From daeccff5d8a54130589564af68fe187ad2a144e2 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Mon, 16 Dec 2019 13:04:38 +0100 Subject: [PATCH 001/665] 3.x: Fix Observable.window (size & time) cancellation and abandonment (#6761) --- .../operators/flowable/FlowableWindow.java | 10 +- ... => FlowableWindowSubscribeIntercept.java} | 4 +- .../flowable/FlowableWindowTimed.java | 27 +- .../observable/ObservableWindow.java | 20 +- .../ObservableWindowSubscribeIntercept.java | 46 + .../observable/ObservableWindowTimed.java | 830 ++++++++---------- .../ObservableWindowWithSizeTest.java | 166 +++- .../ObservableWindowWithTimeTest.java | 150 ++++ 8 files changed, 774 insertions(+), 479 deletions(-) rename src/main/java/io/reactivex/rxjava3/internal/operators/flowable/{WindowSubscribeIntercept.java => FlowableWindowSubscribeIntercept.java} (90%) create mode 100644 src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowSubscribeIntercept.java diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java index f66d38169e..ae027e2d04 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java @@ -92,14 +92,14 @@ public void onNext(T t) { long i = index; UnicastProcessor w = window; - WindowSubscribeIntercept intercept = null; + FlowableWindowSubscribeIntercept intercept = null; if (i == 0) { getAndIncrement(); w = UnicastProcessor.create(bufferSize, this); window = w; - intercept = new WindowSubscribeIntercept(w); + intercept = new FlowableWindowSubscribeIntercept(w); downstream.onNext(intercept); } @@ -211,7 +211,7 @@ public void onSubscribe(Subscription s) { public void onNext(T t) { long i = index; - WindowSubscribeIntercept intercept = null; + FlowableWindowSubscribeIntercept intercept = null; UnicastProcessor w = window; if (i == 0) { getAndIncrement(); @@ -219,7 +219,7 @@ public void onNext(T t) { w = UnicastProcessor.create(bufferSize, this); window = w; - intercept = new WindowSubscribeIntercept(w); + intercept = new FlowableWindowSubscribeIntercept(w); downstream.onNext(intercept); } @@ -477,7 +477,7 @@ void drain() { break; } - WindowSubscribeIntercept intercept = new WindowSubscribeIntercept(t); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(t); a.onNext(intercept); if (intercept.tryAbandon()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/WindowSubscribeIntercept.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowSubscribeIntercept.java similarity index 90% rename from src/main/java/io/reactivex/rxjava3/internal/operators/flowable/WindowSubscribeIntercept.java rename to src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowSubscribeIntercept.java index 4fcbdee225..8f6cf0a2da 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/WindowSubscribeIntercept.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowSubscribeIntercept.java @@ -25,13 +25,13 @@ * @param the element type of the flow. * @since 3.0.0 */ -final class WindowSubscribeIntercept extends Flowable { +final class FlowableWindowSubscribeIntercept extends Flowable { final FlowableProcessor window; final AtomicBoolean once; - WindowSubscribeIntercept(FlowableProcessor source) { + FlowableWindowSubscribeIntercept(FlowableProcessor source) { this.window = source; this.once = new AtomicBoolean(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java index c340c0e0db..0dfec4e4af 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java @@ -28,7 +28,6 @@ import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.BackpressureHelper; import io.reactivex.rxjava3.processors.UnicastProcessor; -import io.reactivex.rxjava3.subscribers.SerializedSubscriber; public final class FlowableWindowTimed extends AbstractFlowableWithUpstream> { final long timespan; @@ -53,23 +52,21 @@ public FlowableWindowTimed(Flowable source, } @Override - protected void subscribeActual(Subscriber> s) { - SerializedSubscriber> actual = new SerializedSubscriber>(s); - + protected void subscribeActual(Subscriber> downstream) { if (timespan == timeskip) { if (maxSize == Long.MAX_VALUE) { source.subscribe(new WindowExactUnboundedSubscriber( - actual, + downstream, timespan, unit, scheduler, bufferSize)); return; } source.subscribe(new WindowExactBoundedSubscriber( - actual, + downstream, timespan, unit, scheduler, bufferSize, maxSize, restartTimerOnMaxSize)); return; } - source.subscribe(new WindowSkipSubscriber(actual, + source.subscribe(new WindowSkipSubscriber(downstream, timespan, timeskip, unit, scheduler.createWorker(), bufferSize)); } @@ -100,8 +97,8 @@ abstract static class AbstractWindowSubscriber final AtomicInteger windowCount; - AbstractWindowSubscriber(Subscriber> actual, long timespan, TimeUnit unit, int bufferSize) { - this.downstream = actual; + AbstractWindowSubscriber(Subscriber> downstream, long timespan, TimeUnit unit, int bufferSize) { + this.downstream = downstream; this.queue = new MpscLinkedQueue(); this.timespan = timespan; this.unit = unit; @@ -204,7 +201,7 @@ void createFirstWindow() { emitted = 1; - WindowSubscribeIntercept intercept = new WindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); downstream.onNext(intercept); timer.replace(scheduler.schedulePeriodicallyDirect(this, timespan, timespan, unit)); @@ -293,7 +290,7 @@ else if (!isEmpty) { window = UnicastProcessor.create(bufferSize, windowRunnable); this.window = window; - WindowSubscribeIntercept intercept = new WindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); downstream.onNext(intercept); if (intercept.tryAbandon()) { @@ -372,7 +369,7 @@ void createFirstWindow() { windowCount.getAndIncrement(); window = UnicastProcessor.create(bufferSize, this); - WindowSubscribeIntercept intercept = new WindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); downstream.onNext(intercept); Runnable boundaryTask = new WindowBoundaryRunnable(this, 1L); @@ -510,7 +507,7 @@ UnicastProcessor createNewWindow(UnicastProcessor window) { window = UnicastProcessor.create(bufferSize, this); this.window = window; - WindowSubscribeIntercept intercept = new WindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); downstream.onNext(intercept); if (restartTimerOnMaxSize) { @@ -573,7 +570,7 @@ void createFirstWindow() { UnicastProcessor window = UnicastProcessor.create(bufferSize, this); windows.add(window); - WindowSubscribeIntercept intercept = new WindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); downstream.onNext(intercept); worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); @@ -647,7 +644,7 @@ void drain() { UnicastProcessor window = UnicastProcessor.create(bufferSize, this); windows.add(window); - WindowSubscribeIntercept intercept = new WindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); downstream.onNext(intercept); worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java index 08eb0327a6..3f11f3952a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java @@ -77,14 +77,17 @@ public void onSubscribe(Disposable d) { @Override public void onNext(T t) { UnicastSubject w = window; + ObservableWindowSubscribeIntercept intercept = null; if (w == null && !cancelled) { w = UnicastSubject.create(capacityHint, this); window = w; - downstream.onNext(w); + intercept = new ObservableWindowSubscribeIntercept(w); + downstream.onNext(intercept); } if (w != null) { w.onNext(t); + if (++size >= count) { size = 0; window = null; @@ -93,6 +96,12 @@ public void onNext(T t) { upstream.dispose(); } } + + if (intercept != null && intercept.tryAbandon()) { + w.onComplete(); + w = null; + window = null; + } } } @@ -180,11 +189,14 @@ public void onNext(T t) { long s = skip; + ObservableWindowSubscribeIntercept intercept = null; + if (i % s == 0 && !cancelled) { wip.getAndIncrement(); UnicastSubject w = UnicastSubject.create(capacityHint, this); + intercept = new ObservableWindowSubscribeIntercept(w); ws.offer(w); - downstream.onNext(w); + downstream.onNext(intercept); } long c = firstEmission + 1; @@ -205,6 +217,10 @@ public void onNext(T t) { } index = i + 1; + + if (intercept != null && intercept.tryAbandon()) { + intercept.window.onComplete(); + } } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowSubscribeIntercept.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowSubscribeIntercept.java new file mode 100644 index 0000000000..da6f96f2a3 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowSubscribeIntercept.java @@ -0,0 +1,46 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.operators.observable; + +import java.util.concurrent.atomic.AtomicBoolean; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.subjects.Subject; + +/** + * Wrapper for a Subject that detects an incoming subscriber. + * @param the element type of the flow. + * @since 3.0.0 + */ +final class ObservableWindowSubscribeIntercept extends Observable { + + final Subject window; + + final AtomicBoolean once; + + ObservableWindowSubscribeIntercept(Subject source) { + this.window = source; + this.once = new AtomicBoolean(); + } + + @Override + protected void subscribeActual(Observer s) { + window.subscribe(s); + once.set(true); + } + + boolean tryAbandon() { + return !once.get() && once.compareAndSet(false, true); + } +} \ No newline at end of file diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java index cdd64cd6ae..3913bf2911 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java @@ -15,17 +15,16 @@ import java.util.*; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.*; -import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.core.Observable; import io.reactivex.rxjava3.core.Observer; +import io.reactivex.rxjava3.core.Scheduler; import io.reactivex.rxjava3.core.Scheduler.Worker; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.observers.QueueDrainObserver; +import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; -import io.reactivex.rxjava3.internal.util.NotificationLite; -import io.reactivex.rxjava3.observers.SerializedObserver; import io.reactivex.rxjava3.subjects.UnicastSubject; public final class ObservableWindowTimed extends AbstractObservableWithUpstream> { @@ -37,8 +36,7 @@ public final class ObservableWindowTimed extends AbstractObservableWithUpstre final int bufferSize; final boolean restartTimerOnMaxSize; - public ObservableWindowTimed( - ObservableSource source, + public ObservableWindowTimed(Observable source, long timespan, long timeskip, TimeUnit unit, Scheduler scheduler, long maxSize, int bufferSize, boolean restartTimerOnMaxSize) { super(source); @@ -52,223 +50,282 @@ public ObservableWindowTimed( } @Override - public void subscribeActual(Observer> t) { - SerializedObserver> actual = new SerializedObserver>(t); - + protected void subscribeActual(Observer> downstream) { if (timespan == timeskip) { if (maxSize == Long.MAX_VALUE) { source.subscribe(new WindowExactUnboundedObserver( - actual, + downstream, timespan, unit, scheduler, bufferSize)); return; } source.subscribe(new WindowExactBoundedObserver( - actual, + downstream, timespan, unit, scheduler, bufferSize, maxSize, restartTimerOnMaxSize)); return; } - source.subscribe(new WindowSkipObserver(actual, + source.subscribe(new WindowSkipObserver(downstream, timespan, timeskip, unit, scheduler.createWorker(), bufferSize)); } - static final class WindowExactUnboundedObserver - extends QueueDrainObserver> - implements Observer, Disposable, Runnable { + abstract static class AbstractWindowObserver + extends AtomicInteger + implements Observer, Disposable { + private static final long serialVersionUID = 5724293814035355511L; + + final Observer> downstream; + + final SimplePlainQueue queue; + final long timespan; final TimeUnit unit; - final Scheduler scheduler; final int bufferSize; - Disposable upstream; + long emitted; - UnicastSubject window; + volatile boolean done; + Throwable error; - final SequentialDisposable timer = new SequentialDisposable(); + Disposable upstream; - static final Object NEXT = new Object(); + final AtomicBoolean downstreamCancelled; - volatile boolean terminated; + volatile boolean upstreamCancelled; - WindowExactUnboundedObserver(Observer> actual, long timespan, TimeUnit unit, - Scheduler scheduler, int bufferSize) { - super(actual, new MpscLinkedQueue()); + final AtomicInteger windowCount; + + AbstractWindowObserver(Observer> downstream, long timespan, TimeUnit unit, int bufferSize) { + this.downstream = downstream; + this.queue = new MpscLinkedQueue(); this.timespan = timespan; this.unit = unit; - this.scheduler = scheduler; this.bufferSize = bufferSize; + this.downstreamCancelled = new AtomicBoolean(); + this.windowCount = new AtomicInteger(1); } @Override - public void onSubscribe(Disposable d) { + public final void onSubscribe(Disposable d) { if (DisposableHelper.validate(this.upstream, d)) { this.upstream = d; - window = UnicastSubject.create(bufferSize); - - Observer> a = downstream; - a.onSubscribe(this); - - a.onNext(window); + downstream.onSubscribe(this); - if (!cancelled) { - Disposable task = scheduler.schedulePeriodicallyDirect(this, timespan, timespan, unit); - timer.replace(task); - } + createFirstWindow(); } } + abstract void createFirstWindow(); + @Override - public void onNext(T t) { - if (terminated) { - return; - } - if (fastEnter()) { - window.onNext(t); - if (leave(-1) == 0) { - return; - } - } else { - queue.offer(NotificationLite.next(t)); - if (!enter()) { - return; - } - } - drainLoop(); + public final void onNext(T t) { + queue.offer(t); + drain(); } @Override - public void onError(Throwable t) { + public final void onError(Throwable t) { error = t; done = true; - if (enter()) { - drainLoop(); - } - - downstream.onError(t); + drain(); } @Override - public void onComplete() { + public final void onComplete() { done = true; - if (enter()) { - drainLoop(); - } + drain(); + } - downstream.onComplete(); + @Override + public final void dispose() { + if (downstreamCancelled.compareAndSet(false, true)) { + windowDone(); + } } @Override - public void dispose() { - cancelled = true; + public final boolean isDisposed() { + return downstreamCancelled.get(); + } + + final void windowDone() { + if (windowCount.decrementAndGet() == 0) { + cleanupResources(); + upstream.dispose(); + upstreamCancelled = true; + drain(); + } + } + + abstract void cleanupResources(); + + abstract void drain(); + } + + static final class WindowExactUnboundedObserver + extends AbstractWindowObserver + implements Runnable { + + private static final long serialVersionUID = 1155822639622580836L; + + final Scheduler scheduler; + + UnicastSubject window; + + final SequentialDisposable timer; + + static final Object NEXT_WINDOW = new Object(); + + final Runnable windowRunnable; + + WindowExactUnboundedObserver(Observer> actual, long timespan, TimeUnit unit, + Scheduler scheduler, int bufferSize) { + super(actual, timespan, unit, bufferSize); + this.scheduler = scheduler; + this.timer = new SequentialDisposable(); + this.windowRunnable = new WindowRunnable(); } @Override - public boolean isDisposed() { - return cancelled; + void createFirstWindow() { + if (!downstreamCancelled.get()) { + windowCount.getAndIncrement(); + window = UnicastSubject.create(bufferSize, windowRunnable); + + emitted = 1; + + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + downstream.onNext(intercept); + + timer.replace(scheduler.schedulePeriodicallyDirect(this, timespan, timespan, unit)); + + if (intercept.tryAbandon()) { + window.onComplete(); + } + } } @Override public void run() { - if (cancelled) { - terminated = true; - } - queue.offer(NEXT); - if (enter()) { - drainLoop(); - } + queue.offer(NEXT_WINDOW); + drain(); } - void drainLoop() { + @Override + void drain() { + if (getAndIncrement() != 0) { + return; + } - final MpscLinkedQueue q = (MpscLinkedQueue)queue; - final Observer> a = downstream; - UnicastSubject w = window; + final SimplePlainQueue queue = this.queue; + final Observer> downstream = this.downstream; + UnicastSubject window = this.window; int missed = 1; for (;;) { - for (;;) { - boolean term = terminated; // NOPMD - - boolean d = done; - - Object o = q.poll(); - - if (d && (o == null || o == NEXT)) { - window = null; - q.clear(); - Throwable err = error; - if (err != null) { - w.onError(err); + if (upstreamCancelled) { + queue.clear(); + window = null; + this.window = null; + } else { + boolean isDone = done; + Object o = queue.poll(); + boolean isEmpty = o == null; + + if (isDone && isEmpty) { + Throwable ex = error; + if (ex != null) { + if (window != null) { + window.onError(ex); + } + downstream.onError(ex); } else { - w.onComplete(); + if (window != null) { + window.onComplete(); + } + downstream.onComplete(); } - timer.dispose(); - return; + cleanupResources(); + upstreamCancelled = true; + continue; } + else if (!isEmpty) { - if (o == null) { - break; - } + if (o == NEXT_WINDOW) { + if (window != null) { + window.onComplete(); + window = null; + this.window = null; + } + if (downstreamCancelled.get()) { + timer.dispose(); + } else { + emitted++; - if (o == NEXT) { - w.onComplete(); - if (!term) { - w = UnicastSubject.create(bufferSize); - window = w; + windowCount.getAndIncrement(); + window = UnicastSubject.create(bufferSize, windowRunnable); + this.window = window; - a.onNext(w); - } else { - upstream.dispose(); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + downstream.onNext(intercept); + + if (intercept.tryAbandon()) { + window.onComplete(); + } + } + } else if (window != null) { + @SuppressWarnings("unchecked") + T item = (T)o; + window.onNext(item); } + continue; } - - w.onNext(NotificationLite.getValue(o)); } - missed = leave(-missed); + missed = addAndGet(-missed); if (missed == 0) { break; } } } + + @Override + void cleanupResources() { + timer.dispose(); + } + + final class WindowRunnable implements Runnable { + @Override + public void run() { + windowDone(); + } + } } static final class WindowExactBoundedObserver - extends QueueDrainObserver> - implements Disposable { - final long timespan; - final TimeUnit unit; + extends AbstractWindowObserver + implements Runnable { + private static final long serialVersionUID = -6130475889925953722L; + final Scheduler scheduler; - final int bufferSize; final boolean restartTimerOnMaxSize; final long maxSize; - final Scheduler.Worker worker; long count; - long producerIndex; - - Disposable upstream; - UnicastSubject window; - volatile boolean terminated; - - final SequentialDisposable timer = new SequentialDisposable(); + final SequentialDisposable timer; WindowExactBoundedObserver( Observer> actual, long timespan, TimeUnit unit, Scheduler scheduler, int bufferSize, long maxSize, boolean restartTimerOnMaxSize) { - super(actual, new MpscLinkedQueue()); - this.timespan = timespan; - this.unit = unit; + super(actual, timespan, unit, bufferSize); this.scheduler = scheduler; - this.bufferSize = bufferSize; this.maxSize = maxSize; this.restartTimerOnMaxSize = restartTimerOnMaxSize; if (restartTimerOnMaxSize) { @@ -276,415 +333,288 @@ static final class WindowExactBoundedObserver } else { worker = null; } + this.timer = new SequentialDisposable(); } @Override - public void onSubscribe(Disposable d) { - if (DisposableHelper.validate(this.upstream, d)) { - this.upstream = d; - - Observer> a = downstream; - - a.onSubscribe(this); - - if (cancelled) { - return; - } + void createFirstWindow() { + if (!downstreamCancelled.get()) { + emitted = 1; - UnicastSubject w = UnicastSubject.create(bufferSize); - window = w; + windowCount.getAndIncrement(); + window = UnicastSubject.create(bufferSize, this); - a.onNext(w); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + downstream.onNext(intercept); - Disposable task; - ConsumerIndexHolder consumerIndexHolder = new ConsumerIndexHolder(producerIndex, this); + Runnable boundaryTask = new WindowBoundaryRunnable(this, 1L); if (restartTimerOnMaxSize) { - task = worker.schedulePeriodically(consumerIndexHolder, timespan, timespan, unit); + timer.replace(worker.schedulePeriodically(boundaryTask, timespan, timespan, unit)); } else { - task = scheduler.schedulePeriodicallyDirect(consumerIndexHolder, timespan, timespan, unit); + timer.replace(scheduler.schedulePeriodicallyDirect(boundaryTask, timespan, timespan, unit)); } - timer.replace(task); - } - } - - @Override - public void onNext(T t) { - if (terminated) { - return; - } - - if (fastEnter()) { - UnicastSubject w = window; - w.onNext(t); - - long c = count + 1; - - if (c >= maxSize) { - producerIndex++; - count = 0; - - w.onComplete(); - - w = UnicastSubject.create(bufferSize); - window = w; - downstream.onNext(w); - if (restartTimerOnMaxSize) { - Disposable tm = timer.get(); - tm.dispose(); - Disposable task = worker.schedulePeriodically( - new ConsumerIndexHolder(producerIndex, this), timespan, timespan, unit); - - DisposableHelper.replace(timer, task); - } - } else { - count = c; - } - - if (leave(-1) == 0) { - return; - } - } else { - queue.offer(NotificationLite.next(t)); - if (!enter()) { - return; + if (intercept.tryAbandon()) { + window.onComplete(); } } - drainLoop(); } @Override - public void onError(Throwable t) { - error = t; - done = true; - if (enter()) { - drainLoop(); - } - - downstream.onError(t); + public void run() { + windowDone(); } @Override - public void onComplete() { - done = true; - if (enter()) { - drainLoop(); + void cleanupResources() { + timer.dispose(); + Worker w = worker; + if (w != null) { + w.dispose(); } - - downstream.onComplete(); } - @Override - public void dispose() { - cancelled = true; + void boundary(WindowBoundaryRunnable sender) { + queue.offer(sender); + drain(); } @Override - public boolean isDisposed() { - return cancelled; - } - - void disposeTimer() { - DisposableHelper.dispose(timer); - Worker w = worker; - if (w != null) { - w.dispose(); + void drain() { + if (getAndIncrement() != 0) { + return; } - } - - void drainLoop() { - final MpscLinkedQueue q = (MpscLinkedQueue)queue; - final Observer> a = downstream; - UnicastSubject w = window; int missed = 1; - for (;;) { + final SimplePlainQueue queue = this.queue; + final Observer> downstream = this.downstream; + UnicastSubject window = this.window; - for (;;) { - if (terminated) { - upstream.dispose(); - q.clear(); - disposeTimer(); - return; - } - - boolean d = done; + for (;;) { - Object o = q.poll(); + if (upstreamCancelled) { + queue.clear(); + window = null; + this.window = null; + } else { - boolean empty = o == null; - boolean isHolder = o instanceof ConsumerIndexHolder; + boolean isDone = done; + Object o = queue.poll(); + boolean isEmpty = o == null; - if (d && (empty || isHolder)) { - window = null; - q.clear(); - Throwable err = error; - if (err != null) { - w.onError(err); + if (isDone && isEmpty) { + Throwable ex = error; + if (ex != null) { + if (window != null) { + window.onError(ex); + } + downstream.onError(ex); } else { - w.onComplete(); + if (window != null) { + window.onComplete(); + } + downstream.onComplete(); } - disposeTimer(); - return; - } - - if (empty) { - break; - } - - if (isHolder) { - ConsumerIndexHolder consumerIndexHolder = (ConsumerIndexHolder) o; - if (!restartTimerOnMaxSize || producerIndex == consumerIndexHolder.index) { - w.onComplete(); - count = 0; - w = UnicastSubject.create(bufferSize); - window = w; - - a.onNext(w); + cleanupResources(); + upstreamCancelled = true; + continue; + } else if (!isEmpty) { + if (o instanceof WindowBoundaryRunnable) { + WindowBoundaryRunnable boundary = (WindowBoundaryRunnable) o; + if (boundary.index == emitted || !restartTimerOnMaxSize) { + this.count = 0; + window = createNewWindow(window); + } + } else if (window != null) { + @SuppressWarnings("unchecked") + T item = (T)o; + window.onNext(item); + + long count = this.count + 1; + if (count == maxSize) { + this.count = 0; + window = createNewWindow(window); + } else { + this.count = count; + } } + continue; } + } - w.onNext(NotificationLite.getValue(o)); - long c = count + 1; - - if (c >= maxSize) { - producerIndex++; - count = 0; + missed = addAndGet(-missed); + if (missed == 0) { + break; + } + } + } - w.onComplete(); + UnicastSubject createNewWindow(UnicastSubject window) { + if (window != null) { + window.onComplete(); + window = null; + } - w = UnicastSubject.create(bufferSize); - window = w; - downstream.onNext(w); + if (downstreamCancelled.get()) { + cleanupResources(); + } else { + long emitted = this.emitted; + this.emitted = ++emitted; - if (restartTimerOnMaxSize) { - Disposable tm = timer.get(); - tm.dispose(); + windowCount.getAndIncrement(); + window = UnicastSubject.create(bufferSize, this); + this.window = window; - Disposable task = worker.schedulePeriodically( - new ConsumerIndexHolder(producerIndex, this), timespan, timespan, unit); - if (!timer.compareAndSet(tm, task)) { - task.dispose(); - } - } + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + downstream.onNext(intercept); - } else { - count = c; - } + if (restartTimerOnMaxSize) { + timer.update(worker.schedulePeriodically(new WindowBoundaryRunnable(this, emitted), timespan, timespan, unit)); } - missed = leave(-missed); - if (missed == 0) { - break; + if (intercept.tryAbandon()) { + window.onComplete(); } } + + return window; } - static final class ConsumerIndexHolder implements Runnable { - final long index; + static final class WindowBoundaryRunnable implements Runnable { + final WindowExactBoundedObserver parent; - ConsumerIndexHolder(long index, WindowExactBoundedObserver parent) { - this.index = index; + + final long index; + + WindowBoundaryRunnable(WindowExactBoundedObserver parent, long index) { this.parent = parent; + this.index = index; } @Override public void run() { - WindowExactBoundedObserver p = parent; - - if (!p.cancelled) { - p.queue.offer(this); - } else { - p.terminated = true; - } - if (p.enter()) { - p.drainLoop(); - } + parent.boundary(this); } } } static final class WindowSkipObserver - extends QueueDrainObserver> - implements Disposable, Runnable { - final long timespan; + extends AbstractWindowObserver + implements Runnable { + private static final long serialVersionUID = -7852870764194095894L; + final long timeskip; - final TimeUnit unit; final Scheduler.Worker worker; - final int bufferSize; final List> windows; - Disposable upstream; - - volatile boolean terminated; - WindowSkipObserver(Observer> actual, long timespan, long timeskip, TimeUnit unit, Worker worker, int bufferSize) { - super(actual, new MpscLinkedQueue()); - this.timespan = timespan; + super(actual, timespan, unit, bufferSize); this.timeskip = timeskip; - this.unit = unit; this.worker = worker; - this.bufferSize = bufferSize; this.windows = new LinkedList>(); } @Override - public void onSubscribe(Disposable d) { - if (DisposableHelper.validate(this.upstream, d)) { - this.upstream = d; - - downstream.onSubscribe(this); - - if (cancelled) { - return; - } - - final UnicastSubject w = UnicastSubject.create(bufferSize); - windows.add(w); + void createFirstWindow() { + if (!downstreamCancelled.get()) { + emitted = 1; - downstream.onNext(w); - worker.schedule(new CompletionTask(w), timespan, unit); + windowCount.getAndIncrement(); + UnicastSubject window = UnicastSubject.create(bufferSize, this); + windows.add(window); - worker.schedulePeriodically(this, timeskip, timeskip, unit); - } + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + downstream.onNext(intercept); - } + worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); + worker.schedulePeriodically(new WindowBoundaryRunnable(this, true), timeskip, timeskip, unit); - @Override - public void onNext(T t) { - if (fastEnter()) { - for (UnicastSubject w : windows) { - w.onNext(t); - } - if (leave(-1) == 0) { - return; + if (intercept.tryAbandon()) { + window.onComplete(); + windows.remove(window); } - } else { - queue.offer(t); - if (!enter()) { - return; - } - } - drainLoop(); - } - - @Override - public void onError(Throwable t) { - error = t; - done = true; - if (enter()) { - drainLoop(); } - - downstream.onError(t); - } - - @Override - public void onComplete() { - done = true; - if (enter()) { - drainLoop(); - } - - downstream.onComplete(); } @Override - public void dispose() { - cancelled = true; + void cleanupResources() { + worker.dispose(); } @Override - public boolean isDisposed() { - return cancelled; - } - - void complete(UnicastSubject w) { - queue.offer(new SubjectWork(w, false)); - if (enter()) { - drainLoop(); + void drain() { + if (getAndIncrement() != 0) { + return; } - } - - @SuppressWarnings("unchecked") - void drainLoop() { - final MpscLinkedQueue q = (MpscLinkedQueue)queue; - final Observer> a = downstream; - final List> ws = windows; int missed = 1; + final SimplePlainQueue queue = this.queue; + final Observer> downstream = this.downstream; + final List> windows = this.windows; for (;;) { - - for (;;) { - if (terminated) { - upstream.dispose(); - q.clear(); - ws.clear(); - worker.dispose(); - return; - } - - boolean d = done; - - Object v = q.poll(); - - boolean empty = v == null; - boolean sw = v instanceof SubjectWork; - - if (d && (empty || sw)) { - q.clear(); - Throwable e = error; - if (e != null) { - for (UnicastSubject w : ws) { - w.onError(e); + if (upstreamCancelled) { + queue.clear(); + windows.clear(); + } else { + boolean isDone = done; + Object o = queue.poll(); + boolean isEmpty = o == null; + + if (isDone && isEmpty) { + Throwable ex = error; + if (ex != null) { + for (UnicastSubject window : windows) { + window.onError(ex); } + downstream.onError(ex); } else { - for (UnicastSubject w : ws) { - w.onComplete(); + for (UnicastSubject window : windows) { + window.onComplete(); } + downstream.onComplete(); } - ws.clear(); - worker.dispose(); - return; - } + cleanupResources(); + upstreamCancelled = true; + continue; + } else if (!isEmpty) { + if (o == WINDOW_OPEN) { + if (!downstreamCancelled.get()) { + long emitted = this.emitted; + this.emitted = ++emitted; - if (empty) { - break; - } + windowCount.getAndIncrement(); + UnicastSubject window = UnicastSubject.create(bufferSize, this); + windows.add(window); - if (sw) { - SubjectWork work = (SubjectWork)v; + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + downstream.onNext(intercept); - if (work.open) { - if (cancelled) { - continue; - } + worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); - final UnicastSubject w = UnicastSubject.create(bufferSize); - ws.add(w); - a.onNext(w); - - worker.schedule(new CompletionTask(w), timespan, unit); + if (intercept.tryAbandon()) { + window.onComplete(); + } + } + } else if (o == WINDOW_CLOSE) { + if (!windows.isEmpty()) { + windows.remove(0).onComplete(); + } } else { - ws.remove(work.w); - work.w.onComplete(); - if (ws.isEmpty() && cancelled) { - terminated = true; + @SuppressWarnings("unchecked") + T item = (T)o; + for (UnicastSubject window : windows) { + window.onNext(item); } } - } else { - for (UnicastSubject w : ws) { - w.onNext((T)v); - } + continue; } } - - missed = leave(-missed); + missed = addAndGet(-missed); if (missed == 0) { break; } @@ -693,37 +623,31 @@ void drainLoop() { @Override public void run() { - - UnicastSubject w = UnicastSubject.create(bufferSize); - - SubjectWork sw = new SubjectWork(w, true); - if (!cancelled) { - queue.offer(sw); - } - if (enter()) { - drainLoop(); - } + windowDone(); } - static final class SubjectWork { - final UnicastSubject w; - final boolean open; - SubjectWork(UnicastSubject w, boolean open) { - this.w = w; - this.open = open; - } + void boundary(boolean isOpen) { + queue.offer(isOpen ? WINDOW_OPEN : WINDOW_CLOSE); + drain(); } - final class CompletionTask implements Runnable { - private final UnicastSubject w; + static final Object WINDOW_OPEN = new Object(); + static final Object WINDOW_CLOSE = new Object(); + + static final class WindowBoundaryRunnable implements Runnable { - CompletionTask(UnicastSubject w) { - this.w = w; + final WindowSkipObserver parent; + + final boolean isOpen; + + WindowBoundaryRunnable(WindowSkipObserver parent, boolean isOpen) { + this.parent = parent; + this.isOpen = isOpen; } @Override public void run() { - complete(w); + parent.boundary(isOpen); } } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java index e90d46bd36..8053b0f532 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java @@ -17,7 +17,7 @@ import java.util.*; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.*; import org.junit.Test; @@ -37,7 +37,7 @@ public class ObservableWindowWithSizeTest extends RxJavaTest { private static List> toLists(Observable> observables) { final List> lists = new ArrayList>(); - Observable.concat(observables.map(new Function, Observable>>() { + Observable.concatEager(observables.map(new Function, Observable>>() { @Override public Observable> apply(Observable xs) { return xs.toList().toObservable(); @@ -376,4 +376,166 @@ public void accept(Observable w) throws Exception { to[0].assertFailure(TestException.class, 1); } + + @Test + public void cancellingWindowCancelsUpstreamSize() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(10) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstreamSize() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(10) + .take(1) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + + inner.get().test().assertResult(1); + } + + @Test + public void cancellingWindowCancelsUpstreamSkip() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(5, 10) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstreamSkip() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(5, 10) + .take(1) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + + inner.get().test().assertResult(1); + } + + @Test + public void cancellingWindowCancelsUpstreamOverlap() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(5, 3) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstreamOverlap() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(5, 3) + .take(1) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + + inner.get().test().assertResult(1); + } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java index ba9544eb9c..271cde0208 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java @@ -949,4 +949,154 @@ public void accept(Observable v) throws Exception { assertFalse("The doOnNext got interrupted!", isInterrupted.get()); } + + @Test + public void cancellingWindowCancelsUpstreamExactTime() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(10, TimeUnit.MINUTES) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstreamExactTime() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(10, TimeUnit.MINUTES) + .take(1) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + + to + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + inner.get().test().assertResult(); + } + + @Test + public void cancellingWindowCancelsUpstreamExactTimeAndSize() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(10, TimeUnit.MINUTES, 100) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstreamExactTimeAndSize() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(10, TimeUnit.MINUTES, 100) + .take(1) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + + to + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + inner.get().test().assertResult(); + } + + @Test + public void cancellingWindowCancelsUpstreamExactTimeSkip() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(10, 15, TimeUnit.MINUTES) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstreamExactTimeSkip() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(10, 15, TimeUnit.MINUTES) + .take(1) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertFalse("Subject still has subscribers!", ps.hasObservers()); + + to + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + inner.get().test().assertResult(); + } } From df30aa19034619d9e501aca19f501c9a450091e0 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Tue, 17 Dec 2019 08:51:21 +0100 Subject: [PATCH 002/665] Bump mockito-core from 3.2.0 to 3.2.4 (#6764) Bumps [mockito-core](https://github.com/mockito/mockito) from 3.2.0 to 3.2.4. - [Release notes](https://github.com/mockito/mockito/releases) - [Commits](https://github.com/mockito/mockito/compare/v3.2.0...v3.2.4) Signed-off-by: dependabot-preview[bot] --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index aaef86c824..439a0c5680 100644 --- a/build.gradle +++ b/build.gradle @@ -6,7 +6,7 @@ buildscript { ext.reactiveStreamsVersion = "1.0.3" ext.junitVersion = "4.12" ext.testNgVersion = "7.0.0" - ext.mockitoVersion = "3.2.0" + ext.mockitoVersion = "3.2.4" ext.jmhLibVersion = "1.21" ext.jmhGradleVersion = "0.5.0" ext.guavaVersion = "28.1-jre" From f96821f14c91752d323877a7874f480fd6b02f12 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Tue, 17 Dec 2019 10:19:08 +0100 Subject: [PATCH 003/665] 3.x: Fix window (boundary, start/end) cancel and abandonment (#6762) --- .../io/reactivex/rxjava3/core/Flowable.java | 20 + .../io/reactivex/rxjava3/core/Observable.java | 70 +++ .../flowable/FlowableWindowBoundary.java | 6 +- .../FlowableWindowBoundarySelector.java | 526 ++++++++++-------- .../observable/ObservableWindowBoundary.java | 6 +- .../ObservableWindowBoundarySelector.java | 488 +++++++++------- .../FlowableWindowWithFlowableTest.java | 69 ++- ...lowableWindowWithStartEndFlowableTest.java | 131 ++++- .../ObservableWindowWithObservableTest.java | 69 ++- .../ObservableWindowWithSizeTest.java | 12 +- ...vableWindowWithStartEndObservableTest.java | 129 ++++- .../ObservableWindowWithTimeTest.java | 12 +- 12 files changed, 1056 insertions(+), 482 deletions(-) diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 5cb5ebd7ed..23e666896d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -17965,6 +17965,11 @@ public final Flowable> window( * Publisher. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Backpressure:
*
The outer Publisher of this operator does not support backpressure as it uses a {@code boundary} Publisher to control data @@ -17995,6 +18000,11 @@ public final Flowable> window(Publisher boundaryIndicator) { * Publisher. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Backpressure:
*
The outer Publisher of this operator does not support backpressure as it uses a {@code boundary} Publisher to control data @@ -18031,6 +18041,11 @@ public final Flowable> window(Publisher boundaryIndicator, in * {@code closingSelector} emits an item. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Backpressure:
*
The outer Publisher of this operator doesn't support backpressure because the emission of new @@ -18068,6 +18083,11 @@ public final Flowable> window( * {@code closingSelector} emits an item. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Backpressure:
*
The outer Publisher of this operator doesn't support backpressure because the emission of new diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index 505513c160..ce0815477f 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -14536,6 +14536,11 @@ public final Observable> window(long count, long skip, int bufferS * current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -14564,6 +14569,11 @@ public final Observable> window(long timespan, long timeskip, Time * current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -14594,6 +14604,11 @@ public final Observable> window(long timespan, long timeskip, Time * current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -14630,6 +14645,11 @@ public final Observable> window(long timespan, long timeskip, Time * ObservableSource emits the current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -14658,6 +14678,11 @@ public final Observable> window(long timespan, TimeUnit unit) { * emits the current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -14690,6 +14715,11 @@ public final Observable> window(long timespan, TimeUnit unit, * emits the current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -14723,6 +14753,11 @@ public final Observable> window(long timespan, TimeUnit unit, * ObservableSource emits the current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -14754,6 +14789,11 @@ public final Observable> window(long timespan, TimeUnit unit, * current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -14788,6 +14828,11 @@ public final Observable> window(long timespan, TimeUnit unit, * current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -14824,6 +14869,11 @@ public final Observable> window(long timespan, TimeUnit unit, * current window and propagates the notification from the source ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -14865,6 +14915,11 @@ public final Observable> window( * ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
@@ -14891,6 +14946,11 @@ public final Observable> window(ObservableSource boundary) * ObservableSource. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
@@ -14922,6 +14982,11 @@ public final Observable> window(ObservableSource boundary, * {@code closingIndicator} emits an item. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
@@ -14953,6 +15018,11 @@ public final Observable> window( * {@code closingIndicator} emits an item. *

* + *

+ * Note that ignoring windows or subscribing later (i.e., on another thread) will result in + * so-called window abandonment where a window may not contain any elements. In this case, subsequent + * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is + * a tradeoff for ensuring upstream cancellation can happen under some race conditions. *

*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java index ba43da6d84..2e07d3eec8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java @@ -240,7 +240,11 @@ void drain() { if (emitted != requested.get()) { emitted++; - downstream.onNext(w); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(w); + downstream.onNext(intercept); + if (intercept.tryAbandon()) { + w.onComplete(); + } } else { SubscriptionHelper.cancel(upstream); boundarySubscriber.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java index 573f0257e9..63ee8ccf3c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java @@ -18,70 +18,85 @@ import org.reactivestreams.*; -import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.*; -import io.reactivex.rxjava3.exceptions.MissingBackpressureException; +import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.disposables.DisposableHelper; import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; -import io.reactivex.rxjava3.internal.subscribers.QueueDrainSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; -import io.reactivex.rxjava3.internal.util.NotificationLite; +import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; import io.reactivex.rxjava3.processors.UnicastProcessor; -import io.reactivex.rxjava3.subscribers.*; public final class FlowableWindowBoundarySelector extends AbstractFlowableWithUpstream> { final Publisher open; - final Function> close; + final Function> closingIndicator; final int bufferSize; public FlowableWindowBoundarySelector( Flowable source, - Publisher open, Function> close, + Publisher open, Function> closingIndicator, int bufferSize) { super(source); this.open = open; - this.close = close; + this.closingIndicator = closingIndicator; this.bufferSize = bufferSize; } @Override protected void subscribeActual(Subscriber> s) { source.subscribe(new WindowBoundaryMainSubscriber( - new SerializedSubscriber>(s), - open, close, bufferSize)); + s, open, closingIndicator, bufferSize)); } static final class WindowBoundaryMainSubscriber - extends QueueDrainSubscriber> - implements Subscription { + extends AtomicInteger + implements FlowableSubscriber, Subscription, Runnable { + private static final long serialVersionUID = 8646217640096099753L; + + final Subscriber> downstream; final Publisher open; - final Function> close; + final Function> closingIndicator; final int bufferSize; final CompositeDisposable resources; - Subscription upstream; + final WindowStartSubscriber startSubscriber; + + final List> windows; + + final SimplePlainQueue queue; + + final AtomicLong windowCount; - final AtomicReference boundary = new AtomicReference(); + final AtomicBoolean downstreamCancelled; - final List> ws; + final AtomicLong requested; + long emitted; - final AtomicLong windows = new AtomicLong(); + volatile boolean upstreamCanceled; - final AtomicBoolean stopWindows = new AtomicBoolean(); + volatile boolean upstreamDone; + volatile boolean openDone; + final AtomicThrowable error; + + Subscription upstream; WindowBoundaryMainSubscriber(Subscriber> actual, - Publisher open, Function> close, int bufferSize) { - super(actual, new MpscLinkedQueue()); + Publisher open, Function> closingIndicator, int bufferSize) { + this.downstream = actual; + this.queue = new MpscLinkedQueue(); this.open = open; - this.close = close; + this.closingIndicator = closingIndicator; this.bufferSize = bufferSize; this.resources = new CompositeDisposable(); - this.ws = new ArrayList>(); - windows.lazySet(1); + this.windows = new ArrayList>(); + this.windowCount = new AtomicLong(1L); + this.downstreamCancelled = new AtomicBoolean(); + this.error = new AtomicThrowable(); + this.startSubscriber = new WindowStartSubscriber(this); + this.requested = new AtomicLong(); } @Override @@ -91,297 +106,340 @@ public void onSubscribe(Subscription s) { downstream.onSubscribe(this); - if (stopWindows.get()) { - return; - } - - OperatorWindowBoundaryOpenSubscriber os = new OperatorWindowBoundaryOpenSubscriber(this); + open.subscribe(startSubscriber); - if (boundary.compareAndSet(null, os)) { - s.request(Long.MAX_VALUE); - open.subscribe(os); - } + s.request(Long.MAX_VALUE); } } @Override public void onNext(T t) { - if (done) { - return; - } - if (fastEnter()) { - for (UnicastProcessor w : ws) { - w.onNext(t); - } - if (leave(-1) == 0) { - return; - } - } else { - queue.offer(NotificationLite.next(t)); - if (!enter()) { - return; - } - } - drainLoop(); + queue.offer(t); + drain(); } @Override public void onError(Throwable t) { - if (done) { - RxJavaPlugins.onError(t); - return; - } - error = t; - done = true; - - if (enter()) { - drainLoop(); - } - - if (windows.decrementAndGet() == 0) { - resources.dispose(); + startSubscriber.cancel(); + resources.dispose(); + if (error.tryAddThrowableOrReport(t)) { + upstreamDone = true; + drain(); } - - downstream.onError(t); } @Override public void onComplete() { - if (done) { - return; + startSubscriber.cancel(); + resources.dispose(); + upstreamDone = true; + drain(); + } + + @Override + public void request(long n) { + if (SubscriptionHelper.validate(n)) { + BackpressureHelper.add(requested, n); } - done = true; + } - if (enter()) { - drainLoop(); + @Override + public void cancel() { + if (downstreamCancelled.compareAndSet(false, true)) { + if (windowCount.decrementAndGet() == 0) { + upstream.cancel(); + startSubscriber.cancel(); + resources.dispose(); + error.tryTerminateAndReport(); + upstreamCanceled = true; + drain(); + } else { + startSubscriber.cancel(); + } } + } - if (windows.decrementAndGet() == 0) { + @Override + public void run() { + if (windowCount.decrementAndGet() == 0) { + upstream.cancel(); + startSubscriber.cancel(); resources.dispose(); + error.tryTerminateAndReport(); + upstreamCanceled = true; + drain(); } + } - downstream.onComplete(); + void open(B startValue) { + queue.offer(new WindowStartItem(startValue)); + drain(); } - void error(Throwable t) { + void openError(Throwable t) { upstream.cancel(); resources.dispose(); - DisposableHelper.dispose(boundary); - - downstream.onError(t); + if (error.tryAddThrowableOrReport(t)) { + upstreamDone = true; + drain(); + } } - @Override - public void request(long n) { - requested(n); + void openComplete() { + openDone = true; + drain(); } - @Override - public void cancel() { - if (stopWindows.compareAndSet(false, true)) { - DisposableHelper.dispose(boundary); - if (windows.decrementAndGet() == 0) { - upstream.cancel(); - } - } + void close(WindowEndSubscriberIntercept what) { + queue.offer(what); + drain(); } - void dispose() { + void closeError(Throwable t) { + upstream.cancel(); + startSubscriber.cancel(); resources.dispose(); - DisposableHelper.dispose(boundary); + if (error.tryAddThrowableOrReport(t)) { + upstreamDone = true; + drain(); + } } - void drainLoop() { - final SimplePlainQueue q = queue; - final Subscriber> a = downstream; - final List> ws = this.ws; + void drain() { + if (getAndIncrement() != 0) { + return; + } + int missed = 1; + final Subscriber> downstream = this.downstream; + final SimplePlainQueue queue = this.queue; + final List> windows = this.windows; for (;;) { - - for (;;) { - boolean d = done; - Object o = q.poll(); - - boolean empty = o == null; - - if (d && empty) { - dispose(); - Throwable e = error; - if (e != null) { - for (UnicastProcessor w : ws) { - w.onError(e); - } - } else { - for (UnicastProcessor w : ws) { - w.onComplete(); - } + if (upstreamCanceled) { + queue.clear(); + windows.clear(); + } else { + boolean isDone = upstreamDone; + Object o = queue.poll(); + boolean isEmpty = o == null; + + if (isDone) { + if (isEmpty || error.get() != null) { + terminateDownstream(downstream); + upstreamCanceled = true; + continue; } - ws.clear(); - return; - } - - if (empty) { - break; } - if (o instanceof WindowOperation) { - @SuppressWarnings("unchecked") - WindowOperation wo = (WindowOperation) o; - - UnicastProcessor w = wo.w; - if (w != null) { - if (ws.remove(wo.w)) { - wo.w.onComplete(); - - if (windows.decrementAndGet() == 0) { - dispose(); - return; + if (!isEmpty) { + if (o instanceof WindowStartItem) { + if (!downstreamCancelled.get()) { + long emitted = this.emitted; + if (requested.get() != emitted) { + this.emitted = ++emitted; + + @SuppressWarnings("unchecked") + B startItem = ((WindowStartItem)o).item; + + Publisher endSource; + try { + endSource = ObjectHelper.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null Publisher"); + } catch (Throwable ex) { + upstream.cancel(); + startSubscriber.cancel(); + resources.dispose(); + Exceptions.throwIfFatal(ex); + error.tryAddThrowableOrReport(ex); + upstreamDone = true; + continue; + } + + windowCount.getAndIncrement(); + UnicastProcessor newWindow = UnicastProcessor.create(bufferSize, this); + WindowEndSubscriberIntercept endSubscriber = new WindowEndSubscriberIntercept(this, newWindow); + + downstream.onNext(endSubscriber); + + if (endSubscriber.tryAbandon()) { + newWindow.onComplete(); + } else { + windows.add(newWindow); + resources.add(endSubscriber); + endSource.subscribe(endSubscriber); + } + } else { + upstream.cancel(); + startSubscriber.cancel(); + resources.dispose(); + error.tryAddThrowableOrReport(new MissingBackpressureException(FlowableWindowTimed.missingBackpressureMessage(emitted))); + upstreamDone = true; } } - continue; - } - - if (stopWindows.get()) { - continue; } + else if (o instanceof WindowEndSubscriberIntercept) { + @SuppressWarnings("unchecked") + UnicastProcessor w = ((WindowEndSubscriberIntercept)o).window; - w = UnicastProcessor.create(bufferSize); - - long r = requested(); - if (r != 0L) { - ws.add(w); - a.onNext(w); - if (r != Long.MAX_VALUE) { - produced(1); - } + windows.remove(w); + resources.delete((Disposable)o); + w.onComplete(); } else { - cancel(); - a.onError(new MissingBackpressureException("Could not deliver new window due to lack of requests")); - continue; - } - - Publisher p; + @SuppressWarnings("unchecked") + T item = (T)o; - try { - p = ObjectHelper.requireNonNull(close.apply(wo.open), "The publisher supplied is null"); - } catch (Throwable e) { - cancel(); - a.onError(e); - continue; - } - - OperatorWindowBoundaryCloseSubscriber cl = new OperatorWindowBoundaryCloseSubscriber(this, w); - - if (resources.add(cl)) { - windows.getAndIncrement(); - - p.subscribe(cl); + for (UnicastProcessor w : windows) { + w.onNext(item); + } } continue; } - - for (UnicastProcessor w : ws) { - w.onNext(NotificationLite.getValue(o)); + else if (openDone && windows.size() == 0) { + upstream.cancel(); + startSubscriber.cancel(); + resources.dispose(); + terminateDownstream(downstream); + upstreamCanceled = true; + continue; } } - missed = leave(-missed); + missed = addAndGet(-missed); if (missed == 0) { break; } } } - @Override - public boolean accept(Subscriber> a, Object v) { - // not used by this operator - return false; + void terminateDownstream(Subscriber downstream) { + Throwable ex = error.terminate(); + if (ex == null) { + for (UnicastProcessor w : windows) { + w.onComplete(); + } + downstream.onComplete(); + } else if (ex != ExceptionHelper.TERMINATED) { + for (UnicastProcessor w : windows) { + w.onError(ex); + } + downstream.onError(ex); + } } - void open(B b) { - queue.offer(new WindowOperation(null, b)); - if (enter()) { - drainLoop(); + static final class WindowStartItem { + + final B item; + + WindowStartItem(B item) { + this.item = item; } } - void close(OperatorWindowBoundaryCloseSubscriber w) { - resources.delete(w); - queue.offer(new WindowOperation(w.w, null)); - if (enter()) { - drainLoop(); + static final class WindowStartSubscriber extends AtomicReference + implements FlowableSubscriber { + + private static final long serialVersionUID = -3326496781427702834L; + + final WindowBoundaryMainSubscriber parent; + + WindowStartSubscriber(WindowBoundaryMainSubscriber parent) { + this.parent = parent; } - } - } - static final class WindowOperation { - final UnicastProcessor w; - final B open; - WindowOperation(UnicastProcessor w, B open) { - this.w = w; - this.open = open; - } - } + @Override + public void onSubscribe(Subscription s) { + if (SubscriptionHelper.setOnce(this, s)) { + s.request(Long.MAX_VALUE); + } + } - static final class OperatorWindowBoundaryOpenSubscriber extends DisposableSubscriber { - final WindowBoundaryMainSubscriber parent; + @Override + public void onNext(B t) { + parent.open(t); + } - OperatorWindowBoundaryOpenSubscriber(WindowBoundaryMainSubscriber parent) { - this.parent = parent; - } + @Override + public void onError(Throwable t) { + parent.openError(t); + } - @Override - public void onNext(B t) { - parent.open(t); - } + @Override + public void onComplete() { + parent.openComplete(); + } - @Override - public void onError(Throwable t) { - parent.error(t); + void cancel() { + SubscriptionHelper.cancel(this); + } } - @Override - public void onComplete() { - parent.onComplete(); - } - } + static final class WindowEndSubscriberIntercept extends Flowable + implements FlowableSubscriber, Disposable { - static final class OperatorWindowBoundaryCloseSubscriber extends DisposableSubscriber { - final WindowBoundaryMainSubscriber parent; - final UnicastProcessor w; + final WindowBoundaryMainSubscriber parent; - boolean done; + final UnicastProcessor window; - OperatorWindowBoundaryCloseSubscriber(WindowBoundaryMainSubscriber parent, UnicastProcessor w) { - this.parent = parent; - this.w = w; - } + final AtomicReference upstream; - @Override - public void onNext(V t) { - cancel(); - onComplete(); - } + final AtomicBoolean once; - @Override - public void onError(Throwable t) { - if (done) { - RxJavaPlugins.onError(t); - return; + WindowEndSubscriberIntercept(WindowBoundaryMainSubscriber parent, UnicastProcessor window) { + this.parent = parent; + this.window = window; + this.upstream = new AtomicReference(); + this.once = new AtomicBoolean(); } - done = true; - parent.error(t); - } - @Override - public void onComplete() { - if (done) { - return; + @Override + public void onSubscribe(Subscription s) { + if (SubscriptionHelper.setOnce(upstream, s)) { + s.request(Long.MAX_VALUE); + } + } + + @Override + public void onNext(V t) { + if (SubscriptionHelper.cancel(upstream)) { + parent.close(this); + } + } + + @Override + public void onError(Throwable t) { + if (isDisposed()) { + RxJavaPlugins.onError(t); + } else { + parent.closeError(t); + } + } + + @Override + public void onComplete() { + parent.close(this); + } + + @Override + public void dispose() { + SubscriptionHelper.cancel(upstream); + } + + @Override + public boolean isDisposed() { + return upstream.get() == SubscriptionHelper.CANCELLED; + } + + @Override + protected void subscribeActual(Subscriber s) { + window.subscribe(s); + once.set(true); + } + + boolean tryAbandon() { + return !once.get() && once.compareAndSet(false, true); } - done = true; - parent.close(this); } } + } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java index cbcdfc3c91..48de711a04 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java @@ -230,7 +230,11 @@ void drain() { window = w; windows.getAndIncrement(); - downstream.onNext(w); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(w); + downstream.onNext(intercept); + if (intercept.tryAbandon()) { + w.onComplete(); + } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java index 35e92c3fec..037c7d0773 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java @@ -24,62 +24,79 @@ import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; import io.reactivex.rxjava3.internal.functions.ObjectHelper; -import io.reactivex.rxjava3.internal.observers.QueueDrainObserver; +import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; -import io.reactivex.rxjava3.internal.util.NotificationLite; -import io.reactivex.rxjava3.observers.*; +import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; import io.reactivex.rxjava3.subjects.UnicastSubject; public final class ObservableWindowBoundarySelector extends AbstractObservableWithUpstream> { final ObservableSource open; - final Function> close; + final Function> closingIndicator; final int bufferSize; public ObservableWindowBoundarySelector( ObservableSource source, - ObservableSource open, Function> close, + ObservableSource open, Function> closingIndicator, int bufferSize) { super(source); this.open = open; - this.close = close; + this.closingIndicator = closingIndicator; this.bufferSize = bufferSize; } @Override public void subscribeActual(Observer> t) { source.subscribe(new WindowBoundaryMainObserver( - new SerializedObserver>(t), - open, close, bufferSize)); + t, open, closingIndicator, bufferSize)); } static final class WindowBoundaryMainObserver - extends QueueDrainObserver> - implements Disposable { + extends AtomicInteger + implements Observer, Disposable, Runnable { + private static final long serialVersionUID = 8646217640096099753L; + + final Observer> downstream; final ObservableSource open; - final Function> close; + final Function> closingIndicator; final int bufferSize; final CompositeDisposable resources; - Disposable upstream; + final WindowStartObserver startObserver; + + final List> windows; + + final SimplePlainQueue queue; + + final AtomicLong windowCount; - final AtomicReference boundary = new AtomicReference(); + final AtomicBoolean downstreamDisposed; - final List> ws; + final AtomicLong requested; + long emitted; - final AtomicLong windows = new AtomicLong(); + volatile boolean upstreamCanceled; - final AtomicBoolean stopWindows = new AtomicBoolean(); + volatile boolean upstreamDone; + volatile boolean openDone; + final AtomicThrowable error; - WindowBoundaryMainObserver(Observer> actual, - ObservableSource open, Function> close, int bufferSize) { - super(actual, new MpscLinkedQueue()); + Disposable upstream; + + WindowBoundaryMainObserver(Observer> downstream, + ObservableSource open, Function> closingIndicator, int bufferSize) { + this.downstream = downstream; + this.queue = new MpscLinkedQueue(); this.open = open; - this.close = close; + this.closingIndicator = closingIndicator; this.bufferSize = bufferSize; this.resources = new CompositeDisposable(); - this.ws = new ArrayList>(); - windows.lazySet(1); + this.windows = new ArrayList>(); + this.windowCount = new AtomicLong(1L); + this.downstreamDisposed = new AtomicBoolean(); + this.error = new AtomicThrowable(); + this.startObserver = new WindowStartObserver(this); + this.requested = new AtomicLong(); } @Override @@ -89,281 +106,320 @@ public void onSubscribe(Disposable d) { downstream.onSubscribe(this); - if (stopWindows.get()) { - return; - } - - OperatorWindowBoundaryOpenObserver os = new OperatorWindowBoundaryOpenObserver(this); - - if (boundary.compareAndSet(null, os)) { - open.subscribe(os); - } + open.subscribe(startObserver); } } @Override public void onNext(T t) { - if (fastEnter()) { - for (UnicastSubject w : ws) { - w.onNext(t); - } - if (leave(-1) == 0) { - return; - } - } else { - queue.offer(NotificationLite.next(t)); - if (!enter()) { - return; - } - } - drainLoop(); + queue.offer(t); + drain(); } @Override public void onError(Throwable t) { - if (done) { - RxJavaPlugins.onError(t); - return; - } - error = t; - done = true; - - if (enter()) { - drainLoop(); - } - - if (windows.decrementAndGet() == 0) { - resources.dispose(); + startObserver.dispose(); + resources.dispose(); + if (error.tryAddThrowableOrReport(t)) { + upstreamDone = true; + drain(); } - - downstream.onError(t); } @Override public void onComplete() { - if (done) { - return; - } - done = true; - - if (enter()) { - drainLoop(); - } - - if (windows.decrementAndGet() == 0) { - resources.dispose(); - } - - downstream.onComplete(); - } - - void error(Throwable t) { - upstream.dispose(); + startObserver.dispose(); resources.dispose(); - onError(t); + upstreamDone = true; + drain(); } @Override public void dispose() { - if (stopWindows.compareAndSet(false, true)) { - DisposableHelper.dispose(boundary); - if (windows.decrementAndGet() == 0) { + if (downstreamDisposed.compareAndSet(false, true)) { + if (windowCount.decrementAndGet() == 0) { upstream.dispose(); + startObserver.dispose(); + resources.dispose(); + error.tryTerminateAndReport(); + upstreamCanceled = true; + drain(); + } else { + startObserver.dispose(); } } } @Override public boolean isDisposed() { - return stopWindows.get(); + return downstreamDisposed.get(); + } + + @Override + public void run() { + if (windowCount.decrementAndGet() == 0) { + upstream.dispose(); + startObserver.dispose(); + resources.dispose(); + error.tryTerminateAndReport(); + upstreamCanceled = true; + drain(); + } + } + + void open(B startValue) { + queue.offer(new WindowStartItem(startValue)); + drain(); } - void disposeBoundary() { + void openError(Throwable t) { + upstream.dispose(); resources.dispose(); - DisposableHelper.dispose(boundary); + if (error.tryAddThrowableOrReport(t)) { + upstreamDone = true; + drain(); + } } - void drainLoop() { - final MpscLinkedQueue q = (MpscLinkedQueue)queue; - final Observer> a = downstream; - final List> ws = this.ws; - int missed = 1; + void openComplete() { + openDone = true; + drain(); + } - for (;;) { + void close(WindowEndObserverIntercept what) { + queue.offer(what); + drain(); + } - for (;;) { - boolean d = done; + void closeError(Throwable t) { + upstream.dispose(); + startObserver.dispose(); + resources.dispose(); + if (error.tryAddThrowableOrReport(t)) { + upstreamDone = true; + drain(); + } + } - Object o = q.poll(); + void drain() { + if (getAndIncrement() != 0) { + return; + } - boolean empty = o == null; + int missed = 1; + final Observer> downstream = this.downstream; + final SimplePlainQueue queue = this.queue; + final List> windows = this.windows; - if (d && empty) { - disposeBoundary(); - Throwable e = error; - if (e != null) { - for (UnicastSubject w : ws) { - w.onError(e); - } - } else { - for (UnicastSubject w : ws) { - w.onComplete(); - } + for (;;) { + if (upstreamCanceled) { + queue.clear(); + windows.clear(); + } else { + boolean isDone = upstreamDone; + Object o = queue.poll(); + boolean isEmpty = o == null; + + if (isDone) { + if (isEmpty || error.get() != null) { + terminateDownstream(downstream); + upstreamCanceled = true; + continue; } - ws.clear(); - return; } - if (empty) { - break; - } + if (!isEmpty) { + if (o instanceof WindowStartItem) { + if (!downstreamDisposed.get()) { + @SuppressWarnings("unchecked") + B startItem = ((WindowStartItem)o).item; + + ObservableSource endSource; + try { + endSource = ObjectHelper.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null ObservableSource"); + } catch (Throwable ex) { + upstream.dispose(); + startObserver.dispose(); + resources.dispose(); + Exceptions.throwIfFatal(ex); + error.tryAddThrowableOrReport(ex); + upstreamDone = true; + continue; + } - if (o instanceof WindowOperation) { - @SuppressWarnings("unchecked") - WindowOperation wo = (WindowOperation) o; + windowCount.getAndIncrement(); + UnicastSubject newWindow = UnicastSubject.create(bufferSize, this); + WindowEndObserverIntercept endObserver = new WindowEndObserverIntercept(this, newWindow); - UnicastSubject w = wo.w; - if (w != null) { - if (ws.remove(wo.w)) { - wo.w.onComplete(); + downstream.onNext(endObserver); - if (windows.decrementAndGet() == 0) { - disposeBoundary(); - return; + if (endObserver.tryAbandon()) { + newWindow.onComplete(); + } else { + windows.add(newWindow); + resources.add(endObserver); + endSource.subscribe(endObserver); } } - continue; - } - - if (stopWindows.get()) { - continue; } + else if (o instanceof WindowEndObserverIntercept) { + @SuppressWarnings("unchecked") + UnicastSubject w = ((WindowEndObserverIntercept)o).window; - w = UnicastSubject.create(bufferSize); - - ws.add(w); - a.onNext(w); - - ObservableSource p; - - try { - p = ObjectHelper.requireNonNull(close.apply(wo.open), "The ObservableSource supplied is null"); - } catch (Throwable e) { - Exceptions.throwIfFatal(e); - stopWindows.set(true); - a.onError(e); - continue; - } - - OperatorWindowBoundaryCloseObserver cl = new OperatorWindowBoundaryCloseObserver(this, w); - - if (resources.add(cl)) { - windows.getAndIncrement(); + windows.remove(w); + resources.delete((Disposable)o); + w.onComplete(); + } else { + @SuppressWarnings("unchecked") + T item = (T)o; - p.subscribe(cl); + for (UnicastSubject w : windows) { + w.onNext(item); + } } continue; } - - for (UnicastSubject w : ws) { - w.onNext(NotificationLite.getValue(o)); + else if (openDone && windows.size() == 0) { + upstream.dispose(); + startObserver.dispose(); + resources.dispose(); + terminateDownstream(downstream); + upstreamCanceled = true; + continue; } } - missed = leave(-missed); + missed = addAndGet(-missed); if (missed == 0) { break; } } } - @Override - public void accept(Observer> a, Object v) { + void terminateDownstream(Observer downstream) { + Throwable ex = error.terminate(); + if (ex == null) { + for (UnicastSubject w : windows) { + w.onComplete(); + } + downstream.onComplete(); + } else if (ex != ExceptionHelper.TERMINATED) { + for (UnicastSubject w : windows) { + w.onError(ex); + } + downstream.onError(ex); + } } - void open(B b) { - queue.offer(new WindowOperation(null, b)); - if (enter()) { - drainLoop(); + static final class WindowStartItem { + + final B item; + + WindowStartItem(B item) { + this.item = item; } } - void close(OperatorWindowBoundaryCloseObserver w) { - resources.delete(w); - queue.offer(new WindowOperation(w.w, null)); - if (enter()) { - drainLoop(); + static final class WindowStartObserver extends AtomicReference + implements Observer { + + private static final long serialVersionUID = -3326496781427702834L; + + final WindowBoundaryMainObserver parent; + + WindowStartObserver(WindowBoundaryMainObserver parent) { + this.parent = parent; } - } - } - static final class WindowOperation { - final UnicastSubject w; - final B open; - WindowOperation(UnicastSubject w, B open) { - this.w = w; - this.open = open; - } - } + @Override + public void onSubscribe(Disposable d) { + DisposableHelper.setOnce(this, d); + } - static final class OperatorWindowBoundaryOpenObserver extends DisposableObserver { - final WindowBoundaryMainObserver parent; + @Override + public void onNext(B t) { + parent.open(t); + } - OperatorWindowBoundaryOpenObserver(WindowBoundaryMainObserver parent) { - this.parent = parent; - } + @Override + public void onError(Throwable t) { + parent.openError(t); + } - @Override - public void onNext(B t) { - parent.open(t); - } + @Override + public void onComplete() { + parent.openComplete(); + } - @Override - public void onError(Throwable t) { - parent.error(t); + void dispose() { + DisposableHelper.dispose(this); + } } - @Override - public void onComplete() { - parent.onComplete(); - } - } + static final class WindowEndObserverIntercept extends Observable + implements Observer, Disposable { - static final class OperatorWindowBoundaryCloseObserver extends DisposableObserver { - final WindowBoundaryMainObserver parent; - final UnicastSubject w; + final WindowBoundaryMainObserver parent; - boolean done; + final UnicastSubject window; - OperatorWindowBoundaryCloseObserver(WindowBoundaryMainObserver parent, UnicastSubject w) { - this.parent = parent; - this.w = w; - } + final AtomicReference upstream; - @Override - public void onNext(V t) { - dispose(); - onComplete(); - } + final AtomicBoolean once; - @Override - public void onError(Throwable t) { - if (done) { - RxJavaPlugins.onError(t); - return; + WindowEndObserverIntercept(WindowBoundaryMainObserver parent, UnicastSubject window) { + this.parent = parent; + this.window = window; + this.upstream = new AtomicReference(); + this.once = new AtomicBoolean(); } - done = true; - parent.error(t); - } - @Override - public void onComplete() { - if (done) { - return; + @Override + public void onSubscribe(Disposable d) { + DisposableHelper.setOnce(upstream, d); + } + + @Override + public void onNext(V t) { + if (DisposableHelper.dispose(upstream)) { + parent.close(this); + } + } + + @Override + public void onError(Throwable t) { + if (isDisposed()) { + RxJavaPlugins.onError(t); + } else { + parent.closeError(t); + } + } + + @Override + public void onComplete() { + parent.close(this); + } + + @Override + public void dispose() { + DisposableHelper.dispose(upstream); + } + + @Override + public boolean isDisposed() { + return upstream.get() == DisposableHelper.DISPOSED; + } + + @Override + protected void subscribeActual(Observer o) { + window.subscribe(o); + once.set(true); + } + + boolean tryAbandon() { + return !once.get() && once.compareAndSet(false, true); } - done = true; - parent.close(this); } } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java index fcd2390f7e..8b001803c0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java @@ -26,7 +26,7 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.*; -import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.functions.Functions; import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -418,6 +418,12 @@ protected void subscribeActual(Subscriber subscriber) { ref.set(subscriber); } }) + .doOnNext(new Consumer>() { + @Override + public void accept(Flowable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) .to(TestHelper.>testConsumer()); ts @@ -674,4 +680,65 @@ public void run() { TestHelper.race(r1, r2); } } + + @Test + public void cancellingWindowCancelsUpstream() { + PublishProcessor pp = PublishProcessor.create(); + + TestSubscriber ts = pp.window(Flowable.just(1).concatWith(Flowable.never())) + .take(1) + .flatMap(new Function, Publisher>() { + @Override + public Publisher apply(Flowable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(pp.hasSubscribers()); + + pp.onNext(1); + + ts + .assertResult(1); + + assertFalse("Processor still has subscribers!", pp.hasSubscribers()); + } + + @Test + public void windowAbandonmentCancelsUpstream() { + PublishProcessor pp = PublishProcessor.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestSubscriber> ts = pp.window(Flowable.never()) + .doOnNext(new Consumer>() { + @Override + public void accept(Flowable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertTrue(pp.hasSubscribers()); + + ts + .assertValueCount(1) + ; + + pp.onNext(1); + + assertTrue(pp.hasSubscribers()); + + ts.cancel(); + + ts + .assertValueCount(1) + .assertNoErrors() + .assertNotComplete(); + + assertFalse("Processor still has subscribers!", pp.hasSubscribers()); + + inner.get().test().assertResult(); + } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java index 3a76a4c7a8..9420794dc5 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java @@ -17,7 +17,7 @@ import java.util.*; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.*; import org.junit.*; import org.reactivestreams.*; @@ -160,7 +160,14 @@ public void noUnsubscribeAndNoLeak() { public Flowable apply(Integer t) { return close; } - }).subscribe(ts); + }) + .doOnNext(new Consumer>() { + @Override + public void accept(Flowable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) + .subscribe(ts); open.onNext(1); source.onNext(1); @@ -197,7 +204,14 @@ public void unsubscribeAll() { public Flowable apply(Integer t) { return close; } - }).subscribe(ts); + }) + .doOnNext(new Consumer>() { + @Override + public void accept(Flowable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) + .subscribe(ts); open.onNext(1); @@ -248,16 +262,6 @@ public Flowable apply(Flowable v) throws Exception { .assertResult(1, 2); } - @Test - public void badSourceCallable() { - TestHelper.checkBadSourceFlowable(new Function, Object>() { - @Override - public Object apply(Flowable f) throws Exception { - return f.window(Flowable.just(1), Functions.justFunction(Flowable.never())); - } - }, false, 1, 1, (Object[])null); - } - @Test public void boundarySelectorNormal() { PublishProcessor source = PublishProcessor.create(); @@ -372,6 +376,12 @@ protected void subscribeActual( }; } }) + .doOnNext(new Consumer>() { + @Override + public void accept(Flowable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) .test() .assertValueCount(1) .assertNoErrors() @@ -406,6 +416,12 @@ public Flowable apply(Integer v) throws Exception { return flowableDisposed(closeDisposed); } }) + .doOnNext(new Consumer>() { + @Override + public void accept(Flowable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) .to(TestHelper.>testConsumer()) .assertSubscribed() .assertNoErrors() @@ -436,4 +452,93 @@ public void mainWindowMissingBackpressure() { assertFalse(source.hasSubscribers()); assertFalse(boundary.hasSubscribers()); } + + @Test + public void cancellingWindowCancelsUpstream() { + PublishProcessor pp = PublishProcessor.create(); + + TestSubscriber ts = pp.window(Flowable.just(1).concatWith(Flowable.never()), Functions.justFunction(Flowable.never())) + .take(1) + .flatMap(new Function, Publisher>() { + @Override + public Publisher apply(Flowable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(pp.hasSubscribers()); + + pp.onNext(1); + + ts + .assertResult(1); + + assertFalse("Processor still has subscribers!", pp.hasSubscribers()); + } + + @Test + public void windowAbandonmentCancelsUpstream() { + PublishProcessor pp = PublishProcessor.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestSubscriber> ts = pp.window(Flowable.just(1).concatWith(Flowable.never()), + Functions.justFunction(Flowable.never())) + .doOnNext(new Consumer>() { + @Override + public void accept(Flowable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertTrue(pp.hasSubscribers()); + + ts + .assertValueCount(1) + ; + + pp.onNext(1); + + assertTrue(pp.hasSubscribers()); + + ts.cancel(); + + ts + .assertValueCount(1) + .assertNoErrors() + .assertNotComplete(); + + assertFalse("Processor still has subscribers!", pp.hasSubscribers()); + + inner.get().test().assertResult(); + } + + @SuppressWarnings("unchecked") + @Test + public void closingIndicatorFunctionCrash() { + + PublishProcessor source = PublishProcessor.create(); + PublishProcessor boundary = PublishProcessor.create(); + + TestSubscriber> ts = source.window(boundary, new Function>() { + @Override + public Publisher apply(Integer end) throws Throwable { + throw new TestException(); + } + }) + .test() + ; + + ts.assertEmpty(); + + boundary.onNext(1); + + ts.assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + assertFalse(boundary.hasSubscribers()); + + } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java index 1ac8c7209b..e6804ddcab 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java @@ -28,7 +28,7 @@ import io.reactivex.rxjava3.core.Observer; import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.*; -import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.functions.Functions; import io.reactivex.rxjava3.observers.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -378,6 +378,12 @@ protected void subscribeActual(Observer observer) { ref.set(observer); } }) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) .to(TestHelper.>testConsumer()); to @@ -636,4 +642,65 @@ public void run() { TestHelper.race(r1, r2); } } + + @Test + public void cancellingWindowCancelsUpstream() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(Observable.never()) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has observers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstream() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(Observable.never()) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + to + .assertValueCount(1) + ; + + ps.onNext(1); + + assertTrue(ps.hasObservers()); + + to.dispose(); + + to + .assertValueCount(1) + .assertNoErrors() + .assertNotComplete(); + + assertFalse("Subject still has observers!", ps.hasObservers()); + + inner.get().test().assertResult(); + } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java index 8053b0f532..b7ef436399 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java @@ -398,7 +398,7 @@ public Observable apply(Observable w) throws Throwable { to .assertResult(1); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); } @Test @@ -426,7 +426,7 @@ public void accept(Observable v) throws Throwable { .assertNoErrors() .assertComplete(); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); inner.get().test().assertResult(1); } @@ -452,7 +452,7 @@ public Observable apply(Observable w) throws Throwable { to .assertResult(1); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); } @Test @@ -480,7 +480,7 @@ public void accept(Observable v) throws Throwable { .assertNoErrors() .assertComplete(); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); inner.get().test().assertResult(1); } @@ -506,7 +506,7 @@ public Observable apply(Observable w) throws Throwable { to .assertResult(1); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); } @Test @@ -534,7 +534,7 @@ public void accept(Observable v) throws Throwable { .assertNoErrors() .assertComplete(); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); inner.get().test().assertResult(1); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java index 90732ddf1e..da02116cd9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java @@ -17,7 +17,7 @@ import java.util.*; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.*; import org.junit.*; @@ -161,7 +161,14 @@ public void noUnsubscribeAndNoLeak() { public Observable apply(Integer t) { return close; } - }).subscribe(to); + }) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) + .subscribe(to); open.onNext(1); source.onNext(1); @@ -199,7 +206,14 @@ public void unsubscribeAll() { public Observable apply(Integer t) { return close; } - }).subscribe(to); + }) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) + .subscribe(to); open.onNext(1); @@ -365,6 +379,12 @@ protected void subscribeActual( }; } }) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) .test() .assertValueCount(1) .assertNoErrors() @@ -399,6 +419,12 @@ public ObservableSource apply(Integer v) throws Exception { return observableDisposed(closeDisposed); } }) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable w) throws Throwable { + w.subscribe(Functions.emptyConsumer(), Functions.emptyConsumer()); // avoid abandonment + } + }) .to(TestHelper.>testConsumer()) .assertSubscribed() .assertNoErrors() @@ -409,4 +435,101 @@ public ObservableSource apply(Integer v) throws Exception { assertTrue(openDisposed.get()); assertTrue(closeDisposed.get()); } + + @Test + public void cancellingWindowCancelsUpstream() { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps.window(Observable.just(1).concatWith(Observable.never()), Functions.justFunction(Observable.never())) + .take(1) + .flatMap(new Function, Observable>() { + @Override + public Observable apply(Observable w) throws Throwable { + return w.take(1); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to + .assertResult(1); + + assertFalse("Subject still has observers!", ps.hasObservers()); + } + + @Test + public void windowAbandonmentCancelsUpstream() { + PublishSubject ps = PublishSubject.create(); + + final AtomicReference> inner = new AtomicReference>(); + + TestObserver> to = ps.window(Observable.just(1).concatWith(Observable.never()), + Functions.justFunction(Observable.never())) + .doOnNext(new Consumer>() { + @Override + public void accept(Observable v) throws Throwable { + inner.set(v); + } + }) + .test(); + + assertTrue(ps.hasObservers()); + + to + .assertValueCount(1) + ; + + ps.onNext(1); + + assertTrue(ps.hasObservers()); + + to.dispose(); + + to + .assertValueCount(1) + .assertNoErrors() + .assertNotComplete(); + + assertFalse("Subject still has observers!", ps.hasObservers()); + + inner.get().test().assertResult(); + } + + @SuppressWarnings("unchecked") + @Test + public void closingIndicatorFunctionCrash() { + + PublishSubject source = PublishSubject.create(); + PublishSubject boundary = PublishSubject.create(); + + TestObserver> to = source.window(boundary, new Function>() { + @Override + public Observable apply(Integer end) throws Throwable { + throw new TestException(); + } + }) + .test() + ; + + to.assertEmpty(); + + boundary.onNext(1); + + to.assertFailure(TestException.class); + + assertFalse(source.hasObservers()); + assertFalse(boundary.hasObservers()); + } + + @SuppressWarnings("unchecked") + @Test + public void mainError() { + Observable.error(new TestException()) + .window(Observable.never(), Functions.justFunction(Observable.never())) + .test() + .assertFailure(TestException.class); + } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java index 271cde0208..8eaebf5991 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java @@ -971,7 +971,7 @@ public Observable apply(Observable w) throws Throwable { to .assertResult(1); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); } @Test @@ -990,7 +990,7 @@ public void accept(Observable v) throws Throwable { }) .test(); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); to .assertValueCount(1) @@ -1021,7 +1021,7 @@ public Observable apply(Observable w) throws Throwable { to .assertResult(1); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); } @Test @@ -1040,7 +1040,7 @@ public void accept(Observable v) throws Throwable { }) .test(); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); to .assertValueCount(1) @@ -1071,7 +1071,7 @@ public Observable apply(Observable w) throws Throwable { to .assertResult(1); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); } @Test @@ -1090,7 +1090,7 @@ public void accept(Observable v) throws Throwable { }) .test(); - assertFalse("Subject still has subscribers!", ps.hasObservers()); + assertFalse("Subject still has observers!", ps.hasObservers()); to .assertValueCount(1) From 735add2c25cb08f175ab3fc5af0270587fada083 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Tue, 17 Dec 2019 17:28:01 +0100 Subject: [PATCH 004/665] 3.x: [Java 8] Upgrade to Java 8, add Flowable.fromX operators (#6765) * 3.x: [Java 8] Upgrade to Java 8, add Flowable.fromX operators * Add NonNull annotation to the new fromX methods * Annotate return type argument to Flowable<@NonNull T> --- build.gradle | 13 +- .../rxjava3/annotations/NonNull.java | 4 +- .../io/reactivex/rxjava3/core/Flowable.java | 175 +++++- .../jdk8/FlowableFromCompletionStage.java | 94 ++++ .../internal/jdk8/FlowableFromStream.java | 299 ++++++++++ .../rxjava3/observers/BaseTestConsumer.java | 5 +- .../io/reactivex/rxjava3/flowable/Burst.java | 2 +- .../jdk8/FlowableFromCompletionStageTest.java | 65 +++ .../jdk8/FlowableFromOptionalTest.java | 38 ++ .../internal/jdk8/FlowableFromStreamTest.java | 515 ++++++++++++++++++ .../jdk8/FromCompletionStageTckTest.java | 48 ++ .../internal/jdk8/FromOptional0TckTest.java | 42 ++ .../internal/jdk8/FromOptional1TckTest.java | 42 ++ .../internal/jdk8/FromStreamTckTest.java | 44 ++ .../internal/operators/observable/Burst.java | 2 +- .../rxjava3/testsupport/TestHelper.java | 106 ++-- .../ParamValidationCheckerTest.java | 10 + 17 files changed, 1419 insertions(+), 85 deletions(-) create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStage.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStageTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromOptionalTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FromCompletionStageTckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional0TckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional1TckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FromStreamTckTest.java diff --git a/build.gradle b/build.gradle index 439a0c5680..4dcead40c4 100644 --- a/build.gradle +++ b/build.gradle @@ -66,15 +66,15 @@ apply plugin: "com.jfrog.bintray" apply plugin: "com.jfrog.artifactory" apply plugin: "eclipse" -sourceCompatibility = JavaVersion.VERSION_1_6 -targetCompatibility = JavaVersion.VERSION_1_6 +sourceCompatibility = JavaVersion.VERSION_1_8 +targetCompatibility = JavaVersion.VERSION_1_8 repositories { mavenCentral() } dependencies { - signature "org.codehaus.mojo.signature:java16:1.1@signature" + signature "org.codehaus.mojo.signature:java18:1.0@signature" api "org.reactivestreams:reactive-streams:$reactiveStreamsVersion" jmh "org.reactivestreams:reactive-streams:$reactiveStreamsVersion" @@ -103,14 +103,9 @@ javadoc { options.stylesheetFile = new File(projectDir, "gradle/stylesheet.css"); options.links( - "https://docs.oracle.com/javase/7/docs/api/", + "https://docs.oracle.com/javase/8/docs/api/", "http://www.reactive-streams.org/reactive-streams-${reactiveStreamsVersion}-javadoc/" ) - - if (JavaVersion.current().isJava7()) { - // "./gradle/stylesheet.css" only supports Java 7 - options.addStringOption("stylesheetfile", rootProject.file("./gradle/stylesheet.css").toString()) - } } animalsniffer { diff --git a/src/main/java/io/reactivex/rxjava3/annotations/NonNull.java b/src/main/java/io/reactivex/rxjava3/annotations/NonNull.java index 06f1d4ee8b..03e5180588 100644 --- a/src/main/java/io/reactivex/rxjava3/annotations/NonNull.java +++ b/src/main/java/io/reactivex/rxjava3/annotations/NonNull.java @@ -19,10 +19,10 @@ import java.lang.annotation.*; /** - * Indicates that a field/parameter/variable/return type is never null. + * Indicates that a field/parameter/variable/type parameter/return type is never null. */ @Documented -@Target(value = {FIELD, METHOD, PARAMETER, LOCAL_VARIABLE}) +@Target(value = {FIELD, METHOD, PARAMETER, LOCAL_VARIABLE, TYPE_PARAMETER, TYPE_USE}) @Retention(value = CLASS) public @interface NonNull { } diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 23e666896d..3fed4742e2 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -14,6 +14,7 @@ import java.util.*; import java.util.concurrent.*; +import java.util.stream.*; import org.reactivestreams.*; @@ -24,6 +25,7 @@ import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.functions.*; import io.reactivex.rxjava3.internal.fuseable.ScalarSupplier; +import io.reactivex.rxjava3.internal.jdk8.*; import io.reactivex.rxjava3.internal.operators.flowable.*; import io.reactivex.rxjava3.internal.operators.mixed.*; import io.reactivex.rxjava3.internal.operators.observable.ObservableFromPublisher; @@ -211,6 +213,7 @@ public static Flowable amb(Iterable> sou @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Flowable ambArray(Publisher... sources) { ObjectHelper.requireNonNull(sources, "sources is null"); int len = sources.length; @@ -1224,7 +1227,6 @@ public static Flowable concat(Publisher> * without interleaving them * @see ReactiveX operators documentation: Concat */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -1261,7 +1263,6 @@ public static Flowable concat(Publisher source1, PublisherReactiveX operators documentation: Concat */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -1303,7 +1304,6 @@ public static Flowable concat( * without interleaving them * @see ReactiveX operators documentation: Concat */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -1341,6 +1341,7 @@ public static Flowable concat( @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Flowable concatArray(Publisher... sources) { if (sources.length == 0) { return empty(); @@ -1373,6 +1374,7 @@ public static Flowable concatArray(Publisher... sources) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Flowable concatArrayDelayError(Publisher... sources) { if (sources.length == 0) { return empty(); @@ -1408,6 +1410,7 @@ public static Flowable concatArrayDelayError(Publisher... so @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Flowable concatArrayEager(Publisher... sources) { return concatArrayEager(bufferSize(), bufferSize(), sources); } @@ -1475,6 +1478,7 @@ public static Flowable concatArrayEager(int maxConcurrency, int prefetch, @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) + @SafeVarargs public static Flowable concatArrayEagerDelayError(Publisher... sources) { return concatArrayEagerDelayError(bufferSize(), bufferSize(), sources); } @@ -1914,6 +1918,7 @@ public static Flowable error(final Throwable throwable) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Flowable fromArray(T... items) { ObjectHelper.requireNonNull(items, "items is null"); if (items.length == 0) { @@ -1977,6 +1982,10 @@ public static Flowable fromCallable(Callable supplier) { *

* Important note: This Publisher is blocking on the thread it gets subscribed on; you cannot cancel it. *

+ * Also note that this operator will consume a {@link CompletionStage}-based {@code Future} subclass (such as + * {@link CompletableFuture}) in a blocking manner as well. Use the {@link #fromCompletionStage(CompletionStage)} + * operator to convert and consume such sources in a non-blocking fashion instead. + *

* Unlike 1.x, canceling the Flowable won't cancel the future. If necessary, one can use composition to achieve the * cancellation effect: {@code futurePublisher.doOnCancel(() -> future.cancel(true));}. *

@@ -1993,6 +2002,7 @@ public static Flowable fromCallable(Callable supplier) { * the resulting Publisher * @return a Flowable that emits the item from the source {@link Future} * @see ReactiveX operators documentation: From + * @see #fromCompletionStage(CompletionStage) */ @CheckReturnValue @NonNull @@ -2016,6 +2026,10 @@ public static Flowable fromFuture(Future future) { * cancellation effect: {@code futurePublisher.doOnCancel(() -> future.cancel(true));}. *

* Important note: This Publisher is blocking on the thread it gets subscribed on; you cannot cancel it. + *

+ * Also note that this operator will consume a {@link CompletionStage}-based {@code Future} subclass (such as + * {@link CompletableFuture}) in a blocking manner as well. Use the {@link #fromCompletionStage(CompletionStage)} + * operator to convert and consume such sources in a non-blocking fashion instead. *

*
Backpressure:
*
The operator honors backpressure from downstream.
@@ -2034,6 +2048,7 @@ public static Flowable fromFuture(Future future) { * the resulting Publisher * @return a Flowable that emits the item from the source {@link Future} * @see ReactiveX operators documentation: From + * @see #fromCompletionStage(CompletionStage) */ @CheckReturnValue @NonNull @@ -2058,6 +2073,10 @@ public static Flowable fromFuture(Future future, long timeou * cancellation effect: {@code futurePublisher.doOnCancel(() -> future.cancel(true));}. *

* Important note: This Publisher is blocking; you cannot cancel it. + *

+ * Also note that this operator will consume a {@link CompletionStage}-based {@code Future} subclass (such as + * {@link CompletableFuture}) in a blocking manner as well. Use the {@link #fromCompletionStage(CompletionStage)} + * operator to convert and consume such sources in a non-blocking fashion instead. *

*
Backpressure:
*
The operator honors backpressure from downstream.
@@ -2079,8 +2098,9 @@ public static Flowable fromFuture(Future future, long timeou * the resulting Publisher * @return a Flowable that emits the item from the source {@link Future} * @see ReactiveX operators documentation: From + * @see #fromCompletionStage(CompletionStage) */ - @SuppressWarnings({ "unchecked", "cast" }) + @SuppressWarnings({ "unchecked" }) @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2119,7 +2139,7 @@ public static Flowable fromFuture(Future future, long timeou * @return a Flowable that emits the item from the source {@link Future} * @see ReactiveX operators documentation: From */ - @SuppressWarnings({ "cast", "unchecked" }) + @SuppressWarnings({ "unchecked" }) @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2148,6 +2168,7 @@ public static Flowable fromFuture(Future future, Scheduler s * resulting Publisher * @return a Flowable that emits each item in the source {@link Iterable} sequence * @see ReactiveX operators documentation: From + * @see #fromStream(Stream) */ @CheckReturnValue @NonNull @@ -2656,7 +2677,6 @@ public static Flowable just(T item) { * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2690,7 +2710,6 @@ public static Flowable just(T item1, T item2) { * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2727,7 +2746,6 @@ public static Flowable just(T item1, T item2, T item3) { * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2767,7 +2785,6 @@ public static Flowable just(T item1, T item2, T item3, T item4) { * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2810,7 +2827,6 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5) * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2856,7 +2872,6 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2905,7 +2920,6 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -2957,7 +2971,6 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -3012,7 +3025,6 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, * @return a Flowable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -4655,7 +4667,6 @@ public static Flowable zip(Iterable> * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -4717,7 +4728,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -4780,7 +4790,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -4845,7 +4854,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -4914,7 +4922,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -4988,7 +4995,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -5065,7 +5071,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -5146,7 +5151,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -5232,7 +5236,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -5322,7 +5325,6 @@ public static Flowable zip( * @return a Flowable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -5398,6 +5400,7 @@ public static Flowable zip( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Flowable zipArray(Function zipper, boolean delayError, int bufferSize, Publisher... sources) { if (sources.length == 0) { @@ -5460,7 +5463,6 @@ public final Single all(Predicate predicate) { * emitted an item or sent a termination notification * @see ReactiveX operators documentation: Amb */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -14548,7 +14550,6 @@ public final Flowable sorted(Comparator sortFunction) { * @see #startWithItem(Object) * @since 3.0.0 */ - @SuppressWarnings("unchecked") @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @@ -14576,7 +14577,6 @@ public final Flowable startWithIterable(Iterable items) { * emitted by the source Publisher * @see ReactiveX operators documentation: StartWith */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -14609,7 +14609,6 @@ public final Flowable startWith(Publisher other) { * @see #startWithIterable(Iterable) * @since 3.0.0 */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) @@ -18620,4 +18619,124 @@ public final TestSubscriber test(long initialRequest, boolean cancel) { // No return ts; } + // ------------------------------------------------------------------------- + // JDK 8 Support + // ------------------------------------------------------------------------- + + /** + * Converts the existing value of the provided optional into a {@link #just(Object)} + * or an empty optional into an {@link #empty()} {@code Flowable} instance. + *

+ * + *

+ * Note that the operator takes an already instantiated optional reference and does not + * by any means create this original optional. If the optional is to be created per + * consumer upon subscription, use {@link #defer(Supplier)} around {@code fromOptional}: + *


+     * Flowable.defer(() -> Flowable.fromOptional(createOptional()));
+     * 
+ *
+ *
Backpressure:
+ *
The returned {@code Flowable} supports backpressure.
+ *
Scheduler:
+ *
{@code fromOptional} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the optional value + * @param optional the optional value to convert into a {@code Flowable} + * @return the new Flowable instance + * @see #just(Object) + * @see #empty() + * @since 3.0.0 + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Flowable<@NonNull T> fromOptional(@NonNull Optional optional) { + ObjectHelper.requireNonNull(optional, "optional is null"); + return optional.map(Flowable::just).orElseGet(Flowable::empty); + } + + /** + * Signals the completion value or error of the given (hot) {@link CompletionStage}-based asynchronous calculation. + *

+ * + *

+ * Note that the operator takes an already instantiated, running or terminated {@code CompletionStage}. + * If the optional is to be created per consumer upon subscription, use {@link #defer(Supplier)} + * around {@code fromCompletionStage}: + *


+     * Flowable.defer(() -> Flowable.fromCompletionStage(createCompletionStage()));
+     * 
+ *

+ * If the {@code CompletionStage} completes with {@code null}, a {@link NullPointerException} is signaled. + *

+ * Canceling the flow can't cancel the execution of the {@code CompletionStage} because {@code CompletionStage} + * itself doesn't support cancellation. Instead, the operator detaches from the {@code CompletionStage}. + *

+ *
Backpressure:
+ *
The returned {@code Flowable} supports backpressure and caches the completion value until the + * downstream is ready to receive it.
+ *
Scheduler:
+ *
{@code fromCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the CompletionStage + * @param stage the CompletionStage to convert to Flowable and signal its terminal value or error + * @return the new Flowable instance + * @since 3.0.0 + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Flowable<@NonNull T> fromCompletionStage(@NonNull CompletionStage stage) { + ObjectHelper.requireNonNull(stage, "stage is null"); + return RxJavaPlugins.onAssembly(new FlowableFromCompletionStage<>(stage)); + } + + /** + * Converts a {@link Stream} into a finite {@code Flowable} and emits its items in the sequence. + *

+ * + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #fromIterable(Iterable)}: + *


+     * Stream<T> stream = ...
+     * Flowable.fromIterable(stream::iterator);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * IntStream intStream = IntStream.rangeClosed(1, 10);
+     * Flowable.fromStream(intStream.boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors backpressure from downstream and iterates the given {@code Stream} + * on demand (i.e., when requested).
+ *
Scheduler:
+ *
{@code fromStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the source {@code Stream} + * @param stream the {@code Stream} of values to emit + * @return the new Flowable instance + * @since 3.0.0 + * @see #fromIterable(Iterable) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Flowable<@NonNull T> fromStream(@NonNull Stream stream) { + ObjectHelper.requireNonNull(stream, "stream is null"); + return RxJavaPlugins.onAssembly(new FlowableFromStream<>(stream)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStage.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStage.java new file mode 100644 index 0000000000..f864cdd96d --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStage.java @@ -0,0 +1,94 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletionStage; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; + +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.internal.subscriptions.DeferredScalarSubscription; + +/** + * Wrap a CompletionStage and signal its outcome. + * @param the element type + * @since 3.0.0 + */ +public final class FlowableFromCompletionStage extends Flowable { + + final CompletionStage stage; + + public FlowableFromCompletionStage(CompletionStage stage) { + this.stage = stage; + } + + @Override + protected void subscribeActual(Subscriber s) { + // We need an indirection because one can't detach from a whenComplete + // and cancellation should not hold onto the stage. + BiConsumerAtomicReference whenReference = new BiConsumerAtomicReference<>(); + CompletionStageHandler handler = new CompletionStageHandler<>(s, whenReference); + whenReference.lazySet(handler); + + s.onSubscribe(handler); + stage.whenComplete(whenReference); + } + + static final class CompletionStageHandler + extends DeferredScalarSubscription + implements BiConsumer { + + private static final long serialVersionUID = 4665335664328839859L; + + final BiConsumerAtomicReference whenReference; + + CompletionStageHandler(Subscriber downstream, BiConsumerAtomicReference whenReference) { + super(downstream); + this.whenReference = whenReference; + } + + @Override + public void accept(T item, Throwable error) { + if (error != null) { + downstream.onError(error); + } + else if (item != null) { + complete(item); + } else { + downstream.onError(new NullPointerException("The CompletionStage terminated with null.")); + } + } + + @Override + public void cancel() { + super.cancel(); + whenReference.set(null); + } + } + + static final class BiConsumerAtomicReference extends AtomicReference> + implements BiConsumer { + + private static final long serialVersionUID = 45838553147237545L; + + @Override + public void accept(T t, Throwable u) { + BiConsumer biConsumer = get(); + if (biConsumer != null) { + biConsumer.accept(t, u); + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java new file mode 100644 index 0000000000..db34238b7a --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java @@ -0,0 +1,299 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; + +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.annotations.*; +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.internal.functions.ObjectHelper; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.internal.subscriptions.*; +import io.reactivex.rxjava3.internal.util.BackpressureHelper; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Wraps a {@link Stream} and emits its values as a Flowable sequence. + * @param the element type of the Stream + * @since 3.0.0 + */ +public final class FlowableFromStream extends Flowable { + + final Stream stream; + + public FlowableFromStream(Stream stream) { + this.stream = stream; + } + + @Override + protected void subscribeActual(Subscriber s) { + Iterator iterator; + try { + iterator = stream.iterator(); + + if (!iterator.hasNext()) { + EmptySubscription.complete(s); + closeSafely(stream); + return; + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + EmptySubscription.error(ex, s); + closeSafely(stream); + return; + } + + if (s instanceof ConditionalSubscriber) { + s.onSubscribe(new StreamConditionalSubscription((ConditionalSubscriber)s, iterator, stream)); + } else { + s.onSubscribe(new StreamSubscription<>(s, iterator, stream)); + } + } + + static void closeSafely(AutoCloseable c) { + try { + c.close(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + RxJavaPlugins.onError(ex); + } + } + + abstract static class AbstractStreamSubscription extends AtomicLong implements QueueSubscription { + + private static final long serialVersionUID = -9082954702547571853L; + + Iterator iterator; + + AutoCloseable closeable; + + volatile boolean cancelled; + + boolean once; + + AbstractStreamSubscription(Iterator iterator, AutoCloseable closeable) { + this.iterator = iterator; + this.closeable = closeable; + } + + @Override + public void request(long n) { + if (SubscriptionHelper.validate(n)) { + if (BackpressureHelper.add(this, n) == 0L) { + run(n); + } + } + } + + abstract void run(long n); + + @Override + public void cancel() { + cancelled = true; + request(1L); + } + + @Override + public int requestFusion(int mode) { + if ((mode & SYNC) != 0) { + lazySet(Long.MAX_VALUE); + return SYNC; + } + return NONE; + } + + @Override + public boolean offer(@NonNull T value) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean offer(@NonNull T v1, @NonNull T v2) { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public T poll() { + if (iterator == null) { + return null; + } + if (!once) { + once = true; + } else { + if (!iterator.hasNext()) { + return null; + } + } + return ObjectHelper.requireNonNull(iterator.next(), "Iterator.next() returned a null value"); + } + + @Override + public boolean isEmpty() { + return iterator == null || !iterator.hasNext(); + } + + @Override + public void clear() { + iterator = null; + AutoCloseable c = closeable; + closeable = null; + if (c != null) { + closeSafely(c); + } + } + } + + static final class StreamSubscription extends AbstractStreamSubscription { + + private static final long serialVersionUID = -9082954702547571853L; + + final Subscriber downstream; + + StreamSubscription(Subscriber downstream, Iterator iterator, AutoCloseable closeable) { + super(iterator, closeable); + this.downstream = downstream; + } + + @Override + public void run(long n) { + long emitted = 0L; + Iterator iterator = this.iterator; + Subscriber downstream = this.downstream; + + for (;;) { + + if (cancelled) { + clear(); + break; + } else { + T next; + try { + next = ObjectHelper.requireNonNull(iterator.next(), "The Stream's Iterator returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + cancelled = true; + continue; + } + + downstream.onNext(next); + + if (cancelled) { + continue; + } + + try { + if (!iterator.hasNext()) { + downstream.onComplete(); + cancelled = true; + continue; + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + cancelled = true; + continue; + } + + if (++emitted != n) { + continue; + } + } + + n = get(); + if (emitted == n) { + if (compareAndSet(n, 0L)) { + break; + } + n = get(); + } + } + } + } + + static final class StreamConditionalSubscription extends AbstractStreamSubscription { + + private static final long serialVersionUID = -9082954702547571853L; + + final ConditionalSubscriber downstream; + + StreamConditionalSubscription(ConditionalSubscriber downstream, Iterator iterator, AutoCloseable closeable) { + super(iterator, closeable); + this.downstream = downstream; + } + + @Override + public void run(long n) { + long emitted = 0L; + Iterator iterator = this.iterator; + ConditionalSubscriber downstream = this.downstream; + + for (;;) { + + if (cancelled) { + clear(); + break; + } else { + T next; + try { + next = ObjectHelper.requireNonNull(iterator.next(), "The Stream's Iterator returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + cancelled = true; + continue; + } + + if (downstream.tryOnNext(next)) { + emitted++; + } + + if (cancelled) { + continue; + } + + try { + if (!iterator.hasNext()) { + downstream.onComplete(); + cancelled = true; + continue; + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + cancelled = true; + continue; + } + + if (emitted != n) { + continue; + } + } + + n = get(); + if (emitted == n) { + if (compareAndSet(n, 0L)) { + break; + } + n = get(); + } + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java b/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java index ee2dcf1853..59aa62e796 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java +++ b/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java @@ -228,7 +228,7 @@ public final U assertError(Throwable error) { * @param errorClass the error class to expect * @return this */ - @SuppressWarnings({ "unchecked", "rawtypes", "cast" }) + @SuppressWarnings({ "unchecked", "rawtypes" }) public final U assertError(Class errorClass) { return (U)assertError((Predicate)Functions.isInstanceOf(errorClass)); } @@ -435,6 +435,7 @@ public final U assertValues(T... values) { * @return this * @since 2.2 */ + @SafeVarargs public final U assertValuesOnly(T... values) { return assertSubscribed() .assertValues(values) @@ -493,6 +494,7 @@ public final U assertValueSequence(Iterable sequence) { * @return this * @see #assertFailure(Class, Object...) */ + @SafeVarargs public final U assertResult(T... values) { return assertSubscribed() .assertValues(values) @@ -507,6 +509,7 @@ public final U assertResult(T... values) { * @param values the expected values, asserted in order * @return this */ + @SafeVarargs public final U assertFailure(Class error, T... values) { return assertSubscribed() .assertValues(values) diff --git a/src/test/java/io/reactivex/rxjava3/flowable/Burst.java b/src/test/java/io/reactivex/rxjava3/flowable/Burst.java index 8acf477242..1e6c255a30 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/Burst.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/Burst.java @@ -52,11 +52,11 @@ protected void subscribeActual(final Subscriber subscriber) { } - @SuppressWarnings("unchecked") public static Builder item(T item) { return items(item); } + @SafeVarargs public static Builder items(T... items) { return new Builder(Arrays.asList(items)); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStageTest.java new file mode 100644 index 0000000000..8a71bbd9b3 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromCompletionStageTest.java @@ -0,0 +1,65 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.subscribers.TestSubscriber; + +public class FlowableFromCompletionStageTest extends RxJavaTest { + + @Test + public void syncSuccess() { + Flowable.fromCompletionStage(CompletableFuture.completedFuture(1)) + .test() + .assertResult(1); + } + + @Test + public void syncFailure() { + CompletableFuture cf = new CompletableFuture<>(); + cf.completeExceptionally(new TestException()); + + Flowable.fromCompletionStage(cf) + .test() + .assertFailure(TestException.class); + } + + @Test + public void syncNull() { + Flowable.fromCompletionStage(CompletableFuture.completedFuture(null)) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void cancel() { + CompletableFuture cf = new CompletableFuture<>(); + + TestSubscriber ts = Flowable.fromCompletionStage(cf) + .test(); + + ts.assertEmpty(); + + ts.cancel(); + + cf.complete(1); + + ts.assertEmpty(); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromOptionalTest.java new file mode 100644 index 0000000000..6676bcee61 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromOptionalTest.java @@ -0,0 +1,38 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Optional; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; + +public class FlowableFromOptionalTest extends RxJavaTest { + + @Test + public void hasValue() { + Flowable.fromOptional(Optional.of(1)) + .test() + .assertResult(1); + } + + @Test + public void empty() { + Flowable.fromOptional(Optional.empty()) + .test() + .assertResult(); + } + +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java new file mode 100644 index 0000000000..940ac95c4d --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java @@ -0,0 +1,515 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; +import java.util.stream.*; + +import org.junit.Test; +import org.reactivestreams.Subscription; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.schedulers.Schedulers; +import io.reactivex.rxjava3.testsupport.*; + +public class FlowableFromStreamTest extends RxJavaTest { + + @Test + public void empty() { + Flowable.fromStream(Stream.of()) + .test() + .assertResult(); + } + + @Test + public void just() { + Flowable.fromStream(Stream.of(1)) + .test() + .assertResult(1); + } + + @Test + public void many() { + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5)) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void manyBackpressured() { + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5)) + .test(0L) + .assertEmpty() + .requestMore(1) + .assertValuesOnly(1) + .requestMore(2) + .assertValuesOnly(1, 2, 3) + .requestMore(2) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void noReuse() { + Flowable source = Flowable.fromStream(Stream.of(1, 2, 3, 4, 5)); + + source + .test() + .assertResult(1, 2, 3, 4, 5); + + source + .test() + .assertFailure(IllegalStateException.class); + } + + @Test + public void take() { + Flowable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .take(5) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void emptyConditional() { + Flowable.fromStream(Stream.of()) + .filter(v -> true) + .test() + .assertResult(); + } + + @Test + public void justConditional() { + Flowable.fromStream(Stream.of(1)) + .filter(v -> true) + .test() + .assertResult(1); + } + + @Test + public void manyConditional() { + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5)) + .filter(v -> true) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void manyBackpressuredConditional() { + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5)) + .filter(v -> true) + .test(0L) + .assertEmpty() + .requestMore(1) + .assertValuesOnly(1) + .requestMore(2) + .assertValuesOnly(1, 2, 3) + .requestMore(2) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void manyConditionalSkip() { + Flowable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .filter(v -> v % 2 == 0) + .test() + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void takeConditional() { + Flowable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .filter(v -> true) + .take(5) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void noOfferNoCrashAfterClear() throws Throwable { + AtomicReference> queue = new AtomicReference<>(); + + Flowable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .subscribe(new FlowableSubscriber() { + @Override + public void onSubscribe(@NonNull Subscription s) { + queue.set((SimpleQueue)s); + } + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + }); + + SimpleQueue q = queue.get(); + TestHelper.assertNoOffer(q); + + assertFalse(q.isEmpty()); + + q.clear(); + + assertNull(q.poll()); + + assertTrue(q.isEmpty()); + + q.clear(); + + assertNull(q.poll()); + + assertTrue(q.isEmpty()); + } + + @Test + public void streamOfNull() { + Flowable.fromStream(Stream.of((Integer)null)) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void streamOfNullConditional() { + Flowable.fromStream(Stream.of((Integer)null)) + .filter(v -> true) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void syncFusionSupport() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + Flowable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .subscribeWith(ts) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + } + + @Test + public void asyncFusionNotSupported() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ASYNC); + + Flowable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .subscribeWith(ts) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + } + + @Test + public void fusedForParallel() { + Flowable.fromStream(IntStream.rangeClosed(1, 1000).boxed()) + .parallel() + .runOn(Schedulers.computation(), 1) + .map(v -> v + 1) + .sequential() + .test() + .awaitDone(5, TimeUnit.SECONDS) + .assertValueCount(1000) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void runToEndCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Stream stream = Stream.of(1, 2, 3, 4, 5).onClose(() -> { throw new TestException(); }); + + Flowable.fromStream(stream) + .test() + .assertResult(1, 2, 3, 4, 5); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void takeCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Stream stream = Stream.of(1, 2, 3, 4, 5).onClose(() -> { throw new TestException(); }); + + Flowable.fromStream(stream) + .take(3) + .test() + .assertResult(1, 2, 3); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void hasNextCrash() { + AtomicInteger v = new AtomicInteger(); + Flowable.fromStream(Stream.generate(() -> { + int value = v.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + })) + .test() + .assertFailure(TestException.class, 0); + } + + @Test + public void hasNextCrashConditional() { + AtomicInteger counter = new AtomicInteger(); + Flowable.fromStream(Stream.generate(() -> { + int value = counter.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + })) + .filter(v -> true) + .test() + .assertFailure(TestException.class, 0); + } + + void requestOneByOneBase(boolean conditional) { + List list = new ArrayList<>(); + + Flowable source = Flowable.fromStream(IntStream.rangeClosed(1, 10).boxed()); + if (conditional) { + source = source.filter(v -> true); + } + + source.subscribe(new FlowableSubscriber() { + + @NonNull Subscription upstream; + + @Override + public void onSubscribe(@NonNull Subscription s) { + this.upstream = s; + s.request(1); + } + + @Override + public void onNext(Integer t) { + list.add(t); + upstream.request(1); + } + + @Override + public void onError(Throwable t) { + list.add(t); + } + + @Override + public void onComplete() { + list.add(100); + } + }); + + assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 100), list); + } + + @Test + public void requestOneByOne() { + requestOneByOneBase(false); + } + + @Test + public void requestOneByOneConditional() { + requestOneByOneBase(true); + } + + void requestRaceBase(boolean conditional) throws Exception { + ExecutorService exec = Executors.newCachedThreadPool(); + try { + for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { + + AtomicInteger counter = new AtomicInteger(); + + int max = 100; + + Flowable source = Flowable.fromStream(IntStream.rangeClosed(1, max).boxed()); + if (conditional) { + source = source.filter(v -> true); + } + + CountDownLatch cdl = new CountDownLatch(1); + + source + .subscribe(new FlowableSubscriber() { + + @NonNull Subscription upstream; + + @Override + public void onSubscribe(@NonNull Subscription s) { + + this.upstream = s; + s.request(1); + + } + + @Override + public void onNext(Integer t) { + counter.getAndIncrement(); + + AtomicInteger sync = new AtomicInteger(2); + exec.submit(() -> { + if (sync.decrementAndGet() != 0) { + while (sync.get() != 0) { } + } + upstream.request(1); + }); + + if (sync.decrementAndGet() != 0) { + while (sync.get() != 0) { } + } + } + + @Override + public void onError(Throwable t) { + t.printStackTrace(); + cdl.countDown(); + } + + @Override + public void onComplete() { + counter.getAndIncrement(); + cdl.countDown(); + } + }); + + assertTrue(cdl.await(60, TimeUnit.SECONDS)); + + assertEquals(max + 1, counter.get()); + } + } finally { + exec.shutdown(); + } + } + + @Test + public void requestRace() throws Exception { + requestRaceBase(false); + } + + @Test + public void requestRaceConditional() throws Exception { + requestRaceBase(true); + } + + @Test + public void closeCalledOnEmpty() { + AtomicInteger calls = new AtomicInteger(); + + Flowable.fromStream(Stream.of().onClose(() -> calls.getAndIncrement())) + .test() + .assertResult(); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledAfterItems() { + AtomicInteger calls = new AtomicInteger(); + + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .test() + .assertResult(1, 2, 3, 4, 5); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnCancel() { + AtomicInteger calls = new AtomicInteger(); + + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .take(3) + .test() + .assertResult(1, 2, 3); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnItemCrash() { + AtomicInteger calls = new AtomicInteger(); + AtomicInteger counter = new AtomicInteger(); + Flowable.fromStream(Stream.generate(() -> { + int value = counter.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + }).onClose(() -> calls.getAndIncrement())) + .test() + .assertFailure(TestException.class, 0); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledAfterItemsConditional() { + AtomicInteger calls = new AtomicInteger(); + + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .filter(v -> true) + .test() + .assertResult(1, 2, 3, 4, 5); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnCancelConditional() { + AtomicInteger calls = new AtomicInteger(); + + Flowable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .filter(v -> true) + .take(3) + .test() + .assertResult(1, 2, 3); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnItemCrashConditional() { + AtomicInteger calls = new AtomicInteger(); + AtomicInteger counter = new AtomicInteger(); + Flowable.fromStream(Stream.generate(() -> { + int value = counter.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + }).onClose(() -> calls.getAndIncrement())) + .filter(v -> true) + .test() + .assertFailure(TestException.class, 0); + + assertEquals(1, calls.get()); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromCompletionStageTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromCompletionStageTckTest.java new file mode 100644 index 0000000000..2c48611cd9 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromCompletionStageTckTest.java @@ -0,0 +1,48 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class FromCompletionStageTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.fromCompletionStage(CompletableFuture.completedFuture(1L)) + ; + } + + @Override + public Publisher createFailedPublisher() { + CompletableFuture cf = new CompletableFuture<>(); + cf.completeExceptionally(new TestException()); + return + Flowable.fromCompletionStage(cf) + ; + } + + @Override + public long maxElementsFromPublisher() { + return 1; + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional0TckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional0TckTest.java new file mode 100644 index 0000000000..76c1246eb3 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional0TckTest.java @@ -0,0 +1,42 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Optional; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +/** + * Test Optional.empty() wrapping. + * @see FromOptional1TckTest + */ +@Test +public class FromOptional0TckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.fromOptional(Optional.empty()) + ; + } + + @Override + public long maxElementsFromPublisher() { + return 0; + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional1TckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional1TckTest.java new file mode 100644 index 0000000000..27e596d708 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromOptional1TckTest.java @@ -0,0 +1,42 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Optional; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +/** + * Test Optional.of wrapping. + * @see FromOptional0TckTest + */ +@Test +public class FromOptional1TckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.fromOptional(Optional.of(1L)) + ; + } + + @Override + public long maxElementsFromPublisher() { + return 1; + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromStreamTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromStreamTckTest.java new file mode 100644 index 0000000000..06f3b02b97 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FromStreamTckTest.java @@ -0,0 +1,44 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +/** + * Test Optional.of wrapping. + * @see FromOptional0TckTest + */ +@Test +public class FromStreamTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.fromStream(IntStream.rangeClosed(1, (int)elements).boxed()) + ; + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Flowable.fromStream(stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java index 4fa76d1628..a7aca59ada 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java @@ -47,11 +47,11 @@ protected void subscribeActual(final Observer observer) { } } - @SuppressWarnings("unchecked") public static Builder item(T item) { return items(item); } + @SafeVarargs public static Builder items(T... items) { return new Builder(Arrays.asList(items)); } diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index 9037d203f6..1f2a834650 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -624,7 +624,7 @@ public static void doubleOnSubscribe(MaybeObserver observer) { * @param source the source to test */ public static void checkDisposed(Flowable source) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); source.subscribe(new FlowableSubscriber() { @Override public void onSubscribe(Subscription s) { @@ -2123,10 +2123,10 @@ protected void subscribeActual(CompletableObserver observer) { public static void checkDisposedMaybe(Function, ? extends MaybeSource> composer) { PublishProcessor pp = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { - new MaybeToFlowable(composer.apply(pp.singleElement())).subscribe(ts); + new MaybeToFlowable<>(composer.apply(pp.singleElement())).subscribe(ts); } catch (Throwable ex) { throw ExceptionHelper.wrapOrThrow(ex); } @@ -2145,7 +2145,7 @@ public static void checkDisposedMaybe(Function, ? extends MaybeS public static void checkDisposedCompletable(Function composer) { PublishProcessor pp = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { new CompletableToFlowable(composer.apply(pp.ignoreElements())).subscribe(ts); @@ -2169,10 +2169,10 @@ public static void checkDisposedCompletable(Function void checkDisposedMaybeToSingle(Function, ? extends SingleSource> composer) { PublishProcessor pp = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { - new SingleToFlowable(composer.apply(pp.singleElement())).subscribe(ts); + new SingleToFlowable<>(composer.apply(pp.singleElement())).subscribe(ts); } catch (Throwable ex) { throw ExceptionHelper.wrapOrThrow(ex); } @@ -2190,6 +2190,7 @@ public static void checkDisposedMaybeToSingle(Function, ? extend * @param ts the TestSubscriber instance * @param classes the array of expected Throwables inside the Composite */ + @SafeVarargs public static void assertCompositeExceptions(TestSubscriberEx ts, Class... classes) { ts .assertSubscribed() @@ -2234,6 +2235,7 @@ public static void assertCompositeExceptions(TestSubscriberEx ts, Object... c * @param to the TestSubscriber instance * @param classes the array of expected Throwables inside the Composite */ + @SafeVarargs public static void assertCompositeExceptions(TestObserverEx to, Class... classes) { to .assertSubscribed() @@ -2278,6 +2280,7 @@ public static void assertCompositeExceptions(TestObserverEx to, Object... cla * @param p the target processor * @param values the values to emit */ + @SafeVarargs public static void emit(Processor p, T... values) { for (T v : values) { p.onNext(v); @@ -2291,6 +2294,7 @@ public static void emit(Processor p, T... values) { * @param p the target subject * @param values the values to emit */ + @SafeVarargs public static void emit(Subject p, T... values) { for (T v : values) { p.onNext(v); @@ -2495,7 +2499,7 @@ protected void subscribeActual(Observer observer) { if (o instanceof ObservableSource) { ObservableSource os = (ObservableSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2517,7 +2521,7 @@ protected void subscribeActual(Observer observer) { if (o instanceof Publisher) { Publisher os = (Publisher) o; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); os.subscribe(ts); @@ -2539,7 +2543,7 @@ protected void subscribeActual(Observer observer) { if (o instanceof SingleSource) { SingleSource os = (SingleSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2561,7 +2565,7 @@ protected void subscribeActual(Observer observer) { if (o instanceof MaybeSource) { MaybeSource os = (MaybeSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2583,7 +2587,7 @@ protected void subscribeActual(Observer observer) { if (o instanceof CompletableSource) { CompletableSource os = (CompletableSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2654,7 +2658,7 @@ protected void subscribeActual(Subscriber subscriber) { if (o instanceof ObservableSource) { ObservableSource os = (ObservableSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2676,7 +2680,7 @@ protected void subscribeActual(Subscriber subscriber) { if (o instanceof Publisher) { Publisher os = (Publisher) o; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); os.subscribe(ts); @@ -2698,7 +2702,7 @@ protected void subscribeActual(Subscriber subscriber) { if (o instanceof SingleSource) { SingleSource os = (SingleSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2720,7 +2724,7 @@ protected void subscribeActual(Subscriber subscriber) { if (o instanceof MaybeSource) { MaybeSource os = (MaybeSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2742,7 +2746,7 @@ protected void subscribeActual(Subscriber subscriber) { if (o instanceof CompletableSource) { CompletableSource os = (CompletableSource) o; - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); os.subscribe(to); @@ -2778,7 +2782,7 @@ public static void checkInvalidParallelSubscribers(ParallelFlowable sourc @SuppressWarnings("unchecked") TestSubscriber[] tss = new TestSubscriber[n + 1]; for (int i = 0; i <= n; i++) { - tss[i] = new TestSubscriber().withTag("" + i); + tss[i] = new TestSubscriber<>().withTag("" + i); } source.subscribe(tss); @@ -2893,7 +2897,7 @@ static final class FlowableStripBoundary extends Flowable implements Flowa @Override public Flowable apply(Flowable upstream) { - return new FlowableStripBoundary(upstream); + return new FlowableStripBoundary<>(upstream); } @Override @@ -2985,7 +2989,7 @@ public void cancel() { } public static FlowableTransformer flowableStripBoundary() { - return new FlowableStripBoundary(null); + return new FlowableStripBoundary<>(null); } static final class ObservableStripBoundary extends Observable implements ObservableTransformer { @@ -2998,7 +3002,7 @@ static final class ObservableStripBoundary extends Observable implements O @Override public Observable apply(Observable upstream) { - return new ObservableStripBoundary(upstream); + return new ObservableStripBoundary<>(upstream); } @Override @@ -3090,23 +3094,23 @@ public boolean isDisposed() { } public static ObservableTransformer observableStripBoundary() { - return new ObservableStripBoundary(null); + return new ObservableStripBoundary<>(null); } public static TestConsumerExConverters testConsumer() { - return new TestConsumerExConverters(false, 0); + return new TestConsumerExConverters<>(false, 0); } public static TestConsumerExConverters testConsumer(boolean cancelled) { - return new TestConsumerExConverters(cancelled, 0); + return new TestConsumerExConverters<>(cancelled, 0); } public static TestConsumerExConverters testConsumer(final int fusionMode, final boolean cancelled) { - return new TestConsumerExConverters(cancelled, fusionMode); + return new TestConsumerExConverters<>(cancelled, fusionMode); } public static TestConsumerExConverters testConsumer(final boolean cancelled, final int fusionMode) { - return new TestConsumerExConverters(cancelled, fusionMode); + return new TestConsumerExConverters<>(cancelled, fusionMode); } public static FlowableConverter> testSubscriber(final long initialRequest) { @@ -3125,7 +3129,7 @@ public static FlowableConverter> testSubscriber(final return new FlowableConverter>() { @Override public TestSubscriberEx apply(Flowable f) { - TestSubscriberEx tse = new TestSubscriberEx(initialRequest); + TestSubscriberEx tse = new TestSubscriberEx<>(initialRequest); if (cancelled) { tse.cancel(); } @@ -3153,7 +3157,7 @@ public static final class TestConsumerExConverters implements @Override public TestObserverEx apply(Completable upstream) { - TestObserverEx toe = new TestObserverEx(); + TestObserverEx toe = new TestObserverEx<>(); if (cancelled) { toe.dispose(); } @@ -3163,7 +3167,7 @@ public TestObserverEx apply(Completable upstream) { @Override public TestObserverEx apply(Maybe upstream) { - TestObserverEx toe = new TestObserverEx(); + TestObserverEx toe = new TestObserverEx<>(); if (cancelled) { toe.dispose(); } @@ -3173,7 +3177,7 @@ public TestObserverEx apply(Maybe upstream) { @Override public TestObserverEx apply(Single upstream) { - TestObserverEx toe = new TestObserverEx(); + TestObserverEx toe = new TestObserverEx<>(); if (cancelled) { toe.dispose(); } @@ -3183,7 +3187,7 @@ public TestObserverEx apply(Single upstream) { @Override public TestObserverEx apply(Observable upstream) { - TestObserverEx toe = new TestObserverEx(); + TestObserverEx toe = new TestObserverEx<>(); if (cancelled) { toe.dispose(); } @@ -3193,7 +3197,7 @@ public TestObserverEx apply(Observable upstream) { @Override public TestSubscriberEx apply(Flowable upstream) { - TestSubscriberEx tse = new TestSubscriberEx(); + TestSubscriberEx tse = new TestSubscriberEx<>(); if (cancelled) { tse.dispose(); } @@ -3202,8 +3206,9 @@ public TestSubscriberEx apply(Flowable upstream) { } } + @SafeVarargs public static TestSubscriberEx assertValueSet(TestSubscriberEx ts, T... values) { - Set expectedSet = new HashSet(Arrays.asList(values)); + Set expectedSet = new HashSet<>(Arrays.asList(values)); for (T t : ts.values()) { if (!expectedSet.contains(t)) { throw ts.failWith("Item not in the set: " + BaseTestConsumer.valueAndClass(t)); @@ -3212,8 +3217,9 @@ public static TestSubscriberEx assertValueSet(TestSubscriberEx ts, T.. return ts; } + @SafeVarargs public static TestObserverEx assertValueSet(TestObserverEx to, T... values) { - Set expectedSet = new HashSet(Arrays.asList(values)); + Set expectedSet = new HashSet<>(Arrays.asList(values)); for (T t : to.values()) { if (!expectedSet.contains(t)) { throw to.failWith("Item not in the set: " + BaseTestConsumer.valueAndClass(t)); @@ -3280,35 +3286,35 @@ public Integer apply(Integer v) throws Throwable { .to(transform); if (result instanceof MaybeSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((MaybeSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof SingleSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((SingleSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof CompletableSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((CompletableSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof ObservableSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((ObservableSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof Publisher) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); disposable.set(Disposables.fromSubscription(ts)); ((Publisher)result) @@ -3348,35 +3354,35 @@ public Integer apply(Integer v) throws Throwable { .to(transform); if (result instanceof MaybeSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((MaybeSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof SingleSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((SingleSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof CompletableSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((CompletableSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof ObservableSource) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); disposable.set(to); ((ObservableSource)result) .subscribe(to); to.assertEmpty(); } else if (result instanceof Publisher) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); disposable.set(Disposables.fromSubscription(ts)); ((Publisher)result) @@ -3419,4 +3425,18 @@ public static long awaitGC(long oneSleep, int maxLoop, long expectedMemoryUsage) } return bean.getHeapMemoryUsage().getUsed(); } + + /** + * Enable thracking of the global errors for the duration of the action. + * @param action the action to run with a list of errors encountered + * @throws Throwable the exception rethrown from the action + */ + public static void withErrorTracking(Consumer> action) throws Throwable { + List errors = trackPluginErrors(); + try { + action.accept(errors); + } finally { + RxJavaPlugins.reset(); + } + } } diff --git a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java index d8b5d14f7e..2d4387d543 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java +++ b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java @@ -14,8 +14,10 @@ package io.reactivex.rxjava3.validators; import java.lang.reflect.*; +import java.time.Duration; import java.util.*; import java.util.concurrent.*; +import java.util.stream.*; import org.junit.Test; import org.reactivestreams.*; @@ -584,6 +586,14 @@ public void checkParallelFlowable() { defaultValues.put(ParallelFailureHandling.class, ParallelFailureHandling.ERROR); + // JDK 8 types + + defaultValues.put(Optional.class, Optional.of(1)); + defaultValues.put(CompletionStage.class, CompletableFuture.completedFuture(1)); + defaultValues.put(Stream.class, Stream.of(1, 2, 3)); + defaultValues.put(Duration.class, Duration.ofSeconds(1)); + defaultValues.put(Collector.class, Collectors.toList()); + @SuppressWarnings("rawtypes") class MixedConverters implements FlowableConverter, ObservableConverter, SingleConverter, MaybeConverter, CompletableConverter, ParallelFlowableConverter { From 2392349ccae27a0916d594c115f356c9040c3b98 Mon Sep 17 00:00:00 2001 From: Jake Wharton Date: Tue, 17 Dec 2019 11:48:50 -0500 Subject: [PATCH 005/665] Remove deprecated method from 2.x (#6769) --- .../rxjava3/internal/functions/ObjectHelper.java | 13 ------------- .../internal/functions/ObjectHelperTest.java | 6 ------ 2 files changed, 19 deletions(-) diff --git a/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java b/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java index c13b1850f2..981d697ec8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java @@ -128,17 +128,4 @@ public boolean test(Object o1, Object o2) { return ObjectHelper.equals(o1, o2); } } - - /** - * Trap null-check attempts on primitives. - * @param value the value to check - * @param message the message to print - * @return the value - * @deprecated this method should not be used as there is no need - * to check primitives for nullness. - */ - @Deprecated - public static long requireNonNull(long value, String message) { - throw new InternalError("Null check on a primitive: " + message); - } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java index c4ae28f883..75c78f4746 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java @@ -67,10 +67,4 @@ public void compareLong() { assertEquals(0, ObjectHelper.compare(0L, 0L)); assertEquals(1, ObjectHelper.compare(2L, 0L)); } - - @SuppressWarnings("deprecation") - @Test(expected = InternalError.class) - public void requireNonNullPrimitive() { - ObjectHelper.requireNonNull(0, "value"); - } } From 652f2a1e54bb011553c627cf3e93335f0032f034 Mon Sep 17 00:00:00 2001 From: Jake Wharton Date: Tue, 17 Dec 2019 12:20:06 -0500 Subject: [PATCH 006/665] Replace some ObjecsHelpers methods with Java 8 APIs (#6770) --- .../reactivex/rxjava3/core/Notification.java | 3 +- .../rxjava3/internal/functions/Functions.java | 2 +- .../internal/functions/ObjectHelper.java | 42 +------------------ .../operators/maybe/MaybeContains.java | 4 +- .../operators/single/SingleEquals.java | 4 +- .../schedulers/TrampolineScheduler.java | 5 +-- .../internal/util/NotificationLite.java | 4 +- .../rxjava3/observers/BaseTestConsumer.java | 8 ++-- .../rxjava3/schedulers/TestScheduler.java | 5 +-- .../reactivex/rxjava3/schedulers/Timed.java | 5 ++- .../internal/functions/ObjectHelperTest.java | 21 ---------- .../testsupport/BaseTestConsumerEx.java | 6 +-- .../rxjava3/testsupport/TestHelper.java | 5 +-- 13 files changed, 27 insertions(+), 87 deletions(-) diff --git a/src/main/java/io/reactivex/rxjava3/core/Notification.java b/src/main/java/io/reactivex/rxjava3/core/Notification.java index 23139896cb..b3270483a7 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Notification.java +++ b/src/main/java/io/reactivex/rxjava3/core/Notification.java @@ -16,6 +16,7 @@ import io.reactivex.rxjava3.annotations.*; import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.NotificationLite; +import java.util.Objects; /** * Represents the reactive signal types: onNext, onError and onComplete and @@ -95,7 +96,7 @@ public Throwable getError() { public boolean equals(Object obj) { if (obj instanceof Notification) { Notification n = (Notification) obj; - return ObjectHelper.equals(value, n.value); + return Objects.equals(value, n.value); } return false; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java index 87aae2e1e8..2e44b86d04 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java +++ b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java @@ -275,7 +275,7 @@ static final class EqualsPredicate implements Predicate { @Override public boolean test(T t) throws Exception { - return ObjectHelper.equals(t, value); + return Objects.equals(t, value); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java b/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java index 981d697ec8..65f5a70c82 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.functions; import io.reactivex.rxjava3.functions.BiPredicate; +import java.util.Objects; /** * Utility methods containing the backport of Java 7's Objects utility class. @@ -41,45 +42,6 @@ public static T requireNonNull(T object, String message) { return object; } - /** - * Compares two potentially null objects with each other using Object.equals. - * @param o1 the first object - * @param o2 the second object - * @return the comparison result - */ - public static boolean equals(Object o1, Object o2) { // NOPMD - return o1 == o2 || (o1 != null && o1.equals(o2)); - } - - /** - * Returns the hashCode of a non-null object or zero for a null object. - * @param o the object to get the hashCode for. - * @return the hashCode - */ - public static int hashCode(Object o) { - return o != null ? o.hashCode() : 0; - } - - /** - * Compares two integer values similar to Integer.compare. - * @param v1 the first value - * @param v2 the second value - * @return the comparison result - */ - public static int compare(int v1, int v2) { - return v1 < v2 ? -1 : (v1 > v2 ? 1 : 0); - } - - /** - * Compares two long values similar to Long.compare. - * @param v1 the first value - * @param v2 the second value - * @return the comparison result - */ - public static int compare(long v1, long v2) { - return v1 < v2 ? -1 : (v1 > v2 ? 1 : 0); - } - static final BiPredicate EQUALS = new BiObjectPredicate(); /** @@ -125,7 +87,7 @@ public static long verifyPositive(long value, String paramName) { static final class BiObjectPredicate implements BiPredicate { @Override public boolean test(Object o1, Object o2) { - return ObjectHelper.equals(o1, o2); + return Objects.equals(o1, o2); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeContains.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeContains.java index 55b68fec3d..8737c6ae37 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeContains.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeContains.java @@ -16,8 +16,8 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.HasUpstreamMaybeSource; +import java.util.Objects; /** * Signals true if the source signals a value that is object-equals with the provided @@ -81,7 +81,7 @@ public void onSubscribe(Disposable d) { @Override public void onSuccess(Object value) { upstream = DisposableHelper.DISPOSED; - downstream.onSuccess(ObjectHelper.equals(value, this.value)); + downstream.onSuccess(Objects.equals(value, this.value)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleEquals.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleEquals.java index 8a9f62473f..0a839ab747 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleEquals.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleEquals.java @@ -13,11 +13,11 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class SingleEquals extends Single { @@ -68,7 +68,7 @@ public void onSuccess(T value) { values[index] = value; if (count.incrementAndGet() == 2) { - downstream.onSuccess(ObjectHelper.equals(values[0], values[1])); + downstream.onSuccess(Objects.equals(values[0], values[1])); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java index cc61d797e9..71c054258b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java @@ -23,7 +23,6 @@ import io.reactivex.rxjava3.core.Scheduler; import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** @@ -165,9 +164,9 @@ static final class TimedRunnable implements Comparable { @Override public int compareTo(TimedRunnable that) { - int result = ObjectHelper.compare(execTime, that.execTime); + int result = Long.compare(execTime, that.execTime); if (result == 0) { - return ObjectHelper.compare(count, that.count); + return Integer.compare(count, that.count); } return result; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/NotificationLite.java b/src/main/java/io/reactivex/rxjava3/internal/util/NotificationLite.java index dbc54c788a..c37aa71da2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/NotificationLite.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/NotificationLite.java @@ -14,11 +14,11 @@ import java.io.Serializable; +import java.util.Objects; import org.reactivestreams.*; import io.reactivex.rxjava3.core.Observer; import io.reactivex.rxjava3.disposables.Disposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Lightweight notification handling utility class. @@ -52,7 +52,7 @@ public int hashCode() { public boolean equals(Object obj) { if (obj instanceof ErrorNotification) { ErrorNotification n = (ErrorNotification) obj; - return ObjectHelper.equals(e, n.e); + return Objects.equals(e, n.e); } return false; } diff --git a/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java b/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java index 59aa62e796..f8a0356a75 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java +++ b/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java @@ -284,7 +284,7 @@ public final U assertValue(T value) { throw fail("expected: " + valueAndClass(value) + " but was: " + values); } T v = values.get(0); - if (!ObjectHelper.equals(value, v)) { + if (!Objects.equals(value, v)) { throw fail("expected: " + valueAndClass(value) + " but was: " + valueAndClass(v)); } return (U)this; @@ -330,7 +330,7 @@ public final U assertValueAt(int index, T value) { } T v = values.get(index); - if (!ObjectHelper.equals(value, v)) { + if (!Objects.equals(value, v)) { throw fail("expected: " + valueAndClass(value) + " but was: " + valueAndClass(v)); } return (U)this; @@ -421,7 +421,7 @@ public final U assertValues(T... values) { for (int i = 0; i < s; i++) { T v = this.values.get(i); T u = values[i]; - if (!ObjectHelper.equals(u, v)) { + if (!Objects.equals(u, v)) { throw fail("Values at position " + i + " differ; expected: " + valueAndClass(u) + " but was: " + valueAndClass(v)); } } @@ -466,7 +466,7 @@ public final U assertValueSequence(Iterable sequence) { T u = expectedIterator.next(); T v = actualIterator.next(); - if (!ObjectHelper.equals(u, v)) { + if (!Objects.equals(u, v)) { throw fail("Values at position " + i + " differ; expected: " + valueAndClass(u) + " but was: " + valueAndClass(v)); } i++; diff --git a/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java b/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java index 33a5b943e1..8b70760236 100644 --- a/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java @@ -20,7 +20,6 @@ import io.reactivex.rxjava3.core.Scheduler; import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * A special, non thread-safe scheduler for testing operators that require @@ -76,9 +75,9 @@ public String toString() { @Override public int compareTo(TimedRunnable o) { if (time == o.time) { - return ObjectHelper.compare(count, o.count); + return Long.compare(count, o.count); } - return ObjectHelper.compare(time, o.time); + return Long.compare(time, o.time); } } diff --git a/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java b/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java index b7ac9c828f..fff21ad682 100644 --- a/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java +++ b/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.schedulers; +import java.util.Objects; import java.util.concurrent.TimeUnit; import io.reactivex.rxjava3.annotations.NonNull; @@ -80,9 +81,9 @@ public long time(@NonNull TimeUnit unit) { public boolean equals(Object other) { if (other instanceof Timed) { Timed o = (Timed) other; - return ObjectHelper.equals(value, o.value) + return Objects.equals(value, o.value) && time == o.time - && ObjectHelper.equals(unit, o.unit); + && Objects.equals(unit, o.unit); } return false; } diff --git a/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java index 75c78f4746..16c182061d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/functions/ObjectHelperTest.java @@ -27,13 +27,6 @@ public void utilityClass() { TestHelper.checkUtilityClass(ObjectHelper.class); } - @Test - public void hashCodeOf() { - assertEquals(0, ObjectHelper.hashCode(null)); - - assertEquals(((Integer)1).hashCode(), ObjectHelper.hashCode(1)); - } - @Test public void verifyPositiveInt() throws Exception { assertEquals(1, ObjectHelper.verifyPositive(1, "param")); @@ -53,18 +46,4 @@ public void verifyPositiveIntFail() throws Exception { public void verifyPositiveLongFail() throws Exception { assertEquals(-1L, ObjectHelper.verifyPositive(-1L, "param")); } - - @Test - public void compare() { - assertEquals(-1, ObjectHelper.compare(0, 2)); - assertEquals(0, ObjectHelper.compare(0, 0)); - assertEquals(1, ObjectHelper.compare(2, 0)); - } - - @Test - public void compareLong() { - assertEquals(-1, ObjectHelper.compare(0L, 2L)); - assertEquals(0, ObjectHelper.compare(0L, 0L)); - assertEquals(1, ObjectHelper.compare(2L, 0L)); - } } diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java b/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java index d997a065ba..2c68b0151a 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java @@ -16,10 +16,10 @@ import java.util.List; import io.reactivex.rxjava3.functions.Predicate; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.QueueFuseable; import io.reactivex.rxjava3.internal.util.ExceptionHelper; import io.reactivex.rxjava3.observers.BaseTestConsumer; +import java.util.Objects; /** * Base class with shared infrastructure to support TestSubscriber and TestObserver. @@ -74,7 +74,7 @@ public final U assertNever(T value) { for (int i = 0; i < s; i++) { T v = this.values.get(i); - if (ObjectHelper.equals(v, value)) { + if (Objects.equals(v, value)) { throw fail("Value at position " + i + " is equal to " + valueAndClass(value) + "; Expected them to be different"); } } @@ -158,7 +158,7 @@ public final U assertErrorMessage(String message) { if (s == 1) { Throwable e = errors.get(0); String errorMessage = e.getMessage(); - if (!ObjectHelper.equals(message, errorMessage)) { + if (!Objects.equals(message, errorMessage)) { throw fail("Error message differs; exptected: " + message + " but was: " + errorMessage); } } else { diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index 1f2a834650..1ce53f01ae 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -36,7 +36,6 @@ import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.operators.completable.CompletableToFlowable; import io.reactivex.rxjava3.internal.operators.maybe.MaybeToFlowable; @@ -196,7 +195,7 @@ public static void assertError(List list, int index, Class list, int index, Class Date: Tue, 17 Dec 2019 14:06:44 -0500 Subject: [PATCH 007/665] Replace ObjectHelper.requireNonNull with Objects.requireNonNull (#6771) --- .../reactivex/rxjava3/core/Completable.java | 145 +-- .../io/reactivex/rxjava3/core/Flowable.java | 954 +++++++++--------- .../java/io/reactivex/rxjava3/core/Maybe.java | 355 +++---- .../reactivex/rxjava3/core/Notification.java | 5 +- .../io/reactivex/rxjava3/core/Observable.java | 950 ++++++++--------- .../io/reactivex/rxjava3/core/Single.java | 301 +++--- .../disposables/CompositeDisposable.java | 17 +- .../rxjava3/disposables/Disposables.java | 11 +- .../disposables/ReferenceDisposable.java | 4 +- .../flowables/ConnectableFlowable.java | 5 +- .../disposables/DisposableHelper.java | 4 +- .../disposables/ListCompositeDisposable.java | 17 +- .../rxjava3/internal/functions/Functions.java | 16 +- .../internal/functions/ObjectHelper.java | 16 - .../internal/jdk8/FlowableFromStream.java | 8 +- .../CompletableConcatIterable.java | 6 +- .../completable/CompletableDefer.java | 5 +- .../completable/CompletableErrorSupplier.java | 5 +- .../CompletableMergeDelayErrorIterable.java | 6 +- .../completable/CompletableMergeIterable.java | 6 +- .../completable/CompletableResumeNext.java | 4 +- .../completable/CompletableUsing.java | 4 +- .../AbstractFlowableWithUpstream.java | 5 +- .../flowable/FlowableBlockingSubscribe.java | 13 +- .../operators/flowable/FlowableBuffer.java | 7 +- .../flowable/FlowableBufferBoundary.java | 5 +- .../flowable/FlowableBufferExactBoundary.java | 6 +- .../flowable/FlowableBufferTimed.java | 15 +- .../operators/flowable/FlowableCollect.java | 5 +- .../flowable/FlowableCollectSingle.java | 5 +- .../flowable/FlowableCombineLatest.java | 10 +- .../operators/flowable/FlowableConcatMap.java | 6 +- .../flowable/FlowableConcatMapEager.java | 4 +- .../flowable/FlowableConcatMapScheduler.java | 6 +- .../operators/flowable/FlowableDebounce.java | 4 +- .../operators/flowable/FlowableDefer.java | 5 +- .../flowable/FlowableDematerialize.java | 5 +- .../operators/flowable/FlowableDistinct.java | 6 +- .../operators/flowable/FlowableFlatMap.java | 4 +- .../flowable/FlowableFlatMapCompletable.java | 4 +- ...FlowableFlatMapCompletableCompletable.java | 4 +- .../flowable/FlowableFlatMapMaybe.java | 4 +- .../flowable/FlowableFlatMapSingle.java | 4 +- .../flowable/FlowableFlattenIterable.java | 6 +- .../operators/flowable/FlowableFromArray.java | 5 +- .../flowable/FlowableFromCallable.java | 6 +- .../flowable/FlowableFromIterable.java | 4 +- .../flowable/FlowableFromSupplier.java | 7 +- .../operators/flowable/FlowableGroupJoin.java | 7 +- .../flowable/FlowableInternalHelper.java | 7 +- .../operators/flowable/FlowableJoin.java | 9 +- .../operators/flowable/FlowableMap.java | 13 +- .../flowable/FlowableMapNotification.java | 9 +- .../flowable/FlowableOnErrorNext.java | 5 +- .../flowable/FlowableOnErrorReturn.java | 5 +- .../flowable/FlowablePublishMulticast.java | 4 +- .../operators/flowable/FlowableReduce.java | 5 +- .../flowable/FlowableReduceMaybe.java | 5 +- .../flowable/FlowableReduceSeedSingle.java | 5 +- .../flowable/FlowableReduceWithSingle.java | 5 +- .../flowable/FlowableRepeatWhen.java | 4 +- .../operators/flowable/FlowableRetryWhen.java | 5 +- .../flowable/FlowableScalarXMap.java | 7 +- .../operators/flowable/FlowableScan.java | 5 +- .../operators/flowable/FlowableScanSeed.java | 6 +- .../operators/flowable/FlowableSwitchMap.java | 4 +- .../operators/flowable/FlowableTimeout.java | 6 +- .../operators/flowable/FlowableUsing.java | 4 +- .../FlowableWindowBoundarySelector.java | 3 +- .../flowable/FlowableWithLatestFrom.java | 4 +- .../flowable/FlowableWithLatestFromMany.java | 6 +- .../operators/flowable/FlowableZip.java | 4 +- .../flowable/FlowableZipIterable.java | 8 +- .../operators/maybe/MaybeConcatIterable.java | 6 +- .../internal/operators/maybe/MaybeDefer.java | 5 +- .../maybe/MaybeFlatMapBiSelector.java | 8 +- .../maybe/MaybeFlatMapCompletable.java | 4 +- .../maybe/MaybeFlatMapIterableFlowable.java | 6 +- .../maybe/MaybeFlatMapIterableObservable.java | 4 +- .../maybe/MaybeFlatMapNotification.java | 8 +- .../operators/maybe/MaybeFlatMapSingle.java | 4 +- .../maybe/MaybeFlatMapSingleElement.java | 4 +- .../operators/maybe/MaybeFlatten.java | 4 +- .../internal/operators/maybe/MaybeLift.java | 5 +- .../internal/operators/maybe/MaybeMap.java | 5 +- .../operators/maybe/MaybeMergeArray.java | 4 +- .../operators/maybe/MaybeOnErrorNext.java | 6 +- .../operators/maybe/MaybeOnErrorReturn.java | 5 +- .../internal/operators/maybe/MaybeUsing.java | 4 +- .../operators/maybe/MaybeZipArray.java | 6 +- .../operators/maybe/MaybeZipIterable.java | 4 +- .../mixed/FlowableConcatMapCompletable.java | 4 +- .../mixed/FlowableConcatMapMaybe.java | 4 +- .../mixed/FlowableConcatMapSingle.java | 4 +- .../mixed/FlowableSwitchMapCompletable.java | 4 +- .../mixed/FlowableSwitchMapMaybe.java | 4 +- .../mixed/FlowableSwitchMapSingle.java | 4 +- .../mixed/MaybeFlatMapObservable.java | 6 +- .../mixed/MaybeFlatMapPublisher.java | 6 +- .../mixed/ObservableConcatMapCompletable.java | 4 +- .../mixed/ObservableConcatMapMaybe.java | 4 +- .../mixed/ObservableConcatMapSingle.java | 4 +- .../mixed/ObservableSwitchMapCompletable.java | 4 +- .../mixed/ObservableSwitchMapMaybe.java | 4 +- .../mixed/ObservableSwitchMapSingle.java | 4 +- .../operators/mixed/ScalarXMapZHelper.java | 9 +- .../mixed/SingleFlatMapObservable.java | 6 +- .../ObservableBlockingSubscribe.java | 7 +- .../observable/ObservableBuffer.java | 3 +- .../observable/ObservableBufferBoundary.java | 5 +- .../ObservableBufferExactBoundary.java | 6 +- .../observable/ObservableBufferTimed.java | 15 +- .../observable/ObservableCollect.java | 5 +- .../observable/ObservableCollectSingle.java | 5 +- .../observable/ObservableCombineLatest.java | 4 +- .../observable/ObservableConcatMap.java | 6 +- .../observable/ObservableConcatMapEager.java | 4 +- .../ObservableConcatMapScheduler.java | 6 +- .../observable/ObservableDebounce.java | 4 +- .../operators/observable/ObservableDefer.java | 5 +- .../observable/ObservableDematerialize.java | 5 +- .../observable/ObservableDistinct.java | 6 +- .../observable/ObservableFlatMap.java | 3 +- .../ObservableFlatMapCompletable.java | 4 +- ...servableFlatMapCompletableCompletable.java | 4 +- .../observable/ObservableFlatMapMaybe.java | 4 +- .../observable/ObservableFlatMapSingle.java | 4 +- .../observable/ObservableFlattenIterable.java | 4 +- .../observable/ObservableFromArray.java | 5 +- .../observable/ObservableFromIterable.java | 6 +- .../observable/ObservableGroupBy.java | 3 +- .../observable/ObservableGroupJoin.java | 7 +- .../observable/ObservableInternalHelper.java | 7 +- .../operators/observable/ObservableJoin.java | 9 +- .../operators/observable/ObservableLift.java | 5 +- .../operators/observable/ObservableMap.java | 7 +- .../observable/ObservableMapNotification.java | 9 +- .../observable/ObservablePublishSelector.java | 4 +- .../observable/ObservableReduceMaybe.java | 5 +- .../ObservableReduceSeedSingle.java | 5 +- .../ObservableReduceWithSingle.java | 5 +- .../observable/ObservableRepeatWhen.java | 4 +- .../observable/ObservableReplay.java | 5 +- .../observable/ObservableRetryWhen.java | 4 +- .../observable/ObservableScalarXMap.java | 6 +- .../operators/observable/ObservableScan.java | 5 +- .../observable/ObservableScanSeed.java | 7 +- .../observable/ObservableSwitchMap.java | 4 +- .../observable/ObservableTimeout.java | 6 +- .../operators/observable/ObservableUsing.java | 4 +- .../ObservableWindowBoundarySelector.java | 3 +- .../observable/ObservableWithLatestFrom.java | 4 +- .../ObservableWithLatestFromMany.java | 6 +- .../operators/observable/ObservableZip.java | 4 +- .../observable/ObservableZipIterable.java | 8 +- .../operators/parallel/ParallelCollect.java | 5 +- .../operators/parallel/ParallelConcatMap.java | 7 +- .../parallel/ParallelDoOnNextTry.java | 7 +- .../operators/parallel/ParallelFilterTry.java | 7 +- .../operators/parallel/ParallelMap.java | 9 +- .../operators/parallel/ParallelMapTry.java | 11 +- .../operators/parallel/ParallelPeek.java | 19 +- .../operators/parallel/ParallelReduce.java | 7 +- .../parallel/ParallelReduceFull.java | 6 +- .../operators/single/SingleDefer.java | 5 +- .../operators/single/SingleDematerialize.java | 5 +- .../operators/single/SingleFlatMap.java | 4 +- .../single/SingleFlatMapCompletable.java | 4 +- .../single/SingleFlatMapIterableFlowable.java | 6 +- .../SingleFlatMapIterableObservable.java | 4 +- .../operators/single/SingleFlatMapMaybe.java | 4 +- .../single/SingleFlatMapPublisher.java | 4 +- .../operators/single/SingleFromCallable.java | 4 +- .../operators/single/SingleFromSupplier.java | 5 +- .../internal/operators/single/SingleLift.java | 5 +- .../internal/operators/single/SingleMap.java | 5 +- .../operators/single/SingleResumeNext.java | 4 +- .../operators/single/SingleUsing.java | 4 +- .../operators/single/SingleZipArray.java | 6 +- .../operators/single/SingleZipIterable.java | 3 +- .../subscriptions/SubscriptionArbiter.java | 4 +- .../subscriptions/SubscriptionHelper.java | 4 +- .../internal/util/EndConsumerHelper.java | 10 +- .../internal/util/ExceptionHelper.java | 2 +- .../observables/ConnectableObservable.java | 5 +- .../ResourceCompletableObserver.java | 4 +- .../observers/ResourceMaybeObserver.java | 4 +- .../rxjava3/observers/ResourceObserver.java | 4 +- .../observers/ResourceSingleObserver.java | 4 +- .../rxjava3/parallel/ParallelFlowable.java | 72 +- .../rxjava3/plugins/RxJavaPlugins.java | 24 +- .../rxjava3/processors/BehaviorProcessor.java | 6 +- .../rxjava3/processors/ReplayProcessor.java | 4 +- .../rxjava3/processors/UnicastProcessor.java | 5 +- .../reactivex/rxjava3/schedulers/Timed.java | 3 +- .../rxjava3/subjects/BehaviorSubject.java | 4 +- .../rxjava3/subjects/ReplaySubject.java | 4 +- .../rxjava3/subjects/UnicastSubject.java | 3 +- .../subscribers/ResourceSubscriber.java | 4 +- 199 files changed, 1957 insertions(+), 1930 deletions(-) diff --git a/src/main/java/io/reactivex/rxjava3/core/Completable.java b/src/main/java/io/reactivex/rxjava3/core/Completable.java index 06183c66db..f25320d66b 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Completable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Completable.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.core; +import java.util.Objects; import java.util.concurrent.*; import org.reactivestreams.Publisher; @@ -120,7 +121,7 @@ public abstract class Completable implements CompletableSource { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable ambArray(final CompletableSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return complete(); } @@ -149,7 +150,7 @@ public static Completable ambArray(final CompletableSource... sources) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable amb(final Iterable sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableAmb(null, sources)); } @@ -187,7 +188,7 @@ public static Completable complete() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable concatArray(CompletableSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return complete(); } else @@ -213,7 +214,7 @@ public static Completable concatArray(CompletableSource... sources) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable concat(Iterable sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableConcatIterable(sources)); } @@ -261,7 +262,7 @@ public static Completable concat(Publisher sources) @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) public static Completable concat(Publisher sources, int prefetch) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new CompletableConcat(sources, prefetch)); } @@ -311,7 +312,7 @@ public static Completable concat(Publisher sources, @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable create(CompletableOnSubscribe source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new CompletableCreate(source)); } @@ -334,7 +335,7 @@ public static Completable create(CompletableOnSubscribe source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable unsafeCreate(CompletableSource source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); if (source instanceof Completable) { throw new IllegalArgumentException("Use of unsafeCreate(Completable)!"); } @@ -356,7 +357,7 @@ public static Completable unsafeCreate(CompletableSource source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable defer(final Supplier completableSupplier) { - ObjectHelper.requireNonNull(completableSupplier, "completableSupplier"); + Objects.requireNonNull(completableSupplier, "completableSupplier"); return RxJavaPlugins.onAssembly(new CompletableDefer(completableSupplier)); } @@ -380,7 +381,7 @@ public static Completable defer(final Supplier comp @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable error(final Supplier errorSupplier) { - ObjectHelper.requireNonNull(errorSupplier, "errorSupplier is null"); + Objects.requireNonNull(errorSupplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new CompletableErrorSupplier(errorSupplier)); } @@ -400,7 +401,7 @@ public static Completable error(final Supplier errorSupplie @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable error(final Throwable error) { - ObjectHelper.requireNonNull(error, "error is null"); + Objects.requireNonNull(error, "error is null"); return RxJavaPlugins.onAssembly(new CompletableError(error)); } @@ -428,7 +429,7 @@ public static Completable error(final Throwable error) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromAction(final Action run) { - ObjectHelper.requireNonNull(run, "run is null"); + Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new CompletableFromAction(run)); } @@ -457,7 +458,7 @@ public static Completable fromAction(final Action run) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromCallable(final Callable callable) { - ObjectHelper.requireNonNull(callable, "callable is null"); + Objects.requireNonNull(callable, "callable is null"); return RxJavaPlugins.onAssembly(new CompletableFromCallable(callable)); } @@ -478,7 +479,7 @@ public static Completable fromCallable(final Callable callable) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromFuture(final Future future) { - ObjectHelper.requireNonNull(future, "future is null"); + Objects.requireNonNull(future, "future is null"); return fromAction(Functions.futureAction(future)); } @@ -503,7 +504,7 @@ public static Completable fromFuture(final Future future) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromMaybe(final MaybeSource maybe) { - ObjectHelper.requireNonNull(maybe, "maybe is null"); + Objects.requireNonNull(maybe, "maybe is null"); return RxJavaPlugins.onAssembly(new MaybeIgnoreElementCompletable(maybe)); } @@ -531,7 +532,7 @@ public static Completable fromMaybe(final MaybeSource maybe) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromRunnable(final Runnable run) { - ObjectHelper.requireNonNull(run, "run is null"); + Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new CompletableFromRunnable(run)); } @@ -553,7 +554,7 @@ public static Completable fromRunnable(final Runnable run) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromObservable(final ObservableSource observable) { - ObjectHelper.requireNonNull(observable, "observable is null"); + Objects.requireNonNull(observable, "observable is null"); return RxJavaPlugins.onAssembly(new CompletableFromObservable(observable)); } @@ -591,7 +592,7 @@ public static Completable fromObservable(final ObservableSource observabl @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromPublisher(final Publisher publisher) { - ObjectHelper.requireNonNull(publisher, "publisher is null"); + Objects.requireNonNull(publisher, "publisher is null"); return RxJavaPlugins.onAssembly(new CompletableFromPublisher(publisher)); } @@ -613,7 +614,7 @@ public static Completable fromPublisher(final Publisher publisher) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromSingle(final SingleSource single) { - ObjectHelper.requireNonNull(single, "single is null"); + Objects.requireNonNull(single, "single is null"); return RxJavaPlugins.onAssembly(new CompletableFromSingle(single)); } @@ -643,7 +644,7 @@ public static Completable fromSingle(final SingleSource single) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable fromSupplier(final Supplier supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new CompletableFromSupplier(supplier)); } @@ -678,7 +679,7 @@ public static Completable fromSupplier(final Supplier supplier) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable mergeArray(CompletableSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return complete(); } else @@ -719,7 +720,7 @@ public static Completable mergeArray(CompletableSource... sources) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable merge(Iterable sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableMergeIterable(sources)); } @@ -822,7 +823,7 @@ public static Completable merge(Publisher sources, @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) private static Completable merge0(Publisher sources, int maxConcurrency, boolean delayErrors) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new CompletableMerge(sources, maxConcurrency, delayErrors)); } @@ -845,7 +846,7 @@ private static Completable merge0(Publisher sources @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable mergeArrayDelayError(CompletableSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableMergeDelayErrorArray(sources)); } @@ -867,7 +868,7 @@ public static Completable mergeArrayDelayError(CompletableSource... sources) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable mergeDelayError(Iterable sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableMergeDelayErrorIterable(sources)); } @@ -973,8 +974,8 @@ public static Completable timer(long delay, TimeUnit unit) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public static Completable timer(final long delay, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableTimer(delay, unit, scheduler)); } @@ -1047,9 +1048,9 @@ public static Completable using( final Function completableFunction, final Consumer disposer, final boolean eager) { - ObjectHelper.requireNonNull(resourceSupplier, "resourceSupplier is null"); - ObjectHelper.requireNonNull(completableFunction, "completableFunction is null"); - ObjectHelper.requireNonNull(disposer, "disposer is null"); + Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); + Objects.requireNonNull(completableFunction, "completableFunction is null"); + Objects.requireNonNull(disposer, "disposer is null"); return RxJavaPlugins.onAssembly(new CompletableUsing(resourceSupplier, completableFunction, disposer, eager)); } @@ -1071,7 +1072,7 @@ public static Completable using( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable wrap(CompletableSource source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); if (source instanceof Completable) { return RxJavaPlugins.onAssembly((Completable)source); } @@ -1096,7 +1097,7 @@ public static Completable wrap(CompletableSource source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable ambWith(CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -1120,7 +1121,7 @@ public final Completable ambWith(CompletableSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Observable andThen(ObservableSource next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return RxJavaPlugins.onAssembly(new CompletableAndThenObservable(this, next)); } @@ -1148,7 +1149,7 @@ public final Observable andThen(ObservableSource next) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable andThen(Publisher next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return RxJavaPlugins.onAssembly(new CompletableAndThenPublisher(this, next)); } @@ -1172,7 +1173,7 @@ public final Flowable andThen(Publisher next) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single andThen(SingleSource next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return RxJavaPlugins.onAssembly(new SingleDelayWithCompletable(next, this)); } @@ -1196,7 +1197,7 @@ public final Single andThen(SingleSource next) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe andThen(MaybeSource next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return RxJavaPlugins.onAssembly(new MaybeDelayWithCompletable(next, this)); } @@ -1218,7 +1219,7 @@ public final Maybe andThen(MaybeSource next) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Completable andThen(CompletableSource next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return RxJavaPlugins.onAssembly(new CompletableAndThenCompletable(this, next)); } @@ -1267,7 +1268,7 @@ public final void blockingAwait() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final boolean blockingAwait(long timeout, TimeUnit unit) { - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(unit, "unit is null"); BlockingMultiObserver observer = new BlockingMultiObserver(); subscribe(observer); return observer.blockingAwait(timeout, unit); @@ -1312,7 +1313,7 @@ public final Completable cache() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Completable compose(CompletableTransformer transformer) { - return wrap(ObjectHelper.requireNonNull(transformer, "transformer is null").apply(this)); + return wrap(Objects.requireNonNull(transformer, "transformer is null").apply(this)); } /** @@ -1335,7 +1336,7 @@ public final Completable compose(CompletableTransformer transformer) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable concatWith(CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new CompletableAndThenCompletable(this, other)); } @@ -1399,8 +1400,8 @@ public final Completable delay(long delay, TimeUnit unit, Scheduler scheduler) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Completable delay(final long delay, final TimeUnit unit, final Scheduler scheduler, final boolean delayError) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableDelay(this, delay, unit, scheduler, delayError)); } @@ -1530,7 +1531,7 @@ public final Completable doOnError(Consumer onError) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable doOnEvent(final Consumer onEvent) { - ObjectHelper.requireNonNull(onEvent, "onEvent is null"); + Objects.requireNonNull(onEvent, "onEvent is null"); return RxJavaPlugins.onAssembly(new CompletableDoOnEvent(this, onEvent)); } @@ -1558,12 +1559,12 @@ private Completable doOnLifecycle( final Action onTerminate, final Action onAfterTerminate, final Action onDispose) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); - ObjectHelper.requireNonNull(onTerminate, "onTerminate is null"); - ObjectHelper.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); - ObjectHelper.requireNonNull(onDispose, "onDispose is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onTerminate, "onTerminate is null"); + Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); + Objects.requireNonNull(onDispose, "onDispose is null"); return RxJavaPlugins.onAssembly(new CompletablePeek(this, onSubscribe, onError, onComplete, onTerminate, onAfterTerminate, onDispose)); } @@ -1657,7 +1658,7 @@ public final Completable doAfterTerminate(final Action onAfterTerminate) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable doFinally(Action onFinally) { - ObjectHelper.requireNonNull(onFinally, "onFinally is null"); + Objects.requireNonNull(onFinally, "onFinally is null"); return RxJavaPlugins.onAssembly(new CompletableDoFinally(this, onFinally)); } @@ -1795,7 +1796,7 @@ public final Completable doFinally(Action onFinally) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable lift(final CompletableOperator onLift) { - ObjectHelper.requireNonNull(onLift, "onLift is null"); + Objects.requireNonNull(onLift, "onLift is null"); return RxJavaPlugins.onAssembly(new CompletableLift(this, onLift)); } @@ -1837,7 +1838,7 @@ public final Single> materialize() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable mergeWith(CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return mergeArray(this, other); } @@ -1857,7 +1858,7 @@ public final Completable mergeWith(CompletableSource other) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Completable observeOn(final Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableObserveOn(this, scheduler)); } @@ -1895,7 +1896,7 @@ public final Completable onErrorComplete() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable onErrorComplete(final Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new CompletableOnErrorComplete(this, predicate)); } @@ -1918,7 +1919,7 @@ public final Completable onErrorComplete(final Predicate pred @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable onErrorResumeNext(final Function errorMapper) { - ObjectHelper.requireNonNull(errorMapper, "errorMapper is null"); + Objects.requireNonNull(errorMapper, "errorMapper is null"); return RxJavaPlugins.onAssembly(new CompletableResumeNext(this, errorMapper)); } @@ -2177,7 +2178,7 @@ public final Completable retryWhen(Function, ? exten @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Completable startWith(CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return concatArray(other, this); } @@ -2199,7 +2200,7 @@ public final Completable startWith(CompletableSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Observable startWith(Observable other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return other.concatWith(this.toObservable()); } /** @@ -2224,7 +2225,7 @@ public final Observable startWith(Observable other) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable startWith(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return this.toFlowable().startWith(other); } @@ -2269,12 +2270,12 @@ public final Disposable subscribe() { @SchedulerSupport(SchedulerSupport.NONE) @Override public final void subscribe(CompletableObserver observer) { - ObjectHelper.requireNonNull(observer, "observer is null"); + Objects.requireNonNull(observer, "observer is null"); try { observer = RxJavaPlugins.onSubscribe(this, observer); - ObjectHelper.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null CompletableObserver. Please check the handler provided to RxJavaPlugins.setOnCompletableSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); + Objects.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null CompletableObserver. Please check the handler provided to RxJavaPlugins.setOnCompletableSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); subscribeActual(observer); } catch (NullPointerException ex) { // NOPMD @@ -2346,8 +2347,8 @@ public final E subscribeWith(E observer) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Disposable subscribe(final Action onComplete, final Consumer onError) { - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); CallbackCompletableObserver observer = new CallbackCompletableObserver(onError, onComplete); subscribe(observer); @@ -2374,7 +2375,7 @@ public final Disposable subscribe(final Action onComplete, final Consumer R to(@NonNull CompletableConverter converter) { - return ObjectHelper.requireNonNull(converter, "converter is null").apply(this); + return Objects.requireNonNull(converter, "converter is null").apply(this); } /** @@ -2658,7 +2659,7 @@ public final Observable toObservable() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single toSingle(final Supplier completionValueSupplier) { - ObjectHelper.requireNonNull(completionValueSupplier, "completionValueSupplier is null"); + Objects.requireNonNull(completionValueSupplier, "completionValueSupplier is null"); return RxJavaPlugins.onAssembly(new CompletableToSingle(this, completionValueSupplier, null)); } @@ -2680,7 +2681,7 @@ public final Single toSingle(final Supplier completionValueS @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single toSingleDefault(final T completionValue) { - ObjectHelper.requireNonNull(completionValue, "completionValue is null"); + Objects.requireNonNull(completionValue, "completionValue is null"); return RxJavaPlugins.onAssembly(new CompletableToSingle(this, null, completionValue)); } @@ -2701,7 +2702,7 @@ public final Single toSingleDefault(final T completionValue) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Completable unsubscribeOn(final Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableDisposeOn(this, scheduler)); } // ------------------------------------------------------------------------- diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 3fed4742e2..514845efbc 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -184,7 +184,7 @@ public abstract class Flowable implements Publisher { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable amb(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableAmb(null, sources)); } @@ -215,7 +215,7 @@ public static Flowable amb(Iterable> sou @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs public static Flowable ambArray(Publisher... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); int len = sources.length; if (len == 0) { return empty(); @@ -324,11 +324,11 @@ public static Flowable combineLatestArray(Publisher[] sou @NonNull @BackpressureSupport(BackpressureKind.FULL) public static Flowable combineLatestArray(Publisher[] sources, Function combiner, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return empty(); } - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableCombineLatest(sources, combiner, bufferSize, false)); } @@ -423,8 +423,8 @@ public static Flowable combineLatest(Iterable Flowable combineLatest(Iterable> sources, Function combiner, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableCombineLatest(sources, combiner, bufferSize, false)); } @@ -520,8 +520,8 @@ public static Flowable combineLatestDelayError(Publisher[ @BackpressureSupport(BackpressureKind.FULL) public static Flowable combineLatestDelayError(Publisher[] sources, Function combiner, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (sources.length == 0) { return empty(); @@ -620,8 +620,8 @@ public static Flowable combineLatestDelayError(Iterable Flowable combineLatestDelayError(Iterable> sources, Function combiner, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableCombineLatest(sources, combiner, bufferSize, true)); } @@ -665,8 +665,8 @@ public static Flowable combineLatestDelayError(Iterable Flowable combineLatest( Publisher source1, Publisher source2, BiFunction combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return combineLatestArray(new Publisher[] { source1, source2 }, Functions.toFunction(combiner), bufferSize()); } @@ -714,9 +714,9 @@ public static Flowable combineLatest( Publisher source1, Publisher source2, Publisher source3, Function3 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return combineLatestArray(new Publisher[] { source1, source2, source3 }, Functions.toFunction(combiner), bufferSize()); } @@ -767,10 +767,10 @@ public static Flowable combineLatest( Publisher source1, Publisher source2, Publisher source3, Publisher source4, Function4 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4 }, Functions.toFunction(combiner), bufferSize()); } @@ -825,11 +825,11 @@ public static Flowable combineLatest( Publisher source3, Publisher source4, Publisher source5, Function5 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5 }, Functions.toFunction(combiner), bufferSize()); } @@ -887,12 +887,12 @@ public static Flowable combineLatest( Publisher source3, Publisher source4, Publisher source5, Publisher source6, Function6 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6 }, Functions.toFunction(combiner), bufferSize()); } @@ -954,13 +954,13 @@ public static Flowable combineLatest( Publisher source5, Publisher source6, Publisher source7, Function7 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6, source7 }, Functions.toFunction(combiner), bufferSize()); } @@ -1025,14 +1025,14 @@ public static Flowable combineLatest( Publisher source5, Publisher source6, Publisher source7, Publisher source8, Function8 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6, source7, source8 }, Functions.toFunction(combiner), bufferSize()); } @@ -1101,15 +1101,15 @@ public static Flowable combineLatest( Publisher source7, Publisher source8, Publisher source9, Function9 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); - ObjectHelper.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source9, "source9 is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6, source7, source8, source9 }, Functions.toFunction(combiner), bufferSize()); } @@ -1137,7 +1137,7 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable concat(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); // unlike general sources, fromIterable can only throw on a boundary because it is consumed only there return fromIterable(sources).concatMapDelayError((Function)Functions.identity(), false, 2); } @@ -1232,8 +1232,8 @@ public static Flowable concat(Publisher> @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable concat(Publisher source1, Publisher source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return concatArray(source1, source2); } @@ -1270,9 +1270,9 @@ public static Flowable concat(Publisher source1, Publisher Flowable concat( Publisher source1, Publisher source2, Publisher source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return concatArray(source1, source2, source3); } @@ -1311,10 +1311,10 @@ public static Flowable concat( public static Flowable concat( Publisher source1, Publisher source2, Publisher source3, Publisher source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return concatArray(source1, source2, source3, source4); } @@ -1446,7 +1446,7 @@ public static Flowable concatArrayEager(Publisher... sources @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "rawtypes", "unchecked" }) public static Flowable concatArrayEager(int maxConcurrency, int prefetch, Publisher... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapEager(new FlowableFromArray(sources), Functions.identity(), maxConcurrency, prefetch, ErrorMode.IMMEDIATE)); @@ -1541,7 +1541,7 @@ public static Flowable concatArrayEagerDelayError(int maxConcurrency, int @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable concatDelayError(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return fromIterable(sources).concatMapDelayError((Function)Functions.identity()); } @@ -1647,7 +1647,7 @@ public static Flowable concatEager(Publisher Flowable concatEager(Publisher> sources, int maxConcurrency, int prefetch) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapEagerPublisher(sources, Functions.identity(), maxConcurrency, prefetch, ErrorMode.IMMEDIATE)); @@ -1707,7 +1707,7 @@ public static Flowable concatEager(Iterable Flowable concatEager(Iterable> sources, int maxConcurrency, int prefetch) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapEager(new FlowableFromIterable(sources), Functions.identity(), maxConcurrency, prefetch, ErrorMode.IMMEDIATE)); @@ -1770,8 +1770,8 @@ public static Flowable concatEager(Iterable Flowable create(FlowableOnSubscribe source, BackpressureStrategy mode) { - ObjectHelper.requireNonNull(source, "source is null"); - ObjectHelper.requireNonNull(mode, "mode is null"); + Objects.requireNonNull(source, "source is null"); + Objects.requireNonNull(mode, "mode is null"); return RxJavaPlugins.onAssembly(new FlowableCreate(source, mode)); } @@ -1807,7 +1807,7 @@ public static Flowable create(FlowableOnSubscribe source, Backpressure @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable defer(Supplier> supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new FlowableDefer(supplier)); } @@ -1862,7 +1862,7 @@ public static Flowable empty() { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable error(Supplier supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new FlowableError(supplier)); } @@ -1891,7 +1891,7 @@ public static Flowable error(Supplier supplier) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable error(final Throwable throwable) { - ObjectHelper.requireNonNull(throwable, "throwable is null"); + Objects.requireNonNull(throwable, "throwable is null"); return error(Functions.justSupplier(throwable)); } @@ -1920,7 +1920,7 @@ public static Flowable error(final Throwable throwable) { @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs public static Flowable fromArray(T... items) { - ObjectHelper.requireNonNull(items, "items is null"); + Objects.requireNonNull(items, "items is null"); if (items.length == 0) { return empty(); } @@ -1967,7 +1967,7 @@ public static Flowable fromArray(T... items) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable fromCallable(Callable supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new FlowableFromCallable(supplier)); } @@ -2009,7 +2009,7 @@ public static Flowable fromCallable(Callable supplier) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable fromFuture(Future future) { - ObjectHelper.requireNonNull(future, "future is null"); + Objects.requireNonNull(future, "future is null"); return RxJavaPlugins.onAssembly(new FlowableFromFuture(future, 0L, null)); } @@ -2055,8 +2055,8 @@ public static Flowable fromFuture(Future future) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable fromFuture(Future future, long timeout, TimeUnit unit) { - ObjectHelper.requireNonNull(future, "future is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(future, "future is null"); + Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new FlowableFromFuture(future, timeout, unit)); } @@ -2106,7 +2106,7 @@ public static Flowable fromFuture(Future future, long timeou @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public static Flowable fromFuture(Future future, long timeout, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return fromFuture((Future)future, timeout, unit).subscribeOn(scheduler); } @@ -2145,7 +2145,7 @@ public static Flowable fromFuture(Future future, long timeou @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public static Flowable fromFuture(Future future, Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return fromFuture((Future)future).subscribeOn(scheduler); } @@ -2175,7 +2175,7 @@ public static Flowable fromFuture(Future future, Scheduler s @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable fromIterable(Iterable source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new FlowableFromIterable(source)); } @@ -2215,7 +2215,7 @@ public static Flowable fromPublisher(final Publisher source) if (source instanceof Flowable) { return RxJavaPlugins.onAssembly((Flowable)source); } - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new FlowableFromPublisher(source)); } @@ -2257,7 +2257,7 @@ public static Flowable fromPublisher(final Publisher source) @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable fromSupplier(Supplier supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new FlowableFromSupplier(supplier)); } @@ -2287,7 +2287,7 @@ public static Flowable fromSupplier(Supplier supplier) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable generate(final Consumer> generator) { - ObjectHelper.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(generator, "generator is null"); return generate(Functions.nullSupplier(), FlowableInternalHelper.simpleGenerator(generator), Functions.emptyConsumer()); @@ -2321,7 +2321,7 @@ public static Flowable generate(final Consumer> generator) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable generate(Supplier initialState, final BiConsumer> generator) { - ObjectHelper.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(generator, "generator is null"); return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator), Functions.emptyConsumer()); } @@ -2357,7 +2357,7 @@ public static Flowable generate(Supplier initialState, final BiCons @SchedulerSupport(SchedulerSupport.NONE) public static Flowable generate(Supplier initialState, final BiConsumer> generator, Consumer disposeState) { - ObjectHelper.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(generator, "generator is null"); return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator), disposeState); } @@ -2423,9 +2423,9 @@ public static Flowable generate(Supplier initialState, BiFunction Flowable generate(Supplier initialState, BiFunction, S> generator, Consumer disposeState) { - ObjectHelper.requireNonNull(initialState, "initialState is null"); - ObjectHelper.requireNonNull(generator, "generator is null"); - ObjectHelper.requireNonNull(disposeState, "disposeState is null"); + Objects.requireNonNull(initialState, "initialState is null"); + Objects.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(disposeState, "disposeState is null"); return RxJavaPlugins.onAssembly(new FlowableGenerate(initialState, generator, disposeState)); } @@ -2493,8 +2493,8 @@ public static Flowable interval(long initialDelay, long period, TimeUnit u @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public static Flowable interval(long initialDelay, long period, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableInterval(Math.max(0L, initialDelay), Math.max(0L, period), unit, scheduler)); } @@ -2612,8 +2612,8 @@ public static Flowable intervalRange(long start, long count, long initialD if (start > 0 && end < 0) { throw new IllegalArgumentException("Overflow! start + count is bigger than Long.MAX_VALUE"); } - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableIntervalRange(start, end, Math.max(0L, initialDelay), Math.max(0L, period), unit, scheduler)); } @@ -2653,7 +2653,7 @@ public static Flowable intervalRange(long start, long count, long initialD @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return RxJavaPlugins.onAssembly(new FlowableJust(item)); } @@ -2682,8 +2682,8 @@ public static Flowable just(T item) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); return fromArray(item1, item2); } @@ -2715,9 +2715,9 @@ public static Flowable just(T item1, T item2) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); return fromArray(item1, item2, item3); } @@ -2751,10 +2751,10 @@ public static Flowable just(T item1, T item2, T item3) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3, T item4) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); return fromArray(item1, item2, item3, item4); } @@ -2790,11 +2790,11 @@ public static Flowable just(T item1, T item2, T item3, T item4) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3, T item4, T item5) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); return fromArray(item1, item2, item3, item4, item5); } @@ -2832,12 +2832,12 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5) @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); return fromArray(item1, item2, item3, item4, item5, item6); } @@ -2877,13 +2877,13 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7); } @@ -2925,14 +2925,14 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); - ObjectHelper.requireNonNull(item8, "item8 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item8, "item8 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7, item8); } @@ -2976,15 +2976,15 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); - ObjectHelper.requireNonNull(item8, "item8 is null"); - ObjectHelper.requireNonNull(item9, "item9 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item8, "item8 is null"); + Objects.requireNonNull(item9, "item9 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7, item8, item9); } @@ -3030,16 +3030,16 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9, T item10) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); - ObjectHelper.requireNonNull(item8, "item8 is null"); - ObjectHelper.requireNonNull(item9, "item9 is null"); - ObjectHelper.requireNonNull(item10, "item10 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item8, "item8 is null"); + Objects.requireNonNull(item9, "item9 is null"); + Objects.requireNonNull(item10, "item10 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7, item8, item9, item10); } @@ -3421,8 +3421,8 @@ public static Flowable mergeArray(Publisher... sources) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable merge(Publisher source1, Publisher source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), false, 2); } @@ -3471,9 +3471,9 @@ public static Flowable merge(Publisher source1, Publisher Flowable merge(Publisher source1, Publisher source2, Publisher source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return fromArray(source1, source2, source3).flatMap((Function)Functions.identity(), false, 3); } @@ -3526,10 +3526,10 @@ public static Flowable merge(Publisher source1, Publisher Flowable merge( Publisher source1, Publisher source2, Publisher source3, Publisher source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return fromArray(source1, source2, source3, source4).flatMap((Function)Functions.identity(), false, 4); } @@ -3834,8 +3834,8 @@ public static Flowable mergeArrayDelayError(Publisher... sou @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable mergeDelayError(Publisher source1, Publisher source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), true, 2); } @@ -3877,9 +3877,9 @@ public static Flowable mergeDelayError(Publisher source1, Pu @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable mergeDelayError(Publisher source1, Publisher source2, Publisher source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return fromArray(source1, source2, source3).flatMap((Function)Functions.identity(), true, 3); } @@ -3925,10 +3925,10 @@ public static Flowable mergeDelayError(Publisher source1, Pu public static Flowable mergeDelayError( Publisher source1, Publisher source2, Publisher source3, Publisher source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return fromArray(source1, source2, source3, source4).flatMap((Function)Functions.identity(), true, 4); } @@ -4140,9 +4140,9 @@ public static Single sequenceEqual(Publisher source1, @SchedulerSupport(SchedulerSupport.NONE) public static Single sequenceEqual(Publisher source1, Publisher source2, BiPredicate isEqual, int bufferSize) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(isEqual, "isEqual is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(isEqual, "isEqual is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableSequenceEqualSingle(source1, source2, isEqual, bufferSize)); } @@ -4390,8 +4390,8 @@ public static Flowable timer(long delay, TimeUnit unit) { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public static Flowable timer(long delay, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableTimer(Math.max(0L, delay), unit, scheduler)); } @@ -4419,7 +4419,7 @@ public static Flowable timer(long delay, TimeUnit unit, Scheduler schedule @BackpressureSupport(BackpressureKind.NONE) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable unsafeCreate(Publisher onSubscribe) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); if (onSubscribe instanceof Flowable) { throw new IllegalArgumentException("unsafeCreate(Flowable) should be upgraded"); } @@ -4497,9 +4497,9 @@ public static Flowable using(Supplier resourceSupplier, public static Flowable using(Supplier resourceSupplier, Function> sourceSupplier, Consumer resourceDisposer, boolean eager) { - ObjectHelper.requireNonNull(resourceSupplier, "resourceSupplier is null"); - ObjectHelper.requireNonNull(sourceSupplier, "sourceSupplier is null"); - ObjectHelper.requireNonNull(resourceDisposer, "resourceDisposer is null"); + Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); + Objects.requireNonNull(sourceSupplier, "sourceSupplier is null"); + Objects.requireNonNull(resourceDisposer, "resourceDisposer is null"); return RxJavaPlugins.onAssembly(new FlowableUsing(resourceSupplier, sourceSupplier, resourceDisposer, eager)); } @@ -4552,8 +4552,8 @@ public static Flowable using(Supplier resourceSupplier, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip(Iterable> sources, Function zipper) { - ObjectHelper.requireNonNull(zipper, "zipper is null"); - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableZip(null, sources, zipper, bufferSize(), false)); } @@ -4613,8 +4613,8 @@ public static Flowable zip(Iterable> public static Flowable zip(Iterable> sources, Function zipper, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(zipper, "zipper is null"); - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableZip(null, sources, zipper, bufferSize, delayError)); } @@ -4674,8 +4674,8 @@ public static Flowable zip(Iterable> public static Flowable zip( Publisher source1, Publisher source2, BiFunction zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2); } @@ -4735,8 +4735,8 @@ public static Flowable zip( public static Flowable zip( Publisher source1, Publisher source2, BiFunction zipper, boolean delayError) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize(), source1, source2); } @@ -4797,8 +4797,8 @@ public static Flowable zip( public static Flowable zip( Publisher source1, Publisher source2, BiFunction zipper, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize, source1, source2); } @@ -4861,9 +4861,9 @@ public static Flowable zip( public static Flowable zip( Publisher source1, Publisher source2, Publisher source3, Function3 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3); } @@ -4930,10 +4930,10 @@ public static Flowable zip( Publisher source1, Publisher source2, Publisher source3, Publisher source4, Function4 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4); } @@ -5003,11 +5003,11 @@ public static Flowable zip( Publisher source1, Publisher source2, Publisher source3, Publisher source4, Publisher source5, Function5 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5); } @@ -5079,12 +5079,12 @@ public static Flowable zip( Publisher source1, Publisher source2, Publisher source3, Publisher source4, Publisher source5, Publisher source6, Function6 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6); } @@ -5160,13 +5160,13 @@ public static Flowable zip( Publisher source4, Publisher source5, Publisher source6, Publisher source7, Function7 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7); } @@ -5245,14 +5245,14 @@ public static Flowable zip( Publisher source4, Publisher source5, Publisher source6, Publisher source7, Publisher source8, Function8 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -5335,15 +5335,15 @@ public static Flowable zip( Publisher source7, Publisher source8, Publisher source9, Function9 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); - ObjectHelper.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source9, "source9 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8, source9); } @@ -5406,7 +5406,7 @@ public static Flowable zipArray(Function(sources, null, zipper, bufferSize, delayError)); } @@ -5439,7 +5439,7 @@ public static Flowable zipArray(Function all(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new FlowableAllSingle(this, predicate)); } @@ -5468,7 +5468,7 @@ public final Single all(Predicate predicate) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable ambWith(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -5500,7 +5500,7 @@ public final Flowable ambWith(Publisher other) { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single any(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new FlowableAnySingle(this, predicate)); } @@ -6209,7 +6209,7 @@ public final Flowable> buffer(int count, int skip) { public final > Flowable buffer(int count, int skip, Supplier bufferSupplier) { ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(skip, "skip"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new FlowableBuffer(this, count, skip, bufferSupplier)); } @@ -6358,9 +6358,9 @@ public final Flowable> buffer(long timespan, long timeskip, TimeUnit uni @SchedulerSupport(SchedulerSupport.CUSTOM) public final > Flowable buffer(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler, Supplier bufferSupplier) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new FlowableBufferTimed(this, timespan, timeskip, unit, scheduler, bufferSupplier, Integer.MAX_VALUE, false)); } @@ -6518,9 +6518,9 @@ public final > Flowable buffer( Scheduler scheduler, int count, Supplier bufferSupplier, boolean restartTimerOnMaxSize) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); ObjectHelper.verifyPositive(count, "count"); return RxJavaPlugins.onAssembly(new FlowableBufferTimed(this, timespan, timespan, unit, scheduler, bufferSupplier, count, restartTimerOnMaxSize)); } @@ -6634,9 +6634,9 @@ public final > Flowable b Flowable openingIndicator, Function> closingIndicator, Supplier bufferSupplier) { - ObjectHelper.requireNonNull(openingIndicator, "openingIndicator is null"); - ObjectHelper.requireNonNull(closingIndicator, "closingIndicator is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(openingIndicator, "openingIndicator is null"); + Objects.requireNonNull(closingIndicator, "closingIndicator is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new FlowableBufferBoundary(this, openingIndicator, closingIndicator, bufferSupplier)); } @@ -6746,8 +6746,8 @@ public final Flowable> buffer(Publisher boundaryIndicator, final @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) public final > Flowable buffer(Publisher boundaryIndicator, Supplier bufferSupplier) { - ObjectHelper.requireNonNull(boundaryIndicator, "boundaryIndicator is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(boundaryIndicator, "boundaryIndicator is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new FlowableBufferExactBoundary(this, boundaryIndicator, bufferSupplier)); } @@ -6898,7 +6898,7 @@ public final Flowable cacheWithInitialCapacity(int initialCapacity) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable cast(final Class clazz) { - ObjectHelper.requireNonNull(clazz, "clazz is null"); + Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -6936,8 +6936,8 @@ public final Flowable cast(final Class clazz) { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single collect(Supplier initialItemSupplier, BiConsumer collector) { - ObjectHelper.requireNonNull(initialItemSupplier, "initialItemSupplier is null"); - ObjectHelper.requireNonNull(collector, "collector is null"); + Objects.requireNonNull(initialItemSupplier, "initialItemSupplier is null"); + Objects.requireNonNull(collector, "collector is null"); return RxJavaPlugins.onAssembly(new FlowableCollectSingle(this, initialItemSupplier, collector)); } @@ -6975,7 +6975,7 @@ public final Single collect(Supplier initialItemSupplier, Bi @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single collectInto(final U initialItem, BiConsumer collector) { - ObjectHelper.requireNonNull(initialItem, "initialItem is null"); + Objects.requireNonNull(initialItem, "initialItem is null"); return collect(Functions.justSupplier(initialItem), collector); } @@ -7006,7 +7006,7 @@ public final Single collectInto(final U initialItem, BiConsumer Flowable compose(FlowableTransformer composer) { - return fromPublisher(((FlowableTransformer) ObjectHelper.requireNonNull(composer, "composer is null")).apply(this)); + return fromPublisher(((FlowableTransformer) Objects.requireNonNull(composer, "composer is null")).apply(this)); } /** @@ -7082,7 +7082,7 @@ public final Flowable concatMap(Function Flowable concatMap(Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); if (this instanceof ScalarSupplier) { @SuppressWarnings("unchecked") @@ -7135,9 +7135,9 @@ public final Flowable concatMap(Function Flowable concatMap(Function> mapper, int prefetch, Scheduler scheduler) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - ObjectHelper.requireNonNull(scheduler, "scheduler"); + Objects.requireNonNull(scheduler, "scheduler"); return RxJavaPlugins.onAssembly(new FlowableConcatMapScheduler(this, mapper, prefetch, ErrorMode.IMMEDIATE, scheduler)); } @@ -7197,7 +7197,7 @@ public final Completable concatMapCompletable(Function mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapCompletable(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } @@ -7300,7 +7300,7 @@ public final Completable concatMapCompletableDelayError(Function mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapCompletable(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } @@ -7374,7 +7374,7 @@ public final Flowable concatMapDelayError(Function Flowable concatMapDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); if (this instanceof ScalarSupplier) { @SuppressWarnings("unchecked") @@ -7426,9 +7426,9 @@ public final Flowable concatMapDelayError(Function Flowable concatMapDelayError(Function> mapper, boolean tillTheEnd, int prefetch, Scheduler scheduler) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableConcatMapScheduler(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, scheduler)); } @@ -7487,7 +7487,7 @@ public final Flowable concatMapEager(Function Flowable concatMapEager(Function> mapper, int maxConcurrency, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapEager(this, mapper, maxConcurrency, prefetch, ErrorMode.IMMEDIATE)); @@ -7557,7 +7557,7 @@ public final Flowable concatMapEagerDelayError(Function Flowable concatMapEagerDelayError(Function> mapper, boolean tillTheEnd, int maxConcurrency, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapEager(this, mapper, maxConcurrency, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); @@ -7621,7 +7621,7 @@ public final Flowable concatMapIterable(Function Flowable concatMapIterable(final Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableFlattenIterable(this, mapper, prefetch)); } @@ -7690,7 +7690,7 @@ public final Flowable concatMapMaybe(Function Flowable concatMapMaybe(Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapMaybe(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } @@ -7801,7 +7801,7 @@ public final Flowable concatMapMaybeDelayError(Function Flowable concatMapMaybeDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapMaybe(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } @@ -7870,7 +7870,7 @@ public final Flowable concatMapSingle(Function Flowable concatMapSingle(Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapSingle(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } @@ -7981,7 +7981,7 @@ public final Flowable concatMapSingleDelayError(Function Flowable concatMapSingleDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapSingle(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } @@ -8011,7 +8011,7 @@ public final Flowable concatMapSingleDelayError(Function concatWith(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return concat(this, other); } @@ -8036,7 +8036,7 @@ public final Flowable concatWith(Publisher other) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable concatWith(@NonNull SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableConcatWithSingle(this, other)); } @@ -8061,7 +8061,7 @@ public final Flowable concatWith(@NonNull SingleSource other) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable concatWith(@NonNull MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableConcatWithMaybe(this, other)); } @@ -8088,7 +8088,7 @@ public final Flowable concatWith(@NonNull MaybeSource other) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable concatWith(@NonNull CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableConcatWithCompletable(this, other)); } @@ -8116,7 +8116,7 @@ public final Flowable concatWith(@NonNull CompletableSource other) { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single contains(final Object item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return any(Functions.equalsWith(item)); } @@ -8179,7 +8179,7 @@ public final Single count() { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable debounce(Function> debounceIndicator) { - ObjectHelper.requireNonNull(debounceIndicator, "debounceIndicator is null"); + Objects.requireNonNull(debounceIndicator, "debounceIndicator is null"); return RxJavaPlugins.onAssembly(new FlowableDebounce(this, debounceIndicator)); } @@ -8267,8 +8267,8 @@ public final Flowable debounce(long timeout, TimeUnit unit) { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable debounce(long timeout, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableDebounceTimed(this, timeout, unit, scheduler)); } @@ -8298,7 +8298,7 @@ public final Flowable debounce(long timeout, TimeUnit unit, Scheduler schedul @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable defaultIfEmpty(T defaultItem) { - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return switchIfEmpty(just(defaultItem)); } @@ -8334,7 +8334,7 @@ public final Flowable defaultIfEmpty(T defaultItem) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable delay(final Function> itemDelayIndicator) { - ObjectHelper.requireNonNull(itemDelayIndicator, "itemDelayIndicator is null"); + Objects.requireNonNull(itemDelayIndicator, "itemDelayIndicator is null"); return flatMap(FlowableInternalHelper.itemDelay(itemDelayIndicator)); } @@ -8450,8 +8450,8 @@ public final Flowable delay(long delay, TimeUnit unit, Scheduler scheduler) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable delay(long delay, TimeUnit unit, Scheduler scheduler, boolean delayError) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableDelay(this, Math.max(0L, delay), unit, scheduler, delayError)); } @@ -8519,7 +8519,7 @@ public final Flowable delay(Publisher subscriptionIndicator, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable delaySubscription(Publisher subscriptionIndicator) { - ObjectHelper.requireNonNull(subscriptionIndicator, "subscriptionIndicator is null"); + Objects.requireNonNull(subscriptionIndicator, "subscriptionIndicator is null"); return RxJavaPlugins.onAssembly(new FlowableDelaySubscriptionOther(this, subscriptionIndicator)); } @@ -8632,7 +8632,7 @@ public final Flowable delaySubscription(long delay, TimeUnit unit, Scheduler @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.PASS_THROUGH) public final Flowable dematerialize(Function> selector) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); return RxJavaPlugins.onAssembly(new FlowableDematerialize(this, selector)); } @@ -8752,8 +8752,8 @@ public final Flowable distinct(Function keySelector) { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable distinct(Function keySelector, Supplier> collectionSupplier) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(collectionSupplier, "collectionSupplier is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); return RxJavaPlugins.onAssembly(new FlowableDistinct(this, keySelector, collectionSupplier)); } @@ -8841,7 +8841,7 @@ public final Flowable distinctUntilChanged() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable distinctUntilChanged(Function keySelector) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged(this, keySelector, ObjectHelper.equalsPredicate())); } @@ -8879,7 +8879,7 @@ public final Flowable distinctUntilChanged(Function keySele @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable distinctUntilChanged(BiPredicate comparer) { - ObjectHelper.requireNonNull(comparer, "comparer is null"); + Objects.requireNonNull(comparer, "comparer is null"); return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged(this, Functions.identity(), comparer)); } @@ -8909,7 +8909,7 @@ public final Flowable distinctUntilChanged(BiPredicate @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable doFinally(Action onFinally) { - ObjectHelper.requireNonNull(onFinally, "onFinally is null"); + Objects.requireNonNull(onFinally, "onFinally is null"); return RxJavaPlugins.onAssembly(new FlowableDoFinally(this, onFinally)); } @@ -8936,7 +8936,7 @@ public final Flowable doFinally(Action onFinally) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable doAfterNext(Consumer onAfterNext) { - ObjectHelper.requireNonNull(onAfterNext, "onAfterNext is null"); + Objects.requireNonNull(onAfterNext, "onAfterNext is null"); return RxJavaPlugins.onAssembly(new FlowableDoAfterNext(this, onAfterNext)); } @@ -9047,10 +9047,10 @@ public final Flowable doOnComplete(Action onComplete) { @SchedulerSupport(SchedulerSupport.NONE) private Flowable doOnEach(Consumer onNext, Consumer onError, Action onComplete, Action onAfterTerminate) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); - ObjectHelper.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); return RxJavaPlugins.onAssembly(new FlowableDoOnEach(this, onNext, onError, onComplete, onAfterTerminate)); } @@ -9076,7 +9076,7 @@ private Flowable doOnEach(Consumer onNext, Consumer doOnEach(final Consumer> onNotification) { - ObjectHelper.requireNonNull(onNotification, "onNotification is null"); + Objects.requireNonNull(onNotification, "onNotification is null"); return doOnEach( Functions.notificationOnNext(onNotification), Functions.notificationOnError(onNotification), @@ -9113,7 +9113,7 @@ public final Flowable doOnEach(final Consumer> onNoti @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable doOnEach(final Subscriber subscriber) { - ObjectHelper.requireNonNull(subscriber, "subscriber is null"); + Objects.requireNonNull(subscriber, "subscriber is null"); return doOnEach( FlowableInternalHelper.subscriberOnNext(subscriber), FlowableInternalHelper.subscriberOnError(subscriber), @@ -9177,9 +9177,9 @@ public final Flowable doOnError(Consumer onError) { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable doOnLifecycle(final Consumer onSubscribe, final LongConsumer onRequest, final Action onCancel) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); - ObjectHelper.requireNonNull(onRequest, "onRequest is null"); - ObjectHelper.requireNonNull(onCancel, "onCancel is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onRequest, "onRequest is null"); + Objects.requireNonNull(onCancel, "onCancel is null"); return RxJavaPlugins.onAssembly(new FlowableDoOnLifecycle(this, onSubscribe, onRequest, onCancel)); } @@ -9352,7 +9352,7 @@ public final Single elementAt(long index, T defaultItem) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new FlowableElementAtSingle(this, index, defaultItem)); } @@ -9410,7 +9410,7 @@ public final Single elementAtOrError(long index) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable filter(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new FlowableFilter(this, predicate)); } @@ -9665,7 +9665,7 @@ public final Flowable flatMap(Function Flowable flatMap(Function> mapper, boolean delayErrors, int maxConcurrency, int bufferSize) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (this instanceof ScalarSupplier) { @@ -9716,9 +9716,9 @@ public final Flowable flatMap( Function> onNextMapper, Function> onErrorMapper, Supplier> onCompleteSupplier) { - ObjectHelper.requireNonNull(onNextMapper, "onNextMapper is null"); - ObjectHelper.requireNonNull(onErrorMapper, "onErrorMapper is null"); - ObjectHelper.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); + Objects.requireNonNull(onNextMapper, "onNextMapper is null"); + Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); + Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); return merge(new FlowableMapNotification>(this, onNextMapper, onErrorMapper, onCompleteSupplier)); } @@ -9764,9 +9764,9 @@ public final Flowable flatMap( Function> onErrorMapper, Supplier> onCompleteSupplier, int maxConcurrency) { - ObjectHelper.requireNonNull(onNextMapper, "onNextMapper is null"); - ObjectHelper.requireNonNull(onErrorMapper, "onErrorMapper is null"); - ObjectHelper.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); + Objects.requireNonNull(onNextMapper, "onNextMapper is null"); + Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); + Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); return merge(new FlowableMapNotification>( this, onNextMapper, onErrorMapper, onCompleteSupplier), maxConcurrency); } @@ -9932,8 +9932,8 @@ public final Flowable flatMap(Function Flowable flatMap(final Function> mapper, final BiFunction combiner, boolean delayErrors, int maxConcurrency, int bufferSize) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return flatMap(FlowableInternalHelper.flatMapWithCombiner(mapper, combiner), delayErrors, maxConcurrency, bufferSize); @@ -10020,7 +10020,7 @@ public final Completable flatMapCompletable(Function mapper, boolean delayErrors, int maxConcurrency) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapCompletableCompletable(this, mapper, delayErrors, maxConcurrency)); } @@ -10085,7 +10085,7 @@ public final Flowable flatMapIterable(final Function Flowable flatMapIterable(final Function> mapper, int bufferSize) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableFlattenIterable(this, mapper, bufferSize)); } @@ -10123,8 +10123,8 @@ public final Flowable flatMapIterable(final Function Flowable flatMapIterable(final Function> mapper, final BiFunction resultSelector) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return flatMap(FlowableInternalHelper.flatMapIntoIterable(mapper), resultSelector, false, bufferSize(), bufferSize()); } @@ -10166,8 +10166,8 @@ public final Flowable flatMapIterable(final Function Flowable flatMapIterable(final Function> mapper, final BiFunction resultSelector, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return flatMap(FlowableInternalHelper.flatMapIntoIterable(mapper), resultSelector, false, bufferSize(), prefetch); } @@ -10215,7 +10215,7 @@ public final Flowable flatMapMaybe(Function Flowable flatMapMaybe(Function> mapper, boolean delayErrors, int maxConcurrency) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapMaybe(this, mapper, delayErrors, maxConcurrency)); } @@ -10264,7 +10264,7 @@ public final Flowable flatMapSingle(Function Flowable flatMapSingle(Function> mapper, boolean delayErrors, int maxConcurrency) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapSingle(this, mapper, delayErrors, maxConcurrency)); } @@ -10386,9 +10386,9 @@ public final Disposable forEachWhile(Predicate onNext, Consumer onNext, final Consumer onError, final Action onComplete) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); ForEachWhileSubscriber s = new ForEachWhileSubscriber(onNext, onError, onComplete); subscribe(s); @@ -10706,8 +10706,8 @@ public final Flowable> groupBy(Function Flowable> groupBy(Function keySelector, Function valueSelector, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableGroupBy(this, keySelector, valueSelector, bufferSize, delayError, null)); @@ -10828,10 +10828,10 @@ public final Flowable> groupBy(Function valueSelector, boolean delayError, int bufferSize, Function, ? extends Map> evictingMapFactory) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(evictingMapFactory, "evictingMapFactory is null"); + Objects.requireNonNull(evictingMapFactory, "evictingMapFactory is null"); return RxJavaPlugins.onAssembly(new FlowableGroupBy(this, keySelector, valueSelector, bufferSize, delayError, evictingMapFactory)); } @@ -10879,10 +10879,10 @@ public final Flowable groupJoin( Function> leftEnd, Function> rightEnd, BiFunction, ? extends R> resultSelector) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(leftEnd, "leftEnd is null"); - ObjectHelper.requireNonNull(rightEnd, "rightEnd is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(leftEnd, "leftEnd is null"); + Objects.requireNonNull(rightEnd, "rightEnd is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return RxJavaPlugins.onAssembly(new FlowableGroupJoin( this, other, leftEnd, rightEnd, resultSelector)); } @@ -11001,10 +11001,10 @@ public final Flowable join( Function> leftEnd, Function> rightEnd, BiFunction resultSelector) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(leftEnd, "leftEnd is null"); - ObjectHelper.requireNonNull(rightEnd, "rightEnd is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(leftEnd, "leftEnd is null"); + Objects.requireNonNull(rightEnd, "rightEnd is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return RxJavaPlugins.onAssembly(new FlowableJoin( this, other, leftEnd, rightEnd, resultSelector)); } @@ -11055,7 +11055,7 @@ public final Maybe lastElement() { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single last(T defaultItem) { - ObjectHelper.requireNonNull(defaultItem, "defaultItem"); + Objects.requireNonNull(defaultItem, "defaultItem"); return RxJavaPlugins.onAssembly(new FlowableLastSingle(this, defaultItem)); } @@ -11233,7 +11233,7 @@ public final Single lastOrError() { @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable lift(FlowableOperator lifter) { - ObjectHelper.requireNonNull(lifter, "lifter is null"); + Objects.requireNonNull(lifter, "lifter is null"); return RxJavaPlugins.onAssembly(new FlowableLift(this, lifter)); } @@ -11262,7 +11262,7 @@ public final Flowable lift(FlowableOperator lifte @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable map(Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableMap(this, mapper)); } @@ -11316,7 +11316,7 @@ public final Flowable> materialize() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return merge(this, other); } @@ -11344,7 +11344,7 @@ public final Flowable mergeWith(Publisher other) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(@NonNull SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableMergeWithSingle(this, other)); } @@ -11373,7 +11373,7 @@ public final Flowable mergeWith(@NonNull SingleSource other) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(@NonNull MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableMergeWithMaybe(this, other)); } @@ -11399,7 +11399,7 @@ public final Flowable mergeWith(@NonNull MaybeSource other) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(@NonNull CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableMergeWithCompletable(this, other)); } @@ -11554,7 +11554,7 @@ public final Flowable observeOn(Scheduler scheduler, boolean delayError) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable observeOn(Scheduler scheduler, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableObserveOn(this, scheduler, delayError, bufferSize)); } @@ -11582,7 +11582,7 @@ public final Flowable observeOn(Scheduler scheduler, boolean delayError, int @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable ofType(final Class clazz) { - ObjectHelper.requireNonNull(clazz, "clazz is null"); + Objects.requireNonNull(clazz, "clazz is null"); return filter(Functions.isInstanceOf(clazz)).cast(clazz); } @@ -11760,7 +11760,7 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError, @SchedulerSupport(SchedulerSupport.NONE) public final Flowable onBackpressureBuffer(int capacity, boolean delayError, boolean unbounded, Action onOverflow) { - ObjectHelper.requireNonNull(onOverflow, "onOverflow is null"); + Objects.requireNonNull(onOverflow, "onOverflow is null"); ObjectHelper.verifyPositive(capacity, "capacity"); return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBuffer(this, capacity, unbounded, delayError, onOverflow)); } @@ -11831,7 +11831,7 @@ public final Flowable onBackpressureBuffer(int capacity, Action onOverflow) { @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable onBackpressureBuffer(long capacity, Action onOverflow, BackpressureOverflowStrategy overflowStrategy) { - ObjectHelper.requireNonNull(overflowStrategy, "overflowStrategy is null"); + Objects.requireNonNull(overflowStrategy, "overflowStrategy is null"); ObjectHelper.verifyPositive(capacity, "capacity"); return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBufferStrategy(this, capacity, onOverflow, overflowStrategy)); } @@ -11888,7 +11888,7 @@ public final Flowable onBackpressureDrop() { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable onBackpressureDrop(Consumer onDrop) { - ObjectHelper.requireNonNull(onDrop, "onDrop is null"); + Objects.requireNonNull(onDrop, "onDrop is null"); return RxJavaPlugins.onAssembly(new FlowableOnBackpressureDrop(this, onDrop)); } @@ -11964,7 +11964,7 @@ public final Flowable onBackpressureLatest() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable onErrorResumeNext(Function> resumeFunction) { - ObjectHelper.requireNonNull(resumeFunction, "resumeFunction is null"); + Objects.requireNonNull(resumeFunction, "resumeFunction is null"); return RxJavaPlugins.onAssembly(new FlowableOnErrorNext(this, resumeFunction)); } @@ -12008,7 +12008,7 @@ public final Flowable onErrorResumeNext(Function onErrorResumeWith(final Publisher next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return onErrorResumeNext(Functions.justFunction(next)); } @@ -12048,7 +12048,7 @@ public final Flowable onErrorResumeWith(final Publisher next) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable onErrorReturn(Function valueSupplier) { - ObjectHelper.requireNonNull(valueSupplier, "valueSupplier is null"); + Objects.requireNonNull(valueSupplier, "valueSupplier is null"); return RxJavaPlugins.onAssembly(new FlowableOnErrorReturn(this, valueSupplier)); } @@ -12088,7 +12088,7 @@ public final Flowable onErrorReturn(Function @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable onErrorReturnItem(final T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return onErrorReturn(Functions.justFunction(item)); } @@ -12297,7 +12297,7 @@ public final Flowable publish(Function, ? extends Pub @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable publish(Function, ? extends Publisher> selector, int prefetch) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowablePublishMulticast(this, selector, prefetch, false)); } @@ -12392,7 +12392,7 @@ public final Flowable rebatchRequests(int n) { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Maybe reduce(BiFunction reducer) { - ObjectHelper.requireNonNull(reducer, "reducer is null"); + Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new FlowableReduceMaybe(this, reducer)); } @@ -12454,8 +12454,8 @@ public final Maybe reduce(BiFunction reducer) { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single reduce(R seed, BiFunction reducer) { - ObjectHelper.requireNonNull(seed, "seed is null"); - ObjectHelper.requireNonNull(reducer, "reducer is null"); + Objects.requireNonNull(seed, "seed is null"); + Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new FlowableReduceSeedSingle(this, seed, reducer)); } @@ -12499,8 +12499,8 @@ public final Single reduce(R seed, BiFunction reducer) { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single reduceWith(Supplier seedSupplier, BiFunction reducer) { - ObjectHelper.requireNonNull(seedSupplier, "seedSupplier is null"); - ObjectHelper.requireNonNull(reducer, "reducer is null"); + Objects.requireNonNull(seedSupplier, "seedSupplier is null"); + Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new FlowableReduceWithSingle(this, seedSupplier, reducer)); } @@ -12587,7 +12587,7 @@ public final Flowable repeat(long times) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable repeatUntil(BooleanSupplier stop) { - ObjectHelper.requireNonNull(stop, "stop is null"); + Objects.requireNonNull(stop, "stop is null"); return RxJavaPlugins.onAssembly(new FlowableRepeatUntil(this, stop)); } @@ -12618,7 +12618,7 @@ public final Flowable repeatUntil(BooleanSupplier stop) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable repeatWhen(final Function, ? extends Publisher> handler) { - ObjectHelper.requireNonNull(handler, "handler is null"); + Objects.requireNonNull(handler, "handler is null"); return RxJavaPlugins.onAssembly(new FlowableRepeatWhen(this, handler)); } @@ -12677,7 +12677,7 @@ public final ConnectableFlowable replay() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable replay(Function, ? extends Publisher> selector) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this), selector); } @@ -12717,7 +12717,7 @@ public final Flowable replay(Function, ? extends Publ @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, bufferSize, false), selector); } @@ -12760,7 +12760,7 @@ public final Flowable replay(Function, ? extends Publ @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize, boolean eagerTruncate) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, bufferSize, eagerTruncate), selector); } @@ -12852,10 +12852,10 @@ public final Flowable replay(Function, ? extends Publ @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(selector, "selector is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.multicastSelector( FlowableInternalHelper.replaySupplier(this, bufferSize, time, unit, scheduler, false), selector); } @@ -12907,10 +12907,10 @@ public final Flowable replay(Function, ? extends Publ @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - ObjectHelper.requireNonNull(selector, "selector is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.multicastSelector( FlowableInternalHelper.replaySupplier(this, bufferSize, time, unit, scheduler, eagerTruncate), selector); } @@ -12988,9 +12988,9 @@ public final Flowable replay(Function, ? extends Publ @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable replay(Function, ? extends Publisher> selector, final long time, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(selector, "selector is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, time, unit, scheduler, false), selector); } @@ -13033,9 +13033,9 @@ public final Flowable replay(Function, ? extends Publ @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable replay(Function, ? extends Publisher> selector, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - ObjectHelper.requireNonNull(selector, "selector is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, time, unit, scheduler, eagerTruncate), selector); } @@ -13194,8 +13194,8 @@ public final ConnectableFlowable replay(int bufferSize, long time, TimeUnit u @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableFlowable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.create(this, time, unit, scheduler, bufferSize, false); } @@ -13244,8 +13244,8 @@ public final ConnectableFlowable replay(final int bufferSize, final long time @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableFlowable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.create(this, time, unit, scheduler, bufferSize, eagerTruncate); } @@ -13318,8 +13318,8 @@ public final ConnectableFlowable replay(long time, TimeUnit unit) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableFlowable replay(final long time, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.create(this, time, unit, scheduler, false); } @@ -13359,8 +13359,8 @@ public final ConnectableFlowable replay(final long time, final TimeUnit unit, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableFlowable replay(final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.create(this, time, unit, scheduler, eagerTruncate); } @@ -13420,7 +13420,7 @@ public final Flowable retry() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable retry(BiPredicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new FlowableRetryBiPredicate(this, predicate)); } @@ -13481,7 +13481,7 @@ public final Flowable retry(long times, Predicate predicat if (times < 0) { throw new IllegalArgumentException("times >= 0 required but it was " + times); } - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new FlowableRetryPredicate(this, times, predicate)); } @@ -13523,7 +13523,7 @@ public final Flowable retry(Predicate predicate) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable retryUntil(final BooleanSupplier stop) { - ObjectHelper.requireNonNull(stop, "stop is null"); + Objects.requireNonNull(stop, "stop is null"); return retry(Long.MAX_VALUE, Functions.predicateReverseFor(stop)); } @@ -13610,7 +13610,7 @@ public final Flowable retryUntil(final BooleanSupplier stop) { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable retryWhen( final Function, ? extends Publisher> handler) { - ObjectHelper.requireNonNull(handler, "handler is null"); + Objects.requireNonNull(handler, "handler is null"); return RxJavaPlugins.onAssembly(new FlowableRetryWhen(this, handler)); } @@ -13632,7 +13632,7 @@ public final Flowable retryWhen( @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final void safeSubscribe(Subscriber s) { - ObjectHelper.requireNonNull(s, "s is null"); + Objects.requireNonNull(s, "s is null"); if (s instanceof SafeSubscriber) { subscribe((SafeSubscriber)s); } else { @@ -13733,8 +13733,8 @@ public final Flowable sample(long period, TimeUnit unit, boolean emitLast) { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableSampleTimed(this, period, unit, scheduler, false)); } @@ -13774,8 +13774,8 @@ public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler) @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler, boolean emitLast) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableSampleTimed(this, period, unit, scheduler, emitLast)); } @@ -13806,7 +13806,7 @@ public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler, @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable sample(Publisher sampler) { - ObjectHelper.requireNonNull(sampler, "sampler is null"); + Objects.requireNonNull(sampler, "sampler is null"); return RxJavaPlugins.onAssembly(new FlowableSamplePublisher(this, sampler, false)); } @@ -13844,7 +13844,7 @@ public final Flowable sample(Publisher sampler) { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable sample(Publisher sampler, boolean emitLast) { - ObjectHelper.requireNonNull(sampler, "sampler is null"); + Objects.requireNonNull(sampler, "sampler is null"); return RxJavaPlugins.onAssembly(new FlowableSamplePublisher(this, sampler, emitLast)); } @@ -13877,7 +13877,7 @@ public final Flowable sample(Publisher sampler, boolean emitLast) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable scan(BiFunction accumulator) { - ObjectHelper.requireNonNull(accumulator, "accumulator is null"); + Objects.requireNonNull(accumulator, "accumulator is null"); return RxJavaPlugins.onAssembly(new FlowableScan(this, accumulator)); } @@ -13931,7 +13931,7 @@ public final Flowable scan(BiFunction accumulator) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable scan(final R initialValue, BiFunction accumulator) { - ObjectHelper.requireNonNull(initialValue, "initialValue is null"); + Objects.requireNonNull(initialValue, "initialValue is null"); return scanWith(Functions.justSupplier(initialValue), accumulator); } @@ -13971,8 +13971,8 @@ public final Flowable scan(final R initialValue, BiFunction Flowable scanWith(Supplier seedSupplier, BiFunction accumulator) { - ObjectHelper.requireNonNull(seedSupplier, "seedSupplier is null"); - ObjectHelper.requireNonNull(accumulator, "accumulator is null"); + Objects.requireNonNull(seedSupplier, "seedSupplier is null"); + Objects.requireNonNull(accumulator, "accumulator is null"); return RxJavaPlugins.onAssembly(new FlowableScanSeed(this, seedSupplier, accumulator)); } @@ -14083,7 +14083,7 @@ public final Maybe singleElement() { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single single(T defaultItem) { - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new FlowableSingleSingle(this, defaultItem)); } @@ -14405,8 +14405,8 @@ public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler, @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); // the internal buffer holds pairs of (timestamp, value) so double the default buffer size int s = bufferSize << 1; @@ -14439,7 +14439,7 @@ public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable skipUntil(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableSkipUntil(this, other)); } @@ -14467,7 +14467,7 @@ public final Flowable skipUntil(Publisher other) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable skipWhile(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new FlowableSkipWhile(this, predicate)); } /** @@ -14523,7 +14523,7 @@ public final Flowable sorted() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable sorted(Comparator sortFunction) { - ObjectHelper.requireNonNull(sortFunction, "sortFunction"); + Objects.requireNonNull(sortFunction, "sortFunction"); return toList().toFlowable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.>identity()); } @@ -14582,7 +14582,7 @@ public final Flowable startWithIterable(Iterable items) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable startWith(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return concatArray(other, this); } @@ -14614,7 +14614,7 @@ public final Flowable startWith(Publisher other) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable startWithItem(T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return concatArray(just(item), this); } @@ -14767,9 +14767,9 @@ public final Disposable subscribe(Consumer onNext, Consumer onNext, Consumer onError, Action onComplete) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); LambdaSubscriber ls = new LambdaSubscriber(onNext, onError, onComplete, FlowableInternalHelper.RequestMax.INSTANCE); @@ -14785,7 +14785,7 @@ public final void subscribe(Subscriber s) { if (s instanceof FlowableSubscriber) { subscribe((FlowableSubscriber)s); } else { - ObjectHelper.requireNonNull(s, "s is null"); + Objects.requireNonNull(s, "s is null"); subscribe(new StrictSubscriber(s)); } } @@ -14830,11 +14830,11 @@ public final void subscribe(Subscriber s) { @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) public final void subscribe(FlowableSubscriber s) { - ObjectHelper.requireNonNull(s, "s is null"); + Objects.requireNonNull(s, "s is null"); try { Subscriber z = RxJavaPlugins.onSubscribe(this, s); - ObjectHelper.requireNonNull(z, "The RxJavaPlugins.onSubscribe hook returned a null FlowableSubscriber. Please check the handler provided to RxJavaPlugins.setOnFlowableSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); + Objects.requireNonNull(z, "The RxJavaPlugins.onSubscribe hook returned a null FlowableSubscriber. Please check the handler provided to RxJavaPlugins.setOnFlowableSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); subscribeActual(z); } catch (NullPointerException e) { // NOPMD @@ -14926,7 +14926,7 @@ public final > E subscribeWith(E subscriber) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable subscribeOn(@NonNull Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return subscribeOn(scheduler, !(this instanceof FlowableCreate)); } @@ -14964,7 +14964,7 @@ public final Flowable subscribeOn(@NonNull Scheduler scheduler) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable subscribeOn(@NonNull Scheduler scheduler, boolean requestOn) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableSubscribeOn(this, scheduler, requestOn)); } @@ -14995,7 +14995,7 @@ public final Flowable subscribeOn(@NonNull Scheduler scheduler, boolean reque @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable switchIfEmpty(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableSwitchIfEmpty(this, other)); } @@ -15111,7 +15111,7 @@ public final Flowable switchMap(Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableSwitchMapCompletable(this, mapper, false)); } @@ -15158,7 +15158,7 @@ public final Completable switchMapCompletable(@NonNull Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableSwitchMapCompletable(this, mapper, true)); } @@ -15237,7 +15237,7 @@ public final Flowable switchMapDelayError(Function Flowable switchMap0(Function> mapper, int bufferSize, boolean delayError) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (this instanceof ScalarSupplier) { @SuppressWarnings("unchecked") @@ -15287,7 +15287,7 @@ Flowable switchMap0(Function> @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable switchMapMaybe(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableSwitchMapMaybe(this, mapper, false)); } @@ -15318,7 +15318,7 @@ public final Flowable switchMapMaybe(@NonNull Function Flowable switchMapMaybeDelayError(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableSwitchMapMaybe(this, mapper, true)); } @@ -15359,7 +15359,7 @@ public final Flowable switchMapMaybeDelayError(@NonNull Function Flowable switchMapSingle(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableSwitchMapSingle(this, mapper, false)); } @@ -15390,7 +15390,7 @@ public final Flowable switchMapSingle(@NonNull Function Flowable switchMapSingleDelayError(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableSwitchMapSingle(this, mapper, true)); } @@ -15650,8 +15650,8 @@ public final Flowable takeLast(long count, long time, TimeUnit unit, Schedule @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable takeLast(long count, long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); @@ -15862,7 +15862,7 @@ public final Flowable takeLast(long time, TimeUnit unit, Scheduler scheduler, @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable takeUntil(Predicate stopPredicate) { - ObjectHelper.requireNonNull(stopPredicate, "stopPredicate is null"); + Objects.requireNonNull(stopPredicate, "stopPredicate is null"); return RxJavaPlugins.onAssembly(new FlowableTakeUntilPredicate(this, stopPredicate)); } @@ -15892,7 +15892,7 @@ public final Flowable takeUntil(Predicate stopPredicate) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable takeUntil(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new FlowableTakeUntil(this, other)); } @@ -15921,7 +15921,7 @@ public final Flowable takeUntil(Publisher other) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable takeWhile(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new FlowableTakeWhile(this, predicate)); } @@ -15986,8 +15986,8 @@ public final Flowable throttleFirst(long windowDuration, TimeUnit unit) { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable throttleFirst(long skipDuration, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableThrottleFirstTimed(this, skipDuration, unit, scheduler)); } @@ -16206,8 +16206,8 @@ public final Flowable throttleLatest(long timeout, TimeUnit unit, Scheduler s @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable throttleLatest(long timeout, TimeUnit unit, Scheduler scheduler, boolean emitLast) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableThrottleLatest(this, timeout, unit, scheduler, emitLast)); } @@ -16383,8 +16383,8 @@ public final Flowable> timeInterval(TimeUnit unit) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. public final Flowable> timeInterval(TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableTimeInterval(this, unit, scheduler)); } @@ -16460,7 +16460,7 @@ public final Flowable timeout(Function> @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable timeout(Function> itemTimeoutIndicator, Flowable other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(null, itemTimeoutIndicator, other); } @@ -16523,7 +16523,7 @@ public final Flowable timeout(long timeout, TimeUnit timeUnit) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) public final Flowable timeout(long timeout, TimeUnit timeUnit, Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, Schedulers.computation()); } @@ -16561,7 +16561,7 @@ public final Flowable timeout(long timeout, TimeUnit timeUnit, Publisher timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler, Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, scheduler); } @@ -16634,7 +16634,7 @@ public final Flowable timeout(long timeout, TimeUnit timeUnit, Scheduler sche @SchedulerSupport(SchedulerSupport.NONE) public final Flowable timeout(Publisher firstTimeoutIndicator, Function> itemTimeoutIndicator) { - ObjectHelper.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); + Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, null); } @@ -16682,15 +16682,15 @@ public final Flowable timeout( Publisher firstTimeoutIndicator, Function> itemTimeoutIndicator, Publisher other) { - ObjectHelper.requireNonNull(firstTimeoutIndicator, "firstTimeoutSelector is null"); - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutSelector is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, other); } private Flowable timeout0(long timeout, TimeUnit timeUnit, Publisher other, Scheduler scheduler) { - ObjectHelper.requireNonNull(timeUnit, "timeUnit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(timeUnit, "timeUnit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableTimeoutTimed(this, timeout, timeUnit, scheduler, other)); } @@ -16698,7 +16698,7 @@ private Flowable timeout0( Publisher firstTimeoutIndicator, Function> itemTimeoutIndicator, Publisher other) { - ObjectHelper.requireNonNull(itemTimeoutIndicator, "itemTimeoutIndicator is null"); + Objects.requireNonNull(itemTimeoutIndicator, "itemTimeoutIndicator is null"); return RxJavaPlugins.onAssembly(new FlowableTimeout(this, firstTimeoutIndicator, itemTimeoutIndicator, other)); } @@ -16804,8 +16804,8 @@ public final Flowable> timestamp(TimeUnit unit) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. public final Flowable> timestamp(final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return map(Functions.timestampWith(unit, scheduler)); } @@ -16830,7 +16830,7 @@ public final Flowable> timestamp(final TimeUnit unit, final Scheduler s @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) public final R to(@NonNull FlowableConverter converter) { - return ObjectHelper.requireNonNull(converter, "converter is null").apply(this); + return Objects.requireNonNull(converter, "converter is null").apply(this); } /** @@ -16938,7 +16938,7 @@ public final Single> toList(final int capacityHint) { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final > Single toList(Supplier collectionSupplier) { - ObjectHelper.requireNonNull(collectionSupplier, "collectionSupplier is null"); + Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); return RxJavaPlugins.onAssembly(new FlowableToListSingle(this, collectionSupplier)); } @@ -16973,7 +16973,7 @@ public final > Single toList(Supplier coll @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single> toMap(final Function keySelector) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); } @@ -17012,8 +17012,8 @@ public final Single> toMap(final Function @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single> toMap(final Function keySelector, final Function valueSelector) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); } @@ -17053,8 +17053,8 @@ public final Single> toMap(final Function Single> toMap(final Function keySelector, final Function valueSelector, final Supplier> mapSupplier) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); return collect(mapSupplier, Functions.toMapKeyValueSelector(keySelector, valueSelector)); } @@ -17169,10 +17169,10 @@ public final Single>> toMultimap( final Function valueSelector, final Supplier>> mapSupplier, final Function> collectionFactory) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); - ObjectHelper.requireNonNull(mapSupplier, "mapSupplier is null"); - ObjectHelper.requireNonNull(collectionFactory, "collectionFactory is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(mapSupplier, "mapSupplier is null"); + Objects.requireNonNull(collectionFactory, "collectionFactory is null"); return collect(mapSupplier, Functions.toMultimapKeyValueSelector(keySelector, valueSelector, collectionFactory)); } @@ -17297,7 +17297,7 @@ public final Single> toSortedList() { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single> toSortedList(final Comparator comparator) { - ObjectHelper.requireNonNull(comparator, "comparator is null"); + Objects.requireNonNull(comparator, "comparator is null"); return toList().map(Functions.listSorter(comparator)); } @@ -17333,7 +17333,7 @@ public final Single> toSortedList(final Comparator comparator @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) public final Single> toSortedList(final Comparator comparator, int capacityHint) { - ObjectHelper.requireNonNull(comparator, "comparator is null"); + Objects.requireNonNull(comparator, "comparator is null"); return toList(capacityHint).map(Functions.listSorter(comparator)); } @@ -17395,7 +17395,7 @@ public final Single> toSortedList(int capacityHint) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable unsubscribeOn(Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableUnsubscribeOn(this, scheduler)); } @@ -17638,8 +17638,8 @@ public final Flowable> window(long timespan, long timeskip, TimeUnit ObjectHelper.verifyPositive(bufferSize, "bufferSize"); ObjectHelper.verifyPositive(timespan, "timespan"); ObjectHelper.verifyPositive(timeskip, "timeskip"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new FlowableWindowTimed(this, timespan, timeskip, unit, scheduler, Long.MAX_VALUE, bufferSize, false)); } @@ -17952,8 +17952,8 @@ public final Flowable> window( long timespan, TimeUnit unit, Scheduler scheduler, long count, boolean restart, int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(count, "count"); return RxJavaPlugins.onAssembly(new FlowableWindowTimed(this, timespan, timespan, unit, scheduler, count, bufferSize, restart)); } @@ -18028,7 +18028,7 @@ public final Flowable> window(Publisher boundaryIndicator) { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable> window(Publisher boundaryIndicator, int bufferSize) { - ObjectHelper.requireNonNull(boundaryIndicator, "boundaryIndicator is null"); + Objects.requireNonNull(boundaryIndicator, "boundaryIndicator is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableWindowBoundary(this, boundaryIndicator, bufferSize)); } @@ -18117,8 +18117,8 @@ public final Flowable> window( public final Flowable> window( Publisher openingIndicator, Function> closingIndicator, int bufferSize) { - ObjectHelper.requireNonNull(openingIndicator, "openingIndicator is null"); - ObjectHelper.requireNonNull(closingIndicator, "closingIndicator is null"); + Objects.requireNonNull(openingIndicator, "openingIndicator is null"); + Objects.requireNonNull(closingIndicator, "closingIndicator is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new FlowableWindowBoundarySelector(this, openingIndicator, closingIndicator, bufferSize)); } @@ -18157,8 +18157,8 @@ public final Flowable> window( @SchedulerSupport(SchedulerSupport.NONE) public final Flowable withLatestFrom(Publisher other, BiFunction combiner) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new FlowableWithLatestFrom(this, combiner, other)); } @@ -18195,8 +18195,8 @@ public final Flowable withLatestFrom(Publisher other, @SchedulerSupport(SchedulerSupport.NONE) public final Flowable withLatestFrom(Publisher source1, Publisher source2, Function3 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new Publisher[] { source1, source2 }, f); } @@ -18237,9 +18237,9 @@ public final Flowable withLatestFrom( Publisher source1, Publisher source2, Publisher source3, Function4 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new Publisher[] { source1, source2, source3 }, f); } @@ -18282,10 +18282,10 @@ public final Flowable withLatestFrom( Publisher source1, Publisher source2, Publisher source3, Publisher source4, Function5 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new Publisher[] { source1, source2, source3, source4 }, f); } @@ -18318,8 +18318,8 @@ public final Flowable withLatestFrom( @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable withLatestFrom(Publisher[] others, Function combiner) { - ObjectHelper.requireNonNull(others, "others is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(others, "others is null"); + Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new FlowableWithLatestFromMany(this, others, combiner)); } @@ -18351,8 +18351,8 @@ public final Flowable withLatestFrom(Publisher[] others, Function Flowable withLatestFrom(Iterable> others, Function combiner) { - ObjectHelper.requireNonNull(others, "others is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(others, "others is null"); + Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new FlowableWithLatestFromMany(this, others, combiner)); } @@ -18391,8 +18391,8 @@ public final Flowable withLatestFrom(Iterable> oth @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable zipWith(Iterable other, BiFunction zipper) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(zipper, "zipper is null"); return RxJavaPlugins.onAssembly(new FlowableZipIterable(this, other, zipper)); } @@ -18440,7 +18440,7 @@ public final Flowable zipWith(Iterable other, BiFunction Flowable zipWith(Publisher other, BiFunction zipper) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return zip(this, other, zipper); } @@ -18653,7 +18653,7 @@ public final TestSubscriber test(long initialRequest, boolean cancel) { // No @SchedulerSupport(SchedulerSupport.NONE) @NonNull public static Flowable<@NonNull T> fromOptional(@NonNull Optional optional) { - ObjectHelper.requireNonNull(optional, "optional is null"); + Objects.requireNonNull(optional, "optional is null"); return optional.map(Flowable::just).orElseGet(Flowable::empty); } @@ -18690,7 +18690,7 @@ public final TestSubscriber test(long initialRequest, boolean cancel) { // No @SchedulerSupport(SchedulerSupport.NONE) @NonNull public static Flowable<@NonNull T> fromCompletionStage(@NonNull CompletionStage stage) { - ObjectHelper.requireNonNull(stage, "stage is null"); + Objects.requireNonNull(stage, "stage is null"); return RxJavaPlugins.onAssembly(new FlowableFromCompletionStage<>(stage)); } @@ -18736,7 +18736,7 @@ public final TestSubscriber test(long initialRequest, boolean cancel) { // No @SchedulerSupport(SchedulerSupport.NONE) @NonNull public static Flowable<@NonNull T> fromStream(@NonNull Stream stream) { - ObjectHelper.requireNonNull(stream, "stream is null"); + Objects.requireNonNull(stream, "stream is null"); return RxJavaPlugins.onAssembly(new FlowableFromStream<>(stream)); } } diff --git a/src/main/java/io/reactivex/rxjava3/core/Maybe.java b/src/main/java/io/reactivex/rxjava3/core/Maybe.java index 9bcf753e85..e3f4641ad3 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Maybe.java +++ b/src/main/java/io/reactivex/rxjava3/core/Maybe.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.core; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.concurrent.*; import org.reactivestreams.*; @@ -127,7 +128,7 @@ public abstract class Maybe implements MaybeSource { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe amb(final Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new MaybeAmb(null, sources)); } @@ -178,7 +179,7 @@ public static Maybe ambArray(final MaybeSource... sources) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Flowable concat(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new MaybeConcatIterable(sources)); } @@ -207,8 +208,8 @@ public static Flowable concat(Iterable @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") public static Flowable concat(MaybeSource source1, MaybeSource source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return concatArray(source1, source2); } @@ -240,9 +241,9 @@ public static Flowable concat(MaybeSource source1, MaybeSour @SuppressWarnings("unchecked") public static Flowable concat( MaybeSource source1, MaybeSource source2, MaybeSource source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return concatArray(source1, source2, source3); } @@ -276,10 +277,10 @@ public static Flowable concat( @SuppressWarnings("unchecked") public static Flowable concat( MaybeSource source1, MaybeSource source2, MaybeSource source3, MaybeSource source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return concatArray(source1, source2, source3, source4); } @@ -331,7 +332,7 @@ public static Flowable concat(Publisher Flowable concat(Publisher> sources, int prefetch) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapPublisher(sources, MaybeToPublisher.instance(), prefetch, ErrorMode.IMMEDIATE)); } @@ -356,7 +357,7 @@ public static Flowable concat(Publisher Flowable concatArray(MaybeSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return Flowable.empty(); } @@ -444,7 +445,7 @@ public static Flowable concatArrayEager(MaybeSource... sourc @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Flowable concatDelayError(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return Flowable.fromIterable(sources).concatMapDelayError((Function)MaybeToPublisher.instance()); } @@ -574,7 +575,7 @@ public static Flowable concatEager(Publisher Maybe create(MaybeOnSubscribe onSubscribe) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); return RxJavaPlugins.onAssembly(new MaybeCreate(onSubscribe)); } @@ -594,7 +595,7 @@ public static Maybe create(MaybeOnSubscribe onSubscribe) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe defer(final Supplier> maybeSupplier) { - ObjectHelper.requireNonNull(maybeSupplier, "maybeSupplier is null"); + Objects.requireNonNull(maybeSupplier, "maybeSupplier is null"); return RxJavaPlugins.onAssembly(new MaybeDefer(maybeSupplier)); } @@ -639,7 +640,7 @@ public static Maybe empty() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe error(Throwable exception) { - ObjectHelper.requireNonNull(exception, "exception is null"); + Objects.requireNonNull(exception, "exception is null"); return RxJavaPlugins.onAssembly(new MaybeError(exception)); } @@ -665,7 +666,7 @@ public static Maybe error(Throwable exception) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe error(Supplier supplier) { - ObjectHelper.requireNonNull(supplier, "errorSupplier is null"); + Objects.requireNonNull(supplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new MaybeErrorCallable(supplier)); } @@ -692,7 +693,7 @@ public static Maybe error(Supplier supplier) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromAction(final Action run) { - ObjectHelper.requireNonNull(run, "run is null"); + Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new MaybeFromAction(run)); } @@ -712,7 +713,7 @@ public static Maybe fromAction(final Action run) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromCompletable(CompletableSource completableSource) { - ObjectHelper.requireNonNull(completableSource, "completableSource is null"); + Objects.requireNonNull(completableSource, "completableSource is null"); return RxJavaPlugins.onAssembly(new MaybeFromCompletable(completableSource)); } @@ -732,7 +733,7 @@ public static Maybe fromCompletable(CompletableSource completableSource) @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromSingle(SingleSource singleSource) { - ObjectHelper.requireNonNull(singleSource, "singleSource is null"); + Objects.requireNonNull(singleSource, "singleSource is null"); return RxJavaPlugins.onAssembly(new MaybeFromSingle(singleSource)); } @@ -776,7 +777,7 @@ public static Maybe fromSingle(SingleSource singleSource) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromCallable(@NonNull final Callable callable) { - ObjectHelper.requireNonNull(callable, "callable is null"); + Objects.requireNonNull(callable, "callable is null"); return RxJavaPlugins.onAssembly(new MaybeFromCallable(callable)); } @@ -810,7 +811,7 @@ public static Maybe fromCallable(@NonNull final Callable cal @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromFuture(Future future) { - ObjectHelper.requireNonNull(future, "future is null"); + Objects.requireNonNull(future, "future is null"); return RxJavaPlugins.onAssembly(new MaybeFromFuture(future, 0L, null)); } @@ -848,8 +849,8 @@ public static Maybe fromFuture(Future future) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromFuture(Future future, long timeout, TimeUnit unit) { - ObjectHelper.requireNonNull(future, "future is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(future, "future is null"); + Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new MaybeFromFuture(future, timeout, unit)); } @@ -869,7 +870,7 @@ public static Maybe fromFuture(Future future, long timeout, @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromRunnable(final Runnable run) { - ObjectHelper.requireNonNull(run, "run is null"); + Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new MaybeFromRunnable(run)); } @@ -916,7 +917,7 @@ public static Maybe fromRunnable(final Runnable run) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe fromSupplier(@NonNull final Supplier supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new MaybeFromSupplier(supplier)); } @@ -943,7 +944,7 @@ public static Maybe fromSupplier(@NonNull final Supplier sup @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe just(T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return RxJavaPlugins.onAssembly(new MaybeJust(item)); } @@ -1049,7 +1050,7 @@ public static Flowable merge(Publisher @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "unchecked", "rawtypes" }) public static Flowable merge(Publisher> sources, int maxConcurrency) { - ObjectHelper.requireNonNull(sources, "source is null"); + Objects.requireNonNull(sources, "source is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, MaybeToPublisher.instance(), false, maxConcurrency, 1)); } @@ -1082,7 +1083,7 @@ public static Flowable merge(Publisher @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "unchecked", "rawtypes" }) public static Maybe merge(MaybeSource> source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new MaybeFlatten(source, Functions.identity())); } @@ -1130,8 +1131,8 @@ public static Maybe merge(MaybeSource> public static Flowable merge( MaybeSource source1, MaybeSource source2 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return mergeArray(source1, source2); } @@ -1182,9 +1183,9 @@ public static Flowable merge( MaybeSource source1, MaybeSource source2, MaybeSource source3 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return mergeArray(source1, source2, source3); } @@ -1237,10 +1238,10 @@ public static Flowable merge( MaybeSource source1, MaybeSource source2, MaybeSource source3, MaybeSource source4 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return mergeArray(source1, source2, source3, source4); } @@ -1277,7 +1278,7 @@ public static Flowable merge( @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") public static Flowable mergeArray(MaybeSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return Flowable.empty(); } @@ -1431,7 +1432,7 @@ public static Flowable mergeDelayError(Publisher Flowable mergeDelayError(Publisher> sources, int maxConcurrency) { - ObjectHelper.requireNonNull(sources, "source is null"); + Objects.requireNonNull(sources, "source is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, MaybeToPublisher.instance(), true, maxConcurrency, 1)); } @@ -1470,8 +1471,8 @@ public static Flowable mergeDelayError(Publisher Flowable mergeDelayError(MaybeSource source1, MaybeSource source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return mergeArrayDelayError(source1, source2); } @@ -1513,9 +1514,9 @@ public static Flowable mergeDelayError(MaybeSource source1, @SchedulerSupport(SchedulerSupport.NONE) public static Flowable mergeDelayError(MaybeSource source1, MaybeSource source2, MaybeSource source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return mergeArrayDelayError(source1, source2, source3); } @@ -1560,10 +1561,10 @@ public static Flowable mergeDelayError(MaybeSource source1, public static Flowable mergeDelayError( MaybeSource source1, MaybeSource source2, MaybeSource source3, MaybeSource source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return mergeArrayDelayError(source1, source2, source3, source4); } @@ -1643,9 +1644,9 @@ public static Single sequenceEqual(MaybeSource source1 @SchedulerSupport(SchedulerSupport.NONE) public static Single sequenceEqual(MaybeSource source1, MaybeSource source2, BiPredicate isEqual) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(isEqual, "isEqual is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(isEqual, "isEqual is null"); return RxJavaPlugins.onAssembly(new MaybeEqualSingle(source1, source2, isEqual)); } @@ -1693,8 +1694,8 @@ public static Maybe timer(long delay, TimeUnit unit) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public static Maybe timer(long delay, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new MaybeTimer(Math.max(0L, delay), unit, scheduler)); } @@ -1717,7 +1718,7 @@ public static Maybe unsafeCreate(MaybeSource onSubscribe) { if (onSubscribe instanceof Maybe) { throw new IllegalArgumentException("unsafeCreate(Maybe) should be upgraded"); } - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); return RxJavaPlugins.onAssembly(new MaybeUnsafeCreate(onSubscribe)); } @@ -1785,9 +1786,9 @@ public static Maybe using(Supplier resourceSupplier, public static Maybe using(Supplier resourceSupplier, Function> sourceSupplier, Consumer resourceDisposer, boolean eager) { - ObjectHelper.requireNonNull(resourceSupplier, "resourceSupplier is null"); - ObjectHelper.requireNonNull(sourceSupplier, "sourceSupplier is null"); - ObjectHelper.requireNonNull(resourceDisposer, "disposer is null"); + Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); + Objects.requireNonNull(sourceSupplier, "sourceSupplier is null"); + Objects.requireNonNull(resourceDisposer, "disposer is null"); return RxJavaPlugins.onAssembly(new MaybeUsing(resourceSupplier, sourceSupplier, resourceDisposer, eager)); } @@ -1809,7 +1810,7 @@ public static Maybe wrap(MaybeSource source) { if (source instanceof Maybe) { return RxJavaPlugins.onAssembly((Maybe)source); } - ObjectHelper.requireNonNull(source, "onSubscribe is null"); + Objects.requireNonNull(source, "onSubscribe is null"); return RxJavaPlugins.onAssembly(new MaybeUnsafeCreate(source)); } @@ -1844,8 +1845,8 @@ public static Maybe wrap(MaybeSource source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip(Iterable> sources, Function zipper) { - ObjectHelper.requireNonNull(zipper, "zipper is null"); - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new MaybeZipIterable(sources, zipper)); } @@ -1882,8 +1883,8 @@ public static Maybe zip(Iterable> s public static Maybe zip( MaybeSource source1, MaybeSource source2, BiFunction zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), source1, source2); } @@ -1923,9 +1924,9 @@ public static Maybe zip( public static Maybe zip( MaybeSource source1, MaybeSource source2, MaybeSource source3, Function3 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3); } @@ -1969,10 +1970,10 @@ public static Maybe zip( MaybeSource source1, MaybeSource source2, MaybeSource source3, MaybeSource source4, Function4 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4); } @@ -2019,11 +2020,11 @@ public static Maybe zip( MaybeSource source1, MaybeSource source2, MaybeSource source3, MaybeSource source4, MaybeSource source5, Function5 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5); } @@ -2073,12 +2074,12 @@ public static Maybe zip( MaybeSource source1, MaybeSource source2, MaybeSource source3, MaybeSource source4, MaybeSource source5, MaybeSource source6, Function6 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6); } @@ -2132,13 +2133,13 @@ public static Maybe zip( MaybeSource source4, MaybeSource source5, MaybeSource source6, MaybeSource source7, Function7 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7); } @@ -2195,14 +2196,14 @@ public static Maybe zip( MaybeSource source4, MaybeSource source5, MaybeSource source6, MaybeSource source7, MaybeSource source8, Function8 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -2262,15 +2263,15 @@ public static Maybe zip( MaybeSource source7, MaybeSource source8, MaybeSource source9, Function9 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); - ObjectHelper.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source9, "source9 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8, source9); } @@ -2306,11 +2307,11 @@ public static Maybe zip( @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zipArray(Function zipper, MaybeSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return empty(); } - ObjectHelper.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(zipper, "zipper is null"); return RxJavaPlugins.onAssembly(new MaybeZipArray(sources, zipper)); } @@ -2339,7 +2340,7 @@ public static Maybe zipArray(Function z @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe ambWith(MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -2381,7 +2382,7 @@ public final T blockingGet() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final T blockingGet(T defaultValue) { - ObjectHelper.requireNonNull(defaultValue, "defaultValue is null"); + Objects.requireNonNull(defaultValue, "defaultValue is null"); BlockingMultiObserver observer = new BlockingMultiObserver(); subscribe(observer); return observer.blockingGet(defaultValue); @@ -2429,7 +2430,7 @@ public final Maybe cache() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe cast(final Class clazz) { - ObjectHelper.requireNonNull(clazz, "clazz is null"); + Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -2455,7 +2456,7 @@ public final Maybe cast(final Class clazz) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Maybe compose(MaybeTransformer transformer) { - return wrap(((MaybeTransformer) ObjectHelper.requireNonNull(transformer, "transformer is null")).apply(this)); + return wrap(((MaybeTransformer) Objects.requireNonNull(transformer, "transformer is null")).apply(this)); } /** @@ -2478,7 +2479,7 @@ public final Maybe compose(MaybeTransformer trans @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe concatMap(Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatten(this, mapper)); } @@ -2505,7 +2506,7 @@ public final Maybe concatMap(Function concatWith(MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return concat(this, other); } @@ -2529,7 +2530,7 @@ public final Flowable concatWith(MaybeSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single contains(final Object item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return RxJavaPlugins.onAssembly(new MaybeContains(this, item)); } @@ -2573,7 +2574,7 @@ public final Single count() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single defaultIfEmpty(T defaultItem) { - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new MaybeToSingle(this, defaultItem)); } @@ -2624,8 +2625,8 @@ public final Maybe delay(long delay, TimeUnit unit) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Maybe delay(long delay, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new MaybeDelay(this, Math.max(0L, delay), unit, scheduler)); } @@ -2656,7 +2657,7 @@ public final Maybe delay(long delay, TimeUnit unit, Scheduler scheduler) { @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) public final Maybe delay(Publisher delayIndicator) { - ObjectHelper.requireNonNull(delayIndicator, "delayIndicator is null"); + Objects.requireNonNull(delayIndicator, "delayIndicator is null"); return RxJavaPlugins.onAssembly(new MaybeDelayOtherPublisher(this, delayIndicator)); } @@ -2683,7 +2684,7 @@ public final Maybe delay(Publisher delayIndicator) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe delaySubscription(Publisher subscriptionIndicator) { - ObjectHelper.requireNonNull(subscriptionIndicator, "subscriptionIndicator is null"); + Objects.requireNonNull(subscriptionIndicator, "subscriptionIndicator is null"); return RxJavaPlugins.onAssembly(new MaybeDelaySubscriptionOtherPublisher(this, subscriptionIndicator)); } @@ -2753,7 +2754,7 @@ public final Maybe delaySubscription(long delay, TimeUnit unit, Scheduler sch @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe doAfterSuccess(Consumer onAfterSuccess) { - ObjectHelper.requireNonNull(onAfterSuccess, "onAfterSuccess is null"); + Objects.requireNonNull(onAfterSuccess, "onAfterSuccess is null"); return RxJavaPlugins.onAssembly(new MaybeDoAfterSuccess(this, onAfterSuccess)); } @@ -2783,7 +2784,7 @@ public final Maybe doAfterTerminate(Action onAfterTerminate) { Functions.emptyConsumer(), // onSuccess Functions.emptyConsumer(), // onError Functions.EMPTY_ACTION, // onComplete - ObjectHelper.requireNonNull(onAfterTerminate, "onAfterTerminate is null"), + Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"), Functions.EMPTY_ACTION // dispose )); } @@ -2808,7 +2809,7 @@ public final Maybe doAfterTerminate(Action onAfterTerminate) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe doFinally(Action onFinally) { - ObjectHelper.requireNonNull(onFinally, "onFinally is null"); + Objects.requireNonNull(onFinally, "onFinally is null"); return RxJavaPlugins.onAssembly(new MaybeDoFinally(this, onFinally)); } @@ -2833,7 +2834,7 @@ public final Maybe doOnDispose(Action onDispose) { Functions.emptyConsumer(), // onError Functions.EMPTY_ACTION, // onComplete Functions.EMPTY_ACTION, // (onSuccess | onError | onComplete) after - ObjectHelper.requireNonNull(onDispose, "onDispose is null") + Objects.requireNonNull(onDispose, "onDispose is null") )); } @@ -2859,7 +2860,7 @@ public final Maybe doOnComplete(Action onComplete) { Functions.emptyConsumer(), // onSubscribe Functions.emptyConsumer(), // onSuccess Functions.emptyConsumer(), // onError - ObjectHelper.requireNonNull(onComplete, "onComplete is null"), + Objects.requireNonNull(onComplete, "onComplete is null"), Functions.EMPTY_ACTION, // (onSuccess | onError | onComplete) Functions.EMPTY_ACTION // dispose )); @@ -2884,7 +2885,7 @@ public final Maybe doOnError(Consumer onError) { return RxJavaPlugins.onAssembly(new MaybePeek(this, Functions.emptyConsumer(), // onSubscribe Functions.emptyConsumer(), // onSuccess - ObjectHelper.requireNonNull(onError, "onError is null"), + Objects.requireNonNull(onError, "onError is null"), Functions.EMPTY_ACTION, // onComplete Functions.EMPTY_ACTION, // (onSuccess | onError | onComplete) Functions.EMPTY_ACTION // dispose @@ -2910,7 +2911,7 @@ public final Maybe doOnError(Consumer onError) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Maybe doOnEvent(BiConsumer onEvent) { - ObjectHelper.requireNonNull(onEvent, "onEvent is null"); + Objects.requireNonNull(onEvent, "onEvent is null"); return RxJavaPlugins.onAssembly(new MaybeDoOnEvent(this, onEvent)); } @@ -2929,7 +2930,7 @@ public final Maybe doOnEvent(BiConsumer onEvent @SchedulerSupport(SchedulerSupport.NONE) public final Maybe doOnSubscribe(Consumer onSubscribe) { return RxJavaPlugins.onAssembly(new MaybePeek(this, - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"), + Objects.requireNonNull(onSubscribe, "onSubscribe is null"), Functions.emptyConsumer(), // onSuccess Functions.emptyConsumer(), // onError Functions.EMPTY_ACTION, // onComplete @@ -2961,7 +2962,7 @@ public final Maybe doOnSubscribe(Consumer onSubscribe) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe doOnTerminate(final Action onTerminate) { - ObjectHelper.requireNonNull(onTerminate, "onTerminate is null"); + Objects.requireNonNull(onTerminate, "onTerminate is null"); return RxJavaPlugins.onAssembly(new MaybeDoOnTerminate(this, onTerminate)); } @@ -2983,7 +2984,7 @@ public final Maybe doOnTerminate(final Action onTerminate) { public final Maybe doOnSuccess(Consumer onSuccess) { return RxJavaPlugins.onAssembly(new MaybePeek(this, Functions.emptyConsumer(), // onSubscribe - ObjectHelper.requireNonNull(onSuccess, "onSuccess is null"), + Objects.requireNonNull(onSuccess, "onSuccess is null"), Functions.emptyConsumer(), // onError Functions.EMPTY_ACTION, // onComplete Functions.EMPTY_ACTION, // (onSuccess | onError | onComplete) @@ -3012,7 +3013,7 @@ public final Maybe doOnSuccess(Consumer onSuccess) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe filter(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new MaybeFilter(this, predicate)); } @@ -3037,7 +3038,7 @@ public final Maybe filter(Predicate predicate) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe flatMap(Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatten(this, mapper)); } @@ -3069,9 +3070,9 @@ public final Maybe flatMap( Function> onSuccessMapper, Function> onErrorMapper, Supplier> onCompleteSupplier) { - ObjectHelper.requireNonNull(onSuccessMapper, "onSuccessMapper is null"); - ObjectHelper.requireNonNull(onErrorMapper, "onErrorMapper is null"); - ObjectHelper.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); + Objects.requireNonNull(onSuccessMapper, "onSuccessMapper is null"); + Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); + Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapNotification(this, onSuccessMapper, onErrorMapper, onCompleteSupplier)); } @@ -3102,8 +3103,8 @@ public final Maybe flatMap( @SchedulerSupport(SchedulerSupport.NONE) public final Maybe flatMap(Function> mapper, BiFunction resultSelector) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapBiSelector(this, mapper, resultSelector)); } @@ -3132,7 +3133,7 @@ public final Maybe flatMap(Function Flowable flattenAsFlowable(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapIterableFlowable(this, mapper)); } @@ -3158,7 +3159,7 @@ public final Flowable flattenAsFlowable(final Function Observable flattenAsObservable(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapIterableObservable(this, mapper)); } @@ -3182,7 +3183,7 @@ public final Observable flattenAsObservable(final Function Observable flatMapObservable(Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapObservable(this, mapper)); } @@ -3210,7 +3211,7 @@ public final Observable flatMapObservable(Function Flowable flatMapPublisher(Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapPublisher(this, mapper)); } @@ -3236,7 +3237,7 @@ public final Flowable flatMapPublisher(Function Single flatMapSingle(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapSingle(this, mapper)); } @@ -3264,7 +3265,7 @@ public final Single flatMapSingle(final Function Maybe flatMapSingleElement(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapSingleElement(this, mapper)); } @@ -3288,7 +3289,7 @@ public final Maybe flatMapSingleElement(final Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeFlatMapCompletable(this, mapper)); } @@ -3500,7 +3501,7 @@ public final Single isEmpty() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe lift(final MaybeOperator lift) { - ObjectHelper.requireNonNull(lift, "lift is null"); + Objects.requireNonNull(lift, "lift is null"); return RxJavaPlugins.onAssembly(new MaybeLift(this, lift)); } @@ -3524,7 +3525,7 @@ public final Maybe lift(final MaybeOperator lift) @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe map(Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeMap(this, mapper)); } @@ -3572,7 +3573,7 @@ public final Single> materialize() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return merge(this, other); } @@ -3598,7 +3599,7 @@ public final Flowable mergeWith(MaybeSource other) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Maybe observeOn(final Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new MaybeObserveOn(this, scheduler)); } @@ -3622,7 +3623,7 @@ public final Maybe observeOn(final Scheduler scheduler) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe ofType(final Class clazz) { - ObjectHelper.requireNonNull(clazz, "clazz is null"); + Objects.requireNonNull(clazz, "clazz is null"); return filter(Functions.isInstanceOf(clazz)).cast(clazz); } @@ -3644,7 +3645,7 @@ public final Maybe ofType(final Class clazz) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final R to(@NonNull MaybeConverter converter) { - return ObjectHelper.requireNonNull(converter, "converter is null").apply(this); + return Objects.requireNonNull(converter, "converter is null").apply(this); } /** @@ -3733,7 +3734,7 @@ public final Maybe onErrorComplete() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe onErrorComplete(final Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new MaybeOnErrorComplete(this, predicate)); } @@ -3761,7 +3762,7 @@ public final Maybe onErrorComplete(final Predicate predica @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe onErrorResumeWith(final MaybeSource next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return onErrorResumeNext(Functions.justFunction(next)); } @@ -3788,7 +3789,7 @@ public final Maybe onErrorResumeWith(final MaybeSource next) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe onErrorResumeNext(Function> resumeFunction) { - ObjectHelper.requireNonNull(resumeFunction, "resumeFunction is null"); + Objects.requireNonNull(resumeFunction, "resumeFunction is null"); return RxJavaPlugins.onAssembly(new MaybeOnErrorNext(this, resumeFunction, true)); } @@ -3815,7 +3816,7 @@ public final Maybe onErrorResumeNext(Function onErrorReturn(Function valueSupplier) { - ObjectHelper.requireNonNull(valueSupplier, "valueSupplier is null"); + Objects.requireNonNull(valueSupplier, "valueSupplier is null"); return RxJavaPlugins.onAssembly(new MaybeOnErrorReturn(this, valueSupplier)); } @@ -3841,7 +3842,7 @@ public final Maybe onErrorReturn(Function val @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe onErrorReturnItem(final T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return onErrorReturn(Functions.justFunction(item)); } @@ -3871,7 +3872,7 @@ public final Maybe onErrorReturnItem(final T item) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe onExceptionResumeNext(final MaybeSource next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return RxJavaPlugins.onAssembly(new MaybeOnErrorNext(this, Functions.justFunction(next), false)); } @@ -4114,7 +4115,7 @@ public final Maybe retry(Predicate predicate) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe retryUntil(final BooleanSupplier stop) { - ObjectHelper.requireNonNull(stop, "stop is null"); + Objects.requireNonNull(stop, "stop is null"); return retry(Long.MAX_VALUE, Functions.predicateReverseFor(stop)); } @@ -4298,20 +4299,20 @@ public final Disposable subscribe(Consumer onSuccess, Consumer onSuccess, Consumer onError, Action onComplete) { - ObjectHelper.requireNonNull(onSuccess, "onSuccess is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onSuccess, "onSuccess is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); return subscribeWith(new MaybeCallbackObserver(onSuccess, onError, onComplete)); } @SchedulerSupport(SchedulerSupport.NONE) @Override public final void subscribe(MaybeObserver observer) { - ObjectHelper.requireNonNull(observer, "observer is null"); + Objects.requireNonNull(observer, "observer is null"); observer = RxJavaPlugins.onSubscribe(this, observer); - ObjectHelper.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null MaybeObserver. Please check the handler provided to RxJavaPlugins.setOnMaybeSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); + Objects.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null MaybeObserver. Please check the handler provided to RxJavaPlugins.setOnMaybeSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); try { subscribeActual(observer); @@ -4354,7 +4355,7 @@ public final void subscribe(MaybeObserver observer) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Maybe subscribeOn(Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new MaybeSubscribeOn(this, scheduler)); } @@ -4407,7 +4408,7 @@ public final > E subscribeWith(E observer) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe switchIfEmpty(MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new MaybeSwitchIfEmpty(this, other)); } @@ -4431,7 +4432,7 @@ public final Maybe switchIfEmpty(MaybeSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single switchIfEmpty(SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new MaybeSwitchIfEmptySingle(this, other)); } @@ -4457,7 +4458,7 @@ public final Single switchIfEmpty(SingleSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe takeUntil(MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new MaybeTakeUntilMaybe(this, other)); } @@ -4487,7 +4488,7 @@ public final Maybe takeUntil(MaybeSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe takeUntil(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new MaybeTakeUntilPublisher(this, other)); } @@ -4539,7 +4540,7 @@ public final Maybe timeout(long timeout, TimeUnit timeUnit) { @NonNull @SchedulerSupport(SchedulerSupport.COMPUTATION) public final Maybe timeout(long timeout, TimeUnit timeUnit, MaybeSource fallback) { - ObjectHelper.requireNonNull(fallback, "fallback is null"); + Objects.requireNonNull(fallback, "fallback is null"); return timeout(timeout, timeUnit, Schedulers.computation(), fallback); } @@ -4570,7 +4571,7 @@ public final Maybe timeout(long timeout, TimeUnit timeUnit, MaybeSource timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler, MaybeSource fallback) { - ObjectHelper.requireNonNull(fallback, "fallback is null"); + Objects.requireNonNull(fallback, "fallback is null"); return timeout(timer(timeout, timeUnit, scheduler), fallback); } @@ -4617,7 +4618,7 @@ public final Maybe timeout(long timeout, TimeUnit timeUnit, Scheduler schedul @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe timeout(MaybeSource timeoutIndicator) { - ObjectHelper.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); + Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); return RxJavaPlugins.onAssembly(new MaybeTimeoutMaybe(this, timeoutIndicator, null)); } @@ -4639,8 +4640,8 @@ public final Maybe timeout(MaybeSource timeoutIndicator) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe timeout(MaybeSource timeoutIndicator, MaybeSource fallback) { - ObjectHelper.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); - ObjectHelper.requireNonNull(fallback, "fallback is null"); + Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); + Objects.requireNonNull(fallback, "fallback is null"); return RxJavaPlugins.onAssembly(new MaybeTimeoutMaybe(this, timeoutIndicator, fallback)); } @@ -4664,7 +4665,7 @@ public final Maybe timeout(MaybeSource timeoutIndicator, MaybeSource Maybe timeout(Publisher timeoutIndicator) { - ObjectHelper.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); + Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); return RxJavaPlugins.onAssembly(new MaybeTimeoutPublisher(this, timeoutIndicator, null)); } @@ -4690,8 +4691,8 @@ public final Maybe timeout(Publisher timeoutIndicator) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe timeout(Publisher timeoutIndicator, MaybeSource fallback) { - ObjectHelper.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); - ObjectHelper.requireNonNull(fallback, "fallback is null"); + Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); + Objects.requireNonNull(fallback, "fallback is null"); return RxJavaPlugins.onAssembly(new MaybeTimeoutPublisher(this, timeoutIndicator, fallback)); } @@ -4712,7 +4713,7 @@ public final Maybe timeout(Publisher timeoutIndicator, MaybeSource unsubscribeOn(final Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new MaybeUnsubscribeOn(this, scheduler)); } @@ -4746,7 +4747,7 @@ public final Maybe unsubscribeOn(final Scheduler scheduler) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe zipWith(MaybeSource other, BiFunction zipper) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return zip(this, other, zipper); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Notification.java b/src/main/java/io/reactivex/rxjava3/core/Notification.java index b3270483a7..8f8ae5c4d8 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Notification.java +++ b/src/main/java/io/reactivex/rxjava3/core/Notification.java @@ -14,7 +14,6 @@ package io.reactivex.rxjava3.core; import io.reactivex.rxjava3.annotations.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.NotificationLite; import java.util.Objects; @@ -128,7 +127,7 @@ public String toString() { */ @NonNull public static Notification createOnNext(@NonNull T value) { - ObjectHelper.requireNonNull(value, "value is null"); + Objects.requireNonNull(value, "value is null"); return new Notification(value); } @@ -141,7 +140,7 @@ public static Notification createOnNext(@NonNull T value) { */ @NonNull public static Notification createOnError(@NonNull Throwable error) { - ObjectHelper.requireNonNull(error, "error is null"); + Objects.requireNonNull(error, "error is null"); return new Notification(NotificationLite.error(error)); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index ce0815477f..63406824ba 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -119,7 +119,7 @@ public abstract class Observable implements ObservableSource { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable amb(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableAmb(null, sources)); } @@ -146,7 +146,7 @@ public static Observable amb(Iterable Observable ambArray(ObservableSource... sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); int len = sources.length; if (len == 0) { return empty(); @@ -252,8 +252,8 @@ public static Observable combineLatest(Iterable Observable combineLatest(Iterable> sources, Function combiner, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); // the queue holds a pair of values so we need to double the capacity @@ -345,11 +345,11 @@ public static Observable combineLatestArray(ObservableSource Observable combineLatestArray(ObservableSource[] sources, Function combiner, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return empty(); } - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); // the queue holds a pair of values so we need to double the capacity @@ -392,8 +392,8 @@ public static Observable combineLatestArray(ObservableSource Observable combineLatest( ObservableSource source1, ObservableSource source2, BiFunction combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return combineLatestArray(new ObservableSource[] { source1, source2 }, Functions.toFunction(combiner), bufferSize()); } @@ -436,9 +436,9 @@ public static Observable combineLatest( ObservableSource source1, ObservableSource source2, ObservableSource source3, Function3 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3 }, Functions.toFunction(combiner), bufferSize()); } @@ -484,10 +484,10 @@ public static Observable combineLatest( ObservableSource source1, ObservableSource source2, ObservableSource source3, ObservableSource source4, Function4 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4 }, Functions.toFunction(combiner), bufferSize()); } @@ -537,11 +537,11 @@ public static Observable combineLatest( ObservableSource source3, ObservableSource source4, ObservableSource source5, Function5 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5 }, Functions.toFunction(combiner), bufferSize()); } @@ -594,12 +594,12 @@ public static Observable combineLatest( ObservableSource source3, ObservableSource source4, ObservableSource source5, ObservableSource source6, Function6 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6 }, Functions.toFunction(combiner), bufferSize()); } @@ -656,13 +656,13 @@ public static Observable combineLatest( ObservableSource source5, ObservableSource source6, ObservableSource source7, Function7 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6, source7 }, Functions.toFunction(combiner), bufferSize()); } @@ -722,14 +722,14 @@ public static Observable combineLatest( ObservableSource source5, ObservableSource source6, ObservableSource source7, ObservableSource source8, Function8 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6, source7, source8 }, Functions.toFunction(combiner), bufferSize()); } @@ -793,15 +793,15 @@ public static Observable combineLates ObservableSource source7, ObservableSource source8, ObservableSource source9, Function9 combiner) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); - ObjectHelper.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source9, "source9 is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6, source7, source8, source9 }, Functions.toFunction(combiner), bufferSize()); } @@ -891,7 +891,7 @@ public static Observable combineLatestDelayError(ObservableSource Observable combineLatestDelayError(ObservableSource[] sources, Function combiner, int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(combiner, "combiner is null"); if (sources.length == 0) { return empty(); } @@ -986,8 +986,8 @@ public static Observable combineLatestDelayError(Iterable Observable combineLatestDelayError(Iterable> sources, Function combiner, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); // the queue holds a pair of values so we need to double the capacity @@ -1013,7 +1013,7 @@ public static Observable combineLatestDelayError(Iterable Observable concat(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return fromIterable(sources).concatMapDelayError((Function)Functions.identity(), false, bufferSize()); } @@ -1064,7 +1064,7 @@ public static Observable concat(ObservableSource Observable concat(ObservableSource> sources, int prefetch) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMap(sources, Functions.identity(), prefetch, ErrorMode.IMMEDIATE)); } @@ -1093,8 +1093,8 @@ public static Observable concat(ObservableSource Observable concat(ObservableSource source1, ObservableSource source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return concatArray(source1, source2); } @@ -1126,9 +1126,9 @@ public static Observable concat(ObservableSource source1, Ob public static Observable concat( ObservableSource source1, ObservableSource source2, ObservableSource source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return concatArray(source1, source2, source3); } @@ -1162,10 +1162,10 @@ public static Observable concat( public static Observable concat( ObservableSource source1, ObservableSource source2, ObservableSource source3, ObservableSource source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return concatArray(source1, source2, source3, source4); } @@ -1344,7 +1344,7 @@ public static Observable concatArrayEagerDelayError(int maxConcurrency, i @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable concatDelayError(Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return concatDelayError(fromIterable(sources)); } @@ -1390,7 +1390,7 @@ public static Observable concatDelayError(ObservableSource Observable concatDelayError(ObservableSource> sources, int prefetch, boolean tillTheEnd) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch is null"); return RxJavaPlugins.onAssembly(new ObservableConcatMap(sources, Functions.identity(), prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); } @@ -1549,7 +1549,7 @@ public static Observable concatEager(Iterable Observable create(ObservableOnSubscribe source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new ObservableCreate(source)); } @@ -1581,7 +1581,7 @@ public static Observable create(ObservableOnSubscribe source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable defer(Supplier> supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new ObservableDefer(supplier)); } @@ -1630,7 +1630,7 @@ public static Observable empty() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable error(Supplier errorSupplier) { - ObjectHelper.requireNonNull(errorSupplier, "errorSupplier is null"); + Objects.requireNonNull(errorSupplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableError(errorSupplier)); } @@ -1656,7 +1656,7 @@ public static Observable error(Supplier errorSupplie @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable error(final Throwable exception) { - ObjectHelper.requireNonNull(exception, "exception is null"); + Objects.requireNonNull(exception, "exception is null"); return error(Functions.justSupplier(exception)); } @@ -1680,7 +1680,7 @@ public static Observable error(final Throwable exception) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public static Observable fromArray(T... items) { - ObjectHelper.requireNonNull(items, "items is null"); + Objects.requireNonNull(items, "items is null"); if (items.length == 0) { return empty(); } @@ -1723,7 +1723,7 @@ public static Observable fromArray(T... items) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable fromCallable(Callable supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new ObservableFromCallable(supplier)); } @@ -1757,7 +1757,7 @@ public static Observable fromCallable(Callable supplier) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable fromFuture(Future future) { - ObjectHelper.requireNonNull(future, "future is null"); + Objects.requireNonNull(future, "future is null"); return RxJavaPlugins.onAssembly(new ObservableFromFuture(future, 0L, null)); } @@ -1795,8 +1795,8 @@ public static Observable fromFuture(Future future) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable fromFuture(Future future, long timeout, TimeUnit unit) { - ObjectHelper.requireNonNull(future, "future is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(future, "future is null"); + Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new ObservableFromFuture(future, timeout, unit)); } @@ -1837,7 +1837,7 @@ public static Observable fromFuture(Future future, long time @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public static Observable fromFuture(Future future, long timeout, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); Observable o = fromFuture(future, timeout, unit); return o.subscribeOn(scheduler); } @@ -1873,7 +1873,7 @@ public static Observable fromFuture(Future future, long time @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public static Observable fromFuture(Future future, Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); Observable o = fromFuture(future); return o.subscribeOn(scheduler); } @@ -1899,7 +1899,7 @@ public static Observable fromFuture(Future future, Scheduler @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable fromIterable(Iterable source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new ObservableFromIterable(source)); } @@ -1936,7 +1936,7 @@ public static Observable fromIterable(Iterable source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable fromPublisher(Publisher publisher) { - ObjectHelper.requireNonNull(publisher, "publisher is null"); + Objects.requireNonNull(publisher, "publisher is null"); return RxJavaPlugins.onAssembly(new ObservableFromPublisher(publisher)); } @@ -1973,7 +1973,7 @@ public static Observable fromPublisher(Publisher publisher) @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable fromSupplier(Supplier supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new ObservableFromSupplier(supplier)); } @@ -2002,7 +2002,7 @@ public static Observable fromSupplier(Supplier supplier) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable generate(final Consumer> generator) { - ObjectHelper.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(generator, "generator is null"); return generate(Functions.nullSupplier(), ObservableInternalHelper.simpleGenerator(generator), Functions.emptyConsumer()); } @@ -2034,7 +2034,7 @@ public static Observable generate(final Consumer> generator) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable generate(Supplier initialState, final BiConsumer> generator) { - ObjectHelper.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(generator, "generator is null"); return generate(initialState, ObservableInternalHelper.simpleBiGenerator(generator), Functions.emptyConsumer()); } @@ -2070,7 +2070,7 @@ public static Observable generate( final Supplier initialState, final BiConsumer> generator, Consumer disposeState) { - ObjectHelper.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(generator, "generator is null"); return generate(initialState, ObservableInternalHelper.simpleBiGenerator(generator), disposeState); } @@ -2135,9 +2135,9 @@ public static Observable generate(Supplier initialState, BiFunction @SchedulerSupport(SchedulerSupport.NONE) public static Observable generate(Supplier initialState, BiFunction, S> generator, Consumer disposeState) { - ObjectHelper.requireNonNull(initialState, "initialState is null"); - ObjectHelper.requireNonNull(generator, "generator is null"); - ObjectHelper.requireNonNull(disposeState, "disposeState is null"); + Objects.requireNonNull(initialState, "initialState is null"); + Objects.requireNonNull(generator, "generator is null"); + Objects.requireNonNull(disposeState, "disposeState is null"); return RxJavaPlugins.onAssembly(new ObservableGenerate(initialState, generator, disposeState)); } @@ -2195,8 +2195,8 @@ public static Observable interval(long initialDelay, long period, TimeUnit @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public static Observable interval(long initialDelay, long period, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableInterval(Math.max(0L, initialDelay), Math.max(0L, period), unit, scheduler)); } @@ -2304,8 +2304,8 @@ public static Observable intervalRange(long start, long count, long initia if (start > 0 && end < 0) { throw new IllegalArgumentException("Overflow! start + count is bigger than Long.MAX_VALUE"); } - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableIntervalRange(start, end, Math.max(0L, initialDelay), Math.max(0L, period), unit, scheduler)); } @@ -2342,7 +2342,7 @@ public static Observable intervalRange(long start, long count, long initia @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return RxJavaPlugins.onAssembly(new ObservableJust(item)); } @@ -2369,8 +2369,8 @@ public static Observable just(T item) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); return fromArray(item1, item2); } @@ -2400,9 +2400,9 @@ public static Observable just(T item1, T item2) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); return fromArray(item1, item2, item3); } @@ -2434,10 +2434,10 @@ public static Observable just(T item1, T item2, T item3) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3, T item4) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); return fromArray(item1, item2, item3, item4); } @@ -2471,11 +2471,11 @@ public static Observable just(T item1, T item2, T item3, T item4) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3, T item4, T item5) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); return fromArray(item1, item2, item3, item4, item5); } @@ -2511,12 +2511,12 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); return fromArray(item1, item2, item3, item4, item5, item6); } @@ -2554,13 +2554,13 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7); } @@ -2600,14 +2600,14 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); - ObjectHelper.requireNonNull(item8, "item8 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item8, "item8 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7, item8); } @@ -2649,15 +2649,15 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); - ObjectHelper.requireNonNull(item8, "item8 is null"); - ObjectHelper.requireNonNull(item9, "item9 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item8, "item8 is null"); + Objects.requireNonNull(item9, "item9 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7, item8, item9); } @@ -2701,16 +2701,16 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9, T item10) { - ObjectHelper.requireNonNull(item1, "item1 is null"); - ObjectHelper.requireNonNull(item2, "item2 is null"); - ObjectHelper.requireNonNull(item3, "item3 is null"); - ObjectHelper.requireNonNull(item4, "item4 is null"); - ObjectHelper.requireNonNull(item5, "item5 is null"); - ObjectHelper.requireNonNull(item6, "item6 is null"); - ObjectHelper.requireNonNull(item7, "item7 is null"); - ObjectHelper.requireNonNull(item8, "item8 is null"); - ObjectHelper.requireNonNull(item9, "item9 is null"); - ObjectHelper.requireNonNull(item10, "item10 is null"); + Objects.requireNonNull(item1, "item1 is null"); + Objects.requireNonNull(item2, "item2 is null"); + Objects.requireNonNull(item3, "item3 is null"); + Objects.requireNonNull(item4, "item4 is null"); + Objects.requireNonNull(item5, "item5 is null"); + Objects.requireNonNull(item6, "item6 is null"); + Objects.requireNonNull(item7, "item7 is null"); + Objects.requireNonNull(item8, "item8 is null"); + Objects.requireNonNull(item9, "item9 is null"); + Objects.requireNonNull(item10, "item10 is null"); return fromArray(item1, item2, item3, item4, item5, item6, item7, item8, item9, item10); } @@ -2932,7 +2932,7 @@ public static Observable merge(Iterable Observable merge(ObservableSource> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), false, Integer.MAX_VALUE, bufferSize())); } @@ -2980,7 +2980,7 @@ public static Observable merge(ObservableSource Observable merge(ObservableSource> sources, int maxConcurrency) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), false, maxConcurrency, bufferSize())); } @@ -3023,8 +3023,8 @@ public static Observable merge(ObservableSource Observable merge(ObservableSource source1, ObservableSource source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), false, 2); } @@ -3068,9 +3068,9 @@ public static Observable merge(ObservableSource source1, Obs @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable merge(ObservableSource source1, ObservableSource source2, ObservableSource source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return fromArray(source1, source2, source3).flatMap((Function)Functions.identity(), false, 3); } @@ -3118,10 +3118,10 @@ public static Observable merge(ObservableSource source1, Obs public static Observable merge( ObservableSource source1, ObservableSource source2, ObservableSource source3, ObservableSource source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return fromArray(source1, source2, source3, source4).flatMap((Function)Functions.identity(), false, 4); } @@ -3331,7 +3331,7 @@ public static Observable mergeDelayError(Iterable Observable mergeDelayError(ObservableSource> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), true, Integer.MAX_VALUE, bufferSize())); } @@ -3368,7 +3368,7 @@ public static Observable mergeDelayError(ObservableSource Observable mergeDelayError(ObservableSource> sources, int maxConcurrency) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), true, maxConcurrency, bufferSize())); } @@ -3403,8 +3403,8 @@ public static Observable mergeDelayError(ObservableSource Observable mergeDelayError(ObservableSource source1, ObservableSource source2) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), true, 2); } @@ -3441,9 +3441,9 @@ public static Observable mergeDelayError(ObservableSource so @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable mergeDelayError(ObservableSource source1, ObservableSource source2, ObservableSource source3) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return fromArray(source1, source2, source3).flatMap((Function)Functions.identity(), true, 3); } @@ -3484,10 +3484,10 @@ public static Observable mergeDelayError(ObservableSource so public static Observable mergeDelayError( ObservableSource source1, ObservableSource source2, ObservableSource source3, ObservableSource source4) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return fromArray(source1, source2, source3, source4).flatMap((Function)Functions.identity(), true, 4); } @@ -3709,9 +3709,9 @@ public static Single sequenceEqual(ObservableSource so @SchedulerSupport(SchedulerSupport.NONE) public static Single sequenceEqual(ObservableSource source1, ObservableSource source2, BiPredicate isEqual, int bufferSize) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(isEqual, "isEqual is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(isEqual, "isEqual is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableSequenceEqualSingle(source1, source2, isEqual, bufferSize)); } @@ -3775,7 +3775,7 @@ public static Single sequenceEqual(ObservableSource so @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable switchOnNext(ObservableSource> sources, int bufferSize) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableSwitchMap(sources, Functions.identity(), bufferSize, false)); } @@ -3877,7 +3877,7 @@ public static Observable switchOnNextDelayError(ObservableSource Observable switchOnNextDelayError(ObservableSource> sources, int prefetch) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableSwitchMap(sources, Functions.identity(), prefetch, true)); } @@ -3930,8 +3930,8 @@ public static Observable timer(long delay, TimeUnit unit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public static Observable timer(long delay, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableTimer(Math.max(delay, 0L), unit, scheduler)); } @@ -3951,7 +3951,7 @@ public static Observable timer(long delay, TimeUnit unit, Scheduler schedu @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable unsafeCreate(ObservableSource onSubscribe) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); if (onSubscribe instanceof Observable) { throw new IllegalArgumentException("unsafeCreate(Observable) should be upgraded"); } @@ -4018,9 +4018,9 @@ public static Observable using(Supplier resourceSupplier, @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable using(Supplier resourceSupplier, Function> sourceSupplier, Consumer disposer, boolean eager) { - ObjectHelper.requireNonNull(resourceSupplier, "resourceSupplier is null"); - ObjectHelper.requireNonNull(sourceSupplier, "sourceSupplier is null"); - ObjectHelper.requireNonNull(disposer, "disposer is null"); + Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); + Objects.requireNonNull(sourceSupplier, "sourceSupplier is null"); + Objects.requireNonNull(disposer, "disposer is null"); return RxJavaPlugins.onAssembly(new ObservableUsing(resourceSupplier, sourceSupplier, disposer, eager)); } @@ -4040,7 +4040,7 @@ public static Observable using(Supplier resourceSupplier, @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable wrap(ObservableSource source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); if (source instanceof Observable) { return RxJavaPlugins.onAssembly((Observable)source); } @@ -4095,8 +4095,8 @@ public static Observable wrap(ObservableSource source) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip(Iterable> sources, Function zipper) { - ObjectHelper.requireNonNull(zipper, "zipper is null"); - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableZip(null, sources, zipper, bufferSize(), false)); } @@ -4155,8 +4155,8 @@ public static Observable zip(Iterable Observable zip(Iterable> sources, Function zipper, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(zipper, "zipper is null"); - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableZip(null, sources, zipper, bufferSize, delayError)); } @@ -4211,8 +4211,8 @@ public static Observable zip(Iterable Observable zip( ObservableSource source1, ObservableSource source2, BiFunction zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2); } @@ -4267,8 +4267,8 @@ public static Observable zip( public static Observable zip( ObservableSource source1, ObservableSource source2, BiFunction zipper, boolean delayError) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize(), source1, source2); } @@ -4324,8 +4324,8 @@ public static Observable zip( public static Observable zip( ObservableSource source1, ObservableSource source2, BiFunction zipper, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize, source1, source2); } @@ -4383,9 +4383,9 @@ public static Observable zip( public static Observable zip( ObservableSource source1, ObservableSource source2, ObservableSource source3, Function3 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3); } @@ -4447,10 +4447,10 @@ public static Observable zip( ObservableSource source1, ObservableSource source2, ObservableSource source3, ObservableSource source4, Function4 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4); } @@ -4515,11 +4515,11 @@ public static Observable zip( ObservableSource source1, ObservableSource source2, ObservableSource source3, ObservableSource source4, ObservableSource source5, Function5 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5); } @@ -4586,12 +4586,12 @@ public static Observable zip( ObservableSource source1, ObservableSource source2, ObservableSource source3, ObservableSource source4, ObservableSource source5, ObservableSource source6, Function6 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6); } @@ -4662,13 +4662,13 @@ public static Observable zip( ObservableSource source4, ObservableSource source5, ObservableSource source6, ObservableSource source7, Function7 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7); } @@ -4742,14 +4742,14 @@ public static Observable zip( ObservableSource source4, ObservableSource source5, ObservableSource source6, ObservableSource source7, ObservableSource source8, Function8 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -4826,15 +4826,15 @@ public static Observable zip( ObservableSource source4, ObservableSource source5, ObservableSource source6, ObservableSource source7, ObservableSource source8, ObservableSource source9, Function9 zipper) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); - ObjectHelper.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source9, "source9 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8, source9); } @@ -4895,7 +4895,7 @@ public static Observable zipArray(Function(sources, null, zipper, bufferSize, delayError)); } @@ -4923,7 +4923,7 @@ public static Observable zipArray(Function all(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableAllSingle(this, predicate)); } @@ -4948,7 +4948,7 @@ public final Single all(Predicate predicate) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable ambWith(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -4975,7 +4975,7 @@ public final Observable ambWith(ObservableSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single any(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableAnySingle(this, predicate)); } @@ -5520,7 +5520,7 @@ public final Observable> buffer(int count, int skip) { public final > Observable buffer(int count, int skip, Supplier bufferSupplier) { ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(skip, "skip"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableBuffer(this, count, skip, bufferSupplier)); } @@ -5650,9 +5650,9 @@ public final Observable> buffer(long timespan, long timeskip, TimeUnit u @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final > Observable buffer(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler, Supplier bufferSupplier) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableBufferTimed(this, timespan, timeskip, unit, scheduler, bufferSupplier, Integer.MAX_VALUE, false)); } @@ -5795,9 +5795,9 @@ public final > Observable buffer( Scheduler scheduler, int count, Supplier bufferSupplier, boolean restartTimerOnMaxSize) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); ObjectHelper.verifyPositive(count, "count"); return RxJavaPlugins.onAssembly(new ObservableBufferTimed(this, timespan, timespan, unit, scheduler, bufferSupplier, count, restartTimerOnMaxSize)); } @@ -5899,9 +5899,9 @@ public final > Observable ObservableSource openingIndicator, Function> closingIndicator, Supplier bufferSupplier) { - ObjectHelper.requireNonNull(openingIndicator, "openingIndicator is null"); - ObjectHelper.requireNonNull(closingIndicator, "closingIndicator is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(openingIndicator, "openingIndicator is null"); + Objects.requireNonNull(closingIndicator, "closingIndicator is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableBufferBoundary(this, openingIndicator, closingIndicator, bufferSupplier)); } @@ -5999,8 +5999,8 @@ public final Observable> buffer(ObservableSource boundary, final @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final > Observable buffer(ObservableSource boundary, Supplier bufferSupplier) { - ObjectHelper.requireNonNull(boundary, "boundary is null"); - ObjectHelper.requireNonNull(bufferSupplier, "bufferSupplier is null"); + Objects.requireNonNull(boundary, "boundary is null"); + Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableBufferExactBoundary(this, boundary, bufferSupplier)); } @@ -6138,7 +6138,7 @@ public final Observable cacheWithInitialCapacity(int initialCapacity) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable cast(final Class clazz) { - ObjectHelper.requireNonNull(clazz, "clazz is null"); + Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -6171,8 +6171,8 @@ public final Observable cast(final Class clazz) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single collect(Supplier initialValueSupplier, BiConsumer collector) { - ObjectHelper.requireNonNull(initialValueSupplier, "initialValueSupplier is null"); - ObjectHelper.requireNonNull(collector, "collector is null"); + Objects.requireNonNull(initialValueSupplier, "initialValueSupplier is null"); + Objects.requireNonNull(collector, "collector is null"); return RxJavaPlugins.onAssembly(new ObservableCollectSingle(this, initialValueSupplier, collector)); } @@ -6205,7 +6205,7 @@ public final Single collect(Supplier initialValueSupplier, B @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single collectInto(final U initialValue, BiConsumer collector) { - ObjectHelper.requireNonNull(initialValue, "initialValue is null"); + Objects.requireNonNull(initialValue, "initialValue is null"); return collect(Functions.justSupplier(initialValue), collector); } @@ -6232,7 +6232,7 @@ public final Single collectInto(final U initialValue, BiConsumer Observable compose(ObservableTransformer composer) { - return wrap(((ObservableTransformer) ObjectHelper.requireNonNull(composer, "composer is null")).apply(this)); + return wrap(((ObservableTransformer) Objects.requireNonNull(composer, "composer is null")).apply(this)); } /** @@ -6294,7 +6294,7 @@ public final Observable concatMap(Function Observable concatMap(Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); if (this instanceof ScalarSupplier) { @SuppressWarnings("unchecked") @@ -6336,9 +6336,9 @@ public final Observable concatMap(Function Observable concatMap(Function> mapper, int prefetch, Scheduler scheduler) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableConcatMapScheduler(this, mapper, prefetch, ErrorMode.IMMEDIATE, scheduler)); } @@ -6399,7 +6399,7 @@ public final Observable concatMapDelayError(Function Observable concatMapDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); if (this instanceof ScalarSupplier) { @SuppressWarnings("unchecked") @@ -6441,9 +6441,9 @@ public final Observable concatMapDelayError(Function Observable concatMapDelayError(Function> mapper, boolean tillTheEnd, int prefetch, Scheduler scheduler) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableConcatMapScheduler(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, scheduler)); } @@ -6497,7 +6497,7 @@ public final Observable concatMapEager(Function Observable concatMapEager(Function> mapper, int maxConcurrency, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapEager(this, mapper, ErrorMode.IMMEDIATE, maxConcurrency, prefetch)); @@ -6562,7 +6562,7 @@ public final Observable concatMapEagerDelayError(Function Observable concatMapEagerDelayError(Function> mapper, boolean tillTheEnd, int maxConcurrency, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapEager(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, maxConcurrency, prefetch)); @@ -6611,7 +6611,7 @@ public final Completable concatMapCompletable(Function mapper, int capacityHint) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable(this, mapper, ErrorMode.IMMEDIATE, capacityHint)); } @@ -6701,7 +6701,7 @@ public final Completable concatMapCompletableDelayError(Function mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } @@ -6729,7 +6729,7 @@ public final Completable concatMapCompletableDelayError(Function Observable concatMapIterable(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlattenIterable(this, mapper)); } @@ -6758,7 +6758,7 @@ public final Observable concatMapIterable(final Function Observable concatMapIterable(final Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return concatMap(ObservableInternalHelper.flatMapIntoIterable(mapper), prefetch); } @@ -6816,7 +6816,7 @@ public final Observable concatMapMaybe(Function Observable concatMapMaybe(Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } @@ -6911,7 +6911,7 @@ public final Observable concatMapMaybeDelayError(Function Observable concatMapMaybeDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } @@ -6969,7 +6969,7 @@ public final Observable concatMapSingle(Function Observable concatMapSingle(Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } @@ -7064,7 +7064,7 @@ public final Observable concatMapSingleDelayError(Function Observable concatMapSingleDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } @@ -7088,7 +7088,7 @@ public final Observable concatMapSingleDelayError(Function concatWith(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return concat(this, other); } @@ -7109,7 +7109,7 @@ public final Observable concatWith(ObservableSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable concatWith(@NonNull SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableConcatWithSingle(this, other)); } @@ -7130,7 +7130,7 @@ public final Observable concatWith(@NonNull SingleSource other) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable concatWith(@NonNull MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableConcatWithMaybe(this, other)); } @@ -7151,7 +7151,7 @@ public final Observable concatWith(@NonNull MaybeSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable concatWith(@NonNull CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableConcatWithCompletable(this, other)); } @@ -7174,7 +7174,7 @@ public final Observable concatWith(@NonNull CompletableSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single contains(final Object element) { - ObjectHelper.requireNonNull(element, "element is null"); + Objects.requireNonNull(element, "element is null"); return any(Functions.equalsWith(element)); } @@ -7227,7 +7227,7 @@ public final Single count() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable debounce(Function> debounceSelector) { - ObjectHelper.requireNonNull(debounceSelector, "debounceSelector is null"); + Objects.requireNonNull(debounceSelector, "debounceSelector is null"); return RxJavaPlugins.onAssembly(new ObservableDebounce(this, debounceSelector)); } @@ -7306,8 +7306,8 @@ public final Observable debounce(long timeout, TimeUnit unit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable debounce(long timeout, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableDebounceTimed(this, timeout, unit, scheduler)); } @@ -7330,7 +7330,7 @@ public final Observable debounce(long timeout, TimeUnit unit, Scheduler sched @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable defaultIfEmpty(T defaultItem) { - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return switchIfEmpty(just(defaultItem)); } @@ -7360,7 +7360,7 @@ public final Observable defaultIfEmpty(T defaultItem) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable delay(final Function> itemDelay) { - ObjectHelper.requireNonNull(itemDelay, "itemDelay is null"); + Objects.requireNonNull(itemDelay, "itemDelay is null"); return flatMap(ObservableInternalHelper.itemDelay(itemDelay)); } @@ -7463,8 +7463,8 @@ public final Observable delay(long delay, TimeUnit unit, Scheduler scheduler) @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable delay(long delay, TimeUnit unit, Scheduler scheduler, boolean delayError) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableDelay(this, delay, unit, scheduler, delayError)); } @@ -7524,7 +7524,7 @@ public final Observable delay(ObservableSource subscriptionDelay, @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable delaySubscription(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableDelaySubscriptionOther(this, other)); } @@ -7626,7 +7626,7 @@ public final Observable delaySubscription(long delay, TimeUnit unit, Schedule @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable dematerialize(Function> selector) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); return RxJavaPlugins.onAssembly(new ObservableDematerialize(this, selector)); } @@ -7735,8 +7735,8 @@ public final Observable distinct(Function keySelector) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable distinct(Function keySelector, Supplier> collectionSupplier) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(collectionSupplier, "collectionSupplier is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableDistinct(this, keySelector, collectionSupplier)); } @@ -7816,7 +7816,7 @@ public final Observable distinctUntilChanged() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable distinctUntilChanged(Function keySelector) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged(this, keySelector, ObjectHelper.equalsPredicate())); } @@ -7850,7 +7850,7 @@ public final Observable distinctUntilChanged(Function keySe @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable distinctUntilChanged(BiPredicate comparer) { - ObjectHelper.requireNonNull(comparer, "comparer is null"); + Objects.requireNonNull(comparer, "comparer is null"); return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged(this, Functions.identity(), comparer)); } @@ -7874,7 +7874,7 @@ public final Observable distinctUntilChanged(BiPredicate doAfterNext(Consumer onAfterNext) { - ObjectHelper.requireNonNull(onAfterNext, "onAfterNext is null"); + Objects.requireNonNull(onAfterNext, "onAfterNext is null"); return RxJavaPlugins.onAssembly(new ObservableDoAfterNext(this, onAfterNext)); } @@ -7898,7 +7898,7 @@ public final Observable doAfterNext(Consumer onAfterNext) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable doAfterTerminate(Action onFinally) { - ObjectHelper.requireNonNull(onFinally, "onFinally is null"); + Objects.requireNonNull(onFinally, "onFinally is null"); return doOnEach(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, onFinally); } @@ -7925,7 +7925,7 @@ public final Observable doAfterTerminate(Action onFinally) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable doFinally(Action onFinally) { - ObjectHelper.requireNonNull(onFinally, "onFinally is null"); + Objects.requireNonNull(onFinally, "onFinally is null"); return RxJavaPlugins.onAssembly(new ObservableDoFinally(this, onFinally)); } @@ -7992,10 +7992,10 @@ public final Observable doOnComplete(Action onComplete) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) private Observable doOnEach(Consumer onNext, Consumer onError, Action onComplete, Action onAfterTerminate) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); - ObjectHelper.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); return RxJavaPlugins.onAssembly(new ObservableDoOnEach(this, onNext, onError, onComplete, onAfterTerminate)); } @@ -8016,7 +8016,7 @@ private Observable doOnEach(Consumer onNext, Consumer doOnEach(final Consumer> onNotification) { - ObjectHelper.requireNonNull(onNotification, "onNotification is null"); + Objects.requireNonNull(onNotification, "onNotification is null"); return doOnEach( Functions.notificationOnNext(onNotification), Functions.notificationOnError(onNotification), @@ -8048,7 +8048,7 @@ public final Observable doOnEach(final Consumer> onNo @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable doOnEach(final Observer observer) { - ObjectHelper.requireNonNull(observer, "observer is null"); + Objects.requireNonNull(observer, "observer is null"); return doOnEach( ObservableInternalHelper.observerOnNext(observer), ObservableInternalHelper.observerOnError(observer), @@ -8099,8 +8099,8 @@ public final Observable doOnError(Consumer onError) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable doOnLifecycle(final Consumer onSubscribe, final Action onDispose) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); - ObjectHelper.requireNonNull(onDispose, "onDispose is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onDispose, "onDispose is null"); return RxJavaPlugins.onAssembly(new ObservableDoOnLifecycle(this, onSubscribe, onDispose)); } @@ -8169,7 +8169,7 @@ public final Observable doOnSubscribe(Consumer onSubscrib @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable doOnTerminate(final Action onTerminate) { - ObjectHelper.requireNonNull(onTerminate, "onTerminate is null"); + Objects.requireNonNull(onTerminate, "onTerminate is null"); return doOnEach(Functions.emptyConsumer(), Functions.actionConsumer(onTerminate), onTerminate, Functions.EMPTY_ACTION); @@ -8228,7 +8228,7 @@ public final Single elementAt(long index, T defaultItem) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new ObservableElementAtSingle(this, index, defaultItem)); } @@ -8278,7 +8278,7 @@ public final Single elementAtOrError(long index) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable filter(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableFilter(this, predicate)); } @@ -8462,7 +8462,7 @@ public final Observable flatMap(Function Observable flatMap(Function> mapper, boolean delayErrors, int maxConcurrency, int bufferSize) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (this instanceof ScalarSupplier) { @@ -8506,9 +8506,9 @@ public final Observable flatMap( Function> onNextMapper, Function> onErrorMapper, Supplier> onCompleteSupplier) { - ObjectHelper.requireNonNull(onNextMapper, "onNextMapper is null"); - ObjectHelper.requireNonNull(onErrorMapper, "onErrorMapper is null"); - ObjectHelper.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); + Objects.requireNonNull(onNextMapper, "onNextMapper is null"); + Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); + Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); return merge(new ObservableMapNotification(this, onNextMapper, onErrorMapper, onCompleteSupplier)); } @@ -8547,9 +8547,9 @@ public final Observable flatMap( Function> onErrorMapper, Supplier> onCompleteSupplier, int maxConcurrency) { - ObjectHelper.requireNonNull(onNextMapper, "onNextMapper is null"); - ObjectHelper.requireNonNull(onErrorMapper, "onErrorMapper is null"); - ObjectHelper.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); + Objects.requireNonNull(onNextMapper, "onNextMapper is null"); + Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); + Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); return merge(new ObservableMapNotification(this, onNextMapper, onErrorMapper, onCompleteSupplier), maxConcurrency); } @@ -8719,8 +8719,8 @@ public final Observable flatMap(Function Observable flatMap(final Function> mapper, final BiFunction combiner, boolean delayErrors, int maxConcurrency, int bufferSize) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(combiner, "combiner is null"); return flatMap(ObservableInternalHelper.flatMapWithCombiner(mapper, combiner), delayErrors, maxConcurrency, bufferSize); } @@ -8793,7 +8793,7 @@ public final Completable flatMapCompletable(Function mapper, boolean delayErrors) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMapCompletableCompletable(this, mapper, delayErrors)); } @@ -8819,7 +8819,7 @@ public final Completable flatMapCompletable(Function Observable flatMapIterable(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlattenIterable(this, mapper)); } @@ -8851,8 +8851,8 @@ public final Observable flatMapIterable(final Function Observable flatMapIterable(final Function> mapper, BiFunction resultSelector) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return flatMap(ObservableInternalHelper.flatMapIntoIterable(mapper), resultSelector, false, bufferSize(), bufferSize()); } @@ -8894,7 +8894,7 @@ public final Observable flatMapMaybe(Function Observable flatMapMaybe(Function> mapper, boolean delayErrors) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMapMaybe(this, mapper, delayErrors)); } @@ -8936,7 +8936,7 @@ public final Observable flatMapSingle(Function Observable flatMapSingle(Function> mapper, boolean delayErrors) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMapSingle(this, mapper, delayErrors)); } @@ -9044,9 +9044,9 @@ public final Disposable forEachWhile(Predicate onNext, Consumer onNext, Consumer onError, final Action onComplete) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); ForEachWhileObserver o = new ForEachWhileObserver(onNext, onError, onComplete); subscribe(o); @@ -9274,8 +9274,8 @@ public final Observable> groupBy(Function Observable> groupBy(Function keySelector, Function valueSelector, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableGroupBy(this, keySelector, valueSelector, bufferSize, delayError)); @@ -9320,10 +9320,10 @@ public final Observable groupJoin( Function> rightEnd, BiFunction, ? extends R> resultSelector ) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(leftEnd, "leftEnd is null"); - ObjectHelper.requireNonNull(rightEnd, "rightEnd is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(leftEnd, "leftEnd is null"); + Objects.requireNonNull(rightEnd, "rightEnd is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return RxJavaPlugins.onAssembly(new ObservableGroupJoin( this, other, leftEnd, rightEnd, resultSelector)); } @@ -9427,10 +9427,10 @@ public final Observable join( Function> rightEnd, BiFunction resultSelector ) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(leftEnd, "leftEnd is null"); - ObjectHelper.requireNonNull(rightEnd, "rightEnd is null"); - ObjectHelper.requireNonNull(resultSelector, "resultSelector is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(leftEnd, "leftEnd is null"); + Objects.requireNonNull(rightEnd, "rightEnd is null"); + Objects.requireNonNull(resultSelector, "resultSelector is null"); return RxJavaPlugins.onAssembly(new ObservableJoin( this, other, leftEnd, rightEnd, resultSelector)); } @@ -9474,7 +9474,7 @@ public final Maybe lastElement() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single last(T defaultItem) { - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new ObservableLastSingle(this, defaultItem)); } @@ -9643,7 +9643,7 @@ public final Single lastOrError() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable lift(ObservableOperator lifter) { - ObjectHelper.requireNonNull(lifter, "lifter is null"); + Objects.requireNonNull(lifter, "lifter is null"); return RxJavaPlugins.onAssembly(new ObservableLift(this, lifter)); } @@ -9667,7 +9667,7 @@ public final Observable lift(ObservableOperator l @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable map(Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableMap(this, mapper)); } @@ -9712,7 +9712,7 @@ public final Observable> materialize() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable mergeWith(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return merge(this, other); } @@ -9735,7 +9735,7 @@ public final Observable mergeWith(ObservableSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable mergeWith(@NonNull SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableMergeWithSingle(this, other)); } @@ -9759,7 +9759,7 @@ public final Observable mergeWith(@NonNull SingleSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable mergeWith(@NonNull MaybeSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableMergeWithMaybe(this, other)); } @@ -9780,7 +9780,7 @@ public final Observable mergeWith(@NonNull MaybeSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable mergeWith(@NonNull CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableMergeWithCompletable(this, other)); } @@ -9895,7 +9895,7 @@ public final Observable observeOn(Scheduler scheduler, boolean delayError) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable observeOn(Scheduler scheduler, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableObserveOn(this, scheduler, delayError, bufferSize)); } @@ -9918,7 +9918,7 @@ public final Observable observeOn(Scheduler scheduler, boolean delayError, in @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable ofType(final Class clazz) { - ObjectHelper.requireNonNull(clazz, "clazz is null"); + Objects.requireNonNull(clazz, "clazz is null"); return filter(Functions.isInstanceOf(clazz)).cast(clazz); } @@ -9954,7 +9954,7 @@ public final Observable ofType(final Class clazz) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable onErrorResumeNext(Function> resumeFunction) { - ObjectHelper.requireNonNull(resumeFunction, "resumeFunction is null"); + Objects.requireNonNull(resumeFunction, "resumeFunction is null"); return RxJavaPlugins.onAssembly(new ObservableOnErrorNext(this, resumeFunction)); } @@ -9990,7 +9990,7 @@ public final Observable onErrorResumeNext(Function onErrorResumeWith(final ObservableSource next) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); return onErrorResumeNext(Functions.justFunction(next)); } @@ -10023,7 +10023,7 @@ public final Observable onErrorResumeWith(final ObservableSource @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable onErrorReturn(Function valueSupplier) { - ObjectHelper.requireNonNull(valueSupplier, "valueSupplier is null"); + Objects.requireNonNull(valueSupplier, "valueSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableOnErrorReturn(this, valueSupplier)); } @@ -10056,7 +10056,7 @@ public final Observable onErrorReturn(Function onErrorReturnItem(final T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return onErrorReturn(Functions.justFunction(item)); } @@ -10122,7 +10122,7 @@ public final ConnectableObservable publish() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable publish(Function, ? extends ObservableSource> selector) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); return RxJavaPlugins.onAssembly(new ObservablePublishSelector(this, selector)); } @@ -10157,7 +10157,7 @@ public final Observable publish(Function, ? extends @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Maybe reduce(BiFunction reducer) { - ObjectHelper.requireNonNull(reducer, "reducer is null"); + Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new ObservableReduceMaybe(this, reducer)); } @@ -10214,8 +10214,8 @@ public final Maybe reduce(BiFunction reducer) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single reduce(R seed, BiFunction reducer) { - ObjectHelper.requireNonNull(seed, "seed is null"); - ObjectHelper.requireNonNull(reducer, "reducer is null"); + Objects.requireNonNull(seed, "seed is null"); + Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new ObservableReduceSeedSingle(this, seed, reducer)); } @@ -10254,8 +10254,8 @@ public final Single reduce(R seed, BiFunction reducer) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single reduceWith(Supplier seedSupplier, BiFunction reducer) { - ObjectHelper.requireNonNull(seedSupplier, "seedSupplier is null"); - ObjectHelper.requireNonNull(reducer, "reducer is null"); + Objects.requireNonNull(seedSupplier, "seedSupplier is null"); + Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new ObservableReduceWithSingle(this, seedSupplier, reducer)); } @@ -10330,7 +10330,7 @@ public final Observable repeat(long times) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable repeatUntil(BooleanSupplier stop) { - ObjectHelper.requireNonNull(stop, "stop is null"); + Objects.requireNonNull(stop, "stop is null"); return RxJavaPlugins.onAssembly(new ObservableRepeatUntil(this, stop)); } @@ -10356,7 +10356,7 @@ public final Observable repeatUntil(BooleanSupplier stop) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable repeatWhen(final Function, ? extends ObservableSource> handler) { - ObjectHelper.requireNonNull(handler, "handler is null"); + Objects.requireNonNull(handler, "handler is null"); return RxJavaPlugins.onAssembly(new ObservableRepeatWhen(this, handler)); } @@ -10404,7 +10404,7 @@ public final ConnectableObservable replay() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable replay(Function, ? extends ObservableSource> selector) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this), selector); } @@ -10438,7 +10438,7 @@ public final Observable replay(Function, ? extends @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this, bufferSize, false), selector); } @@ -10475,7 +10475,7 @@ public final Observable replay(Function, ? extends @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize, boolean eagerTruncate) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this, bufferSize, eagerTruncate), selector); } @@ -10556,10 +10556,10 @@ public final Observable replay(Function, ? extends @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.multicastSelector( ObservableInternalHelper.replaySupplier(this, bufferSize, time, unit, scheduler, false), selector); } @@ -10604,10 +10604,10 @@ public final Observable replay(Function, ? extends @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.multicastSelector( ObservableInternalHelper.replaySupplier(this, bufferSize, time, unit, scheduler, eagerTruncate), selector); } @@ -10674,9 +10674,9 @@ public final Observable replay(Function, ? extends @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable replay(Function, ? extends ObservableSource> selector, final long time, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(selector, "selector is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this, time, unit, scheduler, false), selector); } @@ -10713,9 +10713,9 @@ public final Observable replay(Function, ? extends @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable replay(Function, ? extends ObservableSource> selector, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - ObjectHelper.requireNonNull(selector, "selector is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this, time, unit, scheduler, eagerTruncate), selector); } @@ -10854,8 +10854,8 @@ public final ConnectableObservable replay(int bufferSize, long time, TimeUnit @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableObservable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.create(this, time, unit, scheduler, bufferSize, false); } @@ -10898,8 +10898,8 @@ public final ConnectableObservable replay(final int bufferSize, final long ti @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableObservable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.create(this, time, unit, scheduler, bufferSize, eagerTruncate); } @@ -10958,8 +10958,8 @@ public final ConnectableObservable replay(long time, TimeUnit unit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableObservable replay(final long time, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.create(this, time, unit, scheduler, false); } @@ -10994,8 +10994,8 @@ public final ConnectableObservable replay(final long time, final TimeUnit uni @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final ConnectableObservable replay(final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.create(this, time, unit, scheduler, eagerTruncate); } @@ -11046,7 +11046,7 @@ public final Observable retry() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable retry(BiPredicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableRetryBiPredicate(this, predicate)); } @@ -11099,7 +11099,7 @@ public final Observable retry(long times, Predicate predic if (times < 0) { throw new IllegalArgumentException("times >= 0 required but it was " + times); } - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableRetryPredicate(this, times, predicate)); } @@ -11136,7 +11136,7 @@ public final Observable retry(Predicate predicate) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable retryUntil(final BooleanSupplier stop) { - ObjectHelper.requireNonNull(stop, "stop is null"); + Objects.requireNonNull(stop, "stop is null"); return retry(Long.MAX_VALUE, Functions.predicateReverseFor(stop)); } @@ -11217,7 +11217,7 @@ public final Observable retryUntil(final BooleanSupplier stop) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable retryWhen( final Function, ? extends ObservableSource> handler) { - ObjectHelper.requireNonNull(handler, "handler is null"); + Objects.requireNonNull(handler, "handler is null"); return RxJavaPlugins.onAssembly(new ObservableRetryWhen(this, handler)); } @@ -11235,7 +11235,7 @@ public final Observable retryWhen( */ @SchedulerSupport(SchedulerSupport.NONE) public final void safeSubscribe(Observer observer) { - ObjectHelper.requireNonNull(observer, "observer is null"); + Objects.requireNonNull(observer, "observer is null"); if (observer instanceof SafeObserver) { subscribe(observer); } else { @@ -11323,8 +11323,8 @@ public final Observable sample(long period, TimeUnit unit, boolean emitLast) @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable sample(long period, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleTimed(this, period, unit, scheduler, false)); } @@ -11359,8 +11359,8 @@ public final Observable sample(long period, TimeUnit unit, Scheduler schedule @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable sample(long period, TimeUnit unit, Scheduler scheduler, boolean emitLast) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleTimed(this, period, unit, scheduler, emitLast)); } @@ -11385,7 +11385,7 @@ public final Observable sample(long period, TimeUnit unit, Scheduler schedule @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable sample(ObservableSource sampler) { - ObjectHelper.requireNonNull(sampler, "sampler is null"); + Objects.requireNonNull(sampler, "sampler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable(this, sampler, false)); } @@ -11417,7 +11417,7 @@ public final Observable sample(ObservableSource sampler) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable sample(ObservableSource sampler, boolean emitLast) { - ObjectHelper.requireNonNull(sampler, "sampler is null"); + Objects.requireNonNull(sampler, "sampler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable(this, sampler, emitLast)); } @@ -11445,7 +11445,7 @@ public final Observable sample(ObservableSource sampler, boolean emitL @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable scan(BiFunction accumulator) { - ObjectHelper.requireNonNull(accumulator, "accumulator is null"); + Objects.requireNonNull(accumulator, "accumulator is null"); return RxJavaPlugins.onAssembly(new ObservableScan(this, accumulator)); } @@ -11494,7 +11494,7 @@ public final Observable scan(BiFunction accumulator) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable scan(final R initialValue, BiFunction accumulator) { - ObjectHelper.requireNonNull(initialValue, "initialValue is null"); + Objects.requireNonNull(initialValue, "initialValue is null"); return scanWith(Functions.justSupplier(initialValue), accumulator); } @@ -11529,8 +11529,8 @@ public final Observable scan(final R initialValue, BiFunction Observable scanWith(Supplier seedSupplier, BiFunction accumulator) { - ObjectHelper.requireNonNull(seedSupplier, "seedSupplier is null"); - ObjectHelper.requireNonNull(accumulator, "accumulator is null"); + Objects.requireNonNull(seedSupplier, "seedSupplier is null"); + Objects.requireNonNull(accumulator, "accumulator is null"); return RxJavaPlugins.onAssembly(new ObservableScanSeed(this, seedSupplier, accumulator)); } @@ -11621,7 +11621,7 @@ public final Maybe singleElement() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single single(T defaultItem) { - ObjectHelper.requireNonNull(defaultItem, "defaultItem is null"); + Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new ObservableSingleSingle(this, defaultItem)); } @@ -11902,8 +11902,8 @@ public final Observable skipLast(long time, TimeUnit unit, Scheduler schedule @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable skipLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); // the internal buffer holds pairs of (timestamp, value) so double the default buffer size int s = bufferSize << 1; @@ -11931,7 +11931,7 @@ public final Observable skipLast(long time, TimeUnit unit, Scheduler schedule @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable skipUntil(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableSkipUntil(this, other)); } @@ -11954,7 +11954,7 @@ public final Observable skipUntil(ObservableSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable skipWhile(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableSkipWhile(this, predicate)); } @@ -12004,7 +12004,7 @@ public final Observable sorted() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable sorted(Comparator sortFunction) { - ObjectHelper.requireNonNull(sortFunction, "sortFunction is null"); + Objects.requireNonNull(sortFunction, "sortFunction is null"); return toList().toObservable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.>identity()); } @@ -12054,7 +12054,7 @@ public final Observable startWithIterable(Iterable items) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable startWith(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return concatArray(other, this); } @@ -12081,7 +12081,7 @@ public final Observable startWith(ObservableSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable startWithItem(T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return concatArray(just(item), this); } @@ -12213,9 +12213,9 @@ public final Disposable subscribe(Consumer onNext, Consumer onNext, Consumer onError, Action onComplete) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); LambdaObserver ls = new LambdaObserver(onNext, onError, onComplete, Functions.emptyConsumer()); @@ -12227,11 +12227,11 @@ public final Disposable subscribe(Consumer onNext, Consumer observer) { - ObjectHelper.requireNonNull(observer, "observer is null"); + Objects.requireNonNull(observer, "observer is null"); try { observer = RxJavaPlugins.onSubscribe(this, observer); - ObjectHelper.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null Observer. Please change the handler provided to RxJavaPlugins.setOnObservableSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); + Objects.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null Observer. Please change the handler provided to RxJavaPlugins.setOnObservableSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); subscribeActual(observer); } catch (NullPointerException e) { // NOPMD @@ -12309,7 +12309,7 @@ public final > E subscribeWith(E observer) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable subscribeOn(Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableSubscribeOn(this, scheduler)); } @@ -12332,7 +12332,7 @@ public final Observable subscribeOn(Scheduler scheduler) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable switchIfEmpty(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchIfEmpty(this, other)); } @@ -12391,7 +12391,7 @@ public final Observable switchMap(Function Observable switchMap(Function> mapper, int bufferSize) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (this instanceof ScalarSupplier) { @SuppressWarnings("unchecked") @@ -12441,7 +12441,7 @@ public final Observable switchMap(Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable(this, mapper, false)); } @@ -12483,7 +12483,7 @@ public final Completable switchMapCompletable(@NonNull Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable(this, mapper, true)); } @@ -12519,7 +12519,7 @@ public final Completable switchMapCompletableDelayError(@NonNull Function Observable switchMapMaybe(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe(this, mapper, false)); } @@ -12545,7 +12545,7 @@ public final Observable switchMapMaybe(@NonNull Function Observable switchMapMaybeDelayError(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe(this, mapper, true)); } @@ -12576,7 +12576,7 @@ public final Observable switchMapMaybeDelayError(@NonNull Function Observable switchMapSingle(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapSingle(this, mapper, false)); } @@ -12608,7 +12608,7 @@ public final Observable switchMapSingle(@NonNull Function Observable switchMapSingleDelayError(@NonNull Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapSingle(this, mapper, true)); } @@ -12671,7 +12671,7 @@ public final Observable switchMapDelayError(Function Observable switchMapDelayError(Function> mapper, int bufferSize) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (this instanceof ScalarSupplier) { @SuppressWarnings("unchecked") @@ -12894,8 +12894,8 @@ public final Observable takeLast(long count, long time, TimeUnit unit, Schedu @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable takeLast(long count, long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); @@ -13067,7 +13067,7 @@ public final Observable takeLast(long time, TimeUnit unit, Scheduler schedule @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable takeUntil(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableTakeUntil(this, other)); } @@ -13096,7 +13096,7 @@ public final Observable takeUntil(ObservableSource other) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable takeUntil(Predicate stopPredicate) { - ObjectHelper.requireNonNull(stopPredicate, "stopPredicate is null"); + Objects.requireNonNull(stopPredicate, "stopPredicate is null"); return RxJavaPlugins.onAssembly(new ObservableTakeUntilPredicate(this, stopPredicate)); } @@ -13120,7 +13120,7 @@ public final Observable takeUntil(Predicate stopPredicate) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable takeWhile(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableTakeWhile(this, predicate)); } @@ -13176,8 +13176,8 @@ public final Observable throttleFirst(long windowDuration, TimeUnit unit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable throttleFirst(long skipDuration, TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableThrottleFirstTimed(this, skipDuration, unit, scheduler)); } @@ -13363,8 +13363,8 @@ public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler scheduler, boolean emitLast) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableThrottleLatest(this, timeout, unit, scheduler, emitLast)); } @@ -13516,8 +13516,8 @@ public final Observable> timeInterval(TimeUnit unit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. public final Observable> timeInterval(TimeUnit unit, Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableTimeInterval(this, unit, scheduler)); } @@ -13581,7 +13581,7 @@ public final Observable timeout(Function Observable timeout(Function> itemTimeoutIndicator, ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(null, itemTimeoutIndicator, other); } @@ -13634,7 +13634,7 @@ public final Observable timeout(long timeout, TimeUnit timeUnit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) public final Observable timeout(long timeout, TimeUnit timeUnit, ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, Schedulers.computation()); } @@ -13665,7 +13665,7 @@ public final Observable timeout(long timeout, TimeUnit timeUnit, ObservableSo @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler, ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, scheduler); } @@ -13728,7 +13728,7 @@ public final Observable timeout(long timeout, TimeUnit timeUnit, Scheduler sc @SchedulerSupport(SchedulerSupport.NONE) public final Observable timeout(ObservableSource firstTimeoutIndicator, Function> itemTimeoutIndicator) { - ObjectHelper.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); + Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, null); } @@ -13770,15 +13770,15 @@ public final Observable timeout( ObservableSource firstTimeoutIndicator, Function> itemTimeoutIndicator, ObservableSource other) { - ObjectHelper.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, other); } private Observable timeout0(long timeout, TimeUnit timeUnit, ObservableSource other, Scheduler scheduler) { - ObjectHelper.requireNonNull(timeUnit, "timeUnit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(timeUnit, "timeUnit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableTimeoutTimed(this, timeout, timeUnit, scheduler, other)); } @@ -13786,7 +13786,7 @@ private Observable timeout0( ObservableSource firstTimeoutIndicator, Function> itemTimeoutIndicator, ObservableSource other) { - ObjectHelper.requireNonNull(itemTimeoutIndicator, "itemTimeoutIndicator is null"); + Objects.requireNonNull(itemTimeoutIndicator, "itemTimeoutIndicator is null"); return RxJavaPlugins.onAssembly(new ObservableTimeout(this, firstTimeoutIndicator, itemTimeoutIndicator, other)); } @@ -13875,8 +13875,8 @@ public final Observable> timestamp(TimeUnit unit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. public final Observable> timestamp(final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return map(Functions.timestampWith(unit, scheduler)); } @@ -13898,7 +13898,7 @@ public final Observable> timestamp(final TimeUnit unit, final Scheduler @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final R to(@NonNull ObservableConverter converter) { - return ObjectHelper.requireNonNull(converter, "converter is null").apply(this); + return Objects.requireNonNull(converter, "converter is null").apply(this); } /** @@ -13994,7 +13994,7 @@ public final Single> toList(final int capacityHint) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final > Single toList(Supplier collectionSupplier) { - ObjectHelper.requireNonNull(collectionSupplier, "collectionSupplier is null"); + Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableToListSingle(this, collectionSupplier)); } @@ -14025,7 +14025,7 @@ public final > Single toList(Supplier coll @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single> toMap(final Function keySelector) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); } @@ -14061,8 +14061,8 @@ public final Single> toMap(final Function public final Single> toMap( final Function keySelector, final Function valueSelector) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); } @@ -14098,9 +14098,9 @@ public final Single> toMap( final Function keySelector, final Function valueSelector, Supplier> mapSupplier) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); - ObjectHelper.requireNonNull(mapSupplier, "mapSupplier is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(mapSupplier, "mapSupplier is null"); return collect(mapSupplier, Functions.toMapKeyValueSelector(keySelector, valueSelector)); } @@ -14200,10 +14200,10 @@ public final Single>> toMultimap( final Function valueSelector, final Supplier>> mapSupplier, final Function> collectionFactory) { - ObjectHelper.requireNonNull(keySelector, "keySelector is null"); - ObjectHelper.requireNonNull(valueSelector, "valueSelector is null"); - ObjectHelper.requireNonNull(mapSupplier, "mapSupplier is null"); - ObjectHelper.requireNonNull(collectionFactory, "collectionFactory is null"); + Objects.requireNonNull(keySelector, "keySelector is null"); + Objects.requireNonNull(valueSelector, "valueSelector is null"); + Objects.requireNonNull(mapSupplier, "mapSupplier is null"); + Objects.requireNonNull(collectionFactory, "collectionFactory is null"); return collect(mapSupplier, Functions.toMultimapKeyValueSelector(keySelector, valueSelector, collectionFactory)); } @@ -14352,7 +14352,7 @@ public final Single> toSortedList() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single> toSortedList(final Comparator comparator) { - ObjectHelper.requireNonNull(comparator, "comparator is null"); + Objects.requireNonNull(comparator, "comparator is null"); return toList().map(Functions.listSorter(comparator)); } @@ -14383,7 +14383,7 @@ public final Single> toSortedList(final Comparator comparator @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single> toSortedList(final Comparator comparator, int capacityHint) { - ObjectHelper.requireNonNull(comparator, "comparator is null"); + Objects.requireNonNull(comparator, "comparator is null"); return toList(capacityHint).map(Functions.listSorter(comparator)); } @@ -14438,7 +14438,7 @@ public final Single> toSortedList(int capacityHint) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable unsubscribeOn(Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableUnsubscribeOn(this, scheduler)); } @@ -14633,8 +14633,8 @@ public final Observable> window(long timespan, long timeskip, Time ObjectHelper.verifyPositive(timespan, "timespan"); ObjectHelper.verifyPositive(timeskip, "timeskip"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new ObservableWindowTimed(this, timespan, timeskip, unit, scheduler, Long.MAX_VALUE, bufferSize, false)); } @@ -14903,8 +14903,8 @@ public final Observable> window( long timespan, TimeUnit unit, Scheduler scheduler, long count, boolean restart, int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); - ObjectHelper.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(count, "count"); return RxJavaPlugins.onAssembly(new ObservableWindowTimed(this, timespan, timespan, unit, scheduler, count, bufferSize, restart)); } @@ -14970,7 +14970,7 @@ public final Observable> window(ObservableSource boundary) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable> window(ObservableSource boundary, int bufferSize) { - ObjectHelper.requireNonNull(boundary, "boundary is null"); + Objects.requireNonNull(boundary, "boundary is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableWindowBoundary(this, boundary, bufferSize)); } @@ -15046,8 +15046,8 @@ public final Observable> window( public final Observable> window( ObservableSource openingIndicator, Function> closingIndicator, int bufferSize) { - ObjectHelper.requireNonNull(openingIndicator, "openingIndicator is null"); - ObjectHelper.requireNonNull(closingIndicator, "closingIndicator is null"); + Objects.requireNonNull(openingIndicator, "openingIndicator is null"); + Objects.requireNonNull(closingIndicator, "closingIndicator is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableWindowBoundarySelector(this, openingIndicator, closingIndicator, bufferSize)); } @@ -15079,8 +15079,8 @@ public final Observable> window( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable withLatestFrom(ObservableSource other, BiFunction combiner) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new ObservableWithLatestFrom(this, combiner, other)); } @@ -15114,9 +15114,9 @@ public final Observable withLatestFrom(ObservableSource o public final Observable withLatestFrom( ObservableSource o1, ObservableSource o2, Function3 combiner) { - ObjectHelper.requireNonNull(o1, "o1 is null"); - ObjectHelper.requireNonNull(o2, "o2 is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(o1, "o1 is null"); + Objects.requireNonNull(o2, "o2 is null"); + Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new ObservableSource[] { o1, o2 }, f); } @@ -15153,10 +15153,10 @@ public final Observable withLatestFrom( ObservableSource o1, ObservableSource o2, ObservableSource o3, Function4 combiner) { - ObjectHelper.requireNonNull(o1, "o1 is null"); - ObjectHelper.requireNonNull(o2, "o2 is null"); - ObjectHelper.requireNonNull(o3, "o3 is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(o1, "o1 is null"); + Objects.requireNonNull(o2, "o2 is null"); + Objects.requireNonNull(o3, "o3 is null"); + Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new ObservableSource[] { o1, o2, o3 }, f); } @@ -15195,11 +15195,11 @@ public final Observable withLatestFrom( ObservableSource o1, ObservableSource o2, ObservableSource o3, ObservableSource o4, Function5 combiner) { - ObjectHelper.requireNonNull(o1, "o1 is null"); - ObjectHelper.requireNonNull(o2, "o2 is null"); - ObjectHelper.requireNonNull(o3, "o3 is null"); - ObjectHelper.requireNonNull(o4, "o4 is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(o1, "o1 is null"); + Objects.requireNonNull(o2, "o2 is null"); + Objects.requireNonNull(o3, "o3 is null"); + Objects.requireNonNull(o4, "o4 is null"); + Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new ObservableSource[] { o1, o2, o3, o4 }, f); } @@ -15228,8 +15228,8 @@ public final Observable withLatestFrom( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable withLatestFrom(ObservableSource[] others, Function combiner) { - ObjectHelper.requireNonNull(others, "others is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(others, "others is null"); + Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany(this, others, combiner)); } @@ -15257,8 +15257,8 @@ public final Observable withLatestFrom(ObservableSource[] others, Func @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable withLatestFrom(Iterable> others, Function combiner) { - ObjectHelper.requireNonNull(others, "others is null"); - ObjectHelper.requireNonNull(combiner, "combiner is null"); + Objects.requireNonNull(others, "others is null"); + Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany(this, others, combiner)); } @@ -15291,8 +15291,8 @@ public final Observable withLatestFrom(Iterable Observable zipWith(Iterable other, BiFunction zipper) { - ObjectHelper.requireNonNull(other, "other is null"); - ObjectHelper.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(other, "other is null"); + Objects.requireNonNull(zipper, "zipper is null"); return RxJavaPlugins.onAssembly(new ObservableZipIterable(this, other, zipper)); } @@ -15335,7 +15335,7 @@ public final Observable zipWith(Iterable other, BiFunction Observable zipWith(ObservableSource other, BiFunction zipper) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return zip(this, other, zipper); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Single.java b/src/main/java/io/reactivex/rxjava3/core/Single.java index 3cffa8dfad..27fad6d53e 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Single.java +++ b/src/main/java/io/reactivex/rxjava3/core/Single.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.core; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.concurrent.*; import org.reactivestreams.Publisher; @@ -132,7 +133,7 @@ public abstract class Single implements SingleSource { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single amb(final Iterable> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new SingleAmb(null, sources)); } @@ -207,7 +208,7 @@ public static Flowable concat(Iterable Observable concat(ObservableSource> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableConcatMap(sources, SingleInternalHelper.toObservable(), 2, ErrorMode.IMMEDIATE)); } @@ -260,7 +261,7 @@ public static Flowable concat(Publisher Flowable concat(Publisher> sources, int prefetch) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapPublisher(sources, SingleInternalHelper.toFlowable(), prefetch, ErrorMode.IMMEDIATE)); } @@ -292,8 +293,8 @@ public static Flowable concat(Publisher Flowable concat( SingleSource source1, SingleSource source2 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return concat(Flowable.fromArray(source1, source2)); } @@ -327,9 +328,9 @@ public static Flowable concat( SingleSource source1, SingleSource source2, SingleSource source3 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return concat(Flowable.fromArray(source1, source2, source3)); } @@ -365,10 +366,10 @@ public static Flowable concat( SingleSource source1, SingleSource source2, SingleSource source3, SingleSource source4 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return concat(Flowable.fromArray(source1, source2, source3, source4)); } @@ -523,7 +524,7 @@ public static Flowable concatEager(Iterable Single create(SingleOnSubscribe source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new SingleCreate(source)); } @@ -545,7 +546,7 @@ public static Single create(SingleOnSubscribe source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single defer(final Supplier> singleSupplier) { - ObjectHelper.requireNonNull(singleSupplier, "singleSupplier is null"); + Objects.requireNonNull(singleSupplier, "singleSupplier is null"); return RxJavaPlugins.onAssembly(new SingleDefer(singleSupplier)); } @@ -566,7 +567,7 @@ public static Single defer(final Supplier Single error(final Supplier errorSupplier) { - ObjectHelper.requireNonNull(errorSupplier, "errorSupplier is null"); + Objects.requireNonNull(errorSupplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new SingleError(errorSupplier)); } @@ -592,7 +593,7 @@ public static Single error(final Supplier errorSuppl @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single error(final Throwable exception) { - ObjectHelper.requireNonNull(exception, "exception is null"); + Objects.requireNonNull(exception, "exception is null"); return error(Functions.justSupplier(exception)); } @@ -628,7 +629,7 @@ public static Single error(final Throwable exception) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single fromCallable(final Callable callable) { - ObjectHelper.requireNonNull(callable, "callable is null"); + Objects.requireNonNull(callable, "callable is null"); return RxJavaPlugins.onAssembly(new SingleFromCallable(callable)); } @@ -793,7 +794,7 @@ public static Single fromFuture(Future future, Scheduler sch @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single fromPublisher(final Publisher publisher) { - ObjectHelper.requireNonNull(publisher, "publisher is null"); + Objects.requireNonNull(publisher, "publisher is null"); return RxJavaPlugins.onAssembly(new SingleFromPublisher(publisher)); } @@ -817,7 +818,7 @@ public static Single fromPublisher(final Publisher publisher @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single fromObservable(ObservableSource observableSource) { - ObjectHelper.requireNonNull(observableSource, "observableSource is null"); + Objects.requireNonNull(observableSource, "observableSource is null"); return RxJavaPlugins.onAssembly(new ObservableSingleSingle(observableSource, null)); } @@ -855,7 +856,7 @@ public static Single fromObservable(ObservableSource observa @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single fromSupplier(final Supplier supplier) { - ObjectHelper.requireNonNull(supplier, "supplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new SingleFromSupplier(supplier)); } @@ -882,7 +883,7 @@ public static Single fromSupplier(final Supplier supplier) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public static Single just(final T item) { - ObjectHelper.requireNonNull(item, "item is null"); + Objects.requireNonNull(item, "item is null"); return RxJavaPlugins.onAssembly(new SingleJust(item)); } @@ -960,7 +961,7 @@ public static Flowable merge(Iterable @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "unchecked", "rawtypes" }) public static Flowable merge(Publisher> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, SingleInternalHelper.toFlowable(), false, Integer.MAX_VALUE, Flowable.bufferSize())); } @@ -991,7 +992,7 @@ public static Flowable merge(Publisher Single merge(SingleSource> source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new SingleFlatMap, T>(source, (Function)Functions.identity())); } @@ -1039,8 +1040,8 @@ public static Single merge(SingleSource Flowable merge( SingleSource source1, SingleSource source2 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return merge(Flowable.fromArray(source1, source2)); } @@ -1091,9 +1092,9 @@ public static Flowable merge( SingleSource source1, SingleSource source2, SingleSource source3 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return merge(Flowable.fromArray(source1, source2, source3)); } @@ -1146,10 +1147,10 @@ public static Flowable merge( SingleSource source1, SingleSource source2, SingleSource source3, SingleSource source4 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return merge(Flowable.fromArray(source1, source2, source3, source4)); } @@ -1203,7 +1204,7 @@ public static Flowable mergeDelayError(Iterable Flowable mergeDelayError(Publisher> sources) { - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, SingleInternalHelper.toFlowable(), true, Integer.MAX_VALUE, Flowable.bufferSize())); } @@ -1240,8 +1241,8 @@ public static Flowable mergeDelayError(Publisher Flowable mergeDelayError( SingleSource source1, SingleSource source2 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return mergeDelayError(Flowable.fromArray(source1, source2)); } @@ -1281,9 +1282,9 @@ public static Flowable mergeDelayError( SingleSource source1, SingleSource source2, SingleSource source3 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return mergeDelayError(Flowable.fromArray(source1, source2, source3)); } @@ -1325,10 +1326,10 @@ public static Flowable mergeDelayError( SingleSource source1, SingleSource source2, SingleSource source3, SingleSource source4 ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return mergeDelayError(Flowable.fromArray(source1, source2, source3, source4)); } @@ -1391,8 +1392,8 @@ public static Single timer(long delay, TimeUnit unit) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public static Single timer(final long delay, final TimeUnit unit, final Scheduler scheduler) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new SingleTimer(delay, unit, scheduler)); } @@ -1414,8 +1415,8 @@ public static Single timer(final long delay, final TimeUnit unit, final Sc @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single equals(final SingleSource first, final SingleSource second) { // NOPMD - ObjectHelper.requireNonNull(first, "first is null"); - ObjectHelper.requireNonNull(second, "second is null"); + Objects.requireNonNull(first, "first is null"); + Objects.requireNonNull(second, "second is null"); return RxJavaPlugins.onAssembly(new SingleEquals(first, second)); } @@ -1440,7 +1441,7 @@ public static Single equals(final SingleSource first, @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single unsafeCreate(SingleSource onSubscribe) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); if (onSubscribe instanceof Single) { throw new IllegalArgumentException("unsafeCreate(Single) should be upgraded"); } @@ -1510,9 +1511,9 @@ public static Single using( final Function> singleFunction, final Consumer disposer, final boolean eager) { - ObjectHelper.requireNonNull(resourceSupplier, "resourceSupplier is null"); - ObjectHelper.requireNonNull(singleFunction, "singleFunction is null"); - ObjectHelper.requireNonNull(disposer, "disposer is null"); + Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); + Objects.requireNonNull(singleFunction, "singleFunction is null"); + Objects.requireNonNull(disposer, "disposer is null"); return RxJavaPlugins.onAssembly(new SingleUsing(resourceSupplier, singleFunction, disposer, eager)); } @@ -1534,7 +1535,7 @@ public static Single using( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single wrap(SingleSource source) { - ObjectHelper.requireNonNull(source, "source is null"); + Objects.requireNonNull(source, "source is null"); if (source instanceof Single) { return RxJavaPlugins.onAssembly((Single)source); } @@ -1574,8 +1575,8 @@ public static Single wrap(SingleSource source) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single zip(final Iterable> sources, Function zipper) { - ObjectHelper.requireNonNull(zipper, "zipper is null"); - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new SingleZipIterable(sources, zipper)); } @@ -1610,8 +1611,8 @@ public static Single zip( SingleSource source1, SingleSource source2, BiFunction zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), source1, source2); } @@ -1650,9 +1651,9 @@ public static Single zip( SingleSource source3, Function3 zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3); } @@ -1694,10 +1695,10 @@ public static Single zip( SingleSource source3, SingleSource source4, Function4 zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4); } @@ -1743,11 +1744,11 @@ public static Single zip( SingleSource source5, Function5 zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5); } @@ -1796,12 +1797,12 @@ public static Single zip( SingleSource source5, SingleSource source6, Function6 zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6); } @@ -1854,13 +1855,13 @@ public static Single zip( SingleSource source7, Function7 zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7); } @@ -1916,14 +1917,14 @@ public static Single zip( SingleSource source7, SingleSource source8, Function8 zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -1983,15 +1984,15 @@ public static Single zip( SingleSource source9, Function9 zipper ) { - ObjectHelper.requireNonNull(source1, "source1 is null"); - ObjectHelper.requireNonNull(source2, "source2 is null"); - ObjectHelper.requireNonNull(source3, "source3 is null"); - ObjectHelper.requireNonNull(source4, "source4 is null"); - ObjectHelper.requireNonNull(source5, "source5 is null"); - ObjectHelper.requireNonNull(source6, "source6 is null"); - ObjectHelper.requireNonNull(source7, "source7 is null"); - ObjectHelper.requireNonNull(source8, "source8 is null"); - ObjectHelper.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(source9, "source9 is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8, source9); } @@ -2027,8 +2028,8 @@ public static Single zip( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single zipArray(Function zipper, SingleSource... sources) { - ObjectHelper.requireNonNull(zipper, "zipper is null"); - ObjectHelper.requireNonNull(sources, "sources is null"); + Objects.requireNonNull(zipper, "zipper is null"); + Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return error(new NoSuchElementException()); } @@ -2053,7 +2054,7 @@ public static Single zipArray(Function @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") public final Single ambWith(SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -2099,7 +2100,7 @@ public final Single hide() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single compose(SingleTransformer transformer) { - return wrap(((SingleTransformer) ObjectHelper.requireNonNull(transformer, "transformer is null")).apply(this)); + return wrap(((SingleTransformer) Objects.requireNonNull(transformer, "transformer is null")).apply(this)); } /** @@ -2140,7 +2141,7 @@ public final Single cache() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single cast(final Class clazz) { - ObjectHelper.requireNonNull(clazz, "clazz is null"); + Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -2259,8 +2260,8 @@ public final Single delay(final long time, final TimeUnit unit, final Schedul @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Single delay(final long time, final TimeUnit unit, final Scheduler scheduler, boolean delayError) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new SingleDelay(this, time, unit, scheduler, delayError)); } @@ -2284,7 +2285,7 @@ public final Single delay(final long time, final TimeUnit unit, final Schedul @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single delaySubscription(CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new SingleDelayWithCompletable(this, other)); } @@ -2309,7 +2310,7 @@ public final Single delaySubscription(CompletableSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single delaySubscription(SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new SingleDelayWithSingle(this, other)); } @@ -2334,7 +2335,7 @@ public final Single delaySubscription(SingleSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single delaySubscription(ObservableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new SingleDelayWithObservable(this, other)); } @@ -2364,7 +2365,7 @@ public final Single delaySubscription(ObservableSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single delaySubscription(Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new SingleDelayWithPublisher(this, other)); } @@ -2445,7 +2446,7 @@ public final Single delaySubscription(long time, TimeUnit unit, Scheduler sch @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe dematerialize(Function> selector) { - ObjectHelper.requireNonNull(selector, "selector is null"); + Objects.requireNonNull(selector, "selector is null"); return RxJavaPlugins.onAssembly(new SingleDematerialize(this, selector)); } @@ -2469,7 +2470,7 @@ public final Maybe dematerialize(Function> sel @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doAfterSuccess(Consumer onAfterSuccess) { - ObjectHelper.requireNonNull(onAfterSuccess, "onAfterSuccess is null"); + Objects.requireNonNull(onAfterSuccess, "onAfterSuccess is null"); return RxJavaPlugins.onAssembly(new SingleDoAfterSuccess(this, onAfterSuccess)); } @@ -2498,7 +2499,7 @@ public final Single doAfterSuccess(Consumer onAfterSuccess) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doAfterTerminate(Action onAfterTerminate) { - ObjectHelper.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); + Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); return RxJavaPlugins.onAssembly(new SingleDoAfterTerminate(this, onAfterTerminate)); } @@ -2525,7 +2526,7 @@ public final Single doAfterTerminate(Action onAfterTerminate) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doFinally(Action onFinally) { - ObjectHelper.requireNonNull(onFinally, "onFinally is null"); + Objects.requireNonNull(onFinally, "onFinally is null"); return RxJavaPlugins.onAssembly(new SingleDoFinally(this, onFinally)); } @@ -2547,7 +2548,7 @@ public final Single doFinally(Action onFinally) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doOnSubscribe(final Consumer onSubscribe) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); return RxJavaPlugins.onAssembly(new SingleDoOnSubscribe(this, onSubscribe)); } @@ -2574,7 +2575,7 @@ public final Single doOnSubscribe(final Consumer onSubscr @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doOnTerminate(final Action onTerminate) { - ObjectHelper.requireNonNull(onTerminate, "onTerminate is null"); + Objects.requireNonNull(onTerminate, "onTerminate is null"); return RxJavaPlugins.onAssembly(new SingleDoOnTerminate(this, onTerminate)); } @@ -2596,7 +2597,7 @@ public final Single doOnTerminate(final Action onTerminate) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doOnSuccess(final Consumer onSuccess) { - ObjectHelper.requireNonNull(onSuccess, "onSuccess is null"); + Objects.requireNonNull(onSuccess, "onSuccess is null"); return RxJavaPlugins.onAssembly(new SingleDoOnSuccess(this, onSuccess)); } @@ -2617,7 +2618,7 @@ public final Single doOnSuccess(final Consumer onSuccess) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doOnEvent(final BiConsumer onEvent) { - ObjectHelper.requireNonNull(onEvent, "onEvent is null"); + Objects.requireNonNull(onEvent, "onEvent is null"); return RxJavaPlugins.onAssembly(new SingleDoOnEvent(this, onEvent)); } @@ -2639,7 +2640,7 @@ public final Single doOnEvent(final BiConsumer @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doOnError(final Consumer onError) { - ObjectHelper.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onError, "onError is null"); return RxJavaPlugins.onAssembly(new SingleDoOnError(this, onError)); } @@ -2662,7 +2663,7 @@ public final Single doOnError(final Consumer onError) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single doOnDispose(final Action onDispose) { - ObjectHelper.requireNonNull(onDispose, "onDispose is null"); + Objects.requireNonNull(onDispose, "onDispose is null"); return RxJavaPlugins.onAssembly(new SingleDoOnDispose(this, onDispose)); } @@ -2687,7 +2688,7 @@ public final Single doOnDispose(final Action onDispose) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Maybe filter(Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate is null"); + Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new MaybeFilterSingle(this, predicate)); } @@ -2711,7 +2712,7 @@ public final Maybe filter(Predicate predicate) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single flatMap(Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleFlatMap(this, mapper)); } @@ -2735,7 +2736,7 @@ public final Single flatMap(Function Maybe flatMapMaybe(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleFlatMapMaybe(this, mapper)); } @@ -2764,7 +2765,7 @@ public final Maybe flatMapMaybe(final Function Flowable flatMapPublisher(Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleFlatMapPublisher(this, mapper)); } @@ -2793,7 +2794,7 @@ public final Flowable flatMapPublisher(Function Flowable flattenAsFlowable(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleFlatMapIterableFlowable(this, mapper)); } @@ -2819,7 +2820,7 @@ public final Flowable flattenAsFlowable(final Function Observable flattenAsObservable(final Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleFlatMapIterableObservable(this, mapper)); } @@ -2843,7 +2844,7 @@ public final Observable flattenAsObservable(final Function Observable flatMapObservable(Function> mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleFlatMapObservable(this, mapper)); } @@ -2868,7 +2869,7 @@ public final Observable flatMapObservable(Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleFlatMapCompletable(this, mapper)); } @@ -3042,7 +3043,7 @@ public final T blockingGet() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single lift(final SingleOperator lift) { - ObjectHelper.requireNonNull(lift, "lift is null"); + Objects.requireNonNull(lift, "lift is null"); return RxJavaPlugins.onAssembly(new SingleLift(this, lift)); } @@ -3066,7 +3067,7 @@ public final Single lift(final SingleOperator lif @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single map(Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleMap(this, mapper)); } @@ -3130,8 +3131,8 @@ public final Single contains(Object value) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single contains(final Object value, final BiPredicate comparer) { - ObjectHelper.requireNonNull(value, "value is null"); - ObjectHelper.requireNonNull(comparer, "comparer is null"); + Objects.requireNonNull(value, "value is null"); + Objects.requireNonNull(comparer, "comparer is null"); return RxJavaPlugins.onAssembly(new SingleContains(this, value, comparer)); } @@ -3184,7 +3185,7 @@ public final Flowable mergeWith(SingleSource other) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Single observeOn(final Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new SingleObserveOn(this, scheduler)); } @@ -3218,7 +3219,7 @@ public final Single observeOn(final Scheduler scheduler) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single onErrorReturn(final Function resumeFunction) { - ObjectHelper.requireNonNull(resumeFunction, "resumeFunction is null"); + Objects.requireNonNull(resumeFunction, "resumeFunction is null"); return RxJavaPlugins.onAssembly(new SingleOnErrorReturn(this, resumeFunction, null)); } @@ -3238,7 +3239,7 @@ public final Single onErrorReturn(final Function resu @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single onErrorReturnItem(final T value) { - ObjectHelper.requireNonNull(value, "value is null"); + Objects.requireNonNull(value, "value is null"); return RxJavaPlugins.onAssembly(new SingleOnErrorReturn(this, null, value)); } @@ -3273,7 +3274,7 @@ public final Single onErrorReturnItem(final T value) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single onErrorResumeWith(final SingleSource resumeSingleInCaseOfError) { - ObjectHelper.requireNonNull(resumeSingleInCaseOfError, "resumeSingleInCaseOfError is null"); + Objects.requireNonNull(resumeSingleInCaseOfError, "resumeSingleInCaseOfError is null"); return onErrorResumeNext(Functions.justFunction(resumeSingleInCaseOfError)); } @@ -3310,7 +3311,7 @@ public final Single onErrorResumeWith(final SingleSource resumeS @SchedulerSupport(SchedulerSupport.NONE) public final Single onErrorResumeNext( final Function> resumeFunctionInCaseOfError) { - ObjectHelper.requireNonNull(resumeFunctionInCaseOfError, "resumeFunctionInCaseOfError is null"); + Objects.requireNonNull(resumeFunctionInCaseOfError, "resumeFunctionInCaseOfError is null"); return RxJavaPlugins.onAssembly(new SingleResumeNext(this, resumeFunctionInCaseOfError)); } @@ -3614,7 +3615,7 @@ public final Disposable subscribe() { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Disposable subscribe(final BiConsumer onCallback) { - ObjectHelper.requireNonNull(onCallback, "onCallback is null"); + Objects.requireNonNull(onCallback, "onCallback is null"); BiConsumerSingleObserver observer = new BiConsumerSingleObserver(onCallback); subscribe(observer); @@ -3672,8 +3673,8 @@ public final Disposable subscribe(Consumer onSuccess) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Disposable subscribe(final Consumer onSuccess, final Consumer onError) { - ObjectHelper.requireNonNull(onSuccess, "onSuccess is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onSuccess, "onSuccess is null"); + Objects.requireNonNull(onError, "onError is null"); ConsumerSingleObserver observer = new ConsumerSingleObserver(onSuccess, onError); subscribe(observer); @@ -3683,11 +3684,11 @@ public final Disposable subscribe(final Consumer onSuccess, final Con @SchedulerSupport(SchedulerSupport.NONE) @Override public final void subscribe(SingleObserver observer) { - ObjectHelper.requireNonNull(observer, "observer is null"); + Objects.requireNonNull(observer, "observer is null"); observer = RxJavaPlugins.onSubscribe(this, observer); - ObjectHelper.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null SingleObserver. Please check the handler provided to RxJavaPlugins.setOnSingleSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); + Objects.requireNonNull(observer, "The RxJavaPlugins.onSubscribe hook returned a null SingleObserver. Please check the handler provided to RxJavaPlugins.setOnSingleSubscribe for invalid null returns. Further reading: https://github.com/ReactiveX/RxJava/wiki/Plugins"); try { subscribeActual(observer); @@ -3763,7 +3764,7 @@ public final > E subscribeWith(E observer) { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Single subscribeOn(final Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new SingleSubscribeOn(this, scheduler)); } @@ -3788,7 +3789,7 @@ public final Single subscribeOn(final Scheduler scheduler) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single takeUntil(final CompletableSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return takeUntil(new CompletableToFlowable(other)); } @@ -3820,7 +3821,7 @@ public final Single takeUntil(final CompletableSource other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single takeUntil(final Publisher other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new SingleTakeUntil(this, other)); } @@ -3846,7 +3847,7 @@ public final Single takeUntil(final Publisher other) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single takeUntil(final SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return takeUntil(new SingleToFlowable(other)); } @@ -3912,7 +3913,7 @@ public final Single timeout(long timeout, TimeUnit unit, Scheduler scheduler) @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Single timeout(long timeout, TimeUnit unit, Scheduler scheduler, SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(timeout, unit, scheduler, other); } @@ -3940,13 +3941,13 @@ public final Single timeout(long timeout, TimeUnit unit, Scheduler scheduler, @NonNull @SchedulerSupport(SchedulerSupport.COMPUTATION) public final Single timeout(long timeout, TimeUnit unit, SingleSource other) { - ObjectHelper.requireNonNull(other, "other is null"); + Objects.requireNonNull(other, "other is null"); return timeout0(timeout, unit, Schedulers.computation(), other); } private Single timeout0(final long timeout, final TimeUnit unit, final Scheduler scheduler, final SingleSource other) { - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new SingleTimeout(this, timeout, unit, scheduler, other)); } @@ -3970,7 +3971,7 @@ private Single timeout0(final long timeout, final TimeUnit unit, final Schedu @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final R to(@NonNull SingleConverter converter) { - return ObjectHelper.requireNonNull(converter, "converter is null").apply(this); + return Objects.requireNonNull(converter, "converter is null").apply(this); } /** @@ -4095,7 +4096,7 @@ public final Observable toObservable() { @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) public final Single unsubscribeOn(final Scheduler scheduler) { - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new SingleUnsubscribeOn(this, scheduler)); } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java index 245c41305b..c9c0a8988c 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java @@ -16,7 +16,6 @@ import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.exceptions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.*; /** @@ -41,10 +40,10 @@ public CompositeDisposable() { * @throws NullPointerException if {@code disposables} or any of its array items is null */ public CompositeDisposable(@NonNull Disposable... disposables) { - ObjectHelper.requireNonNull(disposables, "disposables is null"); + Objects.requireNonNull(disposables, "disposables is null"); this.resources = new OpenHashSet(disposables.length + 1); for (Disposable d : disposables) { - ObjectHelper.requireNonNull(d, "A Disposable in the disposables array is null"); + Objects.requireNonNull(d, "A Disposable in the disposables array is null"); this.resources.add(d); } } @@ -55,10 +54,10 @@ public CompositeDisposable(@NonNull Disposable... disposables) { * @throws NullPointerException if {@code disposables} or any of its items is null */ public CompositeDisposable(@NonNull Iterable disposables) { - ObjectHelper.requireNonNull(disposables, "disposables is null"); + Objects.requireNonNull(disposables, "disposables is null"); this.resources = new OpenHashSet(); for (Disposable d : disposables) { - ObjectHelper.requireNonNull(d, "A Disposable item in the disposables sequence is null"); + Objects.requireNonNull(d, "A Disposable item in the disposables sequence is null"); this.resources.add(d); } } @@ -95,7 +94,7 @@ public boolean isDisposed() { */ @Override public boolean add(@NonNull Disposable disposable) { - ObjectHelper.requireNonNull(disposable, "disposable is null"); + Objects.requireNonNull(disposable, "disposable is null"); if (!disposed) { synchronized (this) { if (!disposed) { @@ -121,7 +120,7 @@ public boolean add(@NonNull Disposable disposable) { * @throws NullPointerException if {@code disposables} or any of its array items is null */ public boolean addAll(@NonNull Disposable... disposables) { - ObjectHelper.requireNonNull(disposables, "disposables is null"); + Objects.requireNonNull(disposables, "disposables is null"); if (!disposed) { synchronized (this) { if (!disposed) { @@ -131,7 +130,7 @@ public boolean addAll(@NonNull Disposable... disposables) { resources = set; } for (Disposable d : disposables) { - ObjectHelper.requireNonNull(d, "A Disposable in the disposables array is null"); + Objects.requireNonNull(d, "A Disposable in the disposables array is null"); set.add(d); } return true; @@ -168,7 +167,7 @@ public boolean remove(@NonNull Disposable disposable) { */ @Override public boolean delete(@NonNull Disposable disposable) { - ObjectHelper.requireNonNull(disposable, "disposables is null"); + Objects.requireNonNull(disposable, "disposables is null"); if (disposed) { return false; } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java b/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java index d566b744f4..2ff83d64ff 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.disposables; +import java.util.Objects; import java.util.concurrent.Future; import org.reactivestreams.Subscription; @@ -41,7 +42,7 @@ private Disposables() { */ @NonNull public static Disposable fromRunnable(@NonNull Runnable run) { - ObjectHelper.requireNonNull(run, "run is null"); + Objects.requireNonNull(run, "run is null"); return new RunnableDisposable(run); } @@ -53,7 +54,7 @@ public static Disposable fromRunnable(@NonNull Runnable run) { */ @NonNull public static Disposable fromAction(@NonNull Action run) { - ObjectHelper.requireNonNull(run, "run is null"); + Objects.requireNonNull(run, "run is null"); return new ActionDisposable(run); } @@ -65,7 +66,7 @@ public static Disposable fromAction(@NonNull Action run) { */ @NonNull public static Disposable fromFuture(@NonNull Future future) { - ObjectHelper.requireNonNull(future, "future is null"); + Objects.requireNonNull(future, "future is null"); return fromFuture(future, true); } @@ -78,7 +79,7 @@ public static Disposable fromFuture(@NonNull Future future) { */ @NonNull public static Disposable fromFuture(@NonNull Future future, boolean allowInterrupt) { - ObjectHelper.requireNonNull(future, "future is null"); + Objects.requireNonNull(future, "future is null"); return new FutureDisposable(future, allowInterrupt); } @@ -90,7 +91,7 @@ public static Disposable fromFuture(@NonNull Future future, boolean allowInte */ @NonNull public static Disposable fromSubscription(@NonNull Subscription subscription) { - ObjectHelper.requireNonNull(subscription, "subscription is null"); + Objects.requireNonNull(subscription, "subscription is null"); return new SubscriptionDisposable(subscription); } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/ReferenceDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/ReferenceDisposable.java index e5032d694b..eea584ab0b 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/ReferenceDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/ReferenceDisposable.java @@ -13,10 +13,10 @@ package io.reactivex.rxjava3.disposables; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.annotations.NonNull; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Base class for Disposable containers that manage some other type that @@ -29,7 +29,7 @@ abstract class ReferenceDisposable extends AtomicReference implements Disp private static final long serialVersionUID = 6537757548749041217L; ReferenceDisposable(T value) { - super(ObjectHelper.requireNonNull(value, "value is null")); + super(Objects.requireNonNull(value, "value is null")); } protected abstract void onDisposed(@NonNull T value); diff --git a/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java b/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java index 8b09993ac2..6916d219bf 100644 --- a/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.flowables; +import java.util.Objects; import java.util.concurrent.TimeUnit; import org.reactivestreams.Subscriber; @@ -235,8 +236,8 @@ public final Flowable refCount(int subscriberCount, long timeout, TimeUnit un @BackpressureSupport(BackpressureKind.PASS_THROUGH) public final Flowable refCount(int subscriberCount, long timeout, TimeUnit unit, Scheduler scheduler) { ObjectHelper.verifyPositive(subscriberCount, "subscriberCount"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableRefCount(this, subscriberCount, timeout, unit, scheduler)); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/disposables/DisposableHelper.java b/src/main/java/io/reactivex/rxjava3/internal/disposables/DisposableHelper.java index 3a8c5d7029..1f2d2122d1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/disposables/DisposableHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/disposables/DisposableHelper.java @@ -13,11 +13,11 @@ package io.reactivex.rxjava3.internal.disposables; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.ProtocolViolationException; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** @@ -75,7 +75,7 @@ public static boolean set(AtomicReference field, Disposable d) { * @return true if the operation succeeded, false */ public static boolean setOnce(AtomicReference field, Disposable d) { - ObjectHelper.requireNonNull(d, "d is null"); + Objects.requireNonNull(d, "d is null"); if (!field.compareAndSet(null, d)) { d.dispose(); if (field.get() != DISPOSED) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java b/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java index f3afcbf2c0..90928b64c6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java @@ -16,7 +16,6 @@ import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.ExceptionHelper; /** @@ -32,19 +31,19 @@ public ListCompositeDisposable() { } public ListCompositeDisposable(Disposable... resources) { - ObjectHelper.requireNonNull(resources, "resources is null"); + Objects.requireNonNull(resources, "resources is null"); this.resources = new LinkedList(); for (Disposable d : resources) { - ObjectHelper.requireNonNull(d, "Disposable item is null"); + Objects.requireNonNull(d, "Disposable item is null"); this.resources.add(d); } } public ListCompositeDisposable(Iterable resources) { - ObjectHelper.requireNonNull(resources, "resources is null"); + Objects.requireNonNull(resources, "resources is null"); this.resources = new LinkedList(); for (Disposable d : resources) { - ObjectHelper.requireNonNull(d, "Disposable item is null"); + Objects.requireNonNull(d, "Disposable item is null"); this.resources.add(d); } } @@ -74,7 +73,7 @@ public boolean isDisposed() { @Override public boolean add(Disposable d) { - ObjectHelper.requireNonNull(d, "d is null"); + Objects.requireNonNull(d, "d is null"); if (!disposed) { synchronized (this) { if (!disposed) { @@ -93,7 +92,7 @@ public boolean add(Disposable d) { } public boolean addAll(Disposable... ds) { - ObjectHelper.requireNonNull(ds, "ds is null"); + Objects.requireNonNull(ds, "ds is null"); if (!disposed) { synchronized (this) { if (!disposed) { @@ -103,7 +102,7 @@ public boolean addAll(Disposable... ds) { resources = set; } for (Disposable d : ds) { - ObjectHelper.requireNonNull(d, "d is null"); + Objects.requireNonNull(d, "d is null"); set.add(d); } return true; @@ -127,7 +126,7 @@ public boolean remove(Disposable d) { @Override public boolean delete(Disposable d) { - ObjectHelper.requireNonNull(d, "Disposable item is null"); + Objects.requireNonNull(d, "Disposable item is null"); if (disposed) { return false; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java index 2e44b86d04..2dcbbdea50 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java +++ b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java @@ -34,46 +34,46 @@ private Functions() { } public static Function toFunction(final BiFunction f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array2Func(f); } public static Function toFunction(final Function3 f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array3Func(f); } public static Function toFunction(final Function4 f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array4Func(f); } public static Function toFunction(final Function5 f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array5Func(f); } public static Function toFunction( final Function6 f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array6Func(f); } public static Function toFunction( final Function7 f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array7Func(f); } public static Function toFunction( final Function8 f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array8Func(f); } public static Function toFunction( final Function9 f) { - ObjectHelper.requireNonNull(f, "f is null"); + Objects.requireNonNull(f, "f is null"); return new Array9Func(f); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java b/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java index 65f5a70c82..05bceafa1d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/functions/ObjectHelper.java @@ -26,22 +26,6 @@ private ObjectHelper() { throw new IllegalStateException("No instances!"); } - /** - * Verifies if the object is not null and returns it or throws a NullPointerException - * with the given message. - * @param the value type - * @param object the object to verify - * @param message the message to use with the NullPointerException - * @return the object itself - * @throws NullPointerException if object is null - */ - public static T requireNonNull(T object, String message) { - if (object == null) { - throw new NullPointerException(message); - } - return object; - } - static final BiPredicate EQUALS = new BiObjectPredicate(); /** diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java index db34238b7a..0f4e4b16ba 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.jdk8; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Stream; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.annotations.*; import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.BackpressureHelper; @@ -140,7 +140,7 @@ public T poll() { return null; } } - return ObjectHelper.requireNonNull(iterator.next(), "Iterator.next() returned a null value"); + return Objects.requireNonNull(iterator.next(), "Iterator.next() returned a null value"); } @Override @@ -184,7 +184,7 @@ public void run(long n) { } else { T next; try { - next = ObjectHelper.requireNonNull(iterator.next(), "The Stream's Iterator returned a null value"); + next = Objects.requireNonNull(iterator.next(), "The Stream's Iterator returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); @@ -252,7 +252,7 @@ public void run(long n) { } else { T next; try { - next = ObjectHelper.requireNonNull(iterator.next(), "The Stream's Iterator returned a null value"); + next = Objects.requireNonNull(iterator.next(), "The Stream's Iterator returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatIterable.java index c36f546c16..ef801a1646 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatIterable.java @@ -14,13 +14,13 @@ package io.reactivex.rxjava3.internal.operators.completable; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; public final class CompletableConcatIterable extends Completable { final Iterable sources; @@ -35,7 +35,7 @@ public void subscribeActual(CompletableObserver observer) { Iterator it; try { - it = ObjectHelper.requireNonNull(sources.iterator(), "The iterator returned is null"); + it = Objects.requireNonNull(sources.iterator(), "The iterator returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptyDisposable.error(e, observer); @@ -109,7 +109,7 @@ void next() { CompletableSource c; try { - c = ObjectHelper.requireNonNull(a.next(), "The CompletableSource returned is null"); + c = Objects.requireNonNull(a.next(), "The CompletableSource returned is null"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDefer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDefer.java index 0e28d0d28f..c451413f49 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDefer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDefer.java @@ -17,7 +17,8 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; public final class CompletableDefer extends Completable { @@ -32,7 +33,7 @@ protected void subscribeActual(CompletableObserver observer) { CompletableSource c; try { - c = ObjectHelper.requireNonNull(completableSupplier.get(), "The completableSupplier returned a null CompletableSource"); + c = Objects.requireNonNull(completableSupplier.get(), "The completableSupplier returned a null CompletableSource"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptyDisposable.error(e, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableErrorSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableErrorSupplier.java index 12b39277db..bd4250c447 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableErrorSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableErrorSupplier.java @@ -17,7 +17,8 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; public final class CompletableErrorSupplier extends Completable { @@ -32,7 +33,7 @@ protected void subscribeActual(CompletableObserver observer) { Throwable error; try { - error = ObjectHelper.requireNonNull(errorSupplier.get(), "The error returned is null"); + error = Objects.requireNonNull(errorSupplier.get(), "The error returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); error = e; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeDelayErrorIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeDelayErrorIterable.java index 6d420cf044..55efbd8915 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeDelayErrorIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeDelayErrorIterable.java @@ -14,12 +14,12 @@ package io.reactivex.rxjava3.internal.operators.completable; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.CompositeDisposable; import io.reactivex.rxjava3.exceptions.Exceptions; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.completable.CompletableMergeDelayErrorArray.*; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -40,7 +40,7 @@ public void subscribeActual(final CompletableObserver observer) { Iterator iterator; try { - iterator = ObjectHelper.requireNonNull(sources.iterator(), "The source iterator returned is null"); + iterator = Objects.requireNonNull(sources.iterator(), "The source iterator returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); observer.onError(e); @@ -77,7 +77,7 @@ public void subscribeActual(final CompletableObserver observer) { CompletableSource c; try { - c = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null CompletableSource"); + c = Objects.requireNonNull(iterator.next(), "The iterator returned a null CompletableSource"); } catch (Throwable e) { Exceptions.throwIfFatal(e); errors.tryAddThrowableOrReport(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterable.java index 86c08d359c..b0e5b87bf7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterable.java @@ -14,12 +14,12 @@ package io.reactivex.rxjava3.internal.operators.completable; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.Exceptions; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class CompletableMergeIterable extends Completable { @@ -41,7 +41,7 @@ public void subscribeActual(final CompletableObserver observer) { Iterator iterator; try { - iterator = ObjectHelper.requireNonNull(sources.iterator(), "The source iterator returned is null"); + iterator = Objects.requireNonNull(sources.iterator(), "The source iterator returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); observer.onError(e); @@ -74,7 +74,7 @@ public void subscribeActual(final CompletableObserver observer) { CompletableSource c; try { - c = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null CompletableSource"); + c = Objects.requireNonNull(iterator.next(), "The iterator returned a null CompletableSource"); } catch (Throwable e) { Exceptions.throwIfFatal(e); set.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableResumeNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableResumeNext.java index 63227fb391..1d3bd7a3a0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableResumeNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableResumeNext.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.completable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; public final class CompletableResumeNext extends Completable { @@ -79,7 +79,7 @@ public void onError(Throwable e) { CompletableSource c; try { - c = ObjectHelper.requireNonNull(errorMapper.apply(e), "The errorMapper returned a null CompletableSource"); + c = Objects.requireNonNull(errorMapper.apply(e), "The errorMapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(new CompositeException(e, ex)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java index ab29ce4ae8..7a4e99b332 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.completable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class CompletableUsing extends Completable { @@ -54,7 +54,7 @@ protected void subscribeActual(CompletableObserver observer) { CompletableSource source; try { - source = ObjectHelper.requireNonNull(completableFunction.apply(resource), "The completableFunction returned a null CompletableSource"); + source = Objects.requireNonNull(completableFunction.apply(resource), "The completableFunction returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (eager) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/AbstractFlowableWithUpstream.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/AbstractFlowableWithUpstream.java index 76a68d37b3..d83f65c347 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/AbstractFlowableWithUpstream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/AbstractFlowableWithUpstream.java @@ -16,9 +16,10 @@ import org.reactivestreams.Publisher; import io.reactivex.rxjava3.core.Flowable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.HasUpstreamPublisher; +import java.util.Objects; + /** * Abstract base class for operators that take an upstream * source {@link Publisher}. @@ -39,7 +40,7 @@ abstract class AbstractFlowableWithUpstream extends Flowable implements * @param source the source (upstream) Publisher instance, not null (verified) */ AbstractFlowableWithUpstream(Flowable source) { - this.source = ObjectHelper.requireNonNull(source, "source is null"); + this.source = Objects.requireNonNull(source, "source is null"); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java index 899ae6c21c..14654d35ee 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.*; import org.reactivestreams.*; @@ -103,9 +104,9 @@ public static void subscribe(Publisher o) { */ public static void subscribe(Publisher o, final Consumer onNext, final Consumer onError, final Action onComplete) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); subscribe(o, new LambdaSubscriber(onNext, onError, onComplete, Functions.REQUEST_MAX)); } @@ -120,9 +121,9 @@ public static void subscribe(Publisher o, final Consumer void subscribe(Publisher o, final Consumer onNext, final Consumer onError, final Action onComplete, int bufferSize) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); ObjectHelper.verifyPositive(bufferSize, "number > 0 required"); subscribe(o, new BoundedSubscriber(onNext, onError, onComplete, Functions.boundedConsumer(bufferSize), bufferSize)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java index 0da3aea64e..08f89e99e2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java @@ -21,7 +21,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -105,7 +104,7 @@ public void onNext(T t) { if (b == null) { try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); + b = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); @@ -227,7 +226,7 @@ public void onNext(T t) { if (i++ == 0) { try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); + b = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); @@ -378,7 +377,7 @@ public void onNext(T t) { C b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); + b = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java index 4e8103e1a6..47f6e6bf06 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java @@ -22,7 +22,6 @@ import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; @@ -181,8 +180,8 @@ void open(Open token) { Publisher p; C buf; try { - buf = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null Collection"); - p = ObjectHelper.requireNonNull(bufferClose.apply(token), "The bufferClose returned a null Publisher"); + buf = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null Collection"); + p = Objects.requireNonNull(bufferClose.apply(token), "The bufferClose returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); SubscriptionHelper.cancel(upstream); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java index 22d139eb89..a1ca188b13 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Collection; +import java.util.Objects; import org.reactivestreams.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; import io.reactivex.rxjava3.internal.subscribers.QueueDrainSubscriber; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -73,7 +73,7 @@ public void onSubscribe(Subscription s) { U b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancelled = true; @@ -153,7 +153,7 @@ void next() { U next; try { - next = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + next = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java index 4057a06abb..0b3faed1c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java @@ -25,7 +25,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; import io.reactivex.rxjava3.internal.subscribers.QueueDrainSubscriber; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -110,7 +109,7 @@ public void onSubscribe(Subscription s) { U b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); @@ -187,7 +186,7 @@ public void run() { U next; try { - next = ObjectHelper.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); + next = Objects.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); @@ -258,7 +257,7 @@ public void onSubscribe(Subscription s) { final U b; // NOPMD try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); w.dispose(); @@ -339,7 +338,7 @@ public void run() { final U b; // NOPMD try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); @@ -424,7 +423,7 @@ public void onSubscribe(Subscription s) { U b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); w.dispose(); @@ -468,7 +467,7 @@ public void onNext(T t) { fastPathOrderedEmitMax(b, false, this); try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); @@ -550,7 +549,7 @@ public void run() { U next; try { - next = ObjectHelper.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); + next = Objects.requireNonNull(bufferSupplier.get(), "The supplied buffer is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java index d5b4dd8233..80345f2f3b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java @@ -17,10 +17,11 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class FlowableCollect extends AbstractFlowableWithUpstream { final Supplier initialSupplier; @@ -36,7 +37,7 @@ public FlowableCollect(Flowable source, Supplier initialSupplier protected void subscribeActual(Subscriber s) { U u; try { - u = ObjectHelper.requireNonNull(initialSupplier.get(), "The initial value supplied is null"); + u = Objects.requireNonNull(initialSupplier.get(), "The initial value supplied is null"); } catch (Throwable e) { EmptySubscription.error(e, s); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java index b48f4d11fc..fcb5590b49 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java @@ -19,11 +19,12 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.FuseToFlowable; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class FlowableCollectSingle extends Single implements FuseToFlowable { final Flowable source; @@ -41,7 +42,7 @@ public FlowableCollectSingle(Flowable source, Supplier initialSu protected void subscribeActual(SingleObserver observer) { U u; try { - u = ObjectHelper.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); + u = Objects.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); } catch (Throwable e) { EmptyDisposable.error(e, observer); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java index 70f7f313bc..24306072bc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.flowable.FlowableMap.MapSubscriber; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -82,7 +82,7 @@ public void subscribeActual(Subscriber s) { Iterator> it; try { - it = ObjectHelper.requireNonNull(iterable.iterator(), "The iterator returned is null"); + it = Objects.requireNonNull(iterable.iterator(), "The iterator returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptySubscription.error(e, s); @@ -108,7 +108,7 @@ public void subscribeActual(Subscriber s) { Publisher p; try { - p = ObjectHelper.requireNonNull(it.next(), "The publisher returned by the iterator is null"); + p = Objects.requireNonNull(it.next(), "The publisher returned by the iterator is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptySubscription.error(e, s); @@ -359,7 +359,7 @@ void drainAsync() { R w; try { - w = ObjectHelper.requireNonNull(combiner.apply(va), "The combiner returned a null value"); + w = Objects.requireNonNull(combiner.apply(va), "The combiner returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); @@ -472,7 +472,7 @@ public R poll() throws Throwable { return null; } T[] a = (T[])queue.poll(); - R r = ObjectHelper.requireNonNull(combiner.apply(a), "The combiner returned a null value"); + R r = Objects.requireNonNull(combiner.apply(a), "The combiner returned a null value"); ((CombineLatestInnerSubscriber)e).requestOne(); return r; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java index a11f206dde..f757fb5f3b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import org.reactivestreams.*; @@ -19,7 +20,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -277,7 +277,7 @@ void drain() { Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); + p = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); @@ -479,7 +479,7 @@ void drain() { Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); + p = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java index 29c2077226..9aee3eaebb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimpleQueue; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.subscribers.*; @@ -115,7 +115,7 @@ public void onNext(T t) { Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); + p = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java index 828bb5a5fa..05c984b533 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import org.reactivestreams.*; @@ -19,7 +20,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.operators.flowable.FlowableConcatMap.*; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; @@ -287,7 +287,7 @@ public void run() { Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); + p = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); @@ -474,7 +474,7 @@ public void run() { Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); + p = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java index 53f3c4e833..ab86164622 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.BackpressureHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -88,7 +88,7 @@ public void onNext(T t) { Publisher p; try { - p = ObjectHelper.requireNonNull(debounceSelector.apply(t), "The publisher supplied is null"); + p = Objects.requireNonNull(debounceSelector.apply(t), "The publisher supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefer.java index 6117d56f92..6c3d8f9008 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefer.java @@ -18,9 +18,10 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.EmptySubscription; +import java.util.Objects; + public final class FlowableDefer extends Flowable { final Supplier> supplier; public FlowableDefer(Supplier> supplier) { @@ -31,7 +32,7 @@ public FlowableDefer(Supplier> supplier) { public void subscribeActual(Subscriber s) { Publisher pub; try { - pub = ObjectHelper.requireNonNull(supplier.get(), "The publisher supplied is null"); + pub = Objects.requireNonNull(supplier.get(), "The publisher supplied is null"); } catch (Throwable t) { Exceptions.throwIfFatal(t); EmptySubscription.error(t, s); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java index d19bf4f037..d5e46e6d23 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java @@ -18,10 +18,11 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class FlowableDematerialize extends AbstractFlowableWithUpstream { final Function> selector; @@ -74,7 +75,7 @@ public void onNext(T item) { Notification notification; try { - notification = ObjectHelper.requireNonNull(selector.apply(item), "The selector returned a null Notification"); + notification = Objects.requireNonNull(selector.apply(item), "The selector returned a null Notification"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java index 870710bd87..1cd4be2458 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Collection; +import java.util.Objects; import org.reactivestreams.Subscriber; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.QueueFuseable; import io.reactivex.rxjava3.internal.subscribers.BasicFuseableSubscriber; import io.reactivex.rxjava3.internal.subscriptions.EmptySubscription; @@ -77,7 +77,7 @@ public void onNext(T value) { boolean b; try { - key = ObjectHelper.requireNonNull(keySelector.apply(value), "The keySelector returned a null key"); + key = Objects.requireNonNull(keySelector.apply(value), "The keySelector returned a null key"); b = collection.add(key); } catch (Throwable ex) { fail(ex); @@ -125,7 +125,7 @@ public T poll() throws Throwable { for (;;) { T v = qs.poll(); - if (v == null || collection.add(ObjectHelper.requireNonNull(keySelector.apply(v), "The keySelector returned a null key"))) { + if (v == null || collection.add(Objects.requireNonNull(keySelector.apply(v), "The keySelector returned a null key"))) { return v; } else { if (sourceMode == QueueFuseable.ASYNC) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java index de53c5d5b8..1c14ea54af 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.*; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; @@ -128,7 +128,7 @@ public void onNext(T t) { } Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); + p = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java index 5c20e9d75a..5c544add23 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import org.reactivestreams.*; @@ -23,7 +24,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -106,7 +106,7 @@ public void onNext(T value) { CompletableSource cs; try { - cs = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java index 52a719ca44..94f0fa8653 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.Subscription; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.FuseToFlowable; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -114,7 +114,7 @@ public void onNext(T value) { CompletableSource cs; try { - cs = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java index f6cb69c84a..18c2e21f63 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; @@ -115,7 +115,7 @@ public void onNext(T t) { MaybeSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); + ms = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java index b1ba1fed12..366b48be09 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; @@ -115,7 +115,7 @@ public void onNext(T t) { SingleSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); + ms = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java index 808d964798..de7221cb38 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -297,7 +297,7 @@ void drain() { R v; try { - v = ObjectHelper.requireNonNull(it.next(), "The iterator returned a null value"); + v = Objects.requireNonNull(it.next(), "The iterator returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); current = null; @@ -432,7 +432,7 @@ public R poll() throws Throwable { current = it; } - R r = ObjectHelper.requireNonNull(it.next(), "The iterator returned a null value"); + R r = Objects.requireNonNull(it.next(), "The iterator returned a null value"); if (!it.hasNext()) { current = null; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java index 0b49186290..83371ab447 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java @@ -17,11 +17,12 @@ import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.Flowable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.BackpressureHelper; +import java.util.Objects; + public final class FlowableFromArray extends Flowable { final T[] array; @@ -67,7 +68,7 @@ public final T poll() { } index = i + 1; - return ObjectHelper.requireNonNull(arr[i], "array element is null"); + return Objects.requireNonNull(arr[i], "array element is null"); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java index 7f918c5c51..80afc65910 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.Callable; import org.reactivestreams.Subscriber; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.DeferredScalarSubscription; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -37,7 +37,7 @@ public void subscribeActual(Subscriber s) { T t; try { - t = ObjectHelper.requireNonNull(callable.call(), "The callable returned a null value"); + t = Objects.requireNonNull(callable.call(), "The callable returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (deferred.isCancelled()) { @@ -53,6 +53,6 @@ public void subscribeActual(Subscriber s) { @Override public T get() throws Throwable { - return ObjectHelper.requireNonNull(callable.call(), "The callable returned a null value"); + return Objects.requireNonNull(callable.call(), "The callable returned a null value"); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterable.java index c8dc87c0c3..7ae3f3e7d1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterable.java @@ -14,13 +14,13 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Iterator; +import java.util.Objects; import org.reactivestreams.Subscriber; import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.BackpressureHelper; @@ -101,7 +101,7 @@ public final T poll() { return null; } } - return ObjectHelper.requireNonNull(it.next(), "Iterator.next() returned a null value"); + return Objects.requireNonNull(it.next(), "Iterator.next() returned a null value"); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java index bd714ee6c0..7be86d618b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java @@ -18,10 +18,11 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.DeferredScalarSubscription; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Call a Supplier for each incoming Subscriber and signal the returned value or the thrown exception. * @param the value type and element type returned by the supplier and the flow @@ -42,7 +43,7 @@ public void subscribeActual(Subscriber s) { T t; try { - t = ObjectHelper.requireNonNull(supplier.get(), "The supplier returned a null value"); + t = Objects.requireNonNull(supplier.get(), "The supplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (deferred.isCancelled()) { @@ -58,6 +59,6 @@ public void subscribeActual(Subscriber s) { @Override public T get() throws Throwable { - return ObjectHelper.requireNonNull(supplier.get(), "The supplier returned a null value"); + return Objects.requireNonNull(supplier.get(), "The supplier returned a null value"); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java index 213073c672..ccd80c53d9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java @@ -25,7 +25,6 @@ import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimpleQueue; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; @@ -247,7 +246,7 @@ void drain() { Publisher p; try { - p = ObjectHelper.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null Publisher"); + p = Objects.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null Publisher"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -269,7 +268,7 @@ void drain() { R w; try { - w = ObjectHelper.requireNonNull(resultSelector.apply(left, up), "The resultSelector returned a null value"); + w = Objects.requireNonNull(resultSelector.apply(left, up), "The resultSelector returned a null value"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -299,7 +298,7 @@ else if (mode == RIGHT_VALUE) { Publisher p; try { - p = ObjectHelper.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null Publisher"); + p = Objects.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null Publisher"); } catch (Throwable exc) { fail(exc, a, q); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java index 243d58b9b8..f8f97191fd 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.TimeUnit; import org.reactivestreams.*; @@ -76,7 +77,7 @@ static final class ItemDelayFunction implements Function> @Override public Publisher apply(final T v) throws Throwable { - Publisher p = ObjectHelper.requireNonNull(itemDelay.apply(v), "The itemDelay returned a null Publisher"); + Publisher p = Objects.requireNonNull(itemDelay.apply(v), "The itemDelay returned a null Publisher"); return new FlowableTakePublisher(p, 1).map(Functions.justFunction(v)).defaultIfEmpty(v); } } @@ -164,7 +165,7 @@ static final class FlatMapWithCombinerOuter implements Function apply(final T t) throws Throwable { @SuppressWarnings("unchecked") - Publisher u = (Publisher)ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); + Publisher u = (Publisher)Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); return new FlowableMapPublisher(u, new FlatMapWithCombinerInner(combiner, t)); } } @@ -184,7 +185,7 @@ static final class FlatMapIntoIterable implements Function @Override public Publisher apply(T t) throws Throwable { - return new FlowableFromIterable(ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); + return new FlowableFromIterable(Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java index 591cdb159a..df8eff7049 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java @@ -22,7 +22,6 @@ import io.reactivex.rxjava3.disposables.CompositeDisposable; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimpleQueue; import io.reactivex.rxjava3.internal.operators.flowable.FlowableGroupJoin.*; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; @@ -223,7 +222,7 @@ void drain() { Publisher p; try { - p = ObjectHelper.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null Publisher"); + p = Objects.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null Publisher"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -250,7 +249,7 @@ void drain() { R w; try { - w = ObjectHelper.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); + w = Objects.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -284,7 +283,7 @@ else if (mode == RIGHT_VALUE) { Publisher p; try { - p = ObjectHelper.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null Publisher"); + p = Objects.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null Publisher"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -311,7 +310,7 @@ else if (mode == RIGHT_VALUE) { R w; try { - w = ObjectHelper.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); + w = Objects.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); } catch (Throwable exc) { fail(exc, a, q); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMap.java index cec093d278..208baca0b2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMap.java @@ -18,10 +18,11 @@ import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscribers.*; +import java.util.Objects; + public final class FlowableMap extends AbstractFlowableWithUpstream { final Function mapper; public FlowableMap(Flowable source, Function mapper) { @@ -60,7 +61,7 @@ public void onNext(T t) { U v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); + v = Objects.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); } catch (Throwable ex) { fail(ex); return; @@ -77,7 +78,7 @@ public int requestFusion(int mode) { @Override public U poll() throws Throwable { T t = qs.poll(); - return t != null ? ObjectHelper.requireNonNull(mapper.apply(t), "The mapper function returned a null value.") : null; + return t != null ? Objects.requireNonNull(mapper.apply(t), "The mapper function returned a null value.") : null; } } @@ -103,7 +104,7 @@ public void onNext(T t) { U v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); + v = Objects.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); } catch (Throwable ex) { fail(ex); return; @@ -120,7 +121,7 @@ public boolean tryOnNext(T t) { U v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); + v = Objects.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); } catch (Throwable ex) { fail(ex); return true; @@ -137,7 +138,7 @@ public int requestFusion(int mode) { @Override public U poll() throws Throwable { T t = qs.poll(); - return t != null ? ObjectHelper.requireNonNull(mapper.apply(t), "The mapper function returned a null value.") : null; + return t != null ? Objects.requireNonNull(mapper.apply(t), "The mapper function returned a null value.") : null; } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java index 9998bee06b..cdfb0b0bb5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java @@ -18,9 +18,10 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscribers.SinglePostCompleteSubscriber; +import java.util.Objects; + public final class FlowableMapNotification extends AbstractFlowableWithUpstream { final Function onNextMapper; @@ -66,7 +67,7 @@ public void onNext(T t) { R p; try { - p = ObjectHelper.requireNonNull(onNextMapper.apply(t), "The onNext publisher returned is null"); + p = Objects.requireNonNull(onNextMapper.apply(t), "The onNext publisher returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); @@ -82,7 +83,7 @@ public void onError(Throwable t) { R p; try { - p = ObjectHelper.requireNonNull(onErrorMapper.apply(t), "The onError publisher returned is null"); + p = Objects.requireNonNull(onErrorMapper.apply(t), "The onError publisher returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(new CompositeException(t, e)); @@ -97,7 +98,7 @@ public void onComplete() { R p; try { - p = ObjectHelper.requireNonNull(onCompleteSupplier.get(), "The onComplete publisher returned is null"); + p = Objects.requireNonNull(onCompleteSupplier.get(), "The onComplete publisher returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java index 7a186cdec0..1d939022b4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java @@ -18,10 +18,11 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionArbiter; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class FlowableOnErrorNext extends AbstractFlowableWithUpstream { final Function> nextSupplier; @@ -90,7 +91,7 @@ public void onError(Throwable t) { Publisher p; try { - p = ObjectHelper.requireNonNull(nextSupplier.apply(t), "The nextSupplier returned a null Publisher"); + p = Objects.requireNonNull(nextSupplier.apply(t), "The nextSupplier returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(new CompositeException(t, e)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java index df57dd5c00..4a908cc113 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java @@ -18,9 +18,10 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscribers.SinglePostCompleteSubscriber; +import java.util.Objects; + public final class FlowableOnErrorReturn extends AbstractFlowableWithUpstream { final Function valueSupplier; public FlowableOnErrorReturn(Flowable source, Function valueSupplier) { @@ -54,7 +55,7 @@ public void onNext(T t) { public void onError(Throwable t) { T v; try { - v = ObjectHelper.requireNonNull(valueSupplier.apply(t), "The valueSupplier returned a null value"); + v = Objects.requireNonNull(valueSupplier.apply(t), "The valueSupplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(new CompositeException(t, ex)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java index cc36e86821..702955bf02 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java @@ -16,6 +16,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -24,7 +25,6 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.*; @@ -60,7 +60,7 @@ protected void subscribeActual(Subscriber s) { Publisher other; try { - other = ObjectHelper.requireNonNull(selector.apply(mp), "selector returned a null Publisher"); + other = Objects.requireNonNull(selector.apply(mp), "selector returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptySubscription.error(ex, s); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java index 4da1d68d99..9d3d3f4794 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java @@ -18,10 +18,11 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Reduces a sequence via a function into a single value or signals NoSuchElementException for * an empty source. @@ -77,7 +78,7 @@ public void onNext(T t) { value = t; } else { try { - value = ObjectHelper.requireNonNull(reducer.apply(v, t), "The reducer returned a null value"); + value = Objects.requireNonNull(reducer.apply(v, t), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java index 72b4f1ac28..2de3b633fe 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java @@ -19,11 +19,12 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Reduce a Flowable into a single value exposed as Single or signal NoSuchElementException. * @@ -105,7 +106,7 @@ public void onNext(T t) { value = t; } else { try { - value = ObjectHelper.requireNonNull(reducer.apply(v, t), "The reducer returned a null value"); + value = Objects.requireNonNull(reducer.apply(v, t), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java index cf489963ed..ec570639c6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java @@ -19,10 +19,11 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Reduce a sequence of values, starting from a seed value and by using * an accumulator function and return the last accumulated value. @@ -81,7 +82,7 @@ public void onNext(T value) { R v = this.value; if (v != null) { try { - this.value = ObjectHelper.requireNonNull(reducer.apply(v, value), "The reducer returned a null value"); + this.value = Objects.requireNonNull(reducer.apply(v, value), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java index ca8be6779e..343d72c4e3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java @@ -19,9 +19,10 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.flowable.FlowableReduceSeedSingle.ReduceSeedObserver; +import java.util.Objects; + /** * Reduce a sequence of values, starting from a generated seed value and by using * an accumulator function and return the last accumulated value. @@ -48,7 +49,7 @@ protected void subscribeActual(SingleObserver observer) { R seed; try { - seed = ObjectHelper.requireNonNull(seedSupplier.get(), "The seedSupplier returned a null value"); + seed = Objects.requireNonNull(seedSupplier.get(), "The seedSupplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java index 021b4dd387..dcd807aa2a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.processors.*; import io.reactivex.rxjava3.subscribers.SerializedSubscriber; @@ -44,7 +44,7 @@ public void subscribeActual(Subscriber s) { Publisher when; try { - when = ObjectHelper.requireNonNull(handler.apply(processor), "handler returned a null Publisher"); + when = Objects.requireNonNull(handler.apply(processor), "handler returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptySubscription.error(ex, s); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java index ff7ddb84af..7307664353 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java @@ -18,12 +18,13 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.flowable.FlowableRepeatWhen.*; import io.reactivex.rxjava3.internal.subscriptions.EmptySubscription; import io.reactivex.rxjava3.processors.*; import io.reactivex.rxjava3.subscribers.SerializedSubscriber; +import java.util.Objects; + public final class FlowableRetryWhen extends AbstractFlowableWithUpstream { final Function, ? extends Publisher> handler; @@ -42,7 +43,7 @@ public void subscribeActual(Subscriber s) { Publisher when; try { - when = ObjectHelper.requireNonNull(handler.apply(processor), "handler returned a null Publisher"); + when = Objects.requireNonNull(handler.apply(processor), "handler returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptySubscription.error(ex, s); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java index 999cf48a51..2ef89c0023 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java @@ -18,10 +18,11 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Utility classes to work with scalar-sourced XMap operators (where X == { flat, concat, switch }). */ @@ -64,7 +65,7 @@ public static boolean tryScalarXMapSubscribe(Publisher source, Publisher r; try { - r = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); + r = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptySubscription.error(ex, subscriber); @@ -133,7 +134,7 @@ static final class ScalarXMapFlowable extends Flowable { public void subscribeActual(Subscriber s) { Publisher other; try { - other = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null Publisher"); + other = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null Publisher"); } catch (Throwable e) { EmptySubscription.error(e, s); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java index 3dadb2968b..2fab24a484 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java @@ -18,10 +18,11 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class FlowableScan extends AbstractFlowableWithUpstream { final BiFunction accumulator; public FlowableScan(Flowable source, BiFunction accumulator) { @@ -71,7 +72,7 @@ public void onNext(T t) { T u; try { - u = ObjectHelper.requireNonNull(accumulator.apply(v, t), "The value returned by the accumulator is null"); + u = Objects.requireNonNull(accumulator.apply(v, t), "The value returned by the accumulator is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java index c1875f3070..da0b1870da 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -19,7 +20,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -41,7 +41,7 @@ protected void subscribeActual(Subscriber s) { R r; try { - r = ObjectHelper.requireNonNull(seedSupplier.get(), "The seed supplied is null"); + r = Objects.requireNonNull(seedSupplier.get(), "The seed supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptySubscription.error(e, s); @@ -109,7 +109,7 @@ public void onNext(T t) { R v = value; try { - v = ObjectHelper.requireNonNull(accumulator.apply(v, t), "The accumulator returned a null value"); + v = Objects.requireNonNull(accumulator.apply(v, t), "The accumulator returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java index e77110b6a0..c618a0252b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; @@ -110,7 +110,7 @@ public void onNext(T t) { Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(t), "The publisher returned is null"); + p = Objects.requireNonNull(mapper.apply(t), "The publisher returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java index b45755fc30..c5879c2e74 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.*; @@ -23,7 +24,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.SequentialDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.flowable.FlowableTimeoutTimed.TimeoutSupport; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -108,7 +108,7 @@ public void onNext(T t) { Publisher itemTimeoutPublisher; try { - itemTimeoutPublisher = ObjectHelper.requireNonNull( + itemTimeoutPublisher = Objects.requireNonNull( itemTimeoutIndicator.apply(t), "The itemTimeoutIndicator returned a null Publisher."); } catch (Throwable ex) { @@ -243,7 +243,7 @@ public void onNext(T t) { Publisher itemTimeoutPublisher; try { - itemTimeoutPublisher = ObjectHelper.requireNonNull( + itemTimeoutPublisher = Objects.requireNonNull( itemTimeoutIndicator.apply(t), "The itemTimeoutIndicator returned a null Publisher."); } catch (Throwable ex) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java index 9c0dc74d4c..7e2e91c672 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import org.reactivestreams.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -54,7 +54,7 @@ public void subscribeActual(Subscriber s) { Publisher source; try { - source = ObjectHelper.requireNonNull(sourceSupplier.apply(resource), "The sourceSupplier returned a null Publisher"); + source = Objects.requireNonNull(sourceSupplier.apply(resource), "The sourceSupplier returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); try { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java index 63ee8ccf3c..4e38351ef7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java @@ -22,7 +22,6 @@ import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; @@ -244,7 +243,7 @@ void drain() { Publisher endSource; try { - endSource = ObjectHelper.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null Publisher"); + endSource = Objects.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null Publisher"); } catch (Throwable ex) { upstream.cancel(); startSubscriber.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java index c3ec3ab5f9..049befd1f4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.subscribers.SerializedSubscriber; @@ -84,7 +84,7 @@ public boolean tryOnNext(T t) { if (u != null) { R r; try { - r = ObjectHelper.requireNonNull(combiner.apply(t, u), "The combiner returned a null value"); + r = Objects.requireNonNull(combiner.apply(t, u), "The combiner returned a null value"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java index 7c3b06ed8c..1d1be81e05 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Arrays; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.*; @@ -174,7 +174,7 @@ public boolean tryOnNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(combiner.apply(objects), "The combiner returned a null value"); + v = Objects.requireNonNull(combiner.apply(objects), "The combiner returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancel(); @@ -297,7 +297,7 @@ void dispose() { final class SingletonArrayFunc implements Function { @Override public R apply(T t) throws Throwable { - return ObjectHelper.requireNonNull(combiner.apply(new Object[] { t }), "The combiner returned a null value"); + return Objects.requireNonNull(combiner.apply(new Object[] { t }), "The combiner returned a null value"); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java index 09f1a59a7f..27f1f8e2dc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Arrays; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -229,7 +229,7 @@ void drain() { R v; try { - v = ObjectHelper.requireNonNull(zipper.apply(values.clone()), "The zipper returned a null value"); + v = Objects.requireNonNull(zipper.apply(values.clone()), "The zipper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancelAll(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java index fad3df6e8e..a41c591fe2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java @@ -14,13 +14,13 @@ package io.reactivex.rxjava3.internal.operators.flowable; import java.util.Iterator; +import java.util.Objects; import org.reactivestreams.*; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -41,7 +41,7 @@ public void subscribeActual(Subscriber t) { Iterator it; try { - it = ObjectHelper.requireNonNull(other.iterator(), "The iterator returned by other is null"); + it = Objects.requireNonNull(other.iterator(), "The iterator returned by other is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptySubscription.error(e, t); @@ -99,7 +99,7 @@ public void onNext(T t) { U u; try { - u = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + u = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); } catch (Throwable e) { error(e); return; @@ -107,7 +107,7 @@ public void onNext(T t) { V v; try { - v = ObjectHelper.requireNonNull(zipper.apply(t, u), "The zipper function returned a null value"); + v = Objects.requireNonNull(zipper.apply(t, u), "The zipper function returned a null value"); } catch (Throwable e) { error(e); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java index b01b58c6a0..f3abd7ddb8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.internal.disposables.SequentialDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.*; @@ -45,7 +45,7 @@ protected void subscribeActual(Subscriber s) { Iterator> it; try { - it = ObjectHelper.requireNonNull(sources.iterator(), "The sources Iterable returned a null Iterator"); + it = Objects.requireNonNull(sources.iterator(), "The sources Iterable returned a null Iterator"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptySubscription.error(ex, s); @@ -169,7 +169,7 @@ void drain() { MaybeSource source; try { - source = ObjectHelper.requireNonNull(sources.next(), "The source Iterator returned a null MaybeSource"); + source = Objects.requireNonNull(sources.next(), "The source Iterator returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); a.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDefer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDefer.java index 3146b4781d..ccba4e0041 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDefer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDefer.java @@ -17,7 +17,8 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; /** * Defers the creation of the actual Maybe the incoming MaybeObserver is subscribed to. @@ -37,7 +38,7 @@ protected void subscribeActual(MaybeObserver observer) { MaybeSource source; try { - source = ObjectHelper.requireNonNull(maybeSupplier.get(), "The maybeSupplier returned a null MaybeSource"); + source = Objects.requireNonNull(maybeSupplier.get(), "The maybeSupplier returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java index c463567227..afb27373c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,12 +21,11 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps a source item to another MaybeSource then calls a BiFunction with the * original item and the secondary item to generate the final result. - * + * * @param the main value type * @param the second value type * @param the result value type @@ -85,7 +85,7 @@ public void onSuccess(T value) { MaybeSource next; try { - next = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null MaybeSource"); + next = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); inner.downstream.onError(ex); @@ -139,7 +139,7 @@ public void onSuccess(U value) { R r; try { - r = ObjectHelper.requireNonNull(resultSelector.apply(t, value), "The resultSelector returned a null value"); + r = Objects.requireNonNull(resultSelector.apply(t, value), "The resultSelector returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java index 7ad1008421..fb8bb9d9f1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps the success value of the source MaybeSource into a Completable. @@ -80,7 +80,7 @@ public void onSuccess(T value) { CompletableSource cs; try { - cs = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java index 9c8aef2ba9..8460405dc9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import org.reactivestreams.Subscriber; @@ -24,7 +25,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.BackpressureHelper; @@ -211,7 +211,7 @@ void drain() { R v; try { - v = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + v = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); a.onError(ex); @@ -283,7 +283,7 @@ public R poll() throws Exception { Iterator iterator = it; if (iterator != null) { - R v = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + R v = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); if (!iterator.hasNext()) { it = null; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java index 8c2b4816c9..afbb56f8e9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; import java.util.Iterator; +import java.util.Objects; import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.BasicQueueDisposable; /** @@ -193,7 +193,7 @@ public R poll() throws Exception { Iterator iterator = it; if (iterator != null) { - R v = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + R v = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); if (!iterator.hasNext()) { it = null; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java index 39007e5117..2f50f8cd7d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps a value into a MaybeSource and relays its signal. @@ -102,7 +102,7 @@ public void onSuccess(T value) { MaybeSource source; try { - source = ObjectHelper.requireNonNull(onSuccessMapper.apply(value), "The onSuccessMapper returned a null MaybeSource"); + source = Objects.requireNonNull(onSuccessMapper.apply(value), "The onSuccessMapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); @@ -117,7 +117,7 @@ public void onError(Throwable e) { MaybeSource source; try { - source = ObjectHelper.requireNonNull(onErrorMapper.apply(e), "The onErrorMapper returned a null MaybeSource"); + source = Objects.requireNonNull(onErrorMapper.apply(e), "The onErrorMapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(new CompositeException(e, ex)); @@ -132,7 +132,7 @@ public void onComplete() { MaybeSource source; try { - source = ObjectHelper.requireNonNull(onCompleteSupplier.get(), "The onCompleteSupplier returned a null MaybeSource"); + source = Objects.requireNonNull(onCompleteSupplier.get(), "The onCompleteSupplier returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java index 011cbd9ab7..38ec9417d9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps the success value of the source MaybeSource into a Single. @@ -81,7 +81,7 @@ public void onSuccess(T value) { SingleSource ss; try { - ss = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null SingleSource"); + ss = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java index e823b06936..b917a5885e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps the success value of the source MaybeSource into a Single. @@ -82,7 +82,7 @@ public void onSuccess(T value) { SingleSource ss; try { - ss = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null SingleSource"); + ss = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java index 837ba675ef..325c4db2ea 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps a value into a MaybeSource and relays its signal. @@ -85,7 +85,7 @@ public void onSuccess(T value) { MaybeSource source; try { - source = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null MaybeSource"); + source = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeLift.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeLift.java index 38cb73972c..8e40bbd45d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeLift.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeLift.java @@ -16,7 +16,8 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; /** * Calls a MaybeOperator for the incoming MaybeObserver. @@ -38,7 +39,7 @@ protected void subscribeActual(MaybeObserver observer) { MaybeObserver lifted; try { - lifted = ObjectHelper.requireNonNull(operator.apply(observer), "The operator returned a null MaybeObserver"); + lifted = Objects.requireNonNull(operator.apply(observer), "The operator returned a null MaybeObserver"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMap.java index c2a14f994d..167e21d7fb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMap.java @@ -18,7 +18,8 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; /** * Maps the upstream success value into some other value. @@ -79,7 +80,7 @@ public void onSuccess(T value) { R v; try { - v = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null item"); + v = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null item"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java index d5e5069d22..2910a2e803 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimpleQueue; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.*; @@ -327,7 +327,7 @@ static final class MpscFillOnceSimpleQueue @Override public boolean offer(T value) { - ObjectHelper.requireNonNull(value, "value is null"); + Objects.requireNonNull(value, "value is null"); int idx = producerIndex.getAndIncrement(); if (idx < length()) { lazySet(idx, value); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java index 15ec49c3a9..901016f39a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,11 +21,10 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Subscribes to the MaybeSource returned by a function if the main source signals an onError. - * + * * @param the value type */ public final class MaybeOnErrorNext extends AbstractMaybeWithUpstream { @@ -97,7 +97,7 @@ public void onError(Throwable e) { MaybeSource m; try { - m = ObjectHelper.requireNonNull(resumeFunction.apply(e), "The resumeFunction returned a null MaybeSource"); + m = Objects.requireNonNull(resumeFunction.apply(e), "The resumeFunction returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(new CompositeException(e, ex)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java index 5c61179678..c7ab7b8524 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java @@ -18,7 +18,8 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; /** * Returns a value generated via a function if the main source signals an onError. @@ -82,7 +83,7 @@ public void onError(Throwable e) { T v; try { - v = ObjectHelper.requireNonNull(valueSupplier.apply(e), "The valueSupplier returned a null value"); + v = Objects.requireNonNull(valueSupplier.apply(e), "The valueSupplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(new CompositeException(e, ex)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsing.java index 628c4dda23..0e490d6f40 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsing.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** @@ -65,7 +65,7 @@ protected void subscribeActual(MaybeObserver observer) { MaybeSource source; try { - source = ObjectHelper.requireNonNull(sourceSupplier.apply(resource), "The sourceSupplier returned a null MaybeSource"); + source = Objects.requireNonNull(sourceSupplier.apply(resource), "The sourceSupplier returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (eager) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java index 9d83976966..d4bf3b0a77 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class MaybeZipArray extends Maybe { @@ -108,7 +108,7 @@ void innerSuccess(T value, int index) { R v; try { - v = ObjectHelper.requireNonNull(zipper.apply(values), "The zipper returned a null value"); + v = Objects.requireNonNull(zipper.apply(values), "The zipper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); @@ -190,7 +190,7 @@ public void onComplete() { final class SingletonArrayFunc implements Function { @Override public R apply(T t) throws Throwable { - return ObjectHelper.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); + return Objects.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java index efcd10b136..7c2d85c97b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java @@ -14,12 +14,12 @@ package io.reactivex.rxjava3.internal.operators.maybe; import java.util.Arrays; +import java.util.Objects; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.maybe.MaybeZipArray.ZipCoordinator; public final class MaybeZipIterable extends Maybe { @@ -82,7 +82,7 @@ protected void subscribeActual(MaybeObserver observer) { final class SingletonArrayFunc implements Function { @Override public R apply(T t) throws Throwable { - return ObjectHelper.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); + return Objects.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java index 6e16c9b6a9..986af99049 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.Subscription; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; @@ -223,7 +223,7 @@ void drain() { CompletableSource cs; try { - cs = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); queue.clear(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java index 39fa04b174..cd8b9fa77f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; @@ -251,7 +251,7 @@ void drain() { MaybeSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null MaybeSource"); + ms = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java index cd5e857274..78334d4a1f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; @@ -246,7 +246,7 @@ void drain() { SingleSource ss; try { - ss = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null SingleSource"); + ss = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java index f354d8f057..a2d66067ab 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import org.reactivestreams.Subscription; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.AtomicThrowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -96,7 +96,7 @@ public void onNext(T t) { CompletableSource c; try { - c = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null CompletableSource"); + c = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java index 56858d5d22..11ea0a4c77 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -116,7 +116,7 @@ public void onNext(T t) { MaybeSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); + ms = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java index ac0880082b..c41b52f6e5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -116,7 +116,7 @@ public void onNext(T t) { SingleSource ss; try { - ss = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); + ss = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java index a4dde1242f..891c2a0f33 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,12 +21,11 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps the success value of a Maybe onto an ObservableSource and * relays its signals to the downstream observer. - * + * * @param the success value type of the Maybe source * @param the result type of the ObservableSource and this operator * @since 2.1.15 @@ -99,7 +99,7 @@ public void onSuccess(T t) { ObservableSource o; try { - o = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); + o = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java index 66a446f3f3..1ac179bcca 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -22,13 +23,12 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; /** * Maps the success value of a Maybe onto a Publisher and * relays its signals to the downstream subscriber. - * + * * @param the success value type of the Maybe source * @param the result type of the Publisher and this operator * @since 2.1.15 @@ -109,7 +109,7 @@ public void onSuccess(T t) { Publisher p; try { - p = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); + p = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java index 4921722b5d..44f322e39d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.*; @@ -224,7 +224,7 @@ void drain() { try { T v = queue.poll(); if (v != null) { - cs = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null CompletableSource"); empty = false; } } catch (Throwable ex) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java index 17e4de9050..c76b0115cb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.*; @@ -224,7 +224,7 @@ void drain() { MaybeSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null MaybeSource"); + ms = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java index 91428a80db..34d7d04734 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.*; @@ -219,7 +219,7 @@ void drain() { SingleSource ss; try { - ss = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null SingleSource"); + ss = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java index 06a09676d0..a7a6a7fd99 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.AtomicThrowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -94,7 +94,7 @@ public void onNext(T t) { CompletableSource c; try { - c = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null CompletableSource"); + c = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java index b85bc0e2c2..bf3c9f43c2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.AtomicThrowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -109,7 +109,7 @@ public void onNext(T t) { MaybeSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); + ms = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java index 3343f93d97..bc19c88b55 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.AtomicThrowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -109,7 +109,7 @@ public void onNext(T t) { SingleSource ss; try { - ss = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); + ss = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ScalarXMapZHelper.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ScalarXMapZHelper.java index 24f231e45c..d7fe0e3445 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ScalarXMapZHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ScalarXMapZHelper.java @@ -17,10 +17,11 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.maybe.MaybeToObservable; import io.reactivex.rxjava3.internal.operators.single.SingleToObservable; +import java.util.Objects; + /** * Utility class to extract a value from a scalar source reactive type, * map it to a 0-1 type then subscribe the output type's consumer to it, @@ -55,7 +56,7 @@ static boolean tryAsCompletable(Object source, try { T item = supplier.get(); if (item != null) { - cs = ObjectHelper.requireNonNull(mapper.apply(item), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(item), "The mapper returned a null CompletableSource"); } } catch (Throwable ex) { Exceptions.throwIfFatal(ex); @@ -94,7 +95,7 @@ static boolean tryAsMaybe(Object source, try { T item = supplier.get(); if (item != null) { - cs = ObjectHelper.requireNonNull(mapper.apply(item), "The mapper returned a null MaybeSource"); + cs = Objects.requireNonNull(mapper.apply(item), "The mapper returned a null MaybeSource"); } } catch (Throwable ex) { Exceptions.throwIfFatal(ex); @@ -133,7 +134,7 @@ static boolean tryAsSingle(Object source, try { T item = supplier.get(); if (item != null) { - cs = ObjectHelper.requireNonNull(mapper.apply(item), "The mapper returned a null SingleSource"); + cs = Objects.requireNonNull(mapper.apply(item), "The mapper returned a null SingleSource"); } } catch (Throwable ex) { Exceptions.throwIfFatal(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java index 9591783a8e..bfb466bac1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.mixed; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,12 +21,11 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps the success value of a Single onto an ObservableSource and * relays its signals to the downstream observer. - * + * * @param the success value type of the Single source * @param the result type of the ObservableSource and this operator * @since 2.1.15 @@ -99,7 +99,7 @@ public void onSuccess(T t) { ObservableSource o; try { - o = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); + o = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java index c35f927c0b..c02bcfca88 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.*; import io.reactivex.rxjava3.core.*; @@ -97,9 +98,9 @@ public static void subscribe(ObservableSource o) { */ public static void subscribe(ObservableSource o, final Consumer onNext, final Consumer onError, final Action onComplete) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(onError, "onError is null"); - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); subscribe(o, new LambdaObserver(onNext, onError, onComplete, Functions.emptyConsumer())); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java index d0ae0b89cc..2b09748a61 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java @@ -22,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.ExceptionHelper; public final class ObservableBuffer> extends AbstractObservableWithUpstream { @@ -68,7 +67,7 @@ static final class BufferExactObserver> imple boolean createBuffer() { U b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "Empty buffer supplied"); + b = Objects.requireNonNull(bufferSupplier.get(), "Empty buffer supplied"); } catch (Throwable t) { Exceptions.throwIfFatal(t); buffer = null; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java index 961c6cd374..cef9ef41d8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java @@ -22,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.AtomicThrowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -172,8 +171,8 @@ void open(Open token) { ObservableSource p; C buf; try { - buf = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null Collection"); - p = ObjectHelper.requireNonNull(bufferClose.apply(token), "The bufferClose returned a null ObservableSource"); + buf = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null Collection"); + p = Objects.requireNonNull(bufferClose.apply(token), "The bufferClose returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); DisposableHelper.dispose(upstream); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java index 474e993653..7d76354cf9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java @@ -14,13 +14,13 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.Collection; +import java.util.Objects; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.QueueDrainObserver; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; import io.reactivex.rxjava3.internal.util.QueueDrainHelper; @@ -69,7 +69,7 @@ public void onSubscribe(Disposable d) { U b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); cancelled = true; @@ -148,7 +148,7 @@ void next() { U next; try { - next = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + next = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java index 107b011b2d..3b12c3d0f5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java @@ -24,7 +24,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.QueueDrainObserver; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; import io.reactivex.rxjava3.internal.util.QueueDrainHelper; @@ -110,7 +109,7 @@ public void onSubscribe(Disposable d) { U b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); dispose(); @@ -184,7 +183,7 @@ public void run() { U next; try { - next = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); + next = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); @@ -246,7 +245,7 @@ public void onSubscribe(Disposable d) { final U b; // NOPMD try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); d.dispose(); @@ -328,7 +327,7 @@ public void run() { final U b; // NOPMD try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); + b = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); @@ -427,7 +426,7 @@ public void onSubscribe(Disposable d) { U b; try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); d.dispose(); @@ -469,7 +468,7 @@ public void onNext(T t) { fastPathOrderedEmit(b, false, this); try { - b = ObjectHelper.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); + b = Objects.requireNonNull(bufferSupplier.get(), "The buffer supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); @@ -541,7 +540,7 @@ public void run() { U next; try { - next = ObjectHelper.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); + next = Objects.requireNonNull(bufferSupplier.get(), "The bufferSupplier returned a null buffer"); } catch (Throwable e) { Exceptions.throwIfFatal(e); dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java index 3b81a32ad9..6471d7538c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java @@ -16,9 +16,10 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class ObservableCollect extends AbstractObservableWithUpstream { final Supplier initialSupplier; final BiConsumer collector; @@ -34,7 +35,7 @@ public ObservableCollect(ObservableSource source, protected void subscribeActual(Observer t) { U u; try { - u = ObjectHelper.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); + u = Objects.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); } catch (Throwable e) { EmptyDisposable.error(e, t); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java index a63812a9ab..09c86cb24a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java @@ -16,10 +16,11 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.FuseToObservable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class ObservableCollectSingle extends Single implements FuseToObservable { final ObservableSource source; @@ -38,7 +39,7 @@ public ObservableCollectSingle(ObservableSource source, protected void subscribeActual(SingleObserver t) { U u; try { - u = ObjectHelper.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); + u = Objects.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); } catch (Throwable e) { EmptyDisposable.error(e, t); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java index 6a379d142c..0cc8e3e45e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -188,7 +188,7 @@ void drain() { R v; try { - v = ObjectHelper.requireNonNull(combiner.apply(s), "The combiner returned a null value"); + v = Objects.requireNonNull(combiner.apply(s), "The combiner returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); errors.tryAddThrowableOrReport(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java index 665306db95..b96b1ec8a0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -19,7 +20,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.*; @@ -208,7 +208,7 @@ void drain() { ObservableSource o; try { - o = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); + o = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); dispose(); @@ -436,7 +436,7 @@ void drain() { ObservableSource o; try { - o = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null ObservableSource"); + o = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancelled = true; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java index 47c4099b83..fe2bed8c8c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.ArrayDeque; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import io.reactivex.rxjava3.core.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.observers.*; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; @@ -271,7 +271,7 @@ public void drain() { break; } - source = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null ObservableSource"); + source = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java index efe4b79fc6..f45dd92f1f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -19,7 +20,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.*; @@ -218,7 +218,7 @@ public void run() { ObservableSource o; try { - o = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); + o = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); dispose(); @@ -459,7 +459,7 @@ public void run() { ObservableSource o; try { - o = ObjectHelper.requireNonNull(mapper.apply(v), "The mapper returned a null ObservableSource"); + o = Objects.requireNonNull(mapper.apply(v), "The mapper returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancelled = true; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java index d2e1f8cf18..20f98731c1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.observers.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -81,7 +81,7 @@ public void onNext(T t) { ObservableSource p; try { - p = ObjectHelper.requireNonNull(debounceSelector.apply(t), "The ObservableSource supplied is null"); + p = Objects.requireNonNull(debounceSelector.apply(t), "The ObservableSource supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDefer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDefer.java index c1f8dc41ab..15b3394af7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDefer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDefer.java @@ -17,7 +17,8 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; public final class ObservableDefer extends Observable { final Supplier> supplier; @@ -29,7 +30,7 @@ public ObservableDefer(Supplier> supplie public void subscribeActual(Observer observer) { ObservableSource pub; try { - pub = ObjectHelper.requireNonNull(supplier.get(), "The supplier returned a null ObservableSource"); + pub = Objects.requireNonNull(supplier.get(), "The supplier returned a null ObservableSource"); } catch (Throwable t) { Exceptions.throwIfFatal(t); EmptyDisposable.error(t, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java index 59da21cd47..45b42e1ecc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java @@ -18,9 +18,10 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class ObservableDematerialize extends AbstractObservableWithUpstream { final Function> selector; @@ -83,7 +84,7 @@ public void onNext(T item) { Notification notification; try { - notification = ObjectHelper.requireNonNull(selector.apply(item), "The selector returned a null Notification"); + notification = Objects.requireNonNull(selector.apply(item), "The selector returned a null Notification"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java index 98c197156c..aadeafb2b1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java @@ -14,13 +14,13 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.Collection; +import java.util.Objects; import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.BasicFuseableObserver; import io.reactivex.rxjava3.internal.util.ExceptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -74,7 +74,7 @@ public void onNext(T value) { boolean b; try { - key = ObjectHelper.requireNonNull(keySelector.apply(value), "The keySelector returned a null key"); + key = Objects.requireNonNull(keySelector.apply(value), "The keySelector returned a null key"); b = collection.add(key); } catch (Throwable ex) { fail(ex); @@ -120,7 +120,7 @@ public T poll() throws Throwable { for (;;) { T v = qd.poll(); - if (v == null || collection.add(ObjectHelper.requireNonNull(keySelector.apply(v), "The keySelector returned a null key"))) { + if (v == null || collection.add(Objects.requireNonNull(keySelector.apply(v), "The keySelector returned a null key"))) { return v; } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java index 596719176e..7e01d7cbc1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java @@ -22,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.*; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -117,7 +116,7 @@ public void onNext(T t) { } ObservableSource p; try { - p = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); + p = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java index 26f0505a14..75fc80ac61 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.annotations.Nullable; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.BasicIntQueueDisposable; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -88,7 +88,7 @@ public void onNext(T value) { CompletableSource cs; try { - cs = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java index b9a9213769..ddf9602a05 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.FuseToObservable; import io.reactivex.rxjava3.internal.util.AtomicThrowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -94,7 +94,7 @@ public void onNext(T value) { CompletableSource cs; try { - cs = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java index 9e323ec346..ac951a571c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -96,7 +96,7 @@ public void onNext(T t) { MaybeSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); + ms = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java index 4ca080a73f..6d5dd3da63 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -96,7 +96,7 @@ public void onNext(T t) { SingleSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); + ms = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java index c6704398a1..3f90f31acd 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java @@ -14,13 +14,13 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.Iterator; +import java.util.Objects; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** @@ -100,7 +100,7 @@ public void onNext(T value) { R v; try { - v = ObjectHelper.requireNonNull(it.next(), "The iterator returned a null value"); + v = Objects.requireNonNull(it.next(), "The iterator returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java index 7c4b09ba62..fb4beaaf21 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java @@ -15,9 +15,10 @@ import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.BasicQueueDisposable; +import java.util.Objects; + public final class ObservableFromArray extends Observable { final T[] array; public ObservableFromArray(T[] array) { @@ -70,7 +71,7 @@ public T poll() { T[] a = array; if (i != a.length) { index = i + 1; - return ObjectHelper.requireNonNull(a[i], "The array element is null"); + return Objects.requireNonNull(a[i], "The array element is null"); } return null; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java index 91811b3c4a..9edf260931 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java @@ -14,12 +14,12 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.Iterator; +import java.util.Objects; import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.BasicQueueDisposable; public final class ObservableFromIterable extends Observable { @@ -88,7 +88,7 @@ void run() { T v; try { - v = ObjectHelper.requireNonNull(it.next(), "The iterator returned a null value"); + v = Objects.requireNonNull(it.next(), "The iterator returned a null value"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); @@ -138,7 +138,7 @@ public T poll() { checkNext = true; } - return ObjectHelper.requireNonNull(it.next(), "The iterator returned a null value"); + return Objects.requireNonNull(it.next(), "The iterator returned a null value"); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java index 821d2e81e1..78edf33cea 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java @@ -23,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.observables.GroupedObservable; @@ -115,7 +114,7 @@ public void onNext(T t) { V v; try { - v = ObjectHelper.requireNonNull(valueSelector.apply(t), "The value supplied is null"); + v = Objects.requireNonNull(valueSelector.apply(t), "The value supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java index b6c9f811f4..7b12579966 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java @@ -26,7 +26,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.ExceptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -243,7 +242,7 @@ void drain() { ObservableSource p; try { - p = ObjectHelper.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null ObservableSource"); + p = Objects.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null ObservableSource"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -265,7 +264,7 @@ void drain() { R w; try { - w = ObjectHelper.requireNonNull(resultSelector.apply(left, up), "The resultSelector returned a null value"); + w = Objects.requireNonNull(resultSelector.apply(left, up), "The resultSelector returned a null value"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -288,7 +287,7 @@ else if (mode == RIGHT_VALUE) { ObservableSource p; try { - p = ObjectHelper.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null ObservableSource"); + p = Objects.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null ObservableSource"); } catch (Throwable exc) { fail(exc, a, q); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java index 1665b99014..fcd5f57fa5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java @@ -12,6 +12,7 @@ */ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.TimeUnit; import io.reactivex.rxjava3.core.*; @@ -73,7 +74,7 @@ static final class ItemDelayFunction implements Function apply(final T v) throws Throwable { - ObservableSource o = ObjectHelper.requireNonNull(itemDelay.apply(v), "The itemDelay returned a null ObservableSource"); + ObservableSource o = Objects.requireNonNull(itemDelay.apply(v), "The itemDelay returned a null ObservableSource"); return new ObservableTake(o, 1).map(Functions.justFunction(v)).defaultIfEmpty(v); } } @@ -161,7 +162,7 @@ static final class FlatMapWithCombinerOuter implements Function apply(final T t) throws Throwable { @SuppressWarnings("unchecked") - ObservableSource u = (ObservableSource)ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); + ObservableSource u = (ObservableSource)Objects.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); return new ObservableMap(u, new FlatMapWithCombinerInner(combiner, t)); } } @@ -181,7 +182,7 @@ static final class FlatMapIntoIterable implements Function apply(T t) throws Throwable { - return new ObservableFromIterable(ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); + return new ObservableFromIterable(Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java index cf98732637..6660d342f3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java @@ -24,7 +24,6 @@ import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.observable.ObservableGroupJoin.*; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.ExceptionHelper; @@ -219,7 +218,7 @@ void drain() { ObservableSource p; try { - p = ObjectHelper.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null ObservableSource"); + p = Objects.requireNonNull(leftEnd.apply(left), "The leftEnd returned a null ObservableSource"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -243,7 +242,7 @@ void drain() { R w; try { - w = ObjectHelper.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); + w = Objects.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -263,7 +262,7 @@ else if (mode == RIGHT_VALUE) { ObservableSource p; try { - p = ObjectHelper.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null ObservableSource"); + p = Objects.requireNonNull(rightEnd.apply(right), "The rightEnd returned a null ObservableSource"); } catch (Throwable exc) { fail(exc, a, q); return; @@ -287,7 +286,7 @@ else if (mode == RIGHT_VALUE) { R w; try { - w = ObjectHelper.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); + w = Objects.requireNonNull(resultSelector.apply(left, right), "The resultSelector returned a null value"); } catch (Throwable exc) { fail(exc, a, q); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLift.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLift.java index f38ac66bd3..4aa45683ff 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLift.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLift.java @@ -15,9 +15,10 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Allows lifting operators into a chain of Observables. * @@ -40,7 +41,7 @@ public ObservableLift(ObservableSource source, ObservableOperator observer) { Observer liftedObserver; try { - liftedObserver = ObjectHelper.requireNonNull(operator.apply(observer), "Operator " + operator + " returned a null Observer"); + liftedObserver = Objects.requireNonNull(operator.apply(observer), "Operator " + operator + " returned a null Observer"); } catch (NullPointerException e) { // NOPMD throw e; } catch (Throwable e) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMap.java index 9bff5de703..4361016ec6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMap.java @@ -16,9 +16,10 @@ import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.BasicFuseableObserver; +import java.util.Objects; + public final class ObservableMap extends AbstractObservableWithUpstream { final Function function; @@ -54,7 +55,7 @@ public void onNext(T t) { U v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); + v = Objects.requireNonNull(mapper.apply(t), "The mapper function returned a null value."); } catch (Throwable ex) { fail(ex); return; @@ -71,7 +72,7 @@ public int requestFusion(int mode) { @Override public U poll() throws Throwable { T t = qd.poll(); - return t != null ? ObjectHelper.requireNonNull(mapper.apply(t), "The mapper function returned a null value.") : null; + return t != null ? Objects.requireNonNull(mapper.apply(t), "The mapper function returned a null value.") : null; } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java index 8ef5a5395d..147c21ae37 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java @@ -18,7 +18,8 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; public final class ObservableMapNotification extends AbstractObservableWithUpstream> { @@ -84,7 +85,7 @@ public void onNext(T t) { ObservableSource p; try { - p = ObjectHelper.requireNonNull(onNextMapper.apply(t), "The onNext ObservableSource returned is null"); + p = Objects.requireNonNull(onNextMapper.apply(t), "The onNext ObservableSource returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); @@ -99,7 +100,7 @@ public void onError(Throwable t) { ObservableSource p; try { - p = ObjectHelper.requireNonNull(onErrorMapper.apply(t), "The onError ObservableSource returned is null"); + p = Objects.requireNonNull(onErrorMapper.apply(t), "The onError ObservableSource returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(new CompositeException(t, e)); @@ -115,7 +116,7 @@ public void onComplete() { ObservableSource p; try { - p = ObjectHelper.requireNonNull(onCompleteSupplier.get(), "The onComplete ObservableSource returned is null"); + p = Objects.requireNonNull(onCompleteSupplier.get(), "The onComplete ObservableSource returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java index ef7d2367fc..8247a6abfc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.subjects.PublishSubject; /** @@ -45,7 +45,7 @@ protected void subscribeActual(Observer observer) { ObservableSource target; try { - target = ObjectHelper.requireNonNull(selector.apply(subject), "The selector returned a null ObservableSource"); + target = Objects.requireNonNull(selector.apply(subject), "The selector returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java index 24d5e5d568..bab7bbb065 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java @@ -18,9 +18,10 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Reduce a sequence of values into a single value via an aggregator function and emit the final value or complete * if the source is empty. @@ -78,7 +79,7 @@ public void onNext(T value) { this.value = value; } else { try { - this.value = ObjectHelper.requireNonNull(reducer.apply(v, value), "The reducer returned a null value"); + this.value = Objects.requireNonNull(reducer.apply(v, value), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java index 4ec8ac8bcc..e6b0d1d0ef 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java @@ -18,9 +18,10 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Reduce a sequence of values, starting from a seed value and by using * an accumulator function and return the last accumulated value. @@ -77,7 +78,7 @@ public void onNext(T value) { R v = this.value; if (v != null) { try { - this.value = ObjectHelper.requireNonNull(reducer.apply(v, value), "The reducer returned a null value"); + this.value = Objects.requireNonNull(reducer.apply(v, value), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java index e7b1b16813..956d7bf7c9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java @@ -17,9 +17,10 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.observable.ObservableReduceSeedSingle.ReduceSeedObserver; +import java.util.Objects; + /** * Reduce a sequence of values, starting from a generated seed value and by using * an accumulator function and return the last accumulated value. @@ -46,7 +47,7 @@ protected void subscribeActual(SingleObserver observer) { R seed; try { - seed = ObjectHelper.requireNonNull(seedSupplier.get(), "The seedSupplier returned a null value"); + seed = Objects.requireNonNull(seedSupplier.get(), "The seedSupplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java index abdf42b872..eff1e248ac 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.subjects.*; @@ -45,7 +45,7 @@ protected void subscribeActual(Observer observer) { ObservableSource other; try { - other = ObjectHelper.requireNonNull(handler.apply(signaller), "The handler returned a null ObservableSource"); + other = Objects.requireNonNull(handler.apply(signaller), "The handler returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java index 1bb4f344d4..ba0811b85c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java @@ -24,7 +24,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.HasUpstreamObservableSource; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.observables.ConnectableObservable; @@ -1054,8 +1053,8 @@ protected void subscribeActual(Observer child) { ConnectableObservable co; ObservableSource observable; try { - co = ObjectHelper.requireNonNull(connectableFactory.get(), "The connectableFactory returned a null ConnectableObservable"); - observable = ObjectHelper.requireNonNull(selector.apply(co), "The selector returned a null ObservableSource"); + co = Objects.requireNonNull(connectableFactory.get(), "The connectableFactory returned a null ConnectableObservable"); + observable = Objects.requireNonNull(selector.apply(co), "The selector returned a null ObservableSource"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptyDisposable.error(e, child); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java index 054887adab..0eb5fbb5b7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.subjects.*; @@ -45,7 +45,7 @@ protected void subscribeActual(Observer observer) { ObservableSource other; try { - other = ObjectHelper.requireNonNull(handler.apply(signaller), "The handler returned a null ObservableSource"); + other = Objects.requireNonNull(handler.apply(signaller), "The handler returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java index 85c13e27e7..128d327e2b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import io.reactivex.rxjava3.annotations.Nullable; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.QueueDisposable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -66,7 +66,7 @@ public static boolean tryScalarXMapSubscribe(ObservableSource source, ObservableSource r; try { - r = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); + r = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); @@ -138,7 +138,7 @@ static final class ScalarXMapObservable extends Observable { public void subscribeActual(Observer observer) { ObservableSource other; try { - other = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null ObservableSource"); + other = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null ObservableSource"); } catch (Throwable e) { EmptyDisposable.error(e, observer); return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java index 026b3ac5eb..94918b6c39 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java @@ -18,9 +18,10 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class ObservableScan extends AbstractObservableWithUpstream { final BiFunction accumulator; public ObservableScan(ObservableSource source, BiFunction accumulator) { @@ -80,7 +81,7 @@ public void onNext(T t) { T u; try { - u = ObjectHelper.requireNonNull(accumulator.apply(v, t), "The value returned by the accumulator is null"); + u = Objects.requireNonNull(accumulator.apply(v, t), "The value returned by the accumulator is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java index e13c6fe189..24e4b580ad 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java @@ -17,9 +17,10 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + public final class ObservableScanSeed extends AbstractObservableWithUpstream { final BiFunction accumulator; final Supplier seedSupplier; @@ -35,7 +36,7 @@ public void subscribeActual(Observer t) { R r; try { - r = ObjectHelper.requireNonNull(seedSupplier.get(), "The seed supplied is null"); + r = Objects.requireNonNull(seedSupplier.get(), "The seed supplied is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptyDisposable.error(e, t); @@ -93,7 +94,7 @@ public void onNext(T t) { R u; try { - u = ObjectHelper.requireNonNull(accumulator.apply(v, t), "The accumulator returned a null value"); + u = Objects.requireNonNull(accumulator.apply(v, t), "The accumulator returned a null value"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java index d9d50fc032..bca0f1beeb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.util.AtomicThrowable; @@ -108,7 +108,7 @@ public void onNext(T t) { ObservableSource p; try { - p = ObjectHelper.requireNonNull(mapper.apply(t), "The ObservableSource returned is null"); + p = Objects.requireNonNull(mapper.apply(t), "The ObservableSource returned is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); upstream.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java index 4fe72feadd..a9ada29730 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.observable.ObservableTimeoutTimed.TimeoutSupport; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -102,7 +102,7 @@ public void onNext(T t) { ObservableSource itemTimeoutObservableSource; try { - itemTimeoutObservableSource = ObjectHelper.requireNonNull( + itemTimeoutObservableSource = Objects.requireNonNull( itemTimeoutIndicator.apply(t), "The itemTimeoutIndicator returned a null ObservableSource."); } catch (Throwable ex) { @@ -231,7 +231,7 @@ public void onNext(T t) { ObservableSource itemTimeoutObservableSource; try { - itemTimeoutObservableSource = ObjectHelper.requireNonNull( + itemTimeoutObservableSource = Objects.requireNonNull( itemTimeoutIndicator.apply(t), "The itemTimeoutIndicator returned a null ObservableSource."); } catch (Throwable ex) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java index 3a4e5d81e2..60fad9e740 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class ObservableUsing extends Observable { @@ -53,7 +53,7 @@ public void subscribeActual(Observer observer) { ObservableSource source; try { - source = ObjectHelper.requireNonNull(sourceSupplier.apply(resource), "The sourceSupplier returned a null ObservableSource"); + source = Objects.requireNonNull(sourceSupplier.apply(resource), "The sourceSupplier returned a null ObservableSource"); } catch (Throwable e) { Exceptions.throwIfFatal(e); try { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java index 037c7d0773..5ea42da118 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java @@ -23,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.SimplePlainQueue; import io.reactivex.rxjava3.internal.queue.MpscLinkedQueue; import io.reactivex.rxjava3.internal.util.*; @@ -236,7 +235,7 @@ void drain() { ObservableSource endSource; try { - endSource = ObjectHelper.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null ObservableSource"); + endSource = Objects.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null ObservableSource"); } catch (Throwable ex) { upstream.dispose(); startObserver.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java index 91b4b0266d..6ac680db30 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.observers.SerializedObserver; public final class ObservableWithLatestFrom extends AbstractObservableWithUpstream { @@ -73,7 +73,7 @@ public void onNext(T t) { if (u != null) { R r; try { - r = ObjectHelper.requireNonNull(combiner.apply(t, u), "The combiner returned a null value"); + r = Objects.requireNonNull(combiner.apply(t, u), "The combiner returned a null value"); } catch (Throwable e) { Exceptions.throwIfFatal(e); dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java index fc64adc42f..73d28032fc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.Arrays; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.annotations.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -164,7 +164,7 @@ public void onNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(combiner.apply(objects), "combiner returned a null value"); + v = Objects.requireNonNull(combiner.apply(objects), "combiner returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); dispose(); @@ -285,7 +285,7 @@ public void dispose() { final class SingletonArrayFunc implements Function { @Override public R apply(T t) throws Throwable { - return ObjectHelper.requireNonNull(combiner.apply(new Object[] { t }), "The combiner returned a null value"); + return Objects.requireNonNull(combiner.apply(new Object[] { t }), "The combiner returned a null value"); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java index b3d40dc2ed..80ae9b174b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.Arrays; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; public final class ObservableZip extends Observable { @@ -195,7 +195,7 @@ public void drain() { R v; try { - v = ObjectHelper.requireNonNull(zipper.apply(os.clone()), "The zipper returned a null value"); + v = Objects.requireNonNull(zipper.apply(os.clone()), "The zipper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterable.java index d86b90395d..d775c58515 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterable.java @@ -14,13 +14,13 @@ package io.reactivex.rxjava3.internal.operators.observable; import java.util.Iterator; +import java.util.Objects; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class ObservableZipIterable extends Observable { @@ -41,7 +41,7 @@ public void subscribeActual(Observer t) { Iterator it; try { - it = ObjectHelper.requireNonNull(other.iterator(), "The iterator returned by other is null"); + it = Objects.requireNonNull(other.iterator(), "The iterator returned by other is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptyDisposable.error(e, t); @@ -109,7 +109,7 @@ public void onNext(T t) { U u; try { - u = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + u = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); } catch (Throwable e) { Exceptions.throwIfFatal(e); error(e); @@ -118,7 +118,7 @@ public void onNext(T t) { V v; try { - v = ObjectHelper.requireNonNull(zipper.apply(t, u), "The zipper function returned a null value"); + v = Objects.requireNonNull(zipper.apply(t, u), "The zipper function returned a null value"); } catch (Throwable e) { Exceptions.throwIfFatal(e); error(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java index d37bcfa1b3..e1da1b76b7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java @@ -17,12 +17,13 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscribers.DeferredScalarSubscriber; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.parallel.ParallelFlowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Reduce the sequence of values in each 'rail' to a single value. * @@ -59,7 +60,7 @@ public void subscribe(Subscriber[] subscribers) { C initialValue; try { - initialValue = ObjectHelper.requireNonNull(initialCollection.get(), "The initialSupplier returned a null value"); + initialValue = Objects.requireNonNull(initialCollection.get(), "The initialSupplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); reportError(subscribers, ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelConcatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelConcatMap.java index 5b7d3ef5c4..1ff745ecba 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelConcatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelConcatMap.java @@ -16,11 +16,12 @@ import org.reactivestreams.*; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.flowable.FlowableConcatMap; import io.reactivex.rxjava3.internal.util.ErrorMode; import io.reactivex.rxjava3.parallel.ParallelFlowable; +import java.util.Objects; + /** * Concatenates the generated Publishers on each rail. * @@ -42,9 +43,9 @@ public ParallelConcatMap( Function> mapper, int prefetch, ErrorMode errorMode) { this.source = source; - this.mapper = ObjectHelper.requireNonNull(mapper, "mapper"); + this.mapper = Objects.requireNonNull(mapper, "mapper"); this.prefetch = prefetch; - this.errorMode = ObjectHelper.requireNonNull(errorMode, "errorMode"); + this.errorMode = Objects.requireNonNull(errorMode, "errorMode"); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java index 0e93abcccb..557d5282c4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java @@ -17,12 +17,13 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.parallel.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Calls a Consumer for each upstream value passing by * and handles any failure with a handler function. @@ -133,7 +134,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = ObjectHelper.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); @@ -244,7 +245,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = ObjectHelper.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java index 12366c30fa..f6eff78249 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java @@ -17,12 +17,13 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.parallel.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Filters each 'rail' of the source ParallelFlowable with a predicate function. * @@ -136,7 +137,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = ObjectHelper.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); @@ -225,7 +226,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = ObjectHelper.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMap.java index b262bfa240..c941887853 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMap.java @@ -18,12 +18,13 @@ import io.reactivex.rxjava3.core.FlowableSubscriber; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.parallel.ParallelFlowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Maps each 'rail' of the source ParallelFlowable with a mapper function. * @@ -110,7 +111,7 @@ public void onNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null value"); + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancel(); @@ -183,7 +184,7 @@ public void onNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null value"); + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancel(); @@ -202,7 +203,7 @@ public boolean tryOnNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null value"); + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java index 8096116ea5..ff8da40c62 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java @@ -17,12 +17,13 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.parallel.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Maps each 'rail' of the source ParallelFlowable with a mapper function * and handle any failure based on a handler function. @@ -129,14 +130,14 @@ public boolean tryOnNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null value"); + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); ParallelFailureHandling h; try { - h = ObjectHelper.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); @@ -242,14 +243,14 @@ public boolean tryOnNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(mapper.apply(t), "The mapper returned a null value"); + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); ParallelFailureHandling h; try { - h = ObjectHelper.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java index 4c15e207d4..50f04d46ec 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java @@ -18,11 +18,12 @@ import io.reactivex.rxjava3.core.FlowableSubscriber; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.parallel.ParallelFlowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Execute a Consumer in each 'rail' for the current element passing through. * @@ -53,14 +54,14 @@ public ParallelPeek(ParallelFlowable source, ) { this.source = source; - this.onNext = ObjectHelper.requireNonNull(onNext, "onNext is null"); - this.onAfterNext = ObjectHelper.requireNonNull(onAfterNext, "onAfterNext is null"); - this.onError = ObjectHelper.requireNonNull(onError, "onError is null"); - this.onComplete = ObjectHelper.requireNonNull(onComplete, "onComplete is null"); - this.onAfterTerminated = ObjectHelper.requireNonNull(onAfterTerminated, "onAfterTerminated is null"); - this.onSubscribe = ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); - this.onRequest = ObjectHelper.requireNonNull(onRequest, "onRequest is null"); - this.onCancel = ObjectHelper.requireNonNull(onCancel, "onCancel is null"); + this.onNext = Objects.requireNonNull(onNext, "onNext is null"); + this.onAfterNext = Objects.requireNonNull(onAfterNext, "onAfterNext is null"); + this.onError = Objects.requireNonNull(onError, "onError is null"); + this.onComplete = Objects.requireNonNull(onComplete, "onComplete is null"); + this.onAfterTerminated = Objects.requireNonNull(onAfterTerminated, "onAfterTerminated is null"); + this.onSubscribe = Objects.requireNonNull(onSubscribe, "onSubscribe is null"); + this.onRequest = Objects.requireNonNull(onRequest, "onRequest is null"); + this.onCancel = Objects.requireNonNull(onCancel, "onCancel is null"); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java index f813149407..87c02c8922 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java @@ -17,12 +17,13 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscribers.DeferredScalarSubscriber; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.parallel.ParallelFlowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Reduce the sequence of values in each 'rail' to a single value. * @@ -58,7 +59,7 @@ public void subscribe(Subscriber[] subscribers) { R initialValue; try { - initialValue = ObjectHelper.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); + initialValue = Objects.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); reportError(subscribers, ex); @@ -115,7 +116,7 @@ public void onNext(T t) { R v; try { - v = ObjectHelper.requireNonNull(reducer.apply(accumulator, t), "The reducer returned a null value"); + v = Objects.requireNonNull(reducer.apply(accumulator, t), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java index 3a8cfa2290..e376c352cb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.parallel; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.parallel.ParallelFlowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -132,7 +132,7 @@ void innerComplete(T value) { if (sp != null) { try { - value = ObjectHelper.requireNonNull(reducer.apply(sp.first, sp.second), "The reducer returned a null value"); + value = Objects.requireNonNull(reducer.apply(sp.first, sp.second), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); innerError(ex); @@ -192,7 +192,7 @@ public void onNext(T t) { } else { try { - v = ObjectHelper.requireNonNull(reducer.apply(v, t), "The reducer returned a null value"); + v = Objects.requireNonNull(reducer.apply(v, t), "The reducer returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); get().cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDefer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDefer.java index fff2ca8d58..24a35fbdee 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDefer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDefer.java @@ -17,7 +17,8 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; public final class SingleDefer extends Single { @@ -32,7 +33,7 @@ protected void subscribeActual(SingleObserver observer) { SingleSource next; try { - next = ObjectHelper.requireNonNull(singleSupplier.get(), "The singleSupplier returned a null SingleSource"); + next = Objects.requireNonNull(singleSupplier.get(), "The singleSupplier returned a null SingleSource"); } catch (Throwable e) { Exceptions.throwIfFatal(e); EmptyDisposable.error(e, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java index 15f57b631b..4582e4d4da 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java @@ -18,7 +18,8 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; /** * Maps the success value of the source to a Notification, then @@ -81,7 +82,7 @@ public void onSuccess(T t) { Notification notification; try { - notification = ObjectHelper.requireNonNull(selector.apply(t), "The selector returned a null Notification"); + notification = Objects.requireNonNull(selector.apply(t), "The selector returned a null Notification"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMap.java index 3128c93e5a..d622f83fae 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMap.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; public final class SingleFlatMap extends Single { final SingleSource source; @@ -74,7 +74,7 @@ public void onSuccess(T value) { SingleSource o; try { - o = ObjectHelper.requireNonNull(mapper.apply(value), "The single returned by the mapper is null"); + o = Objects.requireNonNull(mapper.apply(value), "The single returned by the mapper is null"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java index 03edadb83b..d4a9f54737 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Maps the success value of the source SingleSource into a Completable. @@ -80,7 +80,7 @@ public void onSuccess(T value) { CompletableSource cs; try { - cs = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); + cs = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null CompletableSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java index cc7f2740b9..0d9e4aee0f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.single; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import org.reactivestreams.Subscriber; @@ -24,7 +25,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.*; import io.reactivex.rxjava3.internal.util.BackpressureHelper; @@ -166,7 +166,7 @@ void drain() { R v; try { - v = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + v = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); a.onError(ex); @@ -277,7 +277,7 @@ public R poll() throws Exception { Iterator iterator = it; if (iterator != null) { - R v = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + R v = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); if (!iterator.hasNext()) { it = null; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java index d742a06e1c..795f60a3ca 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.internal.operators.single; import java.util.Iterator; +import java.util.Objects; import io.reactivex.rxjava3.annotations.Nullable; import io.reactivex.rxjava3.core.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.BasicIntQueueDisposable; /** @@ -187,7 +187,7 @@ public R poll() throws Exception { Iterator iterator = it; if (iterator != null) { - R v = ObjectHelper.requireNonNull(iterator.next(), "The iterator returned a null value"); + R v = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); if (!iterator.hasNext()) { it = null; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapMaybe.java index 20e60ab476..d810f8f5c3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapMaybe.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; public final class SingleFlatMapMaybe extends Maybe { @@ -75,7 +75,7 @@ public void onSuccess(T value) { MaybeSource ms; try { - ms = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper returned a null MaybeSource"); + ms = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null MaybeSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java index 0fb91438c8..4a636ba337 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -21,7 +22,6 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; /** @@ -86,7 +86,7 @@ public void onSubscribe(Disposable d) { public void onSuccess(S value) { Publisher f; try { - f = ObjectHelper.requireNonNull(mapper.apply(value), "the mapper returned a null Publisher"); + f = Objects.requireNonNull(mapper.apply(value), "the mapper returned a null Publisher"); } catch (Throwable e) { Exceptions.throwIfFatal(e); downstream.onError(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java index 562032f01e..5300d968b0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java @@ -13,12 +13,12 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.Callable; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.Exceptions; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class SingleFromCallable extends Single { @@ -40,7 +40,7 @@ protected void subscribeActual(SingleObserver observer) { T value; try { - value = ObjectHelper.requireNonNull(callable.call(), "The callable returned a null value"); + value = Objects.requireNonNull(callable.call(), "The callable returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (!d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java index 04d6ca72cf..88dcd7dc94 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java @@ -17,9 +17,10 @@ import io.reactivex.rxjava3.disposables.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import java.util.Objects; + /** * Calls a supplier and emits its value or exception to the incoming SingleObserver. * @param the value type returned @@ -44,7 +45,7 @@ protected void subscribeActual(SingleObserver observer) { T value; try { - value = ObjectHelper.requireNonNull(supplier.get(), "The supplier returned a null value"); + value = Objects.requireNonNull(supplier.get(), "The supplier returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (!d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleLift.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleLift.java index af5f969157..8382b2f1b8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleLift.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleLift.java @@ -16,7 +16,8 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; public final class SingleLift extends Single { @@ -34,7 +35,7 @@ protected void subscribeActual(SingleObserver observer) { SingleObserver sr; try { - sr = ObjectHelper.requireNonNull(onLift.apply(observer), "The onLift returned a null SingleObserver"); + sr = Objects.requireNonNull(onLift.apply(observer), "The onLift returned a null SingleObserver"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMap.java index d6ee948d5a..534b41495c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMap.java @@ -17,7 +17,8 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; + +import java.util.Objects; public final class SingleMap extends Single { final SingleSource source; @@ -54,7 +55,7 @@ public void onSubscribe(Disposable d) { public void onSuccess(T value) { R v; try { - v = ObjectHelper.requireNonNull(mapper.apply(value), "The mapper function returned a null value."); + v = Objects.requireNonNull(mapper.apply(value), "The mapper function returned a null value."); } catch (Throwable e) { Exceptions.throwIfFatal(e); onError(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java index 71ccfdc035..6f768591e8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.observers.ResumeSingleObserver; public final class SingleResumeNext extends Single { @@ -70,7 +70,7 @@ public void onError(Throwable e) { SingleSource source; try { - source = ObjectHelper.requireNonNull(nextFunction.apply(e), "The nextFunction returned a null SingleSource."); + source = Objects.requireNonNull(nextFunction.apply(e), "The nextFunction returned a null SingleSource."); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(new CompositeException(e, ex)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUsing.java index a0cb1d84d9..57f9daf263 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUsing.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class SingleUsing extends Single { @@ -56,7 +56,7 @@ protected void subscribeActual(final SingleObserver observer) { SingleSource source; try { - source = ObjectHelper.requireNonNull(singleFunction.apply(resource), "The singleFunction returned a null SingleSource"); + source = Objects.requireNonNull(singleFunction.apply(resource), "The singleFunction returned a null SingleSource"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java index 9633e94bc7..b207476c95 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.operators.single; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.core.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; public final class SingleZipArray extends Single { @@ -109,7 +109,7 @@ void innerSuccess(T value, int index) { R v; try { - v = ObjectHelper.requireNonNull(zipper.apply(values), "The zipper returned a null value"); + v = Objects.requireNonNull(zipper.apply(values), "The zipper returned a null value"); } catch (Throwable ex) { Exceptions.throwIfFatal(ex); downstream.onError(ex); @@ -179,7 +179,7 @@ public void onError(Throwable e) { final class SingletonArrayFunc implements Function { @Override public R apply(T t) throws Throwable { - return ObjectHelper.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); + return Objects.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java index 215f19a58e..b41538aeb4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java @@ -19,7 +19,6 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.operators.single.SingleZipArray.ZipCoordinator; public final class SingleZipIterable extends Single { @@ -82,7 +81,7 @@ protected void subscribeActual(SingleObserver observer) { final class SingletonArrayFunc implements Function { @Override public R apply(T t) throws Throwable { - return ObjectHelper.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); + return Objects.requireNonNull(zipper.apply(new Object[] { t }), "The zipper returned a null value"); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java index c3fb7955fa..6a50ab07dc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java +++ b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java @@ -25,11 +25,11 @@ * the License for the specific language governing permissions and limitations under the License. */ +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.Subscription; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.BackpressureHelper; /** @@ -78,7 +78,7 @@ public final void setSubscription(Subscription s) { return; } - ObjectHelper.requireNonNull(s, "s is null"); + Objects.requireNonNull(s, "s is null"); if (get() == 0 && compareAndSet(0, 1)) { Subscription a = actual; diff --git a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelper.java b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelper.java index 7080b3472c..8cfc3f5a7c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelper.java @@ -13,12 +13,12 @@ package io.reactivex.rxjava3.internal.subscriptions; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.Subscription; import io.reactivex.rxjava3.exceptions.ProtocolViolationException; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.BackpressureHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -129,7 +129,7 @@ public static boolean set(AtomicReference field, Subscription s) { * @return true if the operation succeeded, false if the target field was not null. */ public static boolean setOnce(AtomicReference field, Subscription s) { - ObjectHelper.requireNonNull(s, "s is null"); + Objects.requireNonNull(s, "s is null"); if (!field.compareAndSet(null, s)) { s.cancel(); if (field.get() != CANCELLED) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/EndConsumerHelper.java b/src/main/java/io/reactivex/rxjava3/internal/util/EndConsumerHelper.java index 1e051a636c..01b95a0a11 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/EndConsumerHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/EndConsumerHelper.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.internal.util; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import org.reactivestreams.Subscription; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.ProtocolViolationException; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -50,7 +50,7 @@ private EndConsumerHelper() { * @return true if successful, false if the upstream was non null */ public static boolean validate(Disposable upstream, Disposable next, Class observer) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); if (upstream != null) { next.dispose(); if (upstream != DisposableHelper.DISPOSED) { @@ -72,7 +72,7 @@ public static boolean validate(Disposable upstream, Disposable next, Class ob * @return true if successful, false if the content of the AtomicReference was non null */ public static boolean setOnce(AtomicReference upstream, Disposable next, Class observer) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); if (!upstream.compareAndSet(null, next)) { next.dispose(); if (upstream.get() != DisposableHelper.DISPOSED) { @@ -95,7 +95,7 @@ public static boolean setOnce(AtomicReference upstream, Disposable n * @return true if successful, false if the upstream was non null */ public static boolean validate(Subscription upstream, Subscription next, Class subscriber) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); if (upstream != null) { next.cancel(); if (upstream != SubscriptionHelper.CANCELLED) { @@ -117,7 +117,7 @@ public static boolean validate(Subscription upstream, Subscription next, Class upstream, Subscription next, Class subscriber) { - ObjectHelper.requireNonNull(next, "next is null"); + Objects.requireNonNull(next, "next is null"); if (!upstream.compareAndSet(null, next)) { next.cancel(); if (upstream.get() != SubscriptionHelper.CANCELLED) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java b/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java index 606368d041..c4aaadb343 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java @@ -165,7 +165,7 @@ public static NullPointerException createNullPointerException(String prefix) { } /** - * Similar to ObjectHelper.requireNonNull but composes the error message via + * Similar to Objects.requireNonNull but composes the error message via * {@link #nullWarning(String)}. * @param the value type * @param value the value to check diff --git a/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java b/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java index 135f183ceb..1a9b942011 100644 --- a/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java +++ b/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.observables; +import java.util.Objects; import java.util.concurrent.TimeUnit; import io.reactivex.rxjava3.annotations.*; @@ -209,8 +210,8 @@ public final Observable refCount(int subscriberCount, long timeout, TimeUnit @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable refCount(int subscriberCount, long timeout, TimeUnit unit, Scheduler scheduler) { ObjectHelper.verifyPositive(subscriberCount, "subscriberCount"); - ObjectHelper.requireNonNull(unit, "unit is null"); - ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + Objects.requireNonNull(unit, "unit is null"); + Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableRefCount(this, subscriberCount, timeout, unit, scheduler)); } diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java index 3a7ab322a6..b57d77f6f5 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java @@ -13,13 +13,13 @@ package io.reactivex.rxjava3.observers; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.CompletableObserver; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** @@ -87,7 +87,7 @@ public abstract class ResourceCompletableObserver implements CompletableObserver * @throws NullPointerException if resource is null */ public final void add(@NonNull Disposable resource) { - ObjectHelper.requireNonNull(resource, "resource is null"); + Objects.requireNonNull(resource, "resource is null"); resources.add(resource); } diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java index f824d3c97a..cb42a70fbd 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java @@ -13,13 +13,13 @@ package io.reactivex.rxjava3.observers; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.MaybeObserver; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** @@ -97,7 +97,7 @@ public abstract class ResourceMaybeObserver implements MaybeObserver, Disp * @throws NullPointerException if resource is null */ public final void add(@NonNull Disposable resource) { - ObjectHelper.requireNonNull(resource, "resource is null"); + Objects.requireNonNull(resource, "resource is null"); resources.add(resource); } diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java index 04f2959595..b6d77a0d5f 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java @@ -13,13 +13,13 @@ package io.reactivex.rxjava3.observers; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.Observer; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** @@ -95,7 +95,7 @@ public abstract class ResourceObserver implements Observer, Disposable { * @throws NullPointerException if resource is null */ public final void add(@NonNull Disposable resource) { - ObjectHelper.requireNonNull(resource, "resource is null"); + Objects.requireNonNull(resource, "resource is null"); resources.add(resource); } diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java index fe8075f75a..af723bf789 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java @@ -13,13 +13,13 @@ package io.reactivex.rxjava3.observers; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.SingleObserver; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** @@ -90,7 +90,7 @@ public abstract class ResourceSingleObserver implements SingleObserver, Di * @throws NullPointerException if resource is null */ public final void add(@NonNull Disposable resource) { - ObjectHelper.requireNonNull(resource, "resource is null"); + Objects.requireNonNull(resource, "resource is null"); resources.add(resource); } diff --git a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java index 38d3505337..afa2eddcf0 100644 --- a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java @@ -113,7 +113,7 @@ public static ParallelFlowable from(@NonNull Publisher sourc @NonNull public static ParallelFlowable from(@NonNull Publisher source, int parallelism, int prefetch) { - ObjectHelper.requireNonNull(source, "source"); + Objects.requireNonNull(source, "source"); ObjectHelper.verifyPositive(parallelism, "parallelism"); ObjectHelper.verifyPositive(prefetch, "prefetch"); @@ -131,7 +131,7 @@ public static ParallelFlowable from(@NonNull Publisher sourc @CheckReturnValue @NonNull public final ParallelFlowable map(@NonNull Function mapper) { - ObjectHelper.requireNonNull(mapper, "mapper"); + Objects.requireNonNull(mapper, "mapper"); return RxJavaPlugins.onAssembly(new ParallelMap(this, mapper)); } @@ -151,8 +151,8 @@ public final ParallelFlowable map(@NonNull Function ParallelFlowable map(@NonNull Function mapper, @NonNull ParallelFailureHandling errorHandler) { - ObjectHelper.requireNonNull(mapper, "mapper"); - ObjectHelper.requireNonNull(errorHandler, "errorHandler is null"); + Objects.requireNonNull(mapper, "mapper"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); return RxJavaPlugins.onAssembly(new ParallelMapTry(this, mapper, errorHandler)); } @@ -173,8 +173,8 @@ public final ParallelFlowable map(@NonNull Function ParallelFlowable map(@NonNull Function mapper, @NonNull BiFunction errorHandler) { - ObjectHelper.requireNonNull(mapper, "mapper"); - ObjectHelper.requireNonNull(errorHandler, "errorHandler is null"); + Objects.requireNonNull(mapper, "mapper"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); return RxJavaPlugins.onAssembly(new ParallelMapTry(this, mapper, errorHandler)); } @@ -187,7 +187,7 @@ public final ParallelFlowable map(@NonNull Function filter(@NonNull Predicate predicate) { - ObjectHelper.requireNonNull(predicate, "predicate"); + Objects.requireNonNull(predicate, "predicate"); return RxJavaPlugins.onAssembly(new ParallelFilter(this, predicate)); } @@ -205,8 +205,8 @@ public final ParallelFlowable filter(@NonNull Predicate predicate) */ @CheckReturnValue public final ParallelFlowable filter(@NonNull Predicate predicate, @NonNull ParallelFailureHandling errorHandler) { - ObjectHelper.requireNonNull(predicate, "predicate"); - ObjectHelper.requireNonNull(errorHandler, "errorHandler is null"); + Objects.requireNonNull(predicate, "predicate"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); return RxJavaPlugins.onAssembly(new ParallelFilterTry(this, predicate, errorHandler)); } @@ -225,8 +225,8 @@ public final ParallelFlowable filter(@NonNull Predicate predicate, */ @CheckReturnValue public final ParallelFlowable filter(@NonNull Predicate predicate, @NonNull BiFunction errorHandler) { - ObjectHelper.requireNonNull(predicate, "predicate"); - ObjectHelper.requireNonNull(errorHandler, "errorHandler is null"); + Objects.requireNonNull(predicate, "predicate"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); return RxJavaPlugins.onAssembly(new ParallelFilterTry(this, predicate, errorHandler)); } @@ -279,7 +279,7 @@ public final ParallelFlowable runOn(@NonNull Scheduler scheduler) { @CheckReturnValue @NonNull public final ParallelFlowable runOn(@NonNull Scheduler scheduler, int prefetch) { - ObjectHelper.requireNonNull(scheduler, "scheduler"); + Objects.requireNonNull(scheduler, "scheduler"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ParallelRunOn(this, scheduler, prefetch)); } @@ -295,7 +295,7 @@ public final ParallelFlowable runOn(@NonNull Scheduler scheduler, int prefetc @CheckReturnValue @NonNull public final Flowable reduce(@NonNull BiFunction reducer) { - ObjectHelper.requireNonNull(reducer, "reducer"); + Objects.requireNonNull(reducer, "reducer"); return RxJavaPlugins.onAssembly(new ParallelReduceFull(this, reducer)); } @@ -313,8 +313,8 @@ public final Flowable reduce(@NonNull BiFunction reducer) { @CheckReturnValue @NonNull public final ParallelFlowable reduce(@NonNull Supplier initialSupplier, @NonNull BiFunction reducer) { - ObjectHelper.requireNonNull(initialSupplier, "initialSupplier"); - ObjectHelper.requireNonNull(reducer, "reducer"); + Objects.requireNonNull(initialSupplier, "initialSupplier"); + Objects.requireNonNull(reducer, "reducer"); return RxJavaPlugins.onAssembly(new ParallelReduce(this, initialSupplier, reducer)); } @@ -449,7 +449,7 @@ public final Flowable sorted(@NonNull Comparator comparator) { @CheckReturnValue @NonNull public final Flowable sorted(@NonNull Comparator comparator, int capacityHint) { - ObjectHelper.requireNonNull(comparator, "comparator is null"); + Objects.requireNonNull(comparator, "comparator is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); int ch = capacityHint / parallelism() + 1; ParallelFlowable> railReduced = reduce(Functions.createArrayList(ch), ListAddBiConsumer.instance()); @@ -483,7 +483,7 @@ public final Flowable> toSortedList(@NonNull Comparator compa @CheckReturnValue @NonNull public final Flowable> toSortedList(@NonNull Comparator comparator, int capacityHint) { - ObjectHelper.requireNonNull(comparator, "comparator is null"); + Objects.requireNonNull(comparator, "comparator is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); int ch = capacityHint / parallelism() + 1; @@ -504,7 +504,7 @@ public final Flowable> toSortedList(@NonNull Comparator compa @CheckReturnValue @NonNull public final ParallelFlowable doOnNext(@NonNull Consumer onNext) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(onNext, "onNext is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, onNext, Functions.emptyConsumer(), @@ -530,8 +530,8 @@ public final ParallelFlowable doOnNext(@NonNull Consumer onNext) { @CheckReturnValue @NonNull public final ParallelFlowable doOnNext(@NonNull Consumer onNext, @NonNull ParallelFailureHandling errorHandler) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(errorHandler, "errorHandler is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); return RxJavaPlugins.onAssembly(new ParallelDoOnNextTry(this, onNext, errorHandler)); } @@ -549,8 +549,8 @@ public final ParallelFlowable doOnNext(@NonNull Consumer onNext, @ @CheckReturnValue @NonNull public final ParallelFlowable doOnNext(@NonNull Consumer onNext, @NonNull BiFunction errorHandler) { - ObjectHelper.requireNonNull(onNext, "onNext is null"); - ObjectHelper.requireNonNull(errorHandler, "errorHandler is null"); + Objects.requireNonNull(onNext, "onNext is null"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); return RxJavaPlugins.onAssembly(new ParallelDoOnNextTry(this, onNext, errorHandler)); } @@ -564,7 +564,7 @@ public final ParallelFlowable doOnNext(@NonNull Consumer onNext, @ @CheckReturnValue @NonNull public final ParallelFlowable doAfterNext(@NonNull Consumer onAfterNext) { - ObjectHelper.requireNonNull(onAfterNext, "onAfterNext is null"); + Objects.requireNonNull(onAfterNext, "onAfterNext is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, Functions.emptyConsumer(), onAfterNext, @@ -586,7 +586,7 @@ public final ParallelFlowable doAfterNext(@NonNull Consumer onAfte @CheckReturnValue @NonNull public final ParallelFlowable doOnError(@NonNull Consumer onError) { - ObjectHelper.requireNonNull(onError, "onError is null"); + Objects.requireNonNull(onError, "onError is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -608,7 +608,7 @@ public final ParallelFlowable doOnError(@NonNull Consumer onError) @CheckReturnValue @NonNull public final ParallelFlowable doOnComplete(@NonNull Action onComplete) { - ObjectHelper.requireNonNull(onComplete, "onComplete is null"); + Objects.requireNonNull(onComplete, "onComplete is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -630,7 +630,7 @@ public final ParallelFlowable doOnComplete(@NonNull Action onComplete) { @CheckReturnValue @NonNull public final ParallelFlowable doAfterTerminated(@NonNull Action onAfterTerminate) { - ObjectHelper.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); + Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -652,7 +652,7 @@ public final ParallelFlowable doAfterTerminated(@NonNull Action onAfterTermin @CheckReturnValue @NonNull public final ParallelFlowable doOnSubscribe(@NonNull Consumer onSubscribe) { - ObjectHelper.requireNonNull(onSubscribe, "onSubscribe is null"); + Objects.requireNonNull(onSubscribe, "onSubscribe is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -674,7 +674,7 @@ public final ParallelFlowable doOnSubscribe(@NonNull Consumer doOnRequest(@NonNull LongConsumer onRequest) { - ObjectHelper.requireNonNull(onRequest, "onRequest is null"); + Objects.requireNonNull(onRequest, "onRequest is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -696,7 +696,7 @@ public final ParallelFlowable doOnRequest(@NonNull LongConsumer onRequest) { @CheckReturnValue @NonNull public final ParallelFlowable doOnCancel(@NonNull Action onCancel) { - ObjectHelper.requireNonNull(onCancel, "onCancel is null"); + Objects.requireNonNull(onCancel, "onCancel is null"); return RxJavaPlugins.onAssembly(new ParallelPeek(this, Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -721,8 +721,8 @@ public final ParallelFlowable doOnCancel(@NonNull Action onCancel) { @CheckReturnValue @NonNull public final ParallelFlowable collect(@NonNull Supplier collectionSupplier, @NonNull BiConsumer collector) { - ObjectHelper.requireNonNull(collectionSupplier, "collectionSupplier is null"); - ObjectHelper.requireNonNull(collector, "collector is null"); + Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); + Objects.requireNonNull(collector, "collector is null"); return RxJavaPlugins.onAssembly(new ParallelCollect(this, collectionSupplier, collector)); } @@ -757,7 +757,7 @@ public static ParallelFlowable fromArray(@NonNull Publisher... publish @CheckReturnValue @NonNull public final R to(@NonNull ParallelFlowableConverter converter) { - return ObjectHelper.requireNonNull(converter, "converter is null").apply(this); + return Objects.requireNonNull(converter, "converter is null").apply(this); } /** @@ -771,7 +771,7 @@ public final R to(@NonNull ParallelFlowableConverter converter) { @CheckReturnValue @NonNull public final ParallelFlowable compose(@NonNull ParallelTransformer composer) { - return RxJavaPlugins.onAssembly(ObjectHelper.requireNonNull(composer, "composer is null").apply(this)); + return RxJavaPlugins.onAssembly(Objects.requireNonNull(composer, "composer is null").apply(this)); } /** @@ -842,7 +842,7 @@ public final ParallelFlowable flatMap( public final ParallelFlowable flatMap( @NonNull Function> mapper, boolean delayError, int maxConcurrency, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ParallelFlatMap(this, mapper, delayError, maxConcurrency, prefetch)); @@ -879,7 +879,7 @@ public final ParallelFlowable concatMap( public final ParallelFlowable concatMap( @NonNull Function> mapper, int prefetch) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ParallelConcatMap(this, mapper, prefetch, ErrorMode.IMMEDIATE)); } @@ -919,7 +919,7 @@ public final ParallelFlowable concatMapDelayError( public final ParallelFlowable concatMapDelayError( @NonNull Function> mapper, int prefetch, boolean tillTheEnd) { - ObjectHelper.requireNonNull(mapper, "mapper is null"); + Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ParallelConcatMap( this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); diff --git a/src/main/java/io/reactivex/rxjava3/plugins/RxJavaPlugins.java b/src/main/java/io/reactivex/rxjava3/plugins/RxJavaPlugins.java index 976289c542..9012e36f12 100644 --- a/src/main/java/io/reactivex/rxjava3/plugins/RxJavaPlugins.java +++ b/src/main/java/io/reactivex/rxjava3/plugins/RxJavaPlugins.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.plugins; import java.lang.Thread.UncaughtExceptionHandler; +import java.util.Objects; import java.util.concurrent.ThreadFactory; import org.reactivestreams.Subscriber; @@ -22,7 +23,6 @@ import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.flowables.ConnectableFlowable; import io.reactivex.rxjava3.functions.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.schedulers.*; import io.reactivex.rxjava3.internal.util.ExceptionHelper; import io.reactivex.rxjava3.observables.ConnectableObservable; @@ -266,7 +266,7 @@ public static Consumer getErrorHandler() { */ @NonNull public static Scheduler initComputationScheduler(@NonNull Supplier defaultScheduler) { - ObjectHelper.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); + Objects.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); Function, ? extends Scheduler> f = onInitComputationHandler; if (f == null) { return callRequireNonNull(defaultScheduler); @@ -282,7 +282,7 @@ public static Scheduler initComputationScheduler(@NonNull Supplier de */ @NonNull public static Scheduler initIoScheduler(@NonNull Supplier defaultScheduler) { - ObjectHelper.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); + Objects.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); Function, ? extends Scheduler> f = onInitIoHandler; if (f == null) { return callRequireNonNull(defaultScheduler); @@ -298,7 +298,7 @@ public static Scheduler initIoScheduler(@NonNull Supplier defaultSche */ @NonNull public static Scheduler initNewThreadScheduler(@NonNull Supplier defaultScheduler) { - ObjectHelper.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); + Objects.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); Function, ? extends Scheduler> f = onInitNewThreadHandler; if (f == null) { return callRequireNonNull(defaultScheduler); @@ -314,7 +314,7 @@ public static Scheduler initNewThreadScheduler(@NonNull Supplier defa */ @NonNull public static Scheduler initSingleScheduler(@NonNull Supplier defaultScheduler) { - ObjectHelper.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); + Objects.requireNonNull(defaultScheduler, "Scheduler Supplier can't be null"); Function, ? extends Scheduler> f = onInitSingleHandler; if (f == null) { return callRequireNonNull(defaultScheduler); @@ -464,7 +464,7 @@ public static Scheduler onNewThreadScheduler(@NonNull Scheduler defaultScheduler */ @NonNull public static Runnable onSchedule(@NonNull Runnable run) { - ObjectHelper.requireNonNull(run, "run is null"); + Objects.requireNonNull(run, "run is null"); Function f = onScheduleHandler; if (f == null) { @@ -1208,7 +1208,7 @@ public static BooleanSupplier getOnBeforeBlocking() { */ @NonNull public static Scheduler createComputationScheduler(@NonNull ThreadFactory threadFactory) { - return new ComputationScheduler(ObjectHelper.requireNonNull(threadFactory, "threadFactory is null")); + return new ComputationScheduler(Objects.requireNonNull(threadFactory, "threadFactory is null")); } /** @@ -1222,7 +1222,7 @@ public static Scheduler createComputationScheduler(@NonNull ThreadFactory thread */ @NonNull public static Scheduler createIoScheduler(@NonNull ThreadFactory threadFactory) { - return new IoScheduler(ObjectHelper.requireNonNull(threadFactory, "threadFactory is null")); + return new IoScheduler(Objects.requireNonNull(threadFactory, "threadFactory is null")); } /** @@ -1236,7 +1236,7 @@ public static Scheduler createIoScheduler(@NonNull ThreadFactory threadFactory) */ @NonNull public static Scheduler createNewThreadScheduler(@NonNull ThreadFactory threadFactory) { - return new NewThreadScheduler(ObjectHelper.requireNonNull(threadFactory, "threadFactory is null")); + return new NewThreadScheduler(Objects.requireNonNull(threadFactory, "threadFactory is null")); } /** @@ -1250,7 +1250,7 @@ public static Scheduler createNewThreadScheduler(@NonNull ThreadFactory threadFa */ @NonNull public static Scheduler createSingleScheduler(@NonNull ThreadFactory threadFactory) { - return new SingleScheduler(ObjectHelper.requireNonNull(threadFactory, "threadFactory is null")); + return new SingleScheduler(Objects.requireNonNull(threadFactory, "threadFactory is null")); } /** @@ -1301,7 +1301,7 @@ static R apply(@NonNull BiFunction f, @NonNull T t, @NonNull @NonNull static Scheduler callRequireNonNull(@NonNull Supplier s) { try { - return ObjectHelper.requireNonNull(s.get(), "Scheduler Supplier result can't be null"); + return Objects.requireNonNull(s.get(), "Scheduler Supplier result can't be null"); } catch (Throwable ex) { throw ExceptionHelper.wrapOrThrow(ex); } @@ -1317,7 +1317,7 @@ static Scheduler callRequireNonNull(@NonNull Supplier s) { */ @NonNull static Scheduler applyRequireNonNull(@NonNull Function, ? extends Scheduler> f, Supplier s) { - return ObjectHelper.requireNonNull(apply(f, s), "Scheduler Supplier result can't be null"); + return Objects.requireNonNull(apply(f, s), "Scheduler Supplier result can't be null"); } /** Helper class, no instances. */ diff --git a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java index 5ac1d1d607..eaf47ecae3 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.processors; +import java.util.Objects; import java.util.concurrent.atomic.*; import java.util.concurrent.locks.*; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.annotations.*; import io.reactivex.rxjava3.exceptions.MissingBackpressureException; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.internal.util.AppendOnlyLinkedArrayList.NonThrowingPredicate; @@ -208,7 +208,7 @@ public static BehaviorProcessor create() { @CheckReturnValue @NonNull public static BehaviorProcessor createDefault(T defaultValue) { - ObjectHelper.requireNonNull(defaultValue, "defaultValue is null"); + Objects.requireNonNull(defaultValue, "defaultValue is null"); return new BehaviorProcessor(defaultValue); } @@ -234,7 +234,7 @@ public static BehaviorProcessor createDefault(T defaultValue) { */ BehaviorProcessor(T defaultValue) { this(); - this.value.lazySet(ObjectHelper.requireNonNull(defaultValue, "defaultValue is null")); + this.value.lazySet(Objects.requireNonNull(defaultValue, "defaultValue is null")); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java index 1ab964a6dd..fcd2cb913f 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java @@ -1052,8 +1052,8 @@ static final class SizeAndTimeBoundReplayBuffer SizeAndTimeBoundReplayBuffer(int maxSize, long maxAge, TimeUnit unit, Scheduler scheduler) { this.maxSize = ObjectHelper.verifyPositive(maxSize, "maxSize"); this.maxAge = ObjectHelper.verifyPositive(maxAge, "maxAge"); - this.unit = ObjectHelper.requireNonNull(unit, "unit is null"); - this.scheduler = ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + this.unit = Objects.requireNonNull(unit, "unit is null"); + this.scheduler = Objects.requireNonNull(scheduler, "scheduler is null"); TimedNode h = new TimedNode(null, 0L); this.tail = h; this.head = h; diff --git a/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java index a3715103d5..4a1862a95f 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.processors; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -222,7 +223,7 @@ public static UnicastProcessor create(boolean delayError) { @CheckReturnValue @NonNull public static UnicastProcessor create(int capacityHint, Runnable onCancelled) { - ObjectHelper.requireNonNull(onCancelled, "onTerminate"); + Objects.requireNonNull(onCancelled, "onTerminate"); return new UnicastProcessor(capacityHint, onCancelled); } @@ -243,7 +244,7 @@ public static UnicastProcessor create(int capacityHint, Runnable onCancel @CheckReturnValue @NonNull public static UnicastProcessor create(int capacityHint, Runnable onCancelled, boolean delayError) { - ObjectHelper.requireNonNull(onCancelled, "onTerminate"); + Objects.requireNonNull(onCancelled, "onTerminate"); return new UnicastProcessor(capacityHint, onCancelled, delayError); } diff --git a/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java b/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java index fff21ad682..845273ff22 100644 --- a/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java +++ b/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java @@ -17,7 +17,6 @@ import java.util.concurrent.TimeUnit; import io.reactivex.rxjava3.annotations.NonNull; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; /** * Holds onto a value along with time information. @@ -39,7 +38,7 @@ public final class Timed { public Timed(@NonNull T value, long time, @NonNull TimeUnit unit) { this.value = value; this.time = time; - this.unit = ObjectHelper.requireNonNull(unit, "unit is null"); + this.unit = Objects.requireNonNull(unit, "unit is null"); } /** diff --git a/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java index 45ddb72a51..68efd5bf8f 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java @@ -13,13 +13,13 @@ package io.reactivex.rxjava3.subjects; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.*; import io.reactivex.rxjava3.annotations.*; import io.reactivex.rxjava3.core.Observer; import io.reactivex.rxjava3.disposables.Disposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.internal.util.AppendOnlyLinkedArrayList.NonThrowingPredicate; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -218,7 +218,7 @@ public static BehaviorSubject createDefault(T defaultValue) { */ BehaviorSubject(T defaultValue) { this(); - this.value.lazySet(ObjectHelper.requireNonNull(defaultValue, "defaultValue is null")); + this.value.lazySet(Objects.requireNonNull(defaultValue, "defaultValue is null")); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java b/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java index 2d9dbe779a..aad2092429 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java @@ -1053,8 +1053,8 @@ static final class SizeAndTimeBoundReplayBuffer SizeAndTimeBoundReplayBuffer(int maxSize, long maxAge, TimeUnit unit, Scheduler scheduler) { this.maxSize = ObjectHelper.verifyPositive(maxSize, "maxSize"); this.maxAge = ObjectHelper.verifyPositive(maxAge, "maxAge"); - this.unit = ObjectHelper.requireNonNull(unit, "unit is null"); - this.scheduler = ObjectHelper.requireNonNull(scheduler, "scheduler is null"); + this.unit = Objects.requireNonNull(unit, "unit is null"); + this.scheduler = Objects.requireNonNull(scheduler, "scheduler is null"); TimedNode h = new TimedNode(null, 0L); this.tail = h; this.head = h; diff --git a/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java index be824a7708..c522343aed 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.subjects; +import java.util.Objects; import java.util.concurrent.atomic.*; import io.reactivex.rxjava3.annotations.*; @@ -288,7 +289,7 @@ public static UnicastSubject create(boolean delayError) { */ UnicastSubject(int capacityHint, Runnable onTerminate, boolean delayError) { this.queue = new SpscLinkedArrayQueue(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); - this.onTerminate = new AtomicReference(ObjectHelper.requireNonNull(onTerminate, "onTerminate")); + this.onTerminate = new AtomicReference(Objects.requireNonNull(onTerminate, "onTerminate")); this.delayError = delayError; this.downstream = new AtomicReference>(); this.once = new AtomicBoolean(); diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java b/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java index 71bdeb0121..1990939311 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java @@ -13,6 +13,7 @@ package io.reactivex.rxjava3.subscribers; +import java.util.Objects; import java.util.concurrent.atomic.*; import org.reactivestreams.Subscription; @@ -20,7 +21,6 @@ import io.reactivex.rxjava3.core.FlowableSubscriber; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.ListCompositeDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.EndConsumerHelper; @@ -110,7 +110,7 @@ public abstract class ResourceSubscriber implements FlowableSubscriber, Di * @throws NullPointerException if resource is null */ public final void add(Disposable resource) { - ObjectHelper.requireNonNull(resource, "resource is null"); + Objects.requireNonNull(resource, "resource is null"); resources.add(resource); } From 0fba7c5530b092d13d29681298545ad98d22249a Mon Sep 17 00:00:00 2001 From: kerr Date: Wed, 18 Dec 2019 07:06:38 +0800 Subject: [PATCH 008/665] Add FunctionalInterface annotations. (#6773) --- .../java/io/reactivex/rxjava3/core/CompletableConverter.java | 1 + .../java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java | 1 + src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java | 1 + src/main/java/io/reactivex/rxjava3/core/CompletableSource.java | 1 + .../java/io/reactivex/rxjava3/core/CompletableTransformer.java | 1 + src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java | 1 + src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java | 1 + src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java | 1 + src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java | 1 + src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java | 1 + src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java | 1 + src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java | 1 + src/main/java/io/reactivex/rxjava3/core/MaybeSource.java | 1 + src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java | 1 + src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java | 1 + .../java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java | 1 + src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java | 1 + src/main/java/io/reactivex/rxjava3/core/ObservableSource.java | 1 + .../java/io/reactivex/rxjava3/core/ObservableTransformer.java | 1 + src/main/java/io/reactivex/rxjava3/core/SingleConverter.java | 1 + src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java | 1 + src/main/java/io/reactivex/rxjava3/core/SingleOperator.java | 1 + src/main/java/io/reactivex/rxjava3/core/SingleSource.java | 1 + src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Action.java | 1 + src/main/java/io/reactivex/rxjava3/functions/BiConsumer.java | 1 + src/main/java/io/reactivex/rxjava3/functions/BiFunction.java | 1 + src/main/java/io/reactivex/rxjava3/functions/BiPredicate.java | 1 + .../java/io/reactivex/rxjava3/functions/BooleanSupplier.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Cancellable.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Consumer.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function3.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function4.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function5.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function6.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function7.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function8.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Function9.java | 1 + src/main/java/io/reactivex/rxjava3/functions/IntFunction.java | 1 + src/main/java/io/reactivex/rxjava3/functions/LongConsumer.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Predicate.java | 1 + src/main/java/io/reactivex/rxjava3/functions/Supplier.java | 1 + .../io/reactivex/rxjava3/internal/fuseable/ScalarSupplier.java | 1 + .../io/reactivex/rxjava3/parallel/ParallelFlowableConverter.java | 1 + .../java/io/reactivex/rxjava3/parallel/ParallelTransformer.java | 1 + 46 files changed, 46 insertions(+) diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableConverter.java b/src/main/java/io/reactivex/rxjava3/core/CompletableConverter.java index 6ef04e2608..bcc00f088b 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableConverter.java @@ -22,6 +22,7 @@ * @param the output type * @since 2.2 */ +@FunctionalInterface public interface CompletableConverter { /** * Applies a function to the upstream Completable and returns a converted value of type {@code R}. diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java index 8dc2531997..6aea056be7 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java @@ -19,6 +19,7 @@ * an instance of a {@link CompletableEmitter} instance that allows pushing * an event in a cancellation-safe manner. */ +@FunctionalInterface public interface CompletableOnSubscribe { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java b/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java index 3a45c9085f..763cd670c6 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java @@ -18,6 +18,7 @@ /** * Interface to map/wrap a downstream observer to an upstream observer. */ +@FunctionalInterface public interface CompletableOperator { /** * Applies a function to the child CompletableObserver and returns a new parent CompletableObserver. diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java b/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java index 390be60b2e..9b754a698a 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java @@ -20,6 +20,7 @@ * * @since 2.0 */ +@FunctionalInterface public interface CompletableSource { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java b/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java index b6acc252b0..91e1a361ba 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java @@ -19,6 +19,7 @@ * Convenience interface and callback used by the compose operator to turn a Completable into another * Completable fluently. */ +@FunctionalInterface public interface CompletableTransformer { /** * Applies a function to the upstream Completable and returns a CompletableSource. diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java b/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java index 0736dc189d..abc89eec4a 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java @@ -23,6 +23,7 @@ * @param the output type * @since 2.2 */ +@FunctionalInterface public interface FlowableConverter { /** * Applies a function to the upstream Flowable and returns a converted value of type {@code R}. diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java index 93d125de2f..2bbc8b57a3 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java @@ -21,6 +21,7 @@ * * @param the value type pushed */ +@FunctionalInterface public interface FlowableOnSubscribe { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java b/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java index 8ce0ab8f7d..6d241f7498 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java @@ -23,6 +23,7 @@ * @param the value type of the downstream * @param the value type of the upstream */ +@FunctionalInterface public interface FlowableOperator { /** * Applies a function to the child Subscriber and returns a new parent Subscriber. diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java b/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java index ecee751040..877ad1916a 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java @@ -23,6 +23,7 @@ * @param the upstream value type * @param the downstream value type */ +@FunctionalInterface public interface FlowableTransformer { /** * Applies a function to the upstream Flowable and returns a Publisher with diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java b/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java index de70b81814..a8e6790e5f 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java @@ -23,6 +23,7 @@ * @param the output type * @since 2.2 */ +@FunctionalInterface public interface MaybeConverter { /** * Applies a function to the upstream Maybe and returns a converted value of type {@code R}. diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java index dbccf870f9..2dde118757 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java @@ -21,6 +21,7 @@ * * @param the value type pushed */ +@FunctionalInterface public interface MaybeOnSubscribe { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java b/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java index b3173eb833..48039a9bd0 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java @@ -20,6 +20,7 @@ * @param the value type of the downstream * @param the value type of the upstream */ +@FunctionalInterface public interface MaybeOperator { /** * Applies a function to the child MaybeObserver and returns a new parent MaybeObserver. diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java b/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java index dd037cc864..17e2b7f694 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java @@ -24,6 +24,7 @@ * @param the element type * @since 2.0 */ +@FunctionalInterface public interface MaybeSource { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java b/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java index d53bfda71d..b5a94bce79 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java @@ -21,6 +21,7 @@ * @param the upstream value type * @param the downstream value type */ +@FunctionalInterface public interface MaybeTransformer { /** * Applies a function to the upstream Maybe and returns a MaybeSource with diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java b/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java index 744c0ef48e..40b432cfea 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java @@ -23,6 +23,7 @@ * @param the output type * @since 2.2 */ +@FunctionalInterface public interface ObservableConverter { /** * Applies a function to the upstream Observable and returns a converted value of type {@code R}. diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java index 1f98911285..43d6252849 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java @@ -21,6 +21,7 @@ * * @param the value type pushed */ +@FunctionalInterface public interface ObservableOnSubscribe { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java b/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java index f7335b6221..76ec06e295 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java @@ -21,6 +21,7 @@ * @param the value type of the downstream * @param the value type of the upstream */ +@FunctionalInterface public interface ObservableOperator { /** * Applies a function to the child Observer and returns a new parent Observer. diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java b/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java index a48011e438..ad5d5c79c3 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java @@ -21,6 +21,7 @@ * @param the element type * @since 2.0 */ +@FunctionalInterface public interface ObservableSource { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java b/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java index 63dc87ca8b..259deefa85 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java @@ -21,6 +21,7 @@ * @param the upstream value type * @param the downstream value type */ +@FunctionalInterface public interface ObservableTransformer { /** * Applies a function to the upstream Observable and returns an ObservableSource with diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java b/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java index 5309dbc160..4adc22b8c5 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java @@ -23,6 +23,7 @@ * @param the output type * @since 2.2 */ +@FunctionalInterface public interface SingleConverter { /** * Applies a function to the upstream Single and returns a converted value of type {@code R}. diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java index 157a8e42dc..46cc6a6f56 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java @@ -21,6 +21,7 @@ * * @param the value type pushed */ +@FunctionalInterface public interface SingleOnSubscribe { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java b/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java index 58ea9d338e..2fc2d80ab6 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java @@ -21,6 +21,7 @@ * @param the value type of the downstream * @param the value type of the upstream */ +@FunctionalInterface public interface SingleOperator { /** * Applies a function to the child SingleObserver and returns a new parent SingleObserver. diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleSource.java b/src/main/java/io/reactivex/rxjava3/core/SingleSource.java index 1a3780c3f6..496462d0e1 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleSource.java @@ -24,6 +24,7 @@ * @param the element type * @since 2.0 */ +@FunctionalInterface public interface SingleSource { /** diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java b/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java index 080a2d7af1..08f9408cdb 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java @@ -21,6 +21,7 @@ * @param the upstream value type * @param the downstream value type */ +@FunctionalInterface public interface SingleTransformer { /** * Applies a function to the upstream Single and returns a SingleSource with diff --git a/src/main/java/io/reactivex/rxjava3/functions/Action.java b/src/main/java/io/reactivex/rxjava3/functions/Action.java index 153eff9375..da0e76e0a6 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Action.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Action.java @@ -16,6 +16,7 @@ /** * A functional interface similar to Runnable but allows throwing a checked exception. */ +@FunctionalInterface public interface Action { /** * Runs the action and optionally throws a checked exception. diff --git a/src/main/java/io/reactivex/rxjava3/functions/BiConsumer.java b/src/main/java/io/reactivex/rxjava3/functions/BiConsumer.java index c060c38331..7fa60ef4c7 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/BiConsumer.java +++ b/src/main/java/io/reactivex/rxjava3/functions/BiConsumer.java @@ -18,6 +18,7 @@ * @param the first value type * @param the second value type */ +@FunctionalInterface public interface BiConsumer { /** diff --git a/src/main/java/io/reactivex/rxjava3/functions/BiFunction.java b/src/main/java/io/reactivex/rxjava3/functions/BiFunction.java index f7e1174c6e..174cc3c6d9 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/BiFunction.java +++ b/src/main/java/io/reactivex/rxjava3/functions/BiFunction.java @@ -21,6 +21,7 @@ * @param the second value type * @param the result type */ +@FunctionalInterface public interface BiFunction { /** diff --git a/src/main/java/io/reactivex/rxjava3/functions/BiPredicate.java b/src/main/java/io/reactivex/rxjava3/functions/BiPredicate.java index 4793e1d086..fc41682ef5 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/BiPredicate.java +++ b/src/main/java/io/reactivex/rxjava3/functions/BiPredicate.java @@ -20,6 +20,7 @@ * @param the first value * @param the second value */ +@FunctionalInterface public interface BiPredicate { /** diff --git a/src/main/java/io/reactivex/rxjava3/functions/BooleanSupplier.java b/src/main/java/io/reactivex/rxjava3/functions/BooleanSupplier.java index dc801e5180..b30087fbd7 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/BooleanSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/functions/BooleanSupplier.java @@ -16,6 +16,7 @@ /** * A functional interface (callback) that returns a boolean value. */ +@FunctionalInterface public interface BooleanSupplier { /** * Returns a boolean value. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Cancellable.java b/src/main/java/io/reactivex/rxjava3/functions/Cancellable.java index 47683eafdc..b9f3420c1b 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Cancellable.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Cancellable.java @@ -17,6 +17,7 @@ * A functional interface that has a single cancel method * that can throw. */ +@FunctionalInterface public interface Cancellable { /** diff --git a/src/main/java/io/reactivex/rxjava3/functions/Consumer.java b/src/main/java/io/reactivex/rxjava3/functions/Consumer.java index 08d94eb928..788622c170 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Consumer.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Consumer.java @@ -17,6 +17,7 @@ * A functional interface (callback) that accepts a single value. * @param the value type */ +@FunctionalInterface public interface Consumer { /** * Consume the given value. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function.java b/src/main/java/io/reactivex/rxjava3/functions/Function.java index 23d06b041b..d81a2dec52 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function.java @@ -22,6 +22,7 @@ * @param the input value type * @param the output value type */ +@FunctionalInterface public interface Function { /** * Apply some calculation to the input value and return some other value. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function3.java b/src/main/java/io/reactivex/rxjava3/functions/Function3.java index bdc1b4af2b..cdc65112e7 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function3.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function3.java @@ -22,6 +22,7 @@ * @param the third value type * @param the result type */ +@FunctionalInterface public interface Function3 { /** * Calculate a value based on the input values. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function4.java b/src/main/java/io/reactivex/rxjava3/functions/Function4.java index 907f742907..84c3ab1449 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function4.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function4.java @@ -23,6 +23,7 @@ * @param the fourth value type * @param the result type */ +@FunctionalInterface public interface Function4 { /** * Calculate a value based on the input values. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function5.java b/src/main/java/io/reactivex/rxjava3/functions/Function5.java index a8468844b7..77f2ff7220 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function5.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function5.java @@ -24,6 +24,7 @@ * @param the fifth value type * @param the result type */ +@FunctionalInterface public interface Function5 { /** * Calculate a value based on the input values. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function6.java b/src/main/java/io/reactivex/rxjava3/functions/Function6.java index 6bc5caec1f..50c25f3df9 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function6.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function6.java @@ -25,6 +25,7 @@ * @param the sixth value type * @param the result type */ +@FunctionalInterface public interface Function6 { /** * Calculate a value based on the input values. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function7.java b/src/main/java/io/reactivex/rxjava3/functions/Function7.java index 32d5a08e41..613800a4c1 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function7.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function7.java @@ -26,6 +26,7 @@ * @param the seventh value type * @param the result type */ +@FunctionalInterface public interface Function7 { /** * Calculate a value based on the input values. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function8.java b/src/main/java/io/reactivex/rxjava3/functions/Function8.java index 8a7a88f9e8..2fda54ea40 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function8.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function8.java @@ -27,6 +27,7 @@ * @param the eighth value type * @param the result type */ +@FunctionalInterface public interface Function8 { /** * Calculate a value based on the input values. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Function9.java b/src/main/java/io/reactivex/rxjava3/functions/Function9.java index 28319eac14..3eb0779e30 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Function9.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Function9.java @@ -28,6 +28,7 @@ * @param the ninth value type * @param the result type */ +@FunctionalInterface public interface Function9 { /** * Calculate a value based on the input values. diff --git a/src/main/java/io/reactivex/rxjava3/functions/IntFunction.java b/src/main/java/io/reactivex/rxjava3/functions/IntFunction.java index 8a135ad491..e01cc13ab5 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/IntFunction.java +++ b/src/main/java/io/reactivex/rxjava3/functions/IntFunction.java @@ -18,6 +18,7 @@ * A functional interface (callback) that takes a primitive value and return value of type T. * @param the returned value type */ +@FunctionalInterface public interface IntFunction { /** * Calculates a value based on a primitive integer input. diff --git a/src/main/java/io/reactivex/rxjava3/functions/LongConsumer.java b/src/main/java/io/reactivex/rxjava3/functions/LongConsumer.java index 6b5b72101c..9af7de4f47 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/LongConsumer.java +++ b/src/main/java/io/reactivex/rxjava3/functions/LongConsumer.java @@ -15,6 +15,7 @@ /** * A functional interface (callback) that consumes a primitive long value. */ +@FunctionalInterface public interface LongConsumer { /** * Consume a primitive long input. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Predicate.java b/src/main/java/io/reactivex/rxjava3/functions/Predicate.java index 3197114158..99450f865b 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Predicate.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Predicate.java @@ -19,6 +19,7 @@ * A functional interface (callback) that returns true or false for the given input value. * @param the first value */ +@FunctionalInterface public interface Predicate { /** * Test the given input value and return a boolean. diff --git a/src/main/java/io/reactivex/rxjava3/functions/Supplier.java b/src/main/java/io/reactivex/rxjava3/functions/Supplier.java index 95ef2e88a1..af5052ab83 100644 --- a/src/main/java/io/reactivex/rxjava3/functions/Supplier.java +++ b/src/main/java/io/reactivex/rxjava3/functions/Supplier.java @@ -22,6 +22,7 @@ * @param the value type returned * @since 3.0.0 */ +@FunctionalInterface public interface Supplier { /** diff --git a/src/main/java/io/reactivex/rxjava3/internal/fuseable/ScalarSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/fuseable/ScalarSupplier.java index 039d730be6..fc0d12323f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/fuseable/ScalarSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/fuseable/ScalarSupplier.java @@ -29,6 +29,7 @@ *

* @param the scalar value type held by the implementing reactive type */ +@FunctionalInterface public interface ScalarSupplier extends Supplier { // overridden to remove the throws Throwable diff --git a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowableConverter.java b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowableConverter.java index ea2d07502a..7033261e65 100644 --- a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowableConverter.java +++ b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowableConverter.java @@ -23,6 +23,7 @@ * @param the output type * @since 2.2 */ +@FunctionalInterface public interface ParallelFlowableConverter { /** * Applies a function to the upstream ParallelFlowable and returns a converted value of type {@code R}. diff --git a/src/main/java/io/reactivex/rxjava3/parallel/ParallelTransformer.java b/src/main/java/io/reactivex/rxjava3/parallel/ParallelTransformer.java index 1a918f0835..7e6a7734d0 100644 --- a/src/main/java/io/reactivex/rxjava3/parallel/ParallelTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/parallel/ParallelTransformer.java @@ -22,6 +22,7 @@ * @param the downstream value type * @since 2.2 */ +@FunctionalInterface public interface ParallelTransformer { /** * Applies a function to the upstream ParallelFlowable and returns a ParallelFlowable with From 2f2cef5b2eae258dce76675be3fd83f1f0c322b6 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Wed, 18 Dec 2019 16:59:38 +0100 Subject: [PATCH 009/665] 3.x: Javadocs package-info made nicer, update README regarding Java 8 (#6778) --- README.md | 23 ++++------ .../rxjava3/annotations/Nullable.java | 4 +- .../rxjava3/annotations/package-info.java | 4 +- .../rxjava3/disposables/package-info.java | 6 +-- .../rxjava3/exceptions/package-info.java | 7 +++- .../rxjava3/observers/package-info.java | 42 +++++++++++++++++-- .../rxjava3/subjects/package-info.java | 2 +- .../rxjava3/subscribers/package-info.java | 9 ++-- 8 files changed, 64 insertions(+), 33 deletions(-) diff --git a/README.md b/README.md index 7c120db700..698aa96f40 100644 --- a/README.md +++ b/README.md @@ -11,10 +11,10 @@ It extends the [observer pattern](http://en.wikipedia.org/wiki/Observer_pattern) #### Version 3.x ([Javadoc](http://reactivex.io/RxJava/3.x/javadoc/)) - single dependency: [Reactive-Streams](https://github.com/reactive-streams/reactive-streams-jvm) -- continued support for Java 6+ & [Android](https://github.com/ReactiveX/RxAndroid) 2.3+ +- Java 8+ ([Android](https://github.com/ReactiveX/RxAndroid) desugar friendly) +- Java 8 lambda-friendly API - fixed API mistakes and many limits of RxJava 2 - intended to be a replacement for RxJava 2 with relatively few binary incompatible changes -- Java 8 lambda-friendly API - non-opinionated about the source of concurrency (threads, pools, event loops, fibers, actors, etc.) - async or synchronous execution - virtual time and schedulers for parameterized concurrency @@ -60,19 +60,6 @@ public class HelloWorld { } ``` -If your platform doesn't support Java 8 lambdas (yet), you have to create an inner class of `Consumer` manually: - -```java -import io.reactivex.rxjava3.functions.Consumer; - -Flowable.just("Hello world") - .subscribe(new Consumer() { - @Override public void accept(String s) { - System.out.println(s); - } - }); -``` - Note that RxJava 3 components now live under `io.reactivex.rxjava3` and the base classes and interfaces live under `io.reactivex.rxjava3.core`. ### Base classes @@ -557,7 +544,7 @@ Binaries and dependency information for Maven, Ivy, Gradle and others can be fou Example for Gradle: ```groovy -compile 'io.reactivex.rxjava3:rxjava:x.y.z' +implementation 'io.reactivex.rxjava3:rxjava:x.y.z' ``` and for Maven: @@ -575,6 +562,8 @@ and for Ivy: ``` +### Snapshots + Snapshots are available via https://oss.jfrog.org/libs-snapshot/io/reactivex/rxjava3/rxjava/ ```groovy @@ -587,6 +576,8 @@ dependencies { } ``` +JavaDoc snapshots are available at http://reactivex.io/RxJava/3.x/javadoc/snapshot + ## Build To build: diff --git a/src/main/java/io/reactivex/rxjava3/annotations/Nullable.java b/src/main/java/io/reactivex/rxjava3/annotations/Nullable.java index c94cef974b..c95ef4a35c 100644 --- a/src/main/java/io/reactivex/rxjava3/annotations/Nullable.java +++ b/src/main/java/io/reactivex/rxjava3/annotations/Nullable.java @@ -19,10 +19,10 @@ import java.lang.annotation.*; /** - * Indicates that a field/parameter/variable/return type may be null. + * Indicates that a field/parameter/variable/type parameter/return type may be null. */ @Documented -@Target(value = {FIELD, METHOD, PARAMETER, LOCAL_VARIABLE}) +@Target(value = {FIELD, METHOD, PARAMETER, LOCAL_VARIABLE, TYPE_PARAMETER, TYPE_USE}) @Retention(value = CLASS) public @interface Nullable { } diff --git a/src/main/java/io/reactivex/rxjava3/annotations/package-info.java b/src/main/java/io/reactivex/rxjava3/annotations/package-info.java index 1f84d214c7..8fa44fd553 100644 --- a/src/main/java/io/reactivex/rxjava3/annotations/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/annotations/package-info.java @@ -15,6 +15,8 @@ */ /** - * Annotations for indicating experimental and beta operators, classes, methods, types or fields. + * Annotations for indicating operator behavior, API stability + * ({@link io.reactivex.rxjava3.annotations.Experimental @Experimental} and {@link io.reactivex.rxjava3.annotations.Beta @Beta}) and + * nullability indicators ({@link io.reactivex.rxjava3.annotations.Nullable Nullable} and {@link io.reactivex.rxjava3.annotations.NonNull NonNull}). */ package io.reactivex.rxjava3.annotations; diff --git a/src/main/java/io/reactivex/rxjava3/disposables/package-info.java b/src/main/java/io/reactivex/rxjava3/disposables/package-info.java index fc3c4221b9..21b276e132 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/package-info.java @@ -15,8 +15,8 @@ */ /** - * Default implementations for Disposable-based resource management - * (Disposable container types) and utility classes to construct - * Disposables from callbacks and other types. + * Default implementations for {@link io.reactivex.rxjava3.disposables.Disposable Disposable}-based resource management + * ({@code Disposable} container types) and utility classes to construct + * {@link io.reactivex.rxjava3.disposables.Disposables Disposables} from callbacks and other types. */ package io.reactivex.rxjava3.disposables; diff --git a/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java b/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java index 1b3340e414..34557ce302 100644 --- a/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java @@ -15,7 +15,10 @@ */ /** - * Exception handling utilities, safe subscriber exception classes, - * lifecycle exception classes. + * Exception handling utilities ({@link io.reactivex.rxjava3.exceptions.Exceptions Exceptions}), + * composite exception container ({@link io.reactivex.rxjava3.exceptions.CompositeException CompositeException}) and + * various lifecycle-reladed ({@link io.reactivex.rxjava3.exceptions.MissingBackpressureException UndeliverableException}) + * and behavior-violation exception types ({@link io.reactivex.rxjava3.exceptions.OnErrorNotImplementedException OnErrorNotImplementedException}, + * {@link io.reactivex.rxjava3.exceptions.MissingBackpressureException MissingBackpressureException}). */ package io.reactivex.rxjava3.exceptions; diff --git a/src/main/java/io/reactivex/rxjava3/observers/package-info.java b/src/main/java/io/reactivex/rxjava3/observers/package-info.java index 89596573d4..dbf39cbef2 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/observers/package-info.java @@ -15,10 +15,44 @@ */ /** - * Default wrappers and implementations for Observer-based consumer classes and interfaces, + * Default wrappers and implementations for observer-based consumer classes and interfaces, * including disposable and resource-tracking variants and - * the {@link io.reactivex.rxjava3.observers.TestObserver} that allows unit testing - * {@link io.reactivex.rxjava3.core.Observable}-, {@link io.reactivex.rxjava3.core.Single}-, {@link io.reactivex.rxjava3.core.Maybe}- - * and {@link io.reactivex.rxjava3.core.Completable}-based flows. + * the {@link io.reactivex.rxjava3.observers.TestObserver TestObserver} that allows unit testing + * {@link io.reactivex.rxjava3.core.Observable Observable}-, {@link io.reactivex.rxjava3.core.Single Single}-, + * {@link io.reactivex.rxjava3.core.Maybe Maybe}- and {@link io.reactivex.rxjava3.core.Completable Completable}-based flows. + *

+ * Available observer variants + *
+ * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Reactive typeBase interfaceSimpleDisposableResource
{@link io.reactivex.rxjava3.core.Observable Observable}{@link io.reactivex.rxjava3.core.Observer Observer}{@link io.reactivex.rxjava3.observers.DefaultObserver DefaultObserver}{@link io.reactivex.rxjava3.observers.DisposableObserver DisposableObserver}{@link io.reactivex.rxjava3.observers.ResourceObserver DisposableObserver}
{@link io.reactivex.rxjava3.core.Maybe Maybe}{@link io.reactivex.rxjava3.core.MaybeObserver MaybeObserver}N/A{@link io.reactivex.rxjava3.observers.DisposableMaybeObserver DisposableMaybeObserver}{@link io.reactivex.rxjava3.observers.ResourceMaybeObserver DisposableMaybeObserver}
{@link io.reactivex.rxjava3.core.Single Single}{@link io.reactivex.rxjava3.core.SingleObserver SingleObserver}N/A{@link io.reactivex.rxjava3.observers.DisposableSingleObserver DisposableSingleObserver}{@link io.reactivex.rxjava3.observers.ResourceSingleObserver DisposableSingleObserver}
{@link io.reactivex.rxjava3.core.Completable Completable}{@link io.reactivex.rxjava3.core.CompletableObserver CompletableObserver}N/A{@link io.reactivex.rxjava3.observers.DisposableCompletableObserver DisposableCompletableObserver}{@link io.reactivex.rxjava3.observers.ResourceCompletableObserver DisposableCompletableObserver}
*/ package io.reactivex.rxjava3.observers; diff --git a/src/main/java/io/reactivex/rxjava3/subjects/package-info.java b/src/main/java/io/reactivex/rxjava3/subjects/package-info.java index 2980d499bb..88fde8391d 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/package-info.java @@ -15,7 +15,7 @@ */ /** - * Classes representing so-called hot sources, aka subjects, that implement a base reactive class and + * Classes representing so-called hot sources, aka subjects, that implement a base reactive class and * the respective consumer type at once to allow forms of multicasting events to multiple * consumers as well as consuming another base reactive type of their kind. *

diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/package-info.java b/src/main/java/io/reactivex/rxjava3/subscribers/package-info.java index 1792e0e460..348792062a 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/package-info.java @@ -15,9 +15,10 @@ */ /** - * Default wrappers and implementations for Subscriber-based consumer classes and interfaces, - * including disposable and resource-tracking variants and - * the {@link io.reactivex.rxjava3.subscribers.TestSubscriber} that allows unit testing - * {@link io.reactivex.rxjava3.core.Flowable}-based flows. + * Default wrappers and implementations for {@link org.reactivestreams.Subscriber Subscriber}-based consumer classes and interfaces, + * including disposable ({@link io.reactivex.rxjava3.subscribers.DisposableSubscriber DisposableSubscriber}) and resource-tracking + * ({@link io.reactivex.rxjava3.subscribers.ResourceSubscriber ResourceSubscriber}) + * variants and the {@link io.reactivex.rxjava3.subscribers.TestSubscriber TestSubscriber} that allows unit testing + * {@link io.reactivex.rxjava3.core.Flowable Flowable}-based flows. */ package io.reactivex.rxjava3.subscribers; From 8d41cc5801eecac4acc38eee7adcee642de24385 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Thu, 19 Dec 2019 09:58:58 +0100 Subject: [PATCH 010/665] 3.x: [Java 8] Implement mapOptional, collector, first/last/single stage (#6775) * 3.x: [Java 8] Implement mapOptional, collector, first/last/single stage * Upgrade the Checkstyle plugin --- build.gradle | 2 +- .../io/reactivex/rxjava3/core/Flowable.java | 260 +++++++++- .../jdk8/FlowableCollectWithCollector.java | 153 ++++++ .../FlowableCollectWithCollectorSingle.java | 164 ++++++ .../jdk8/FlowableFirstStageSubscriber.java | 59 +++ .../jdk8/FlowableLastStageSubscriber.java | 62 +++ .../internal/jdk8/FlowableMapOptional.java | 179 +++++++ .../jdk8/FlowableSingleStageSubscriber.java | 68 +++ .../jdk8/FlowableStageSubscriber.java | 80 +++ .../jdk8/CollectWithCollectorTckTest.java | 45 ++ .../FlowableCollectWithCollectorTest.java | 420 ++++++++++++++++ .../jdk8/FlowableMapOptionalTest.java | 470 +++++++++++++++++ .../FlowableStageSubscriberOrDefaultTest.java | 476 ++++++++++++++++++ .../FlowableStageSubscriberOrErrorTest.java | 470 +++++++++++++++++ .../internal/jdk8/MapOptionalTckTest.java | 38 ++ .../rxjava3/testsupport/TestHelper.java | 17 + .../ParamValidationCheckerTest.java | 6 + 17 files changed, 2967 insertions(+), 2 deletions(-) create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollector.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorSingle.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFirstStageSubscriber.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableLastStageSubscriber.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptional.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableSingleStageSubscriber.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriber.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/CollectWithCollectorTckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptionalTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrDefaultTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrErrorTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MapOptionalTckTest.java diff --git a/build.gradle b/build.gradle index 4dcead40c4..f0406bd3f7 100644 --- a/build.gradle +++ b/build.gradle @@ -16,7 +16,7 @@ buildscript { ext.bintrayVersion = "1.8.4" ext.jfrogExtractorVersion = "4.11.0" ext.bndVersion = "4.3.1" - ext.checkstyleVersion = "6.19" + ext.checkstyleVersion = "8.26" // -------------------------------------- diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 514845efbc..4177a2d9e6 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -6930,6 +6930,7 @@ public final Flowable cast(final Class clazz) { * @return a Single that emits the result of collecting the values emitted by the source Publisher * into a single mutable data structure * @see ReactiveX operators documentation: Reduce + * @see #collect(Collector) */ @CheckReturnValue @NonNull @@ -11256,12 +11257,13 @@ public final Flowable lift(FlowableOperator lifte * @return a Flowable that emits the items from the source Publisher, transformed by the specified * function * @see ReactiveX operators documentation: Map + * @see #mapOptional(Function) */ @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable map(Function mapper) { + public final <@NonNull R> Flowable map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new FlowableMap(this, mapper)); } @@ -18739,4 +18741,260 @@ public final TestSubscriber test(long initialRequest, boolean cancel) { // No Objects.requireNonNull(stream, "stream is null"); return RxJavaPlugins.onAssembly(new FlowableFromStream<>(stream)); } + + /** + * Maps each upstream value into an {@link Optional} and emits the contained item if not empty. + *

+ * + * + *

+ *
Backpressure:
+ *
The operator is a pass-through for downstream requests but issues {@code request(1)} whenever the + * mapped {@code Optional} is empty.
+ *
Scheduler:
+ *
{@code mapOptional} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the non-null output type + * @param mapper the function that receives the upstream item and should return a non-empty {@code Optional} + * to emit as the output or an empty {@code Optional} to skip to the next upstream value + * @return the new Flowable instance + * @since 3.0.0 + * @see #map(Function) + * @see #filter(Predicate) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Flowable mapOptional(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new FlowableMapOptional<>(this, mapper)); + } + + /** + * Collects the finite upstream's values into a container via a Stream {@link Collector} callback set and emits + * it as the success result. + *

+ * + * + *

+ *
Backpressure:
+ *
The operator consumes the upstream in an unbounded manner.
+ *
Scheduler:
+ *
{@code collect} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the non-null result type + * @param the intermediate container type used for the accumulation + * @param collector the interface defining the container supplier, accumulator and finisher functions; + * see {@link Collectors} for some standard implementations + * @return the new Single instance + * @since 3.0.0 + * @see Collectors + * @see #collect(Supplier, BiConsumer) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R, A> Single collect(@NonNull Collector collector) { + Objects.requireNonNull(collector, "collector is null"); + return RxJavaPlugins.onAssembly(new FlowableCollectWithCollectorSingle<>(this, collector)); + } + + /** + * Signals the first upstream item (or the default item if the upstream is empty) via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).firstStage(Optional.empty());
+     * 
+ *
+ *
Backpressure:
+ *
The operator requests one item from upstream and then when received, cancels the upstream.
+ *
Scheduler:
+ *
{@code firstStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + * @see #firstOrErrorStage() + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage firstStage(@Nullable T defaultItem) { + return subscribeWith(new FlowableFirstStageSubscriber<>(true, defaultItem)); + } + + /** + * Signals the only expected upstream item (or the default item if the upstream is empty) + * or signals {@link IllegalArgumentException} if the upstream has more than one item + * via a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).singleStage(Optional.empty());
+     * 
+ *
+ *
Backpressure:
+ *
The operator requests two items from upstream and then when more than one item is received, cancels the upstream.
+ *
Scheduler:
+ *
{@code singleStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + * @see #singleOrErrorStage() + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage singleStage(@Nullable T defaultItem) { + return subscribeWith(new FlowableSingleStageSubscriber<>(true, defaultItem)); + } + + /** + * Signals the last upstream item (or the default item if the upstream is empty) via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).lastStage(Optional.empty());
+     * 
+ *
+ *
Backpressure:
+ *
The operator requests an unbounded number of items from the upstream.
+ *
Scheduler:
+ *
{@code lastStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + * @see #lastOrErrorStage() + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage lastStage(@Nullable T defaultItem) { + return subscribeWith(new FlowableLastStageSubscriber<>(true, defaultItem)); + } + + /** + * Signals the first upstream item or a {@link NoSuchElementException} if the upstream is empty via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ *
Backpressure:
+ *
The operator requests one item from upstream and then when received, cancels the upstream.
+ *
Scheduler:
+ *
{@code firstOrErrorStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + * @see #firstStage(Object) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage firstOrErrorStage() { + return subscribeWith(new FlowableFirstStageSubscriber<>(false, null)); + } + + /** + * Signals the only expected upstream item, a {@link NoSuchElementException} if the upstream is empty + * or signals {@link IllegalArgumentException} if the upstream has more than one item + * via a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ *
Backpressure:
+ *
The operator requests two items from upstream and then when more than one item is received, cancels the upstream.
+ *
Scheduler:
+ *
{@code singleOrErrorStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + * @see #singleStage(Object) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage singleOrErrorStage() { + return subscribeWith(new FlowableSingleStageSubscriber<>(false, null)); + } + + /** + * Signals the last upstream item or a {@link NoSuchElementException} if the upstream is empty via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ *
Backpressure:
+ *
The operator requests an unbounded number of items from the upstream.
+ *
Scheduler:
+ *
{@code lastOrErrorStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + * @see #lastStage(Object) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage lastOrErrorStage() { + return subscribeWith(new FlowableLastStageSubscriber<>(false, null)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollector.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollector.java new file mode 100644 index 0000000000..bc408db304 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollector.java @@ -0,0 +1,153 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Objects; +import java.util.function.*; +import java.util.stream.Collector; + +import org.reactivestreams.*; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.internal.subscriptions.*; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Collect items into a container defined by a Stream {@link Collector} callback set. + * + * @param the upstream value type + * @param
the intermediate accumulator type + * @param the result type + * @since 3.0.0 + */ +public final class FlowableCollectWithCollector extends Flowable { + + final Flowable source; + + final Collector collector; + + public FlowableCollectWithCollector(Flowable source, Collector collector) { + this.source = source; + this.collector = collector; + } + + @Override + protected void subscribeActual(@NonNull Subscriber s) { + A container; + BiConsumer accumulator; + Function finisher; + + try { + container = collector.supplier().get(); + accumulator = collector.accumulator(); + finisher = collector.finisher(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + EmptySubscription.error(ex, s); + return; + } + + source.subscribe(new CollectorSubscriber<>(s, container, accumulator, finisher)); + } + + static final class CollectorSubscriber + extends DeferredScalarSubscription + implements FlowableSubscriber { + + private static final long serialVersionUID = -229544830565448758L; + + final BiConsumer accumulator; + + final Function finisher; + + Subscription upstream; + + boolean done; + + A container; + + CollectorSubscriber(Subscriber downstream, A container, BiConsumer accumulator, Function finisher) { + super(downstream); + this.container = container; + this.accumulator = accumulator; + this.finisher = finisher; + } + + @Override + public void onSubscribe(@NonNull Subscription s) { + if (SubscriptionHelper.validate(this.upstream, s)) { + this.upstream = s; + + downstream.onSubscribe(this); + + s.request(Long.MAX_VALUE); + } + } + + @Override + public void onNext(T t) { + if (done) { + return; + } + try { + accumulator.accept(container, t); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + upstream.cancel(); + onError(ex); + } + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + } else { + done = true; + upstream = SubscriptionHelper.CANCELLED; + this.container = null; + downstream.onError(t); + } + } + + @Override + public void onComplete() { + if (done) { + return; + } + + done = true; + upstream = SubscriptionHelper.CANCELLED; + A container = this.container; + this.container = null; + R result; + try { + result = Objects.requireNonNull(finisher.apply(container), "The finisher returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + + complete(result); + } + + @Override + public void cancel() { + super.cancel(); + upstream.cancel(); + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorSingle.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorSingle.java new file mode 100644 index 0000000000..134de68415 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorSingle.java @@ -0,0 +1,164 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Objects; +import java.util.function.*; +import java.util.stream.Collector; + +import org.reactivestreams.Subscription; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; +import io.reactivex.rxjava3.internal.fuseable.FuseToFlowable; +import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Collect items into a container defined by a Stream {@link Collector} callback set. + * + * @param the upstream value type + * @param the intermediate accumulator type + * @param the result type + * @since 3.0.0 + */ +public final class FlowableCollectWithCollectorSingle extends Single implements FuseToFlowable { + + final Flowable source; + + final Collector collector; + + public FlowableCollectWithCollectorSingle(Flowable source, Collector collector) { + this.source = source; + this.collector = collector; + } + + @Override + public Flowable fuseToFlowable() { + return new FlowableCollectWithCollector<>(source, collector); + } + + @Override + protected void subscribeActual(@NonNull SingleObserver observer) { + A container; + BiConsumer accumulator; + Function finisher; + + try { + container = collector.supplier().get(); + accumulator = collector.accumulator(); + finisher = collector.finisher(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + EmptyDisposable.error(ex, observer); + return; + } + + source.subscribe(new CollectorSingleObserver<>(observer, container, accumulator, finisher)); + } + + static final class CollectorSingleObserver implements FlowableSubscriber, Disposable { + + final SingleObserver downstream; + + final BiConsumer accumulator; + + final Function finisher; + + Subscription upstream; + + boolean done; + + A container; + + CollectorSingleObserver(SingleObserver downstream, A container, BiConsumer accumulator, Function finisher) { + this.downstream = downstream; + this.container = container; + this.accumulator = accumulator; + this.finisher = finisher; + } + + @Override + public void onSubscribe(@NonNull Subscription s) { + if (SubscriptionHelper.validate(this.upstream, s)) { + this.upstream = s; + + downstream.onSubscribe(this); + + s.request(Long.MAX_VALUE); + } + } + + @Override + public void onNext(T t) { + if (done) { + return; + } + try { + accumulator.accept(container, t); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + upstream.cancel(); + onError(ex); + } + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + } else { + done = true; + upstream = SubscriptionHelper.CANCELLED; + this.container = null; + downstream.onError(t); + } + } + + @Override + public void onComplete() { + if (done) { + return; + } + + done = true; + upstream = SubscriptionHelper.CANCELLED; + A container = this.container; + this.container = null; + R result; + try { + result = Objects.requireNonNull(finisher.apply(container), "The finisher returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + + downstream.onSuccess(result); + } + + @Override + public void dispose() { + upstream.cancel(); + upstream = SubscriptionHelper.CANCELLED; + } + + @Override + public boolean isDisposed() { + return upstream == SubscriptionHelper.CANCELLED; + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFirstStageSubscriber.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFirstStageSubscriber.java new file mode 100644 index 0000000000..5af6cc5aac --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFirstStageSubscriber.java @@ -0,0 +1,59 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.NoSuchElementException; + +import org.reactivestreams.Subscription; + +/** + * Signals the first element of the source via the underlying CompletableFuture, + * signals the a default item if the upstream is empty or signals {@link NoSuchElementException}. + * + * @param the element type + * @since 3.0.0 + */ +public final class FlowableFirstStageSubscriber extends FlowableStageSubscriber { + + final boolean hasDefault; + + final T defaultItem; + + public FlowableFirstStageSubscriber(boolean hasDefault, T defaultItem) { + this.hasDefault = hasDefault; + this.defaultItem = defaultItem; + } + + @Override + public void onNext(T t) { + complete(t); + } + + @Override + public void onComplete() { + if (!isDone()) { + clear(); + if (hasDefault) { + complete(defaultItem); + } else { + completeExceptionally(new NoSuchElementException()); + } + } + } + + @Override + protected void afterSubscribe(Subscription s) { + s.request(1); + } + +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableLastStageSubscriber.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableLastStageSubscriber.java new file mode 100644 index 0000000000..c9a93eeb0e --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableLastStageSubscriber.java @@ -0,0 +1,62 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.NoSuchElementException; + +import org.reactivestreams.Subscription; + +/** + * Signals the last element of the source via the underlying CompletableFuture, + * signals the a default item if the upstream is empty or signals {@link NoSuchElementException}. + * + * @param the element type + * @since 3.0.0 + */ +public final class FlowableLastStageSubscriber extends FlowableStageSubscriber { + + final boolean hasDefault; + + final T defaultItem; + + public FlowableLastStageSubscriber(boolean hasDefault, T defaultItem) { + this.hasDefault = hasDefault; + this.defaultItem = defaultItem; + } + + @Override + public void onNext(T t) { + value = t; + } + + @Override + public void onComplete() { + if (!isDone()) { + T v = value; + clear(); + if (v != null) { + complete(v); + } else if (hasDefault) { + complete(defaultItem); + } else { + completeExceptionally(new NoSuchElementException()); + } + } + } + + @Override + protected void afterSubscribe(Subscription s) { + s.request(Long.MAX_VALUE); + } + +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptional.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptional.java new file mode 100644 index 0000000000..3fdcd40a4f --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptional.java @@ -0,0 +1,179 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; + +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; +import io.reactivex.rxjava3.internal.subscribers.*; + +/** + * Map the upstream values into an Optional and emit its value if any. + * @param the upstream element type + * @param the output element type + * @since 3.0.0 + */ +public final class FlowableMapOptional extends Flowable { + + final Flowable source; + + final Function> mapper; + + public FlowableMapOptional(Flowable source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(Subscriber s) { + if (s instanceof ConditionalSubscriber) { + source.subscribe(new MapOptionalConditionalSubscriber<>((ConditionalSubscriber)s, mapper)); + } else { + source.subscribe(new MapOptionalSubscriber<>(s, mapper)); + } + } + + static final class MapOptionalSubscriber extends BasicFuseableSubscriber + implements ConditionalSubscriber { + + final Function> mapper; + + MapOptionalSubscriber(Subscriber downstream, Function> mapper) { + super(downstream); + this.mapper = mapper; + } + + @Override + public void onNext(T t) { + if (!tryOnNext(t)) { + upstream.request(1); + } + } + + @Override + public boolean tryOnNext(T t) { + if (done) { + return true; + } + + if (sourceMode != NONE) { + downstream.onNext(null); + return true; + } + + Optional result; + try { + result = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Optional"); + } catch (Throwable ex) { + fail(ex); + return true; + } + + if (result.isPresent()) { + downstream.onNext(result.get()); + return true; + } + return false; + } + + @Override + public int requestFusion(int mode) { + return transitiveBoundaryFusion(mode); + } + + @Override + public R poll() throws Throwable { + for (;;) { + T item = qs.poll(); + if (item == null) { + return null; + } + Optional result = Objects.requireNonNull(mapper.apply(item), "The mapper returned a null Optional"); + if (result.isPresent()) { + return result.get(); + } + if (sourceMode == ASYNC) { + qs.request(1); + } + } + } + } + + static final class MapOptionalConditionalSubscriber extends BasicFuseableConditionalSubscriber { + + final Function> mapper; + + MapOptionalConditionalSubscriber(ConditionalSubscriber downstream, Function> mapper) { + super(downstream); + this.mapper = mapper; + } + + @Override + public void onNext(T t) { + if (!tryOnNext(t)) { + upstream.request(1); + } + } + + @Override + public boolean tryOnNext(T t) { + if (done) { + return true; + } + + if (sourceMode != NONE) { + downstream.onNext(null); + return true; + } + + Optional result; + try { + result = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Optional"); + } catch (Throwable ex) { + fail(ex); + return true; + } + + if (result.isPresent()) { + return downstream.tryOnNext(result.get()); + } + return false; + } + + @Override + public int requestFusion(int mode) { + return transitiveBoundaryFusion(mode); + } + + @Override + public R poll() throws Throwable { + for (;;) { + T item = qs.poll(); + if (item == null) { + return null; + } + Optional result = Objects.requireNonNull(mapper.apply(item), "The mapper returned a null Optional"); + if (result.isPresent()) { + return result.get(); + } + if (sourceMode == ASYNC) { + qs.request(1); + } + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableSingleStageSubscriber.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableSingleStageSubscriber.java new file mode 100644 index 0000000000..8040ec6f5d --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableSingleStageSubscriber.java @@ -0,0 +1,68 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.NoSuchElementException; + +import org.reactivestreams.Subscription; + +/** + * Signals the only element of the source via the underlying CompletableFuture, + * signals the a default item if the upstream is empty or signals {@link IllegalArgumentException} + * if the upstream has more than one item. + * + * @param the element type + * @since 3.0.0 + */ +public final class FlowableSingleStageSubscriber extends FlowableStageSubscriber { + + final boolean hasDefault; + + final T defaultItem; + + public FlowableSingleStageSubscriber(boolean hasDefault, T defaultItem) { + this.hasDefault = hasDefault; + this.defaultItem = defaultItem; + } + + @Override + public void onNext(T t) { + if (value != null) { + value = null; + completeExceptionally(new IllegalArgumentException("Sequence contains more than one element!")); + } else { + value = t; + } + } + + @Override + public void onComplete() { + if (!isDone()) { + T v = value; + clear(); + if (v != null) { + complete(v); + } else if (hasDefault) { + complete(defaultItem); + } else { + completeExceptionally(new NoSuchElementException()); + } + } + } + + @Override + protected void afterSubscribe(Subscription s) { + s.request(2); + } + +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriber.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriber.java new file mode 100644 index 0000000000..6594b7ba5c --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriber.java @@ -0,0 +1,80 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; + +import org.reactivestreams.Subscription; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.FlowableSubscriber; +import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Base class that extends CompletableFuture and provides basic infrastructure + * to notify watchers upon upstream signals. + * @param the element type + * @since 3.0.0 + */ +abstract class FlowableStageSubscriber extends CompletableFuture implements FlowableSubscriber { + + final AtomicReference upstream = new AtomicReference<>(); + + T value; + + @Override + public final void onSubscribe(@NonNull Subscription s) { + if (SubscriptionHelper.setOnce(upstream, s)) { + afterSubscribe(s); + } + } + + protected abstract void afterSubscribe(Subscription s); + + @Override + public final void onError(Throwable t) { + clear(); + if (!completeExceptionally(t)) { + RxJavaPlugins.onError(t); + } + } + + protected final void cancelUpstream() { + SubscriptionHelper.cancel(upstream); + } + + protected final void clear() { + value = null; + upstream.lazySet(SubscriptionHelper.CANCELLED); + } + + @Override + public final boolean cancel(boolean mayInterruptIfRunning) { + cancelUpstream(); + return super.cancel(mayInterruptIfRunning); + } + + @Override + public final boolean complete(T value) { + cancelUpstream(); + return super.complete(value); + } + + @Override + public final boolean completeExceptionally(Throwable ex) { + cancelUpstream(); + return super.completeExceptionally(ex); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/CollectWithCollectorTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/CollectWithCollectorTckTest.java new file mode 100644 index 0000000000..99fb2dcff9 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/CollectWithCollectorTckTest.java @@ -0,0 +1,45 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.List; +import java.util.stream.Collectors; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class CollectWithCollectorTckTest extends BaseTck> { + + @Override + public Publisher> createPublisher(final long elements) { + return + Flowable.range(0, (int)elements).collect(Collectors.toList()).toFlowable() + ; + } + + @Override + public Publisher> createFailedPublisher() { + return Flowable.error(new TestException()).collect(Collectors.toList()).toFlowable(); + } + + @Override + public long maxElementsFromPublisher() { + return 1; + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorTest.java new file mode 100644 index 0000000000..ecef7665e0 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableCollectWithCollectorTest.java @@ -0,0 +1,420 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.assertFalse; + +import java.io.IOException; +import java.util.*; +import java.util.function.*; +import java.util.stream.*; + +import org.junit.Test; +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class FlowableCollectWithCollectorTest extends RxJavaTest { + + @Test + public void basic() { + Flowable.range(1, 5) + .collect(Collectors.toList()) + .test() + .assertResult(Arrays.asList(1, 2, 3, 4, 5)); + } + + @Test + public void empty() { + Flowable.empty() + .collect(Collectors.toList()) + .test() + .assertResult(Collections.emptyList()); + } + + @Test + public void error() { + Flowable.error(new TestException()) + .collect(Collectors.toList()) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorSupplierCrash() { + Flowable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + throw new TestException(); + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorCrash() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void collectorFinisherCrash() { + Flowable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> { throw new TestException(); }; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorDropSignals() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Flowable source = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onNext(2); + s.onError(new IOException()); + s.onComplete(); + } + }; + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + + TestHelper.assertUndeliverable(errors, 0, IOException.class); + }); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(PublishProcessor.create() + .collect(Collectors.toList())); + } + + @Test + public void onSubscribe() { + TestHelper.checkDoubleOnSubscribeFlowableToSingle(f -> f.collect(Collectors.toList())); + } + + @Test + public void basicToFlowable() { + Flowable.range(1, 5) + .collect(Collectors.toList()) + .toFlowable() + .test() + .assertResult(Arrays.asList(1, 2, 3, 4, 5)); + } + + @Test + public void emptyToFlowable() { + Flowable.empty() + .collect(Collectors.toList()) + .toFlowable() + .test() + .assertResult(Collections.emptyList()); + } + + @Test + public void errorToFlowable() { + Flowable.error(new TestException()) + .collect(Collectors.toList()) + .toFlowable() + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorSupplierCrashToFlowable() { + Flowable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + throw new TestException(); + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toFlowable() + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorCrashToFlowable() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toFlowable() + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void collectorFinisherCrashToFlowable() { + Flowable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> { throw new TestException(); }; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toFlowable() + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorDropSignalsToFlowable() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Flowable source = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onNext(2); + s.onError(new IOException()); + s.onComplete(); + } + }; + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toFlowable() + .test() + .assertFailure(TestException.class); + + TestHelper.assertUndeliverable(errors, 0, IOException.class); + }); + } + + @Test + public void disposeToFlowable() { + TestHelper.checkDisposed(PublishProcessor.create() + .collect(Collectors.toList()).toFlowable()); + } + + @Test + public void onSubscribeToFlowable() { + TestHelper.checkDoubleOnSubscribeFlowable(f -> f.collect(Collectors.toList()).toFlowable()); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptionalTest.java new file mode 100644 index 0000000000..85fc8d75be --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableMapOptionalTest.java @@ -0,0 +1,470 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.assertFalse; + +import java.util.Optional; + +import org.junit.Test; +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.QueueFuseable; +import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class FlowableMapOptionalTest extends RxJavaTest { + + static final Function> MODULO = v -> v % 2 == 0 ? Optional.of(v) : Optional.empty(); + + @Test + public void allPresent() { + Flowable.range(1, 5) + .mapOptional(Optional::of) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void allEmpty() { + Flowable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .test() + .assertResult(); + } + + @Test + public void mixed() { + Flowable.range(1, 10) + .mapOptional(MODULO) + .test() + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void mapperChash() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .mapOptional(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void mapperNull() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .mapOptional(v -> null) + .test() + .assertFailure(NullPointerException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void crashDropsOnNexts() { + Flowable source = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onNext(2); + } + }; + + source + .mapOptional(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void backpressureAll() { + Flowable.range(1, 5) + .mapOptional(Optional::of) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(1, 2) + .requestMore(2) + .assertValuesOnly(1, 2, 3, 4) + .requestMore(1) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void backpressureNone() { + Flowable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .test(1L) + .assertResult(); + } + + @Test + public void backpressureMixed() { + Flowable.range(1, 10) + .mapOptional(MODULO) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(2, 4) + .requestMore(2) + .assertValuesOnly(2, 4, 6, 8) + .requestMore(1) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void syncFusedAll() { + Flowable.range(1, 5) + .mapOptional(Optional::of) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void asyncFusedAll() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(Optional::of) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void boundaryFusedAll() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(Optional::of) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void syncFusedNone() { + Flowable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(); + } + + @Test + public void asyncFusedNone() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(v -> Optional.empty()) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void boundaryFusedNone() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(v -> Optional.empty()) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(); + } + + @Test + public void syncFusedMixed() { + Flowable.range(1, 10) + .mapOptional(MODULO) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void asyncFusedMixed() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + up + .mapOptional(MODULO) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void boundaryFusedMixed() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + up + .mapOptional(MODULO) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void allPresentConditional() { + Flowable.range(1, 5) + .mapOptional(Optional::of) + .filter(v -> true) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void allEmptyConditional() { + Flowable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .test() + .assertResult(); + } + + @Test + public void mixedConditional() { + Flowable.range(1, 10) + .mapOptional(MODULO) + .filter(v -> true) + .test() + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void mapperChashConditional() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .mapOptional(v -> { throw new TestException(); }) + .filter(v -> true) + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void mapperNullConditional() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .mapOptional(v -> null) + .filter(v -> true) + .test() + .assertFailure(NullPointerException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void crashDropsOnNextsConditional() { + Flowable source = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onNext(2); + } + }; + + source + .mapOptional(v -> { throw new TestException(); }) + .filter(v -> true) + .test() + .assertFailure(TestException.class); + } + + @Test + public void backpressureAllConditional() { + Flowable.range(1, 5) + .mapOptional(Optional::of) + .filter(v -> true) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(1, 2) + .requestMore(2) + .assertValuesOnly(1, 2, 3, 4) + .requestMore(1) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void backpressureNoneConditional() { + Flowable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .test(1L) + .assertResult(); + } + + @Test + public void backpressureMixedConditional() { + Flowable.range(1, 10) + .mapOptional(MODULO) + .filter(v -> true) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(2, 4) + .requestMore(2) + .assertValuesOnly(2, 4, 6, 8) + .requestMore(1) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void syncFusedAllConditional() { + Flowable.range(1, 5) + .mapOptional(Optional::of) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void asyncFusedAllConditional() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(Optional::of) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void boundaryFusedAllConditiona() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(Optional::of) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void syncFusedNoneConditional() { + Flowable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(); + } + + @Test + public void asyncFusedNoneConditional() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void boundaryFusedNoneConditional() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5); + + up + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(); + } + + @Test + public void syncFusedMixedConditional() { + Flowable.range(1, 10) + .mapOptional(MODULO) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void asyncFusedMixedConditional() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + up + .mapOptional(MODULO) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void boundaryFusedMixedConditional() { + UnicastProcessor up = UnicastProcessor.create(); + TestHelper.emit(up, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + up + .mapOptional(MODULO) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(2, 4, 6, 8, 10); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrDefaultTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrDefaultTest.java new file mode 100644 index 0000000000..882fd83dfb --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrDefaultTest.java @@ -0,0 +1,476 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.concurrent.*; + +import org.junit.Test; +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class FlowableStageSubscriberOrDefaultTest extends RxJavaTest { + + @Test + public void firstJust() throws Exception { + Integer v = Flowable.just(1) + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void firstEmpty() throws Exception { + Integer v = Flowable.empty() + .firstStage(2) + .toCompletableFuture() + .get(); + + assertEquals((Integer)2, v); + } + + @Test + public void firstCancels() throws Exception { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + Integer v = source + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void firstCompletableFutureCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .firstStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void firstCompletableManualCompleteCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .firstStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void firstCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .firstStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstError() throws Exception { + CompletableFuture cf = Flowable.error(new TestException()) + .firstStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onError(new TestException()); + s.onComplete(); + } + } + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void firstDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + } + } + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void singleJust() throws Exception { + Integer v = Flowable.just(1) + .singleStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void singleEmpty() throws Exception { + Integer v = Flowable.empty() + .singleStage(2) + .toCompletableFuture() + .get(); + + assertEquals((Integer)2, v); + } + + @Test + public void singleTooManyCancels() throws Exception { + ReplayProcessor source = ReplayProcessor.create(); + source.onNext(1); + source.onNext(2); + + TestHelper.assertError(source + .singleStage(null) + .toCompletableFuture(), IllegalArgumentException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void singleCompletableFutureCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .singleStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void singleCompletableManualCompleteCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .singleStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void singleCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .singleStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleError() throws Exception { + CompletableFuture cf = Flowable.error(new TestException()) + .singleStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + s.onError(new TestException()); + s.onComplete(); + } + } + .singleStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void singleDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + } + } + .singleStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void lastJust() throws Exception { + Integer v = Flowable.just(1) + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void lastRange() throws Exception { + Integer v = Flowable.range(1, 5) + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)5, v); + } + + @Test + public void lastEmpty() throws Exception { + Integer v = Flowable.empty() + .lastStage(2) + .toCompletableFuture() + .get(); + + assertEquals((Integer)2, v); + } + + @Test + public void lastCompletableFutureCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .lastStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void lastCompletableManualCompleteCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .lastStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void lastCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .lastStage(null) + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastError() throws Exception { + CompletableFuture cf = Flowable.error(new TestException()) + .lastStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + s.onError(new TestException()); + s.onComplete(); + } + } + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void lastDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + } + } + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrErrorTest.java new file mode 100644 index 0000000000..c34e4739fa --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableStageSubscriberOrErrorTest.java @@ -0,0 +1,470 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.NoSuchElementException; +import java.util.concurrent.*; + +import org.junit.Test; +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class FlowableStageSubscriberOrErrorTest extends RxJavaTest { + + @Test + public void firstJust() throws Exception { + Integer v = Flowable.just(1) + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void firstEmpty() throws Exception { + TestHelper.assertError( + Flowable.empty() + .firstOrErrorStage() + .toCompletableFuture(), NoSuchElementException.class); + } + + @Test + public void firstCancels() throws Exception { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + Integer v = source + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void firstCompletableFutureCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void firstCompletableManualCompleteCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void firstCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstError() throws Exception { + CompletableFuture cf = Flowable.error(new TestException()) + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onError(new TestException()); + s.onComplete(); + } + } + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void firstDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + } + } + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void singleJust() throws Exception { + Integer v = Flowable.just(1) + .singleOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void singleEmpty() throws Exception { + TestHelper.assertError( + Flowable.empty() + .singleOrErrorStage() + .toCompletableFuture(), NoSuchElementException.class); + } + + @Test + public void singleTooManyCancels() throws Exception { + ReplayProcessor source = ReplayProcessor.create(); + source.onNext(1); + source.onNext(2); + + TestHelper.assertError(source + .singleOrErrorStage() + .toCompletableFuture(), IllegalArgumentException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void singleCompletableFutureCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void singleCompletableManualCompleteCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void singleCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleError() throws Exception { + CompletableFuture cf = Flowable.error(new TestException()) + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + s.onError(new TestException()); + s.onComplete(); + } + } + .singleOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void singleDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + } + } + .singleOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void lastJust() throws Exception { + Integer v = Flowable.just(1) + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void lastRange() throws Exception { + Integer v = Flowable.range(1, 5) + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)5, v); + } + + @Test + public void lastEmpty() throws Exception { + TestHelper.assertError(Flowable.empty() + .lastOrErrorStage() + .toCompletableFuture(), NoSuchElementException.class); + } + + @Test + public void lastCompletableFutureCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void lastCompletableManualCompleteCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void lastCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishProcessor source = PublishProcessor.create(); + + CompletableFuture cf = source + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasSubscribers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasSubscribers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastError() throws Exception { + CompletableFuture cf = Flowable.error(new TestException()) + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + s.onError(new TestException()); + s.onComplete(); + } + } + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void lastDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onComplete(); + } + } + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MapOptionalTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MapOptionalTckTest.java new file mode 100644 index 0000000000..b7cb4d9725 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MapOptionalTckTest.java @@ -0,0 +1,38 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Optional; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class MapOptionalTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.range(0, (int)(2 * elements)).mapOptional(v -> v % 2 == 0 ? Optional.of(v) : Optional.empty()) + ; + } + + @Override + public Publisher createFailedPublisher() { + return Flowable.just(1).mapOptional(v -> null).onBackpressureDrop(); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index 1ce53f01ae..865b31a082 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -3438,4 +3438,21 @@ public static void withErrorTracking(Consumer> action) throws Th RxJavaPlugins.reset(); } } + + /** + * Assert if the given CompletableFuture fails with a specified error inside an ExecutionException. + * @param cf the CompletableFuture to test + * @param error the error class expected + */ + public static void assertError(CompletableFuture cf, Class error) { + try { + cf.get(); + fail("Should have thrown!"); + } catch (Throwable ex) { + if (!error.isInstance(ex.getCause())) { + ex.printStackTrace(); + fail("Wrong cause: " + ex.getCause()); + } + } + } } diff --git a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java index 2d4387d543..99f6783b4a 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java +++ b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java @@ -503,6 +503,12 @@ public void checkParallelFlowable() { addOverride(new ParamOverride(Observable.class, 0, ParamMode.ANY, "window", Long.TYPE, TimeUnit.class, Scheduler.class, Long.TYPE, Boolean.TYPE)); addOverride(new ParamOverride(Observable.class, 0, ParamMode.ANY, "window", Long.TYPE, TimeUnit.class, Scheduler.class, Long.TYPE, Boolean.TYPE, Integer.TYPE)); + // null value allowed + + addOverride(new ParamOverride(Flowable.class, 0, ParamMode.ANY, "firstStage", Object.class)); + addOverride(new ParamOverride(Flowable.class, 0, ParamMode.ANY, "singleStage", Object.class)); + addOverride(new ParamOverride(Flowable.class, 0, ParamMode.ANY, "lastStage", Object.class)); + // ----------------------------------------------------------------------------------- ignores = new HashMap>(); From 3d25617651a764b2df7d3817932153fd651bcad3 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Thu, 19 Dec 2019 18:18:05 +0100 Subject: [PATCH 011/665] 3.x: [Java 8] Add AutoCloseable <-> Disposable conversions, nicen docs (#6780) --- .../rxjava3/disposables/ActionDisposable.java | 7 +- .../disposables/AutoCloseableDisposable.java | 45 +++++++ .../rxjava3/disposables/Disposables.java | 58 ++++++--- .../rxjava3/disposables/FutureDisposable.java | 2 +- .../disposables/RunnableDisposable.java | 2 +- .../rxjava3/disposables/DisposablesTest.java | 111 +++++++++++++++--- 6 files changed, 191 insertions(+), 34 deletions(-) create mode 100644 src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java diff --git a/src/main/java/io/reactivex/rxjava3/disposables/ActionDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/ActionDisposable.java index dd74d600c6..32ad478656 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/ActionDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/ActionDisposable.java @@ -17,7 +17,7 @@ import io.reactivex.rxjava3.internal.util.ExceptionHelper; /** - * A Disposable container that manages an Action instance. + * A Disposable container that manages an {@link Action} instance. */ final class ActionDisposable extends ReferenceDisposable { @@ -35,4 +35,9 @@ protected void onDisposed(@NonNull Action value) { throw ExceptionHelper.wrapOrThrow(ex); } } + + @Override + public String toString() { + return "ActionDisposable(disposed=" + isDisposed() + ", " + get() + ")"; + } } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java new file mode 100644 index 0000000000..8c0f2be2dc --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java @@ -0,0 +1,45 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.disposables; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * A disposable container that manages an {@link AutoCloseable} instance. + * @since 3.0.0 + */ +final class AutoCloseableDisposable extends ReferenceDisposable { + + private static final long serialVersionUID = -6646144244598696847L; + + AutoCloseableDisposable(AutoCloseable value) { + super(value); + } + + @Override + protected void onDisposed(@NonNull AutoCloseable value) { + try { + value.close(); + } catch (Throwable ex) { + RxJavaPlugins.onError(ex); + } + } + + @Override + public String toString() { + return "AutoCloseableDisposable(disposed=" + isDisposed() + ", " + get() + ")"; + } + +} diff --git a/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java b/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java index 2ff83d64ff..39b1b76b25 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java @@ -35,8 +35,8 @@ private Disposables() { } /** - * Construct a Disposable by wrapping a Runnable that is - * executed exactly once when the Disposable is disposed. + * Construct a {@code Disposable} by wrapping a {@link Runnable} that is + * executed exactly once when the {@code Disposable} is disposed. * @param run the Runnable to wrap * @return the new Disposable instance */ @@ -47,8 +47,8 @@ public static Disposable fromRunnable(@NonNull Runnable run) { } /** - * Construct a Disposable by wrapping a Action that is - * executed exactly once when the Disposable is disposed. + * Construct a {@code Disposable} by wrapping a {@link Action} that is + * executed exactly once when the {@code Disposable} is disposed. * @param run the Action to wrap * @return the new Disposable instance */ @@ -59,10 +59,13 @@ public static Disposable fromAction(@NonNull Action run) { } /** - * Construct a Disposable by wrapping a Future that is - * cancelled exactly once when the Disposable is disposed. + * Construct a {@code Disposable} by wrapping a {@link Future} that is + * cancelled exactly once when the {@code Disposable} is disposed. + *

+ * The {@code Future} is cancelled with {@code mayInterruptIfRunning == true}. * @param future the Future to wrap * @return the new Disposable instance + * @see #fromFuture(Future, boolean) */ @NonNull public static Disposable fromFuture(@NonNull Future future) { @@ -71,10 +74,10 @@ public static Disposable fromFuture(@NonNull Future future) { } /** - * Construct a Disposable by wrapping a Future that is - * cancelled exactly once when the Disposable is disposed. + * Construct a {@code Disposable} by wrapping a {@link Future} that is + * cancelled exactly once when the {@code Disposable} is disposed. * @param future the Future to wrap - * @param allowInterrupt if true, the future cancel happens via Future.cancel(true) + * @param allowInterrupt if true, the future cancel happens via {@code Future.cancel(true)} * @return the new Disposable instance */ @NonNull @@ -84,8 +87,8 @@ public static Disposable fromFuture(@NonNull Future future, boolean allowInte } /** - * Construct a Disposable by wrapping a Subscription that is - * cancelled exactly once when the Disposable is disposed. + * Construct a {@code Disposable} by wrapping a {@link Subscription} that is + * cancelled exactly once when the {@code Disposable} is disposed. * @param subscription the Runnable to wrap * @return the new Disposable instance */ @@ -96,8 +99,33 @@ public static Disposable fromSubscription(@NonNull Subscription subscription) { } /** - * Returns a new, non-disposed Disposable instance. - * @return a new, non-disposed Disposable instance + * Construct a {@code Disposable} by wrapping an {@link AutoCloseable} that is + * closed exactly once when the {@code Disposable} is disposed. + * @param autoCloseable the AutoCloseable to wrap + * @return the new Disposable instance + * @since 3.0.0 + */ + @NonNull + public static Disposable fromAutoCloseable(@NonNull AutoCloseable autoCloseable) { + Objects.requireNonNull(autoCloseable, "autoCloseable is null"); + return new AutoCloseableDisposable(autoCloseable); + } + + /** + * Construct an {@link AutoCloseable} by wrapping a {@code Disposable} that is + * disposed when the returned {@code AutoCloseable} is closed. + * @param disposable the Disposable instance + * @return the new AutoCloseable instance + * @since 3.0.0 + */ + @NonNull + public static AutoCloseable toAutoCloseable(@NonNull Disposable disposable) { + return disposable::dispose; + } + + /** + * Returns a new, non-disposed {@code Disposable} instance. + * @return a new, non-disposed {@code Disposable} instance */ @NonNull public static Disposable empty() { @@ -105,8 +133,8 @@ public static Disposable empty() { } /** - * Returns a disposed Disposable instance. - * @return a disposed Disposable instance + * Returns a shared, disposed {@code Disposable} instance. + * @return a shared, disposed {@code Disposable} instance */ @NonNull public static Disposable disposed() { diff --git a/src/main/java/io/reactivex/rxjava3/disposables/FutureDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/FutureDisposable.java index 0a900f6ed1..9649bcd7d3 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/FutureDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/FutureDisposable.java @@ -16,7 +16,7 @@ import java.util.concurrent.atomic.AtomicReference; /** - * A Disposable container that cancels a Future instance. + * A Disposable container that cancels a {@link Future} instance. */ final class FutureDisposable extends AtomicReference> implements Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/disposables/RunnableDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/RunnableDisposable.java index 43eee3105c..34203e86a9 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/RunnableDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/RunnableDisposable.java @@ -15,7 +15,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * A disposable container that manages a Runnable instance. + * A disposable container that manages a {@link Runnable} instance. */ final class RunnableDisposable extends ReferenceDisposable { diff --git a/src/test/java/io/reactivex/rxjava3/disposables/DisposablesTest.java b/src/test/java/io/reactivex/rxjava3/disposables/DisposablesTest.java index d5ea5d0d49..244ca873f0 100644 --- a/src/test/java/io/reactivex/rxjava3/disposables/DisposablesTest.java +++ b/src/test/java/io/reactivex/rxjava3/disposables/DisposablesTest.java @@ -25,6 +25,7 @@ import org.reactivestreams.Subscription; import io.reactivex.rxjava3.core.RxJavaTest; +import io.reactivex.rxjava3.exceptions.TestException; import io.reactivex.rxjava3.functions.Action; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -34,11 +35,19 @@ public class DisposablesTest extends RxJavaTest { @Test public void unsubscribeOnlyOnce() { - Runnable dispose = mock(Runnable.class); - Disposable subscription = Disposables.fromRunnable(dispose); - subscription.dispose(); - subscription.dispose(); - verify(dispose, times(1)).run(); + Runnable run = mock(Runnable.class); + + Disposable d = Disposables.fromRunnable(run); + + assertTrue(d.toString(), d.toString().contains("RunnableDisposable(disposed=false, ")); + + d.dispose(); + assertTrue(d.toString(), d.toString().contains("RunnableDisposable(disposed=true, ")); + + d.dispose(); + assertTrue(d.toString(), d.toString().contains("RunnableDisposable(disposed=true, ")); + + verify(run, times(1)).run(); } @Test @@ -61,22 +70,20 @@ public void utilityClass() { } @Test - public void fromAction() { - class AtomicAction extends AtomicBoolean implements Action { + public void fromAction() throws Throwable { + Action action = mock(Action.class); - private static final long serialVersionUID = -1517510584253657229L; + Disposable d = Disposables.fromAction(action); - @Override - public void run() throws Exception { - set(true); - } - } + assertTrue(d.toString(), d.toString().contains("ActionDisposable(disposed=false, ")); - AtomicAction aa = new AtomicAction(); + d.dispose(); + assertTrue(d.toString(), d.toString().contains("ActionDisposable(disposed=true, ")); - Disposables.fromAction(aa).dispose(); + d.dispose(); + assertTrue(d.toString(), d.toString().contains("ActionDisposable(disposed=true, ")); - assertTrue(aa.get()); + verify(action, times(1)).run(); } @Test @@ -174,4 +181,76 @@ public void setOnceTwice() { RxJavaPlugins.reset(); } } + + @Test + public void fromAutoCloseable() { + AtomicInteger counter = new AtomicInteger(); + + AutoCloseable ac = () -> counter.getAndIncrement(); + + Disposable d = Disposables.fromAutoCloseable(ac); + + assertFalse(d.isDisposed()); + assertEquals(0, counter.get()); + assertTrue(d.toString(), d.toString().contains("AutoCloseableDisposable(disposed=false, ")); + + d.dispose(); + + assertTrue(d.isDisposed()); + assertEquals(1, counter.get()); + assertTrue(d.toString(), d.toString().contains("AutoCloseableDisposable(disposed=true, ")); + + d.dispose(); + + assertTrue(d.isDisposed()); + assertEquals(1, counter.get()); + assertTrue(d.toString(), d.toString().contains("AutoCloseableDisposable(disposed=true, ")); + } + + @Test + public void fromAutoCloseableThrows() throws Throwable { + TestHelper.withErrorTracking(errors -> { + AutoCloseable ac = () -> { throw new TestException(); }; + + Disposable d = Disposables.fromAutoCloseable(ac); + + assertFalse(d.isDisposed()); + + assertTrue(errors.isEmpty()); + + d.dispose(); + + assertTrue(d.isDisposed()); + assertEquals(1, errors.size()); + + d.dispose(); + + assertTrue(d.isDisposed()); + assertEquals(1, errors.size()); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void toAutoCloseable() throws Exception { + AtomicInteger counter = new AtomicInteger(); + + Disposable d = Disposables.fromAction(() -> counter.getAndIncrement()); + + AutoCloseable ac = Disposables.toAutoCloseable(d); + + assertFalse(d.isDisposed()); + assertEquals(0, counter.get()); + + ac.close(); + + assertTrue(d.isDisposed()); + assertEquals(1, counter.get()); + + ac.close(); + + assertTrue(d.isDisposed()); + assertEquals(1, counter.get()); + } } From a249f4f1241f1c0ebe8e03dc80bdd16e29962c53 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Thu, 19 Dec 2019 18:37:58 +0100 Subject: [PATCH 012/665] 3.x: [Java 8] Add blockingStream & flatMapStream to Flowable (#6779) --- .../io/reactivex/rxjava3/core/Flowable.java | 285 +++++++++++ .../internal/jdk8/FlowableFlatMapStream.java | 332 +++++++++++++ .../internal/jdk8/FlowableFromStream.java | 10 + .../internal/jdk8/FlatMapStream0HTckTest.java | 40 ++ .../internal/jdk8/FlatMapStream0TckTest.java | 40 ++ .../internal/jdk8/FlatMapStream1HTckTest.java | 40 ++ .../internal/jdk8/FlatMapStream1TckTest.java | 40 ++ .../internal/jdk8/FlatMapStream2HTckTest.java | 47 ++ .../internal/jdk8/FlatMapStream2TckTest.java | 47 ++ .../jdk8/FlowableBlockingStreamTest.java | 107 +++++ .../jdk8/FlowableFlatMapStreamTest.java | 451 ++++++++++++++++++ .../rxjava3/testsupport/TestHelper.java | 22 + 12 files changed, 1461 insertions(+) create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0HTckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0TckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1HTckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1TckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2HTckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2TckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableBlockingStreamTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStreamTest.java diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 4177a2d9e6..09cb859b83 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -18997,4 +18997,289 @@ public final CompletionStage singleOrErrorStage() { public final CompletionStage lastOrErrorStage() { return subscribeWith(new FlowableLastStageSubscriber<>(false, null)); } + + /** + * Creates a sequential {@link Stream} to consume or process this {@code Flowable} in a blocking manner via + * the Java {@code Stream} API. + *

+ * + *

+ * Cancellation of the upstream is done via {@link Stream#close()}, therefore, it is strongly recommended the + * consumption is performed within a try-with-resources construct: + *


+     * Flowable<Integer> source = Flowable.range(1, 10)
+     *        .subscribeOn(Schedulers.computation());
+     *
+     * try (Stream<Integer> stream = source.blockingStream()) {
+     *     stream.limit(3).forEach(System.out::println);
+     * }
+     * 
+ *
+ *
Backpressure:
+ *
The operator requests {@link #bufferSize()} amount upfront and 75% of it after each 75% of the amount received.
+ *
Scheduler:
+ *
{@code blockingStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @return the new Stream instance + * @since 3.0.0 + * @see #blockingStream(int) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final Stream blockingStream() { + return blockingStream(bufferSize()); + } + + /** + * Creates a sequential {@link Stream} to consume or process this {@code Flowable} in a blocking manner via + * the Java {@code Stream} API. + *

+ * + *

+ * Cancellation of the upstream is done via {@link Stream#close()}, therefore, it is strongly recommended the + * consumption is performed within a try-with-resources construct: + *


+     * Flowable<Integer> source = Flowable.range(1, 10)
+     *        .subscribeOn(Schedulers.computation());
+     *
+     * try (Stream<Integer> stream = source.blockingStream(4)) {
+     *     stream.limit(3).forEach(System.out::println);
+     * }
+     * 
+ *
+ *
Backpressure:
+ *
The operator requests the given {@code prefetch} amount upfront and 75% of it after each 75% of the amount received.
+ *
Scheduler:
+ *
{@code blockingStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param prefetch the number of items to request from the upstream to limit the number of + * in-flight items and item generation. + * @return the new Stream instance + * @since 3.0.0 + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final Stream blockingStream(int prefetch) { + Iterator iterator = blockingIterable(prefetch).iterator(); + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, 0), false) + .onClose(() -> ((Disposable)iterator).dispose()); + } + + /** + * Maps each upstream item into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #concatMapIterable(Function)}: + *


+     * source.concatMapIterable(v -> createStream(v)::iterator);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.concatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors the downstream backpressure and consumes the inner stream only on demand. The operator + * prefetches {@link #bufferSize} items of the upstream (then 75% of it after the 75% received) + * and caches them until they are ready to be mapped into {@code Stream}s + * after the current {@code Stream} has been consumed.
+ *
Scheduler:
+ *
{@code concatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @return the new Flowable instance + * @see #concatMap(Function) + * @see #concatMapIterable(Function) + * @see #concatMapStream(Function, int) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Flowable concatMapStream(@NonNull Function> mapper) { + return flatMapStream(mapper, bufferSize()); + } + + /** + * Maps each upstream item into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #concatMapIterable(Function, int)}: + *


+     * source.concatMapIterable(v -> createStream(v)::iterator, 32);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.concatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed(), 32);
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors the downstream backpressure and consumes the inner stream only on demand. The operator + * prefetches the given amount of upstream items and caches them until they are ready to be mapped into {@code Stream}s + * after the current {@code Stream} has been consumed.
+ *
Scheduler:
+ *
{@code concatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @param prefetch the number of upstream items to request upfront, then 75% of this amount after each 75% upstream items received + * @return the new Flowable instance + * @see #concatMap(Function, int) + * @see #concatMapIterable(Function, int) + * @see #flatMapStream(Function, int) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Flowable concatMapStream(@NonNull Function> mapper, int prefetch) { + Objects.requireNonNull(mapper, "mapper is null"); + ObjectHelper.verifyPositive(prefetch, "prefetch"); + return RxJavaPlugins.onAssembly(new FlowableFlatMapStream<>(this, mapper, prefetch)); + } + + /** + * Maps each upstream item into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flatMapIterable(Function)}: + *


+     * source.flatMapIterable(v -> createStream(v)::iterator);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors the downstream backpressure and consumes the inner stream only on demand. The operator + * prefetches {@link #bufferSize} items of the upstream (then 75% of it after the 75% received) + * and caches them until they are ready to be mapped into {@code Stream}s + * after the current {@code Stream} has been consumed.
+ *
Scheduler:
+ *
{@code flatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @return the new Flowable instance + * @see #flatMap(Function) + * @see #flatMapIterable(Function) + * @see #flatMapStream(Function, int) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Flowable flatMapStream(@NonNull Function> mapper) { + return flatMapStream(mapper, bufferSize()); + } + + /** + * Maps each upstream item into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flatMapIterable(Function, int)}: + *


+     * source.flatMapIterable(v -> createStream(v)::iterator, 32);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed(), 32);
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors the downstream backpressure and consumes the inner stream only on demand. The operator + * prefetches the given amount of upstream items and caches them until they are ready to be mapped into {@code Stream}s + * after the current {@code Stream} has been consumed.
+ *
Scheduler:
+ *
{@code flatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @param prefetch the number of upstream items to request upfront, then 75% of this amount after each 75% upstream items received + * @return the new Flowable instance + * @see #flatMap(Function, int) + * @see #flatMapIterable(Function, int) + * @see #concatMapStream(Function, int) + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Flowable flatMapStream(@NonNull Function> mapper, int prefetch) { + Objects.requireNonNull(mapper, "mapper is null"); + ObjectHelper.verifyPositive(prefetch, "prefetch"); + return RxJavaPlugins.onAssembly(new FlowableFlatMapStream<>(this, mapper, prefetch)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java new file mode 100644 index 0000000000..ad1b3ad140 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java @@ -0,0 +1,332 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; +import java.util.concurrent.atomic.*; +import java.util.stream.Stream; + +import org.reactivestreams.*; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.MissingBackpressureException; +import io.reactivex.rxjava3.functions.*; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; +import io.reactivex.rxjava3.internal.subscriptions.*; +import io.reactivex.rxjava3.internal.util.*; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Maps the upstream values onto {@link Stream}s and emits their items in order to the downstream. + * + * @param the upstream element type + * @param the inner {@code Stream} and result element type + * @since 3.0.0 + */ +public final class FlowableFlatMapStream extends Flowable { + + final Flowable source; + + final Function> mapper; + + final int prefetch; + + public FlowableFlatMapStream(Flowable source, Function> mapper, int prefetch) { + this.source = source; + this.mapper = mapper; + this.prefetch = prefetch; + } + + @Override + protected void subscribeActual(Subscriber s) { + if (source instanceof Supplier) { + Stream stream = null; + try { + @SuppressWarnings("unchecked") + T t = ((Supplier)source).get(); + if (t != null) { + stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream"); + } + } catch (Throwable ex) { + EmptySubscription.error(ex, s); + return; + } + + if (stream != null) { + FlowableFromStream.subscribeStream(s, stream); + } else { + EmptySubscription.complete(s); + } + } else { + source.subscribe(new FlatMapStreamSubscriber<>(s, mapper, prefetch)); + } + } + + static final class FlatMapStreamSubscriber extends AtomicInteger + implements FlowableSubscriber, Subscription { + + private static final long serialVersionUID = -5127032662980523968L; + + final Subscriber downstream; + + final Function> mapper; + + final int prefetch; + + final AtomicLong requested; + + SimpleQueue queue; + + Subscription upstream; + + Iterator currentIterator; + + AutoCloseable currentCloseable; + + volatile boolean cancelled; + + volatile boolean upstreamDone; + final AtomicThrowable error; + + long emitted; + + int consumed; + + int sourceMode; + + FlatMapStreamSubscriber(Subscriber downstream, Function> mapper, int prefetch) { + this.downstream = downstream; + this.mapper = mapper; + this.prefetch = prefetch; + this.requested = new AtomicLong(); + this.error = new AtomicThrowable(); + } + + @Override + public void onSubscribe(@NonNull Subscription s) { + if (SubscriptionHelper.validate(this.upstream, s)) { + this.upstream = s; + + if (s instanceof QueueSubscription) { + + @SuppressWarnings("unchecked") + QueueSubscription qs = (QueueSubscription)s; + + int m = qs.requestFusion(QueueFuseable.ANY | QueueFuseable.BOUNDARY); + if (m == QueueFuseable.SYNC) { + sourceMode = m; + queue = qs; + upstreamDone = true; + + downstream.onSubscribe(this); + return; + } + else if (m == QueueFuseable.ASYNC) { + sourceMode = m; + queue = qs; + + downstream.onSubscribe(this); + + s.request(prefetch); + return; + } + } + + queue = new SpscArrayQueue<>(prefetch); + + downstream.onSubscribe(this); + + s.request(prefetch); + } + } + + @Override + public void onNext(T t) { + if (sourceMode != QueueFuseable.ASYNC) { + if (!queue.offer(t)) { + upstream.cancel(); + onError(new MissingBackpressureException("Queue full?!")); + return; + } + } + drain(); + } + + @Override + public void onError(Throwable t) { + if (error.compareAndSet(null, t)) { + upstreamDone = true; + drain(); + } else { + RxJavaPlugins.onError(t); + } + } + + @Override + public void onComplete() { + upstreamDone = true; + drain(); + } + + @Override + public void request(long n) { + if (SubscriptionHelper.validate(n)) { + BackpressureHelper.add(requested, n); + drain(); + } + } + + @Override + public void cancel() { + cancelled = true; + upstream.cancel(); + drain(); + } + + void drain() { + if (getAndIncrement() != 0) { + return; + } + + int missed = 1; + + final Subscriber downstream = this.downstream; + final SimpleQueue queue = this.queue; + final AtomicThrowable error = this.error; + Iterator iterator = this.currentIterator; + long requested = this.requested.get(); + long emitted = this.emitted; + final int limit = prefetch - (prefetch >> 2); + boolean canRequest = sourceMode != QueueFuseable.SYNC; + + for (;;) { + if (cancelled) { + queue.clear(); + clearCurrentSuppressCloseError(); + } else { + boolean isDone = upstreamDone; + if (error.get() != null) { + downstream.onError(error.get()); + cancelled = true; + continue; + } + + if (iterator == null) { + T t; + + try { + t = queue.poll(); + } catch (Throwable ex) { + trySignalError(downstream, ex); + continue; + } + + boolean isEmpty = t == null; + + if (isDone && isEmpty) { + downstream.onComplete(); + cancelled = true; + } + else if (!isEmpty) { + if (canRequest && ++consumed == limit) { + consumed = 0; + upstream.request(limit); + } + + Stream stream; + try { + stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream"); + iterator = stream.iterator(); + + if (iterator.hasNext()) { + currentIterator = iterator; + currentCloseable = stream; + } else { + iterator = null; + } + } catch (Throwable ex) { + trySignalError(downstream, ex); + } + continue; + } + } + if (iterator != null && emitted != requested) { + R item; + + try { + item = Objects.requireNonNull(iterator.next(), "The Stream.Iterator returned a null value"); + } catch (Throwable ex) { + trySignalError(downstream, ex); + continue; + } + + if (!cancelled) { + downstream.onNext(item); + emitted++; + + if (!cancelled) { + try { + if (!iterator.hasNext()) { + iterator = null; + clearCurrentRethrowCloseError(); + } + } catch (Throwable ex) { + trySignalError(downstream, ex); + } + } + } + + continue; + } + } + + this.emitted = emitted; + missed = addAndGet(-missed); + if (missed == 0) { + break; + } + requested = this.requested.get(); + } + } + + void clearCurrentRethrowCloseError() throws Throwable { + currentIterator = null; + AutoCloseable ac = currentCloseable; + currentCloseable = null; + if (ac != null) { + ac.close(); + } + } + + void clearCurrentSuppressCloseError() { + try { + clearCurrentRethrowCloseError(); + } catch (Throwable ex) { + RxJavaPlugins.onError(ex); + } + } + + void trySignalError(Subscriber downstream, Throwable ex) { + if (error.compareAndSet(null, ex)) { + upstream.cancel(); + cancelled = true; + downstream.onError(ex); + } else { + RxJavaPlugins.onError(ex); + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java index 0f4e4b16ba..ecc18fb154 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java @@ -42,6 +42,16 @@ public FlowableFromStream(Stream stream) { @Override protected void subscribeActual(Subscriber s) { + subscribeStream(s, stream); + } + + /** + * Subscribes to the Stream by picking the normal or conditional stream Subscription implementation. + * @param the element type of the flow + * @param s the subscriber to drive + * @param stream the sequence to consume + */ + public static void subscribeStream(Subscriber s, Stream stream) { Iterator iterator; try { iterator = stream.iterator(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0HTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0HTckTest.java new file mode 100644 index 0000000000..57365f5857 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0HTckTest.java @@ -0,0 +1,40 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class FlatMapStream0HTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.just(1).hide().flatMapStream(v -> IntStream.range(0, (int)elements).boxed()) + ; + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Flowable.just(1).hide().flatMapStream(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0TckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0TckTest.java new file mode 100644 index 0000000000..232ded7321 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream0TckTest.java @@ -0,0 +1,40 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class FlatMapStream0TckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.just(1).flatMapStream(v -> IntStream.range(0, (int)elements).boxed()) + ; + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Flowable.just(1).flatMapStream(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1HTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1HTckTest.java new file mode 100644 index 0000000000..aa65a19ca7 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1HTckTest.java @@ -0,0 +1,40 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class FlatMapStream1HTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.range(1, (int)elements).hide().flatMapStream(v -> Stream.of(v)) + ; + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Flowable.just(1).hide().flatMapStream(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1TckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1TckTest.java new file mode 100644 index 0000000000..20e99b1ec1 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream1TckTest.java @@ -0,0 +1,40 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class FlatMapStream1TckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Flowable.range(1, (int)elements).flatMapStream(v -> Stream.of(v)) + ; + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Flowable.just(1).flatMapStream(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2HTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2HTckTest.java new file mode 100644 index 0000000000..0c06e4de40 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2HTckTest.java @@ -0,0 +1,47 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class FlatMapStream2HTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + if (elements % 2 == 0) { + return Flowable.range(0, (int)elements / 2).hide().flatMapStream(v -> Stream.of(v, v + 1)); + } + return + Flowable.range(-1, 1 + (int)elements / 2).hide().flatMapStream(v -> { + if (v != -1) { + return Stream.of(v, v + 1); + } + return Stream.of(v); + }); + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Flowable.just(1).hide().flatMapStream(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2TckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2TckTest.java new file mode 100644 index 0000000000..d799a720c9 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlatMapStream2TckTest.java @@ -0,0 +1,47 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.Flowable; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class FlatMapStream2TckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + if (elements % 2 == 0) { + return Flowable.range(0, (int)elements / 2).flatMapStream(v -> Stream.of(v, v + 1)); + } + return + Flowable.range(-1, 1 + (int)elements / 2).flatMapStream(v -> { + if (v != -1) { + return Stream.of(v, v + 1); + } + return Stream.of(v); + }); + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Flowable.just(1).flatMapStream(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableBlockingStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableBlockingStreamTest.java new file mode 100644 index 0000000000..2cf932f40e --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableBlockingStreamTest.java @@ -0,0 +1,107 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.List; +import java.util.stream.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.processors.UnicastProcessor; +import io.reactivex.rxjava3.schedulers.Schedulers; + +public class FlowableBlockingStreamTest extends RxJavaTest { + + @Test + public void empty() { + try (Stream stream = Flowable.empty().blockingStream()) { + assertEquals(0, stream.toArray().length); + } + } + + @Test + public void just() { + try (Stream stream = Flowable.just(1).blockingStream()) { + assertArrayEquals(new Integer[] { 1 }, stream.toArray(Integer[]::new)); + } + } + + @Test + public void range() { + try (Stream stream = Flowable.range(1, 5).blockingStream()) { + assertArrayEquals(new Integer[] { 1, 2, 3, 4, 5 }, stream.toArray(Integer[]::new)); + } + } + + @Test + public void rangeBackpressured() { + try (Stream stream = Flowable.range(1, 5).blockingStream(1)) { + assertArrayEquals(new Integer[] { 1, 2, 3, 4, 5 }, stream.toArray(Integer[]::new)); + } + } + + @Test + public void rangeAsyncBackpressured() { + try (Stream stream = Flowable.range(1, 1000).subscribeOn(Schedulers.computation()).blockingStream()) { + List list = stream.collect(Collectors.toList()); + + assertEquals(1000, list.size()); + for (int i = 1; i <= 1000; i++) { + assertEquals(i, list.get(i - 1).intValue()); + } + } + } + + @Test + public void rangeAsyncBackpressured1() { + try (Stream stream = Flowable.range(1, 1000).subscribeOn(Schedulers.computation()).blockingStream(1)) { + List list = stream.collect(Collectors.toList()); + + assertEquals(1000, list.size()); + for (int i = 1; i <= 1000; i++) { + assertEquals(i, list.get(i - 1).intValue()); + } + } + } + + @Test + public void error() { + try (Stream stream = Flowable.error(new TestException()).blockingStream()) { + stream.toArray(Integer[]::new); + fail("Should have thrown!"); + } catch (TestException expected) { + // expected + } + } + + @Test + public void close() { + UnicastProcessor up = UnicastProcessor.create(); + up.onNext(1); + up.onNext(2); + up.onNext(3); + up.onNext(4); + up.onNext(5); + + try (Stream stream = up.blockingStream()) { + assertArrayEquals(new Integer[] { 1, 2, 3 }, stream.limit(3).toArray(Integer[]::new)); + } + + assertFalse(up.hasSubscribers()); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStreamTest.java new file mode 100644 index 0000000000..548a66a709 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStreamTest.java @@ -0,0 +1,451 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.*; + +import org.junit.Test; +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.subscribers.TestSubscriber; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class FlowableFlatMapStreamTest extends RxJavaTest { + + @Test + public void empty() { + Flowable.empty() + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertResult(); + } + + @Test + public void emptyHidden() { + Flowable.empty() + .hide() + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertResult(); + } + + @Test + public void just() { + Flowable.just(1) + .flatMapStream(v -> Stream.of(v + 1, v + 2, v + 3, v + 4, v + 5)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void justHidden() { + Flowable.just(1).hide() + .flatMapStream(v -> Stream.of(v + 1, v + 2, v + 3, v + 4, v + 5)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void error() { + Flowable.error(new TestException()) + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertFailure(TestException.class); + } + + @Test + public void supplierFusedError() { + Flowable.fromCallable(() -> { throw new TestException(); }) + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertFailure(TestException.class); + } + + @Test + public void errorHidden() { + Flowable.error(new TestException()) + .hide() + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertFailure(TestException.class); + } + + @Test + public void range() { + Flowable.range(1, 5) + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31, 32, 33, 34, + 40, 41, 42, 43, 44, + 50, 51, 52, 53, 54 + ); + } + + @Test + public void rangeHidden() { + Flowable.range(1, 5) + .hide() + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31, 32, 33, 34, + 40, 41, 42, 43, 44, + 50, 51, 52, 53, 54 + ); + } + + @Test + public void rangeToEmpty() { + Flowable.range(1, 5) + .flatMapStream(v -> Stream.of()) + .test() + .assertResult(); + } + + @Test + public void rangeTake() { + Flowable.range(1, 5) + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .take(12) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31 + ); + } + + @Test + public void rangeTakeHidden() { + Flowable.range(1, 5) + .hide() + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .take(12) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31 + ); + } + + @Test + public void upstreamCancelled() { + PublishProcessor pp = PublishProcessor.create(); + + AtomicInteger calls = new AtomicInteger(); + + TestSubscriber ts = pp + .flatMapStream(v -> Stream.of(v + 1, v + 2).onClose(() -> calls.getAndIncrement())) + .test(1); + + assertTrue(pp.hasSubscribers()); + + pp.onNext(1); + + ts.assertValuesOnly(2); + ts.cancel(); + + assertFalse(pp.hasSubscribers()); + + assertEquals(1, calls.get()); + } + + @Test + public void upstreamCancelledCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + PublishProcessor pp = PublishProcessor.create(); + + TestSubscriber ts = pp + .flatMapStream(v -> Stream.of(v + 1, v + 2).onClose(() -> { throw new TestException(); })) + .test(1); + + assertTrue(pp.hasSubscribers()); + + pp.onNext(1); + + ts.assertValuesOnly(2); + ts.cancel(); + + assertFalse(pp.hasSubscribers()); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void crossMap() { + Flowable.range(1, 1000) + .flatMapStream(v -> IntStream.range(v * 1000, v * 1000 + 1000).boxed()) + .test() + .assertValueCount(1_000_000) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void crossMapHidden() { + Flowable.range(1, 1000) + .hide() + .flatMapStream(v -> IntStream.range(v * 1000, v * 1000 + 1000).boxed()) + .test() + .assertValueCount(1_000_000) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void crossMapBackpressured() { + for (int n = 1; n < 2048; n *= 2) { + Flowable.range(1, 1000) + .flatMapStream(v -> IntStream.range(v * 1000, v * 1000 + 1000).boxed()) + .rebatchRequests(n) + .test() + .withTag("rebatch: " + n) + .assertValueCount(1_000_000) + .assertNoErrors() + .assertComplete(); + } + } + + @Test + public void crossMapBackpressuredHidden() { + for (int n = 1; n < 2048; n *= 2) { + Flowable.range(1, 1000) + .hide() + .flatMapStream(v -> IntStream.range(v * 1000, v * 1000 + 1000).boxed()) + .rebatchRequests(n) + .test() + .withTag("rebatch: " + n) + .assertValueCount(1_000_000) + .assertNoErrors() + .assertComplete(); + } + } + + @Test + public void onSubscribe() { + TestHelper.checkDoubleOnSubscribeFlowable(f -> f.flatMapStream(v -> Stream.of(1, 2))); + } + + @Test + public void badRequest() { + TestHelper.assertBadRequestReported(UnicastProcessor.create().flatMapStream(v -> Stream.of(1, 2))); + } + + @Test + public void queueOverflow() throws Throwable { + TestHelper.withErrorTracking(errors -> { + new Flowable() { + @Override + protected void subscribeActual(Subscriber s) { + s.onSubscribe(new BooleanSubscription()); + s.onNext(1); + s.onNext(2); + s.onNext(3); + s.onError(new TestException()); + } + } + .flatMapStream(v -> Stream.of(1, 2), 1) + .test(0) + .assertFailure(MissingBackpressureException.class); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void mapperThrows() { + Flowable.just(1).hide() + .concatMapStream(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void mapperNull() { + Flowable.just(1).hide() + .concatMapStream(v -> null) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void streamNull() { + Flowable.just(1).hide() + .concatMapStream(v -> Stream.of(1, null)) + .test() + .assertFailure(NullPointerException.class, 1); + } + + @Test + public void hasNextThrows() { + Flowable.just(1).hide() + .concatMapStream(v -> Stream.generate(() -> { throw new TestException(); })) + .test() + .assertFailure(TestException.class); + } + + @Test + public void hasNextThrowsLater() { + AtomicInteger counter = new AtomicInteger(); + Flowable.just(1).hide() + .concatMapStream(v -> Stream.generate(() -> { + if (counter.getAndIncrement() == 0) { + return 1; + } + throw new TestException(); + })) + .test() + .assertFailure(TestException.class, 1); + } + + @Test + public void mapperThrowsWhenUpstreamErrors() throws Throwable { + TestHelper.withErrorTracking(errors -> { + PublishProcessor pp = PublishProcessor.create(); + + AtomicInteger counter = new AtomicInteger(); + + TestSubscriber ts = pp.hide() + .concatMapStream(v -> { + if (counter.getAndIncrement() == 0) { + return Stream.of(1, 2); + } + pp.onError(new IOException()); + throw new TestException(); + }) + .test(); + + pp.onNext(1); + pp.onNext(2); + + ts + .assertFailure(IOException.class, 1, 2); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void rangeBackpressured() { + Flowable.range(1, 5) + .hide() + .concatMapStream(v -> Stream.of(v), 1) + .test(0) + .assertEmpty() + .requestMore(5) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void cancelAfterIteratorNext() throws Exception { + TestSubscriber ts = new TestSubscriber<>(); + + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + ts.cancel(); + return 1; + } + }); + + Flowable.just(1) + .hide() + .concatMapStream(v -> stream) + .subscribe(ts); + + ts.assertEmpty(); + } + + @Test + public void asyncUpstreamFused() { + UnicastProcessor up = UnicastProcessor.create(); + + TestSubscriber ts = up.flatMapStream(v -> Stream.of(1, 2)) + .test(); + + assertTrue(up.hasSubscribers()); + + up.onNext(1); + + ts.assertValuesOnly(1, 2); + + up.onComplete(); + + ts.assertResult(1, 2); + } + + @Test + public void asyncUpstreamFusionBoundary() { + UnicastProcessor up = UnicastProcessor.create(); + + TestSubscriber ts = up + .map(v -> v + 1) + .flatMapStream(v -> Stream.of(1, 2)) + .test(); + + assertTrue(up.hasSubscribers()); + + up.onNext(1); + + ts.assertValuesOnly(1, 2); + + up.onComplete(); + + ts.assertResult(1, 2); + } + + @Test + public void fusedPollCrash() { + UnicastProcessor up = UnicastProcessor.create(); + + TestSubscriber ts = up + .map(v -> { throw new TestException(); }) + .compose(TestHelper.flowableStripBoundary()) + .flatMapStream(v -> Stream.of(1, 2)) + .test(); + + assertTrue(up.hasSubscribers()); + + up.onNext(1); + + assertFalse(up.hasSubscribers()); + + ts.assertFailure(TestException.class); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index 865b31a082..f31475ae58 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -2987,6 +2987,17 @@ public void cancel() { } } + /** + * Strips the {@link QueueFuseable#BOUNDARY} mode flag when the downstream calls {@link QueueSubscription#requestFusion(int)}. + *

+ * By default, many operators use {@link QueueFuseable#BOUNDARY} to indicate upstream side-effects + * should not leak over a fused boundary. However, some tests want to verify if {@link QueueSubscription#poll()} crashes + * are handled correctly and the most convenient way is to crash {@link Flowable#map} that won't fuse with {@code BOUNDARY} + * flag. This transformer strips this flag and thus allows the function of {@code map} to be executed as part of the + * {@code poll()} chain. + * @param the element type of the flow + * @return the new Transformer instance + */ public static FlowableTransformer flowableStripBoundary() { return new FlowableStripBoundary<>(null); } @@ -3092,6 +3103,17 @@ public boolean isDisposed() { } } + /** + * Strips the {@link QueueFuseable#BOUNDARY} mode flag when the downstream calls {@link QueueDisposable#requestFusion(int)}. + *

+ * By default, many operators use {@link QueueFuseable#BOUNDARY} to indicate upstream side-effects + * should not leak over a fused boundary. However, some tests want to verify if {@link QueueDisposable#poll()} crashes + * are handled correctly and the most convenient way is to crash {@link Observable#map} that won't fuse with {@code BOUNDARY} + * flag. This transformer strips this flag and thus allows the function of {@code map} to be executed as part of the + * {@code poll()} chain. + * @param the element type of the flow + * @return the new Transformer instance + */ public static ObservableTransformer observableStripBoundary() { return new ObservableStripBoundary<>(null); } From 166c529eb165b473c9bff11e8fedbde6c471b005 Mon Sep 17 00:00:00 2001 From: Jake Wharton Date: Thu, 19 Dec 2019 13:39:40 -0500 Subject: [PATCH 013/665] Migrate Disposables static factories to Disposable interface (#6781) --- .../io/reactivex/rxjava3/core/Scheduler.java | 4 +- .../rxjava3/disposables/Disposable.java | 123 +++++++++++++++ .../rxjava3/disposables/Disposables.java | 143 ------------------ .../rxjava3/disposables/SerialDisposable.java | 2 +- .../rxjava3/disposables/package-info.java | 2 +- .../completable/CompletableFromAction.java | 2 +- .../completable/CompletableFromCallable.java | 2 +- .../completable/CompletableFromRunnable.java | 2 +- .../completable/CompletableFromSupplier.java | 2 +- .../internal/operators/maybe/MaybeError.java | 4 +- .../operators/maybe/MaybeErrorCallable.java | 4 +- .../operators/maybe/MaybeFromAction.java | 2 +- .../operators/maybe/MaybeFromCallable.java | 2 +- .../operators/maybe/MaybeFromFuture.java | 2 +- .../operators/maybe/MaybeFromRunnable.java | 2 +- .../operators/maybe/MaybeFromSupplier.java | 2 +- .../internal/operators/maybe/MaybeJust.java | 4 +- .../operators/single/SingleFromCallable.java | 2 +- .../operators/single/SingleFromSupplier.java | 2 +- .../internal/operators/single/SingleJust.java | 4 +- .../schedulers/ImmediateThinScheduler.java | 2 +- .../internal/schedulers/SchedulerWhen.java | 16 +- .../schedulers/TrampolineScheduler.java | 2 +- .../rxjava3/schedulers/TestScheduler.java | 4 +- .../rxjava3/completable/CompletableTest.java | 2 +- .../disposables/CompositeDisposableTest.java | 102 ++++++------- ...sposablesTest.java => DisposableTest.java} | 39 +++-- .../disposables/FutureDisposableTest.java | 4 +- .../disposables/SequentialDisposableTest.java | 2 +- .../disposables/SerialDisposableTests.java | 4 +- .../ArrayCompositeDisposableTest.java | 16 +- .../disposables/DisposableHelperTest.java | 12 +- .../ListCompositeDisposableTest.java | 42 ++--- .../observers/BasicFuseableObserverTest.java | 4 +- .../observers/BlockingFirstObserverTest.java | 2 +- .../observers/BlockingMultiObserverTest.java | 2 +- .../observers/DeferredScalarObserverTest.java | 36 ++--- .../DisposableLambdaObserverTest.java | 4 +- .../observers/FutureObserverTest.java | 4 +- .../observers/LambdaObserverTest.java | 6 +- .../observers/QueueDrainObserverTest.java | 20 +-- .../completable/CompletableAmbTest.java | 4 +- .../completable/CompletableConcatTest.java | 6 +- .../completable/CompletableCreateTest.java | 12 +- .../completable/CompletableDetachTest.java | 6 +- .../completable/CompletableDoOnTest.java | 2 +- .../completable/CompletableMergeTest.java | 10 +- .../completable/CompletableTakeUntilTest.java | 10 +- .../completable/CompletableTimeoutTest.java | 2 +- .../CompletableToObservableTest.java | 2 +- .../completable/CompletableUnsafeTest.java | 4 +- .../completable/CompletableUsingTest.java | 4 +- .../flowable/FlowableConcatTest.java | 2 +- .../flowable/FlowableCreateTest.java | 16 +- .../flowable/FlowableDoOnUnsubscribeTest.java | 4 +- .../flowable/FlowableElementAtTest.java | 4 +- .../FlowableFlatMapCompletableTest.java | 4 +- .../flowable/FlowableFlatMapMaybeTest.java | 4 +- .../flowable/FlowableFlatMapSingleTest.java | 4 +- .../flowable/FlowableObserveOnTest.java | 2 +- .../flowable/FlowableRefCountTest.java | 12 +- .../operators/flowable/FlowableUsingTest.java | 2 +- .../flowable/NotificationLiteTest.java | 4 +- .../operators/maybe/MaybeAmbTest.java | 4 +- .../maybe/MaybeCallbackObserverTest.java | 8 +- .../operators/maybe/MaybeConcatArrayTest.java | 4 +- .../operators/maybe/MaybeCreateTest.java | 6 +- .../operators/maybe/MaybeDetachTest.java | 8 +- .../operators/maybe/MaybeDoOnEventTest.java | 2 +- .../operators/maybe/MaybeMergeArrayTest.java | 4 +- .../operators/maybe/MaybePeekTest.java | 6 +- .../operators/maybe/MaybeUsingTest.java | 4 +- .../mixed/FlowableConcatMapMaybeTest.java | 4 +- .../mixed/FlowableConcatMapSingleTest.java | 4 +- .../mixed/FlowableSwitchMapMaybeTest.java | 4 +- .../mixed/FlowableSwitchMapSingleTest.java | 4 +- .../mixed/ObservableConcatMapMaybeTest.java | 8 +- .../mixed/ObservableConcatMapSingleTest.java | 8 +- .../ObservableSwitchMapCompletableTest.java | 4 +- .../mixed/ObservableSwitchMapMaybeTest.java | 8 +- .../mixed/ObservableSwitchMapSingleTest.java | 8 +- .../BlockingObservableNextTest.java | 2 +- .../BlockingObservableToFutureTest.java | 4 +- .../BlockingObservableToIteratorTest.java | 2 +- .../internal/operators/observable/Burst.java | 4 +- .../observable/ObservableAllTest.java | 4 +- .../observable/ObservableAnyTest.java | 6 +- .../observable/ObservableBlockingTest.java | 4 +- .../observable/ObservableBufferTest.java | 36 ++--- .../observable/ObservableCacheTest.java | 6 +- .../ObservableConcatMapCompletableTest.java | 4 +- .../ObservableConcatMapSchedulerTest.java | 6 +- .../observable/ObservableConcatMapTest.java | 6 +- .../observable/ObservableConcatTest.java | 8 +- .../ObservableConcatWithCompletableTest.java | 6 +- .../ObservableConcatWithMaybeTest.java | 10 +- .../ObservableConcatWithSingleTest.java | 10 +- .../observable/ObservableCreateTest.java | 20 +-- .../observable/ObservableDebounceTest.java | 16 +- .../ObservableDematerializeTest.java | 6 +- .../observable/ObservableDistinctTest.java | 2 +- .../ObservableDistinctUntilChangedTest.java | 4 +- .../observable/ObservableDoOnEachTest.java | 18 +-- .../ObservableDoOnSubscribeTest.java | 4 +- .../observable/ObservableElementAtTest.java | 8 +- .../ObservableFlatMapCompletableTest.java | 4 +- .../ObservableFlatMapMaybeTest.java | 6 +- .../ObservableFlatMapSingleTest.java | 6 +- .../observable/ObservableFlatMapTest.java | 2 +- .../observable/ObservableGroupByTest.java | 16 +- .../observable/ObservableJoinTest.java | 6 +- .../ObservableMapNotificationTest.java | 4 +- .../observable/ObservableMaterializeTest.java | 4 +- .../ObservableMergeDelayErrorTest.java | 16 +- .../ObservableMergeMaxConcurrentTest.java | 4 +- .../observable/ObservableMergeTest.java | 18 +-- .../ObservableMergeWithCompletableTest.java | 2 +- .../ObservableMergeWithMaybeTest.java | 4 +- .../ObservableMergeWithSingleTest.java | 4 +- .../observable/ObservableObserveOnTest.java | 4 +- .../ObservableOnErrorResumeNextTest.java | 6 +- .../ObservableOnErrorReturnTest.java | 4 +- .../observable/ObservablePublishTest.java | 8 +- .../observable/ObservableReduceTest.java | 6 +- .../observable/ObservableRefCountTest.java | 30 ++-- .../observable/ObservableRepeatTest.java | 2 +- .../ObservableReplayEagerTruncateTest.java | 8 +- .../observable/ObservableReplayTest.java | 8 +- .../ObservableResourceWrapperTest.java | 6 +- .../observable/ObservableRetryTest.java | 16 +- .../ObservableRetryWithPredicateTest.java | 8 +- .../observable/ObservableSampleTest.java | 2 +- .../observable/ObservableScanTest.java | 6 +- .../observable/ObservableSerializeTest.java | 6 +- .../observable/ObservableSubscribeOnTest.java | 6 +- .../ObservableSwitchIfEmptyTest.java | 4 +- .../observable/ObservableSwitchTest.java | 56 +++---- .../observable/ObservableTakeTest.java | 8 +- .../ObservableTakeUntilPredicateTest.java | 4 +- .../observable/ObservableTakeWhileTest.java | 4 +- .../ObservableThrottleFirstTest.java | 8 +- .../observable/ObservableTimeoutTests.java | 6 +- .../ObservableTimeoutWithSelectorTest.java | 18 +-- .../observable/ObservableTimerTest.java | 4 +- .../ObservableUnsubscribeOnTest.java | 2 +- .../observable/ObservableUsingTest.java | 6 +- .../ObservableWindowWithObservableTest.java | 18 +-- .../ObservableWindowWithSizeTest.java | 2 +- ...vableWindowWithStartEndObservableTest.java | 10 +- .../ObservableWindowWithTimeTest.java | 6 +- .../ObservableWithLatestFromTest.java | 4 +- .../observable/ObservableZipIterableTest.java | 4 +- .../observable/ObservableZipTest.java | 6 +- .../operators/single/SingleCreateTest.java | 12 +- .../operators/single/SingleDelayTest.java | 4 +- .../operators/single/SingleDetachTest.java | 6 +- .../operators/single/SingleDoOnTest.java | 2 +- .../operators/single/SingleMiscTest.java | 4 +- .../operators/single/SingleUsingTest.java | 26 ++-- .../schedulers/DisposeOnCancelTest.java | 2 +- .../schedulers/SchedulerWhenTest.java | 8 +- .../schedulers/SingleSchedulerTest.java | 12 +- .../subscribers/EmptyComponentTest.java | 2 +- .../subscribers/QueueDrainSubscriberTest.java | 24 +-- .../SubscriberResourceWrapperTest.java | 6 +- .../internal/util/AtomicThrowableTest.java | 32 ++-- .../internal/util/BlockingHelperTest.java | 4 +- .../internal/util/EndConsumerHelperTest.java | 38 ++--- .../util/HalfSerializerObserverTest.java | 12 +- .../internal/util/NotificationLiteTest.java | 2 +- .../internal/util/QueueDrainHelperTest.java | 18 +-- .../rxjava3/maybe/MaybeCreateTest.java | 10 +- .../io/reactivex/rxjava3/maybe/MaybeTest.java | 10 +- .../rxjava3/observable/ObservableTest.java | 8 +- .../DisposableCompletableObserverTest.java | 6 +- .../DisposableMaybeObserverTest.java | 6 +- .../observers/DisposableObserverTest.java | 6 +- .../DisposableSingleObserverTest.java | 6 +- .../ResourceCompletableObserverTest.java | 12 +- .../observers/ResourceMaybeObserverTest.java | 14 +- .../observers/ResourceObserverTest.java | 12 +- .../observers/ResourceSingleObserverTest.java | 12 +- .../rxjava3/observers/SafeObserverTest.java | 10 +- .../observers/SerializedObserverTest.java | 32 ++-- .../rxjava3/observers/TestObserverTest.java | 52 +++---- .../rxjava3/plugins/RxJavaPluginsTest.java | 2 +- .../schedulers/CachedThreadSchedulerTest.java | 6 +- .../schedulers/ComputationSchedulerTests.java | 14 +- .../schedulers/NewThreadSchedulerTest.java | 6 +- .../reactivex/rxjava3/single/SingleTest.java | 16 +- .../rxjava3/subjects/AsyncSubjectTest.java | 4 +- .../rxjava3/subjects/BehaviorSubjectTest.java | 4 +- .../subjects/CompletableSubjectTest.java | 4 +- .../rxjava3/subjects/MaybeSubjectTest.java | 4 +- .../ReplaySubjectBoundedConcurrencyTest.java | 6 +- .../ReplaySubjectConcurrencyTest.java | 6 +- .../rxjava3/subjects/ReplaySubjectTest.java | 2 +- .../subjects/SerializedSubjectTest.java | 10 +- .../rxjava3/subjects/SingleSubjectTest.java | 4 +- .../rxjava3/subjects/UnicastSubjectTest.java | 4 +- .../subscribers/ResourceSubscriberTest.java | 6 +- .../rxjava3/testsupport/TestHelper.java | 94 ++++++------ .../testsupport/TestObserverExTest.java | 52 +++---- .../testsupport/TimesteppingScheduler.java | 2 +- 204 files changed, 1037 insertions(+), 1062 deletions(-) delete mode 100644 src/main/java/io/reactivex/rxjava3/disposables/Disposables.java rename src/test/java/io/reactivex/rxjava3/disposables/{DisposablesTest.java => DisposableTest.java} (86%) diff --git a/src/main/java/io/reactivex/rxjava3/core/Scheduler.java b/src/main/java/io/reactivex/rxjava3/core/Scheduler.java index a300cb02ce..80eda093c4 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Scheduler.java +++ b/src/main/java/io/reactivex/rxjava3/core/Scheduler.java @@ -81,7 +81,7 @@ * underlying task-execution scheme supports stopping and restarting itself. *

* If the {@code Scheduler} is shut down or a {@code Worker} is disposed, the {@code schedule} methods - * should return the {@link io.reactivex.rxjava3.disposables.Disposables#disposed()} singleton instance indicating the shut down/disposed + * should return the {@link Disposable#disposed()} singleton instance indicating the shut down/disposed * state to the caller. Since the shutdown or dispose can happen from any thread, the {@code schedule} implementations * should make best effort to cancel tasks immediately after those tasks have been submitted to the * underlying task-execution scheme if the shutdown/dispose was detected after this submission. @@ -349,7 +349,7 @@ public S when(@NonNull Function * If the {@code Worker} is disposed, the {@code schedule} methods - * should return the {@link io.reactivex.rxjava3.disposables.Disposables#disposed()} singleton instance indicating the disposed + * should return the {@link Disposable#disposed()} singleton instance indicating the disposed * state to the caller. Since the {@link #dispose()} call can happen on any thread, the {@code schedule} implementations * should make best effort to cancel tasks immediately after those tasks have been submitted to the * underlying task-execution scheme if the dispose was detected after this submission. diff --git a/src/main/java/io/reactivex/rxjava3/disposables/Disposable.java b/src/main/java/io/reactivex/rxjava3/disposables/Disposable.java index 2fab6e83ed..0b77c28c84 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/Disposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/Disposable.java @@ -12,6 +12,15 @@ */ package io.reactivex.rxjava3.disposables; +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.functions.Action; +import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; +import io.reactivex.rxjava3.internal.functions.Functions; +import org.reactivestreams.Subscription; + +import java.util.Objects; +import java.util.concurrent.Future; + /** * Represents a disposable resource. */ @@ -26,4 +35,118 @@ public interface Disposable { * @return true if this resource has been disposed */ boolean isDisposed(); + + /** + * Construct a {@code Disposable} by wrapping a {@link Runnable} that is + * executed exactly once when the {@code Disposable} is disposed. + * @param run the Runnable to wrap + * @return the new Disposable instance + * @since 3.0.0 + */ + @NonNull + static Disposable fromRunnable(@NonNull Runnable run) { + Objects.requireNonNull(run, "run is null"); + return new RunnableDisposable(run); + } + + /** + * Construct a {@code Disposable} by wrapping a {@link Action} that is + * executed exactly once when the {@code Disposable} is disposed. + * @param run the Action to wrap + * @return the new Disposable instance + * @since 3.0.0 + */ + @NonNull + static Disposable fromAction(@NonNull Action run) { + Objects.requireNonNull(run, "run is null"); + return new ActionDisposable(run); + } + + /** + * Construct a {@code Disposable} by wrapping a {@link Future} that is + * cancelled exactly once when the {@code Disposable} is disposed. + *

+ * The {@code Future} is cancelled with {@code mayInterruptIfRunning == true}. + * @param future the Future to wrap + * @return the new Disposable instance + * @see #fromFuture(Future, boolean) + * @since 3.0.0 + */ + @NonNull + static Disposable fromFuture(@NonNull Future future) { + Objects.requireNonNull(future, "future is null"); + return fromFuture(future, true); + } + + /** + * Construct a {@code Disposable} by wrapping a {@link Future} that is + * cancelled exactly once when the {@code Disposable} is disposed. + * @param future the Future to wrap + * @param allowInterrupt if true, the future cancel happens via {@code Future.cancel(true)} + * @return the new Disposable instance + * @since 3.0.0 + */ + @NonNull + static Disposable fromFuture(@NonNull Future future, boolean allowInterrupt) { + Objects.requireNonNull(future, "future is null"); + return new FutureDisposable(future, allowInterrupt); + } + + /** + * Construct a {@code Disposable} by wrapping a {@link Subscription} that is + * cancelled exactly once when the {@code Disposable} is disposed. + * @param subscription the Runnable to wrap + * @return the new Disposable instance + * @since 3.0.0 + */ + @NonNull + static Disposable fromSubscription(@NonNull Subscription subscription) { + Objects.requireNonNull(subscription, "subscription is null"); + return new SubscriptionDisposable(subscription); + } + + /** + * Construct a {@code Disposable} by wrapping an {@link AutoCloseable} that is + * closed exactly once when the {@code Disposable} is disposed. + * @param autoCloseable the AutoCloseable to wrap + * @return the new Disposable instance + * @since 3.0.0 + */ + @NonNull + static Disposable fromAutoCloseable(@NonNull AutoCloseable autoCloseable) { + Objects.requireNonNull(autoCloseable, "autoCloseable is null"); + return new AutoCloseableDisposable(autoCloseable); + } + + /** + * Construct an {@link AutoCloseable} by wrapping a {@code Disposable} that is + * disposed when the returned {@code AutoCloseable} is closed. + * @param disposable the Disposable instance + * @return the new AutoCloseable instance + * @since 3.0.0 + */ + @NonNull + static AutoCloseable toAutoCloseable(@NonNull Disposable disposable) { + return disposable::dispose; + } + + /** + * Returns a new, non-disposed {@code Disposable} instance. + * @return a new, non-disposed {@code Disposable} instance + * @since 3.0.0 + */ + @NonNull + static Disposable empty() { + return fromRunnable(Functions.EMPTY_RUNNABLE); + } + + /** + * Returns a shared, disposed {@code Disposable} instance. + * @return a shared, disposed {@code Disposable} instance + * @since 3.0.0 + */ + @NonNull + static Disposable disposed() { + return EmptyDisposable.INSTANCE; + } } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java b/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java deleted file mode 100644 index 39b1b76b25..0000000000 --- a/src/main/java/io/reactivex/rxjava3/disposables/Disposables.java +++ /dev/null @@ -1,143 +0,0 @@ -/** - * Copyright (c) 2016-present, RxJava Contributors. - * - * 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 io.reactivex.rxjava3.disposables; - -import java.util.Objects; -import java.util.concurrent.Future; - -import org.reactivestreams.Subscription; - -import io.reactivex.rxjava3.annotations.NonNull; -import io.reactivex.rxjava3.functions.Action; -import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.*; - -/** - * Utility class to help create disposables by wrapping - * other types. - * @since 2.0 - */ -public final class Disposables { - /** Utility class. */ - private Disposables() { - throw new IllegalStateException("No instances!"); - } - - /** - * Construct a {@code Disposable} by wrapping a {@link Runnable} that is - * executed exactly once when the {@code Disposable} is disposed. - * @param run the Runnable to wrap - * @return the new Disposable instance - */ - @NonNull - public static Disposable fromRunnable(@NonNull Runnable run) { - Objects.requireNonNull(run, "run is null"); - return new RunnableDisposable(run); - } - - /** - * Construct a {@code Disposable} by wrapping a {@link Action} that is - * executed exactly once when the {@code Disposable} is disposed. - * @param run the Action to wrap - * @return the new Disposable instance - */ - @NonNull - public static Disposable fromAction(@NonNull Action run) { - Objects.requireNonNull(run, "run is null"); - return new ActionDisposable(run); - } - - /** - * Construct a {@code Disposable} by wrapping a {@link Future} that is - * cancelled exactly once when the {@code Disposable} is disposed. - *

- * The {@code Future} is cancelled with {@code mayInterruptIfRunning == true}. - * @param future the Future to wrap - * @return the new Disposable instance - * @see #fromFuture(Future, boolean) - */ - @NonNull - public static Disposable fromFuture(@NonNull Future future) { - Objects.requireNonNull(future, "future is null"); - return fromFuture(future, true); - } - - /** - * Construct a {@code Disposable} by wrapping a {@link Future} that is - * cancelled exactly once when the {@code Disposable} is disposed. - * @param future the Future to wrap - * @param allowInterrupt if true, the future cancel happens via {@code Future.cancel(true)} - * @return the new Disposable instance - */ - @NonNull - public static Disposable fromFuture(@NonNull Future future, boolean allowInterrupt) { - Objects.requireNonNull(future, "future is null"); - return new FutureDisposable(future, allowInterrupt); - } - - /** - * Construct a {@code Disposable} by wrapping a {@link Subscription} that is - * cancelled exactly once when the {@code Disposable} is disposed. - * @param subscription the Runnable to wrap - * @return the new Disposable instance - */ - @NonNull - public static Disposable fromSubscription(@NonNull Subscription subscription) { - Objects.requireNonNull(subscription, "subscription is null"); - return new SubscriptionDisposable(subscription); - } - - /** - * Construct a {@code Disposable} by wrapping an {@link AutoCloseable} that is - * closed exactly once when the {@code Disposable} is disposed. - * @param autoCloseable the AutoCloseable to wrap - * @return the new Disposable instance - * @since 3.0.0 - */ - @NonNull - public static Disposable fromAutoCloseable(@NonNull AutoCloseable autoCloseable) { - Objects.requireNonNull(autoCloseable, "autoCloseable is null"); - return new AutoCloseableDisposable(autoCloseable); - } - - /** - * Construct an {@link AutoCloseable} by wrapping a {@code Disposable} that is - * disposed when the returned {@code AutoCloseable} is closed. - * @param disposable the Disposable instance - * @return the new AutoCloseable instance - * @since 3.0.0 - */ - @NonNull - public static AutoCloseable toAutoCloseable(@NonNull Disposable disposable) { - return disposable::dispose; - } - - /** - * Returns a new, non-disposed {@code Disposable} instance. - * @return a new, non-disposed {@code Disposable} instance - */ - @NonNull - public static Disposable empty() { - return fromRunnable(Functions.EMPTY_RUNNABLE); - } - - /** - * Returns a shared, disposed {@code Disposable} instance. - * @return a shared, disposed {@code Disposable} instance - */ - @NonNull - public static Disposable disposed() { - return EmptyDisposable.INSTANCE; - } -} diff --git a/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java index 187a94b1e5..d2c7ddee4b 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java @@ -71,7 +71,7 @@ public boolean replace(@Nullable Disposable next) { public Disposable get() { Disposable d = resource.get(); if (d == DisposableHelper.DISPOSED) { - return Disposables.disposed(); + return Disposable.disposed(); } return d; } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/package-info.java b/src/main/java/io/reactivex/rxjava3/disposables/package-info.java index 21b276e132..29812d0ab7 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/package-info.java @@ -17,6 +17,6 @@ /** * Default implementations for {@link io.reactivex.rxjava3.disposables.Disposable Disposable}-based resource management * ({@code Disposable} container types) and utility classes to construct - * {@link io.reactivex.rxjava3.disposables.Disposables Disposables} from callbacks and other types. + * {@link io.reactivex.rxjava3.disposables.Disposable Disposables} from callbacks and other types. */ package io.reactivex.rxjava3.disposables; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromAction.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromAction.java index 5ce4d43547..4cd7fb479c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromAction.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromAction.java @@ -29,7 +29,7 @@ public CompletableFromAction(Action run) { @Override protected void subscribeActual(CompletableObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); try { run.run(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallable.java index ac2c94c921..57764950df 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallable.java @@ -30,7 +30,7 @@ public CompletableFromCallable(Callable callable) { @Override protected void subscribeActual(CompletableObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); try { callable.call(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromRunnable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromRunnable.java index 3dc275e0e1..e94d650310 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromRunnable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromRunnable.java @@ -28,7 +28,7 @@ public CompletableFromRunnable(Runnable runnable) { @Override protected void subscribeActual(CompletableObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); try { runnable.run(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplier.java index 5c6054306a..e3e29188a0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplier.java @@ -33,7 +33,7 @@ public CompletableFromSupplier(Supplier supplier) { @Override protected void subscribeActual(CompletableObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); try { supplier.get(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeError.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeError.java index a850a27b6d..fc2f50066e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeError.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeError.java @@ -14,7 +14,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; import io.reactivex.rxjava3.core.*; -import io.reactivex.rxjava3.disposables.Disposables; +import io.reactivex.rxjava3.disposables.Disposable; /** * Signals a constant Throwable. @@ -31,7 +31,7 @@ public MaybeError(Throwable error) { @Override protected void subscribeActual(MaybeObserver observer) { - observer.onSubscribe(Disposables.disposed()); + observer.onSubscribe(Disposable.disposed()); observer.onError(error); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeErrorCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeErrorCallable.java index f3b631ba36..20bb1e03a8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeErrorCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeErrorCallable.java @@ -14,7 +14,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; import io.reactivex.rxjava3.core.*; -import io.reactivex.rxjava3.disposables.Disposables; +import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Supplier; import io.reactivex.rxjava3.internal.util.ExceptionHelper; @@ -34,7 +34,7 @@ public MaybeErrorCallable(Supplier errorSupplier) { @Override protected void subscribeActual(MaybeObserver observer) { - observer.onSubscribe(Disposables.disposed()); + observer.onSubscribe(Disposable.disposed()); Throwable ex; try { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromAction.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromAction.java index 8c29290deb..5bdf86f904 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromAction.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromAction.java @@ -34,7 +34,7 @@ public MaybeFromAction(Action action) { @Override protected void subscribeActual(MaybeObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); if (!d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallable.java index 4d218eb1ad..088f2e14f0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallable.java @@ -36,7 +36,7 @@ public MaybeFromCallable(Callable callable) { @Override protected void subscribeActual(MaybeObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); if (!d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java index db6c87518e..5d4dbc0f0b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java @@ -41,7 +41,7 @@ public MaybeFromFuture(Future future, long timeout, TimeUnit unit) @Override protected void subscribeActual(MaybeObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); if (!d.isDisposed()) { T v; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java index 672bd4edf5..2b7b658390 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java @@ -34,7 +34,7 @@ public MaybeFromRunnable(Runnable runnable) { @Override protected void subscribeActual(MaybeObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); if (!d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplier.java index 52aba416d9..38f0afdb62 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplier.java @@ -35,7 +35,7 @@ public MaybeFromSupplier(Supplier supplier) { @Override protected void subscribeActual(MaybeObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); if (!d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeJust.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeJust.java index a2c70740a6..54dacb3c7d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeJust.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeJust.java @@ -14,7 +14,7 @@ package io.reactivex.rxjava3.internal.operators.maybe; import io.reactivex.rxjava3.core.*; -import io.reactivex.rxjava3.disposables.Disposables; +import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.fuseable.ScalarSupplier; /** @@ -32,7 +32,7 @@ public MaybeJust(T value) { @Override protected void subscribeActual(MaybeObserver observer) { - observer.onSubscribe(Disposables.disposed()); + observer.onSubscribe(Disposable.disposed()); observer.onSuccess(value); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java index 5300d968b0..a78dde3e36 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallable.java @@ -31,7 +31,7 @@ public SingleFromCallable(Callable callable) { @Override protected void subscribeActual(SingleObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); if (d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java index 88dcd7dc94..e6fd32603f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplier.java @@ -36,7 +36,7 @@ public SingleFromSupplier(Supplier supplier) { @Override protected void subscribeActual(SingleObserver observer) { - Disposable d = Disposables.empty(); + Disposable d = Disposable.empty(); observer.onSubscribe(d); if (d.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleJust.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleJust.java index 8495f931db..4fd2387f96 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleJust.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleJust.java @@ -14,7 +14,7 @@ package io.reactivex.rxjava3.internal.operators.single; import io.reactivex.rxjava3.core.*; -import io.reactivex.rxjava3.disposables.Disposables; +import io.reactivex.rxjava3.disposables.Disposable; public final class SingleJust extends Single { @@ -26,7 +26,7 @@ public SingleJust(T value) { @Override protected void subscribeActual(SingleObserver observer) { - observer.onSubscribe(Disposables.disposed()); + observer.onSubscribe(Disposable.disposed()); observer.onSuccess(value); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ImmediateThinScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ImmediateThinScheduler.java index b1e47267b9..21fbbead83 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ImmediateThinScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ImmediateThinScheduler.java @@ -38,7 +38,7 @@ public final class ImmediateThinScheduler extends Scheduler { static final Disposable DISPOSED; static { - DISPOSED = Disposables.empty(); + DISPOSED = Disposable.empty(); DISPOSED.dispose(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhen.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhen.java index 5a15e61223..ddff75388b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhen.java @@ -1,12 +1,12 @@ /** * Copyright (c) 2016-present, RxJava Contributors. - * + * * 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. @@ -51,7 +51,7 @@ *

* Limit the amount concurrency two at a time without creating a new fix size * thread pool: - * + * *

  * Scheduler limitScheduler = Schedulers.computation().when(workers -> {
  *  // use merge max concurrent to limit the number of concurrent
@@ -69,7 +69,7 @@
  * {@link Flowable#zip(org.reactivestreams.Publisher, org.reactivestreams.Publisher, io.reactivex.rxjava3.functions.BiFunction)} where
  * subscribing to the first {@link Observable} could deadlock the subscription
  * to the second.
- * 
+ *
  * 
  * Scheduler limitScheduler = Schedulers.computation().when(workers -> {
  *  // use merge max concurrent to limit the number of concurrent
@@ -77,12 +77,12 @@
  *  return Completable.merge(Observable.merge(workers, 2));
  * });
  * 
- * + * * Slowing down the rate to no more than than 1 a second. This suffers from the * same problem as the one above I could find an {@link Observable} operator * that limits the rate without dropping the values (aka leaky bucket * algorithm). - * + * *
  * Scheduler slowScheduler = Schedulers.computation().when(workers -> {
  *  // use concatenate to make each worker happen one at a time.
@@ -145,7 +145,7 @@ public Worker createWorker() {
 
     static final Disposable SUBSCRIBED = new SubscribedDisposable();
 
-    static final Disposable DISPOSED = Disposables.disposed();
+    static final Disposable DISPOSED = Disposable.disposed();
 
     @SuppressWarnings("serial")
     abstract static class ScheduledAction extends AtomicReference implements Disposable {
diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java
index 71c054258b..b8a164f9e1 100644
--- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java
+++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java
@@ -120,7 +120,7 @@ Disposable enqueue(Runnable action, long execTime) {
                 return EmptyDisposable.INSTANCE;
             } else {
                 // queue wasn't empty, a parent is already processing so we just add to the end of the queue
-                return Disposables.fromRunnable(new AppendToQueueTask(timedRunnable));
+                return Disposable.fromRunnable(new AppendToQueueTask(timedRunnable));
             }
         }
 
diff --git a/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java b/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java
index 8b70760236..d9247acee2 100644
--- a/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java
+++ b/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java
@@ -166,7 +166,7 @@ public Disposable schedule(@NonNull Runnable run, long delayTime, @NonNull TimeU
             final TimedRunnable timedAction = new TimedRunnable(this, time + unit.toNanos(delayTime), run, counter++);
             queue.add(timedAction);
 
-            return Disposables.fromRunnable(new QueueRemove(timedAction));
+            return Disposable.fromRunnable(new QueueRemove(timedAction));
         }
 
         @NonNull
@@ -177,7 +177,7 @@ public Disposable schedule(@NonNull Runnable run) {
             }
             final TimedRunnable timedAction = new TimedRunnable(this, 0, run, counter++);
             queue.add(timedAction);
-            return Disposables.fromRunnable(new QueueRemove(timedAction));
+            return Disposable.fromRunnable(new QueueRemove(timedAction));
         }
 
         @Override
diff --git a/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java b/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java
index ebe1303bd2..e18d884f54 100644
--- a/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java
@@ -4257,7 +4257,7 @@ public void andThenError() {
         Completable.unsafeCreate(new CompletableSource() {
                 @Override
                 public void subscribe(CompletableObserver co) {
-                    co.onSubscribe(Disposables.empty());
+                    co.onSubscribe(Disposable.empty());
                     co.onError(e);
                 }
             })
diff --git a/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java
index 2d6fec394b..b219e8f9d7 100644
--- a/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java
@@ -33,7 +33,7 @@ public class CompositeDisposableTest extends RxJavaTest {
     public void success() {
         final AtomicInteger counter = new AtomicInteger();
         CompositeDisposable cd = new CompositeDisposable();
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -42,7 +42,7 @@ public void run() {
 
         }));
 
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -63,7 +63,7 @@ public void shouldUnsubscribeAll() throws InterruptedException {
         final int count = 10;
         final CountDownLatch start = new CountDownLatch(1);
         for (int i = 0; i < count; i++) {
-            cd.add(Disposables.fromRunnable(new Runnable() {
+            cd.add(Disposable.fromRunnable(new Runnable() {
 
                 @Override
                 public void run() {
@@ -101,7 +101,7 @@ public void run() {
     public void exception() {
         final AtomicInteger counter = new AtomicInteger();
         CompositeDisposable cd = new CompositeDisposable();
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -110,7 +110,7 @@ public void run() {
 
         }));
 
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -135,7 +135,7 @@ public void run() {
     public void compositeException() {
         final AtomicInteger counter = new AtomicInteger();
         CompositeDisposable cd = new CompositeDisposable();
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -144,7 +144,7 @@ public void run() {
 
         }));
 
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -152,7 +152,7 @@ public void run() {
             }
         }));
 
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -175,8 +175,8 @@ public void run() {
 
     @Test
     public void removeUnsubscribes() {
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         CompositeDisposable cd = new CompositeDisposable();
         cd.add(d1);
@@ -190,8 +190,8 @@ public void removeUnsubscribes() {
 
     @Test
     public void clear() {
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         CompositeDisposable cd = new CompositeDisposable();
         cd.add(d1);
@@ -206,7 +206,7 @@ public void clear() {
         assertTrue(d2.isDisposed());
         assertFalse(cd.isDisposed());
 
-        Disposable d3 = Disposables.empty();
+        Disposable d3 = Disposable.empty();
 
         cd.add(d3);
         cd.dispose();
@@ -219,7 +219,7 @@ public void clear() {
     public void unsubscribeIdempotence() {
         final AtomicInteger counter = new AtomicInteger();
         CompositeDisposable cd = new CompositeDisposable();
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -244,7 +244,7 @@ public void unsubscribeIdempotenceConcurrently()
 
         final int count = 10;
         final CountDownLatch start = new CountDownLatch(1);
-        cd.add(Disposables.fromRunnable(new Runnable() {
+        cd.add(Disposable.fromRunnable(new Runnable() {
 
             @Override
             public void run() {
@@ -301,8 +301,8 @@ public void addingNullDisposableIllegal() {
 
     @Test
     public void initializeVarargs() {
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         CompositeDisposable cd = new CompositeDisposable(d1, d2);
 
@@ -315,8 +315,8 @@ public void initializeVarargs() {
         assertTrue(d1.isDisposed());
         assertTrue(d2.isDisposed());
 
-        Disposable d3 = Disposables.empty();
-        Disposable d4 = Disposables.empty();
+        Disposable d3 = Disposable.empty();
+        Disposable d4 = Disposable.empty();
 
         cd = new CompositeDisposable(d3, d4);
 
@@ -330,8 +330,8 @@ public void initializeVarargs() {
 
     @Test
     public void initializeIterable() {
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         CompositeDisposable cd = new CompositeDisposable(Arrays.asList(d1, d2));
 
@@ -344,8 +344,8 @@ public void initializeIterable() {
         assertTrue(d1.isDisposed());
         assertTrue(d2.isDisposed());
 
-        Disposable d3 = Disposables.empty();
-        Disposable d4 = Disposables.empty();
+        Disposable d3 = Disposable.empty();
+        Disposable d4 = Disposable.empty();
 
         cd = new CompositeDisposable(Arrays.asList(d3, d4));
 
@@ -363,9 +363,9 @@ public void initializeIterable() {
     public void addAll() {
         CompositeDisposable cd = new CompositeDisposable();
 
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
-        Disposable d3 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
+        Disposable d3 = Disposable.empty();
 
         cd.addAll(d1, d2);
         cd.addAll(d3);
@@ -379,8 +379,8 @@ public void addAll() {
         assertTrue(d1.isDisposed());
         assertTrue(d2.isDisposed());
 
-        d1 = Disposables.empty();
-        d2 = Disposables.empty();
+        d1 = Disposable.empty();
+        d2 = Disposable.empty();
 
         cd = new CompositeDisposable();
 
@@ -406,14 +406,14 @@ public void addAfterDisposed() {
         CompositeDisposable cd = new CompositeDisposable();
         cd.dispose();
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         assertFalse(cd.add(d1));
 
         assertTrue(d1.isDisposed());
 
-        d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         assertFalse(cd.addAll(d1, d2));
 
@@ -427,11 +427,11 @@ public void delete() {
 
         CompositeDisposable cd = new CompositeDisposable();
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         assertFalse(cd.delete(d1));
 
-        Disposable d2 = Disposables.empty();
+        Disposable d2 = Disposable.empty();
 
         cd.add(d2);
 
@@ -466,7 +466,7 @@ public void addRace() {
             Runnable run = new Runnable() {
                 @Override
                 public void run() {
-                    cd.add(Disposables.empty());
+                    cd.add(Disposable.empty());
                 }
             };
 
@@ -482,7 +482,7 @@ public void addAllRace() {
             Runnable run = new Runnable() {
                 @Override
                 public void run() {
-                    cd.addAll(Disposables.empty());
+                    cd.addAll(Disposable.empty());
                 }
             };
 
@@ -495,7 +495,7 @@ public void removeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final CompositeDisposable cd = new CompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -515,7 +515,7 @@ public void deleteRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final CompositeDisposable cd = new CompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -535,7 +535,7 @@ public void clearRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final CompositeDisposable cd = new CompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -565,7 +565,7 @@ public void run() {
             Runnable run2 = new Runnable() {
                 @Override
                 public void run() {
-                    cd.add(Disposables.empty());
+                    cd.add(Disposable.empty());
                 }
             };
 
@@ -588,7 +588,7 @@ public void run() {
             Runnable run2 = new Runnable() {
                 @Override
                 public void run() {
-                    cd.addAll(Disposables.empty());
+                    cd.addAll(Disposable.empty());
                 }
             };
 
@@ -601,7 +601,7 @@ public void removeDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final CompositeDisposable cd = new CompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -628,7 +628,7 @@ public void deleteDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final CompositeDisposable cd = new CompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -655,7 +655,7 @@ public void clearDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final CompositeDisposable cd = new CompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -682,7 +682,7 @@ public void sizeDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final CompositeDisposable cd = new CompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -708,14 +708,14 @@ public void run() {
     public void disposeThrowsIAE() {
         CompositeDisposable cd = new CompositeDisposable();
 
-        cd.add(Disposables.fromAction(new Action() {
+        cd.add(Disposable.fromAction(new Action() {
             @Override
             public void run() throws Exception {
                 throw new IllegalArgumentException();
             }
         }));
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         cd.add(d1);
 
@@ -733,14 +733,14 @@ public void run() throws Exception {
     public void disposeThrowsError() {
         CompositeDisposable cd = new CompositeDisposable();
 
-        cd.add(Disposables.fromAction(new Action() {
+        cd.add(Disposable.fromAction(new Action() {
             @Override
             public void run() throws Exception {
                 throw new AssertionError();
             }
         }));
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         cd.add(d1);
 
@@ -758,14 +758,14 @@ public void run() throws Exception {
     public void disposeThrowsCheckedException() {
         CompositeDisposable cd = new CompositeDisposable();
 
-        cd.add(Disposables.fromAction(new Action() {
+        cd.add(Disposable.fromAction(new Action() {
             @Override
             public void run() throws Exception {
                 throw new IOException();
             }
         }));
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         cd.add(d1);
 
@@ -804,7 +804,7 @@ public boolean isDisposed() {
             }
         });
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         cd.add(d1);
 
diff --git a/src/test/java/io/reactivex/rxjava3/disposables/DisposablesTest.java b/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java
similarity index 86%
rename from src/test/java/io/reactivex/rxjava3/disposables/DisposablesTest.java
rename to src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java
index 244ca873f0..69bd6a8cc2 100644
--- a/src/test/java/io/reactivex/rxjava3/disposables/DisposablesTest.java
+++ b/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java
@@ -31,13 +31,13 @@
 import io.reactivex.rxjava3.plugins.RxJavaPlugins;
 import io.reactivex.rxjava3.testsupport.TestHelper;
 
-public class DisposablesTest extends RxJavaTest {
+public class DisposableTest extends RxJavaTest {
 
     @Test
     public void unsubscribeOnlyOnce() {
         Runnable run = mock(Runnable.class);
 
-        Disposable d = Disposables.fromRunnable(run);
+        Disposable d = Disposable.fromRunnable(run);
 
         assertTrue(d.toString(), d.toString().contains("RunnableDisposable(disposed=false, "));
 
@@ -52,7 +52,7 @@ public void unsubscribeOnlyOnce() {
 
     @Test
     public void empty() {
-        Disposable empty = Disposables.empty();
+        Disposable empty = Disposable.empty();
         assertFalse(empty.isDisposed());
         empty.dispose();
         assertTrue(empty.isDisposed());
@@ -60,20 +60,15 @@ public void empty() {
 
     @Test
     public void unsubscribed() {
-        Disposable disposed = Disposables.disposed();
+        Disposable disposed = Disposable.disposed();
         assertTrue(disposed.isDisposed());
     }
 
-    @Test
-    public void utilityClass() {
-        TestHelper.checkUtilityClass(Disposables.class);
-    }
-
     @Test
     public void fromAction() throws Throwable {
         Action action = mock(Action.class);
 
-        Disposable d = Disposables.fromAction(action);
+        Disposable d = Disposable.fromAction(action);
 
         assertTrue(d.toString(), d.toString().contains("ActionDisposable(disposed=false, "));
 
@@ -89,7 +84,7 @@ public void fromAction() throws Throwable {
     @Test
     public void fromActionThrows() {
         try {
-            Disposables.fromAction(new Action() {
+            Disposable.fromAction(new Action() {
                 @Override
                 public void run() throws Exception {
                     throw new IllegalArgumentException();
@@ -101,7 +96,7 @@ public void run() throws Exception {
         }
 
         try {
-            Disposables.fromAction(new Action() {
+            Disposable.fromAction(new Action() {
                 @Override
                 public void run() throws Exception {
                     throw new InternalError();
@@ -113,7 +108,7 @@ public void run() throws Exception {
         }
 
         try {
-            Disposables.fromAction(new Action() {
+            Disposable.fromAction(new Action() {
                 @Override
                 public void run() throws Exception {
                     throw new IOException();
@@ -132,7 +127,7 @@ public void run() throws Exception {
     @Test
     public void disposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Runnable r = new Runnable() {
                 @Override
@@ -147,14 +142,14 @@ public void run() {
 
     @Test(expected = NullPointerException.class)
     public void fromSubscriptionNull() {
-        Disposables.fromSubscription(null);
+        Disposable.fromSubscription(null);
     }
 
     @Test
     public void fromSubscription() {
         Subscription s = mock(Subscription.class);
 
-        Disposables.fromSubscription(s).dispose();
+        Disposable.fromSubscription(s).dispose();
 
         verify(s).cancel();
         verify(s, never()).request(anyInt());
@@ -166,11 +161,11 @@ public void setOnceTwice() {
         try {
 
             AtomicReference target = new AtomicReference();
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             DisposableHelper.setOnce(target, d);
 
-            Disposable d1 = Disposables.empty();
+            Disposable d1 = Disposable.empty();
 
             DisposableHelper.setOnce(target, d1);
 
@@ -188,7 +183,7 @@ public void fromAutoCloseable() {
 
         AutoCloseable ac = () -> counter.getAndIncrement();
 
-        Disposable d = Disposables.fromAutoCloseable(ac);
+        Disposable d = Disposable.fromAutoCloseable(ac);
 
         assertFalse(d.isDisposed());
         assertEquals(0, counter.get());
@@ -212,7 +207,7 @@ public void fromAutoCloseableThrows() throws Throwable {
         TestHelper.withErrorTracking(errors -> {
             AutoCloseable ac = () -> { throw new TestException(); };
 
-            Disposable d = Disposables.fromAutoCloseable(ac);
+            Disposable d = Disposable.fromAutoCloseable(ac);
 
             assertFalse(d.isDisposed());
 
@@ -236,9 +231,9 @@ public void fromAutoCloseableThrows() throws Throwable {
     public void toAutoCloseable() throws Exception {
         AtomicInteger counter = new AtomicInteger();
 
-        Disposable d = Disposables.fromAction(() -> counter.getAndIncrement());
+        Disposable d = Disposable.fromAction(() -> counter.getAndIncrement());
 
-        AutoCloseable ac = Disposables.toAutoCloseable(d);
+        AutoCloseable ac = Disposable.toAutoCloseable(d);
 
         assertFalse(d.isDisposed());
         assertEquals(0, counter.get());
diff --git a/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java
index 7cb13de5a8..a15f018c9b 100644
--- a/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java
@@ -27,7 +27,7 @@ public class FutureDisposableTest extends RxJavaTest {
     @Test
     public void normal() {
         FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, null);
-        Disposable d = Disposables.fromFuture(ft);
+        Disposable d = Disposable.fromFuture(ft);
         assertFalse(d.isDisposed());
 
         d.dispose();
@@ -44,7 +44,7 @@ public void normal() {
     @Test
     public void interruptible() {
         FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, null);
-        Disposable d = Disposables.fromFuture(ft, true);
+        Disposable d = Disposable.fromFuture(ft, true);
         assertFalse(d.isDisposed());
 
         d.dispose();
diff --git a/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java
index 05f544bfa3..dc92b1f1d3 100644
--- a/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java
@@ -56,7 +56,7 @@ public void notDisposedWhenReplaced() {
         final Disposable underlying = mock(Disposable.class);
         serialDisposable.update(underlying);
 
-        serialDisposable.replace(Disposables.empty());
+        serialDisposable.replace(Disposable.empty());
         serialDisposable.dispose();
 
         verify(underlying, never()).dispose();
diff --git a/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java b/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java
index 9ab98dae94..0205d6a85d 100644
--- a/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java
+++ b/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java
@@ -56,7 +56,7 @@ public void notDisposedWhenReplaced() {
         final Disposable underlying = mock(Disposable.class);
         serialDisposable.set(underlying);
 
-        serialDisposable.replace(Disposables.empty());
+        serialDisposable.replace(Disposable.empty());
         serialDisposable.dispose();
 
         verify(underlying, never()).dispose();
@@ -206,7 +206,7 @@ public void run() {
 
     @Test
     public void disposeState() {
-        Disposable empty = Disposables.empty();
+        Disposable empty = Disposable.empty();
         SerialDisposable d = new SerialDisposable(empty);
 
         assertFalse(d.isDisposed());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/disposables/ArrayCompositeDisposableTest.java b/src/test/java/io/reactivex/rxjava3/internal/disposables/ArrayCompositeDisposableTest.java
index c5edd75477..8871cc4e50 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/disposables/ArrayCompositeDisposableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/disposables/ArrayCompositeDisposableTest.java
@@ -27,14 +27,14 @@ public class ArrayCompositeDisposableTest extends RxJavaTest {
     public void normal() {
         ArrayCompositeDisposable acd = new ArrayCompositeDisposable(2);
 
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         assertTrue(acd.setResource(0, d1));
         assertTrue(acd.setResource(1, d2));
 
-        Disposable d3 = Disposables.empty();
-        Disposable d4 = Disposables.empty();
+        Disposable d3 = Disposable.empty();
+        Disposable d4 = Disposable.empty();
 
         acd.replaceResource(0, d3);
         acd.replaceResource(1, d4);
@@ -58,8 +58,8 @@ public void normal() {
         assertTrue(d1.isDisposed());
         assertTrue(d2.isDisposed());
 
-        Disposable d5 = Disposables.empty();
-        Disposable d6 = Disposables.empty();
+        Disposable d5 = Disposable.empty();
+        Disposable d6 = Disposable.empty();
 
         assertFalse(acd.setResource(0, d5));
         acd.replaceResource(1, d6);
@@ -92,7 +92,7 @@ public void replaceRace() {
             Runnable r = new Runnable() {
                 @Override
                 public void run() {
-                    acd.replaceResource(0, Disposables.empty());
+                    acd.replaceResource(0, Disposable.empty());
                 }
             };
 
@@ -108,7 +108,7 @@ public void setRace() {
             Runnable r = new Runnable() {
                 @Override
                 public void run() {
-                    acd.setResource(0, Disposables.empty());
+                    acd.setResource(0, Disposable.empty());
                 }
             };
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java
index ea0c4ea89b..24c8dd784f 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java
@@ -75,7 +75,7 @@ public void setReplace() {
             Runnable r = new Runnable() {
                 @Override
                 public void run() {
-                    DisposableHelper.replace(d, Disposables.empty());
+                    DisposableHelper.replace(d, Disposable.empty());
                 }
             };
 
@@ -91,7 +91,7 @@ public void setRace() {
             Runnable r = new Runnable() {
                 @Override
                 public void run() {
-                    DisposableHelper.set(d, Disposables.empty());
+                    DisposableHelper.set(d, Disposable.empty());
                 }
             };
 
@@ -111,7 +111,7 @@ public void setReplaceNull() {
 
     @Test
     public void dispose() {
-        Disposable u = Disposables.empty();
+        Disposable u = Disposable.empty();
         final AtomicReference d = new AtomicReference(u);
 
         DisposableHelper.dispose(d);
@@ -123,11 +123,11 @@ public void dispose() {
     public void trySet() {
         AtomicReference ref = new AtomicReference();
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         assertTrue(DisposableHelper.trySet(ref, d1));
 
-        Disposable d2 = Disposables.empty();
+        Disposable d2 = Disposable.empty();
 
         assertFalse(DisposableHelper.trySet(ref, d2));
 
@@ -137,7 +137,7 @@ public void trySet() {
 
         DisposableHelper.dispose(ref);
 
-        Disposable d3 = Disposables.empty();
+        Disposable d3 = Disposable.empty();
 
         assertFalse(DisposableHelper.trySet(ref, d3));
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposableTest.java b/src/test/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposableTest.java
index 280cb1c28e..0325d00500 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposableTest.java
@@ -28,8 +28,8 @@ public class ListCompositeDisposableTest extends RxJavaTest {
 
     @Test
     public void constructorAndAddVarargs() {
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         ListCompositeDisposable lcd = new ListCompositeDisposable(d1, d2);
 
@@ -40,8 +40,8 @@ public void constructorAndAddVarargs() {
         assertTrue(d1.isDisposed());
         assertTrue(d2.isDisposed());
 
-        d1 = Disposables.empty();
-        d2 = Disposables.empty();
+        d1 = Disposable.empty();
+        d2 = Disposable.empty();
 
         lcd.addAll(d1, d2);
 
@@ -54,8 +54,8 @@ public void constructorAndAddVarargs() {
 
     @Test
     public void constructorIterable() {
-        Disposable d1 = Disposables.empty();
-        Disposable d2 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
+        Disposable d2 = Disposable.empty();
 
         ListCompositeDisposable lcd = new ListCompositeDisposable(Arrays.asList(d1, d2));
 
@@ -66,8 +66,8 @@ public void constructorIterable() {
         assertTrue(d1.isDisposed());
         assertTrue(d2.isDisposed());
 
-        d1 = Disposables.empty();
-        d2 = Disposables.empty();
+        d1 = Disposable.empty();
+        d2 = Disposable.empty();
 
         lcd.add(d1);
         lcd.addAll(d2);
@@ -103,11 +103,11 @@ public void afterDispose() {
         ListCompositeDisposable lcd = new ListCompositeDisposable();
         lcd.dispose();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         assertFalse(lcd.add(d));
         assertTrue(d.isDisposed());
 
-        d = Disposables.empty();
+        d = Disposable.empty();
         assertFalse(lcd.addAll(d));
         assertTrue(d.isDisposed());
     }
@@ -152,7 +152,7 @@ public boolean isDisposed() {
     @Test
     public void remove() {
         ListCompositeDisposable lcd = new ListCompositeDisposable();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         lcd.add(d);
 
@@ -201,7 +201,7 @@ public void addRace() {
             Runnable run = new Runnable() {
                 @Override
                 public void run() {
-                    cd.add(Disposables.empty());
+                    cd.add(Disposable.empty());
                 }
             };
 
@@ -217,7 +217,7 @@ public void addAllRace() {
             Runnable run = new Runnable() {
                 @Override
                 public void run() {
-                    cd.addAll(Disposables.empty());
+                    cd.addAll(Disposable.empty());
                 }
             };
 
@@ -230,7 +230,7 @@ public void removeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final ListCompositeDisposable cd = new ListCompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -250,7 +250,7 @@ public void deleteRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final ListCompositeDisposable cd = new ListCompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -270,7 +270,7 @@ public void clearRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final ListCompositeDisposable cd = new ListCompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -300,7 +300,7 @@ public void run() {
             Runnable run2 = new Runnable() {
                 @Override
                 public void run() {
-                    cd.add(Disposables.empty());
+                    cd.add(Disposable.empty());
                 }
             };
 
@@ -323,7 +323,7 @@ public void run() {
             Runnable run2 = new Runnable() {
                 @Override
                 public void run() {
-                    cd.addAll(Disposables.empty());
+                    cd.addAll(Disposable.empty());
                 }
             };
 
@@ -336,7 +336,7 @@ public void removeDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final ListCompositeDisposable cd = new ListCompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -363,7 +363,7 @@ public void deleteDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final ListCompositeDisposable cd = new ListCompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
@@ -390,7 +390,7 @@ public void clearDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final ListCompositeDisposable cd = new ListCompositeDisposable();
 
-            final Disposable d1 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
 
             cd.add(d1);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java
index c62f59f831..a63283b938 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java
@@ -13,11 +13,11 @@
 
 package io.reactivex.rxjava3.internal.observers;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.annotations.Nullable;
 import io.reactivex.rxjava3.core.RxJavaTest;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.observers.TestObserver;
 import io.reactivex.rxjava3.testsupport.TestObserverEx;
 
@@ -48,7 +48,7 @@ protected boolean beforeDownstream() {
             }
         };
 
-        o.onSubscribe(Disposables.disposed());
+        o.onSubscribe(Disposable.disposed());
 
         to.assertNotSubscribed();
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java
index e871ec9639..845817379a 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java
@@ -26,7 +26,7 @@ public class BlockingFirstObserverTest extends RxJavaTest {
     @Test
     public void firstValueOnly() {
         BlockingFirstObserver bf = new BlockingFirstObserver();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         bf.onSubscribe(d);
 
         bf.onNext(1);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java
index dddbb1e962..399b5506e3 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java
@@ -30,7 +30,7 @@ public void dispose() {
         BlockingMultiObserver bmo = new BlockingMultiObserver();
         bmo.dispose();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         bmo.onSubscribe(d);
     }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java
index 139d6fa0fc..f6ebd39d1f 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java
@@ -54,9 +54,9 @@ public void normal() {
 
             TakeFirst source = new TakeFirst(to);
 
-            source.onSubscribe(Disposables.empty());
+            source.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
             source.onSubscribe(d);
 
             assertTrue(d.isDisposed());
@@ -77,7 +77,7 @@ public void error() {
 
         TakeFirst source = new TakeFirst(to);
 
-        source.onSubscribe(Disposables.empty());
+        source.onSubscribe(Disposable.empty());
         source.onError(new TestException());
 
         to.assertFailure(TestException.class);
@@ -89,7 +89,7 @@ public void complete() {
 
         TakeFirst source = new TakeFirst(to);
 
-        source.onSubscribe(Disposables.empty());
+        source.onSubscribe(Disposable.empty());
         source.onComplete();
 
         to.assertResult();
@@ -101,7 +101,7 @@ public void dispose() {
 
         TakeFirst source = new TakeFirst(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         source.onSubscribe(d);
 
@@ -122,7 +122,7 @@ public void fused() {
 
             TakeFirst source = new TakeFirst(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             source.onSubscribe(d);
 
@@ -152,7 +152,7 @@ public void fusedReject() {
 
             TakeFirst source = new TakeFirst(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             source.onSubscribe(d);
 
@@ -197,7 +197,7 @@ public void nonfusedTerminateMore() {
 
             TakeLast source = new TakeLast(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             source.onSubscribe(d);
 
@@ -222,7 +222,7 @@ public void nonfusedError() {
 
             TakeLast source = new TakeLast(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             source.onSubscribe(d);
 
@@ -247,7 +247,7 @@ public void fusedTerminateMore() {
 
             TakeLast source = new TakeLast(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             source.onSubscribe(d);
 
@@ -272,7 +272,7 @@ public void fusedError() {
 
             TakeLast source = new TakeLast(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             source.onSubscribe(d);
 
@@ -295,7 +295,7 @@ public void disposed() {
 
         TakeLast source = new TakeLast(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         source.onSubscribe(d);
 
@@ -337,7 +337,7 @@ public void onComplete() {
             }
         });
 
-        source.onSubscribe(Disposables.empty());
+        source.onSubscribe(Disposable.empty());
         source.onNext(1);
         source.onComplete();
 
@@ -350,7 +350,7 @@ public void fusedEmpty() {
 
         TakeLast source = new TakeLast(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         source.onSubscribe(d);
 
@@ -365,7 +365,7 @@ public void nonfusedEmpty() {
 
         TakeLast source = new TakeLast(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         source.onSubscribe(d);
 
@@ -417,7 +417,7 @@ public void onComplete() {
             }
         });
 
-        source.onSubscribe(Disposables.empty());
+        source.onSubscribe(Disposable.empty());
         source.onNext(1);
         source.onComplete();
 
@@ -456,7 +456,7 @@ public void onComplete() {
             }
         });
 
-        source.onSubscribe(Disposables.empty());
+        source.onSubscribe(Disposable.empty());
         source.onNext(1);
         source.onComplete();
 
@@ -503,7 +503,7 @@ public void onComplete() {
             }
         });
 
-        source.onSubscribe(Disposables.empty());
+        source.onSubscribe(Disposable.empty());
         source.onNext(1);
 
         to.assertNoValues().assertNoErrors().assertComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java
index 0de5f0dcaa..cc84102c30 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java
@@ -17,10 +17,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.RxJavaTest;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.Action;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -51,7 +51,7 @@ public void run() throws Exception {
                     }
             );
 
-            o.onSubscribe(Disposables.empty());
+            o.onSubscribe(Disposable.empty());
 
             assertFalse(o.isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java
index 1edecae92c..f9caa33808 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java
@@ -137,11 +137,11 @@ public void onSubscribe() throws Exception {
 
         try {
 
-            Disposable d1 = Disposables.empty();
+            Disposable d1 = Disposable.empty();
 
             fo.onSubscribe(d1);
 
-            Disposable d2 = Disposables.empty();
+            Disposable d2 = Disposable.empty();
 
             fo.onSubscribe(d2);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java
index 7cfd46352a..937e11441d 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java
@@ -202,9 +202,9 @@ public void badSourceOnSubscribe() {
             Observable source = new Observable() {
                 @Override
                 public void subscribeActual(Observer observer) {
-                    Disposable d1 = Disposables.empty();
+                    Disposable d1 = Disposable.empty();
                     observer.onSubscribe(d1);
-                    Disposable d2 = Disposables.empty();
+                    Disposable d2 = Disposable.empty();
                     observer.onSubscribe(d2);
 
                     assertFalse(d1.isDisposed());
@@ -256,7 +256,7 @@ public void badSourceEmitAfterDone() {
             Observable source = new Observable() {
                 @Override
                 public void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java
index b8d82a3ed3..c8823563e0 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java
@@ -80,9 +80,9 @@ public void accept(Observer a, Integer v) {
     @Test
     public void unorderedSlowPath() {
         TestObserver to = new TestObserver();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainObserver qd = createUnordered(to, d);
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         qd.enter();
         qd.onNext(1);
@@ -93,9 +93,9 @@ public void unorderedSlowPath() {
     @Test
     public void orderedSlowPath() {
         TestObserver to = new TestObserver();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainObserver qd = createOrdered(to, d);
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         qd.enter();
         qd.onNext(1);
@@ -106,9 +106,9 @@ public void orderedSlowPath() {
     @Test
     public void orderedSlowPathNonEmptyQueue() {
         TestObserver to = new TestObserver();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainObserver qd = createOrdered(to, d);
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         qd.queue.offer(0);
         qd.onNext(1);
@@ -121,9 +121,9 @@ public void unorderedOnNextRace() {
         for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) {
 
             TestObserver to = new TestObserver();
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
             final QueueDrainObserver qd = createUnordered(to, d);
-            to.onSubscribe(Disposables.empty());
+            to.onSubscribe(Disposable.empty());
 
             Runnable r1 = new Runnable() {
                 @Override
@@ -143,9 +143,9 @@ public void orderedOnNextRace() {
         for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) {
 
             TestObserver to = new TestObserver();
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
             final QueueDrainObserver qd = createOrdered(to, d);
-            to.onSubscribe(Disposables.empty());
+            to.onSubscribe(Disposable.empty());
 
             Runnable r1 = new Runnable() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java
index 643ae8a880..8192f338da 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java
@@ -173,12 +173,12 @@ public void ambArrayOrder() {
     @Test
     public void ambRace() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         CompositeDisposable cd = new CompositeDisposable();
         AtomicBoolean once = new AtomicBoolean();
         Amb a = new Amb(once, cd, to);
-        a.onSubscribe(Disposables.empty());
+        a.onSubscribe(Disposable.empty());
 
         a.onComplete();
         a.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java
index 19cff53d90..3180f0b71a 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java
@@ -18,11 +18,11 @@
 import java.util.*;
 import java.util.concurrent.CountDownLatch;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.*;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription;
@@ -172,7 +172,7 @@ public void arrayFirstCancels() {
         Completable.concatArray(new Completable() {
             @Override
             protected void subscribeActual(CompletableObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 to.dispose();
                 observer.onComplete();
             }
@@ -196,7 +196,7 @@ public void iterableFirstCancels() {
         Completable.concat(Arrays.asList(new Completable() {
             @Override
             protected void subscribeActual(CompletableObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 to.dispose();
                 observer.onComplete();
             }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCreateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCreateTest.java
index 8b7993609f..2ee2550872 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCreateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCreateTest.java
@@ -38,7 +38,7 @@ public void nullArgument() {
     public void basic() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Completable.create(new CompletableOnSubscribe() {
                 @Override
@@ -65,8 +65,8 @@ public void subscribe(CompletableEmitter e) throws Exception {
     public void basicWithCancellable() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d1 = Disposables.empty();
-            final Disposable d2 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
+            final Disposable d2 = Disposable.empty();
 
             Completable.create(new CompletableOnSubscribe() {
                 @Override
@@ -100,7 +100,7 @@ public void cancel() throws Exception {
     public void basicWithError() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Completable.create(new CompletableOnSubscribe() {
                 @Override
@@ -162,7 +162,7 @@ public void onErrorThrows() {
         Completable.create(new CompletableOnSubscribe() {
             @Override
             public void subscribe(CompletableEmitter e) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 e.setDisposable(d);
 
                 try {
@@ -199,7 +199,7 @@ public void onCompleteThrows() {
         Completable.create(new CompletableOnSubscribe() {
             @Override
             public void subscribe(CompletableEmitter e) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 e.setDisposable(d);
 
                 try {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java
index 5225359554..b2dfd16c32 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java
@@ -64,7 +64,7 @@ public void onComplete() {
 
     @Test
     public void cancelDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Completable() {
@@ -90,7 +90,7 @@ protected void subscribeActual(CompletableObserver observer) {
 
     @Test
     public void completeDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Completable() {
@@ -116,7 +116,7 @@ protected void subscribeActual(CompletableObserver observer) {
 
     @Test
     public void errorDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Completable() {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDoOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDoOnTest.java
index fea4bcb84e..c2bf494837 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDoOnTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDoOnTest.java
@@ -82,7 +82,7 @@ public void run() throws Exception {
     public void onSubscribeCrash() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable bs = Disposables.empty();
+            final Disposable bs = Disposable.empty();
 
             new Completable() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java
index 6a7c9aa70d..c1cc06f3c6 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java
@@ -48,7 +48,7 @@ public void cancelAfterFirst() {
         Completable.mergeArray(new Completable() {
             @Override
             protected void subscribeActual(CompletableObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
                 to.dispose();
             }
@@ -65,7 +65,7 @@ public void cancelAfterFirstDelayError() {
         Completable.mergeArrayDelayError(new Completable() {
             @Override
             protected void subscribeActual(CompletableObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
                 to.dispose();
             }
@@ -84,7 +84,7 @@ public void onErrorAfterComplete() {
             Completable.mergeArrayDelayError(Completable.complete(), new Completable() {
                 @Override
                 protected void subscribeActual(CompletableObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                     co[0] = observer;
                 }
@@ -410,7 +410,7 @@ public void innerDoubleOnError() {
             Completable.mergeDelayError(Flowable.just(new Completable() {
                 @Override
                 protected void subscribeActual(CompletableObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     o[0] = observer;
                 }
@@ -433,7 +433,7 @@ public void innerIsDisposed() {
         Completable.mergeDelayError(Flowable.just(new Completable() {
             @Override
             protected void subscribeActual(CompletableObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 assertFalse(((Disposable)observer).isDisposed());
 
                 to.dispose();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java
index b8b48fe9c0..1b4bb17263 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java
@@ -18,10 +18,10 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.observers.TestObserver;
 import io.reactivex.rxjava3.plugins.RxJavaPlugins;
@@ -145,7 +145,7 @@ public void mainErrorLate() {
             new Completable() {
                 @Override
                 protected void subscribeActual(CompletableObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException());
                 }
             }.takeUntil(Completable.complete())
@@ -167,7 +167,7 @@ public void mainCompleteLate() {
             new Completable() {
                 @Override
                 protected void subscribeActual(CompletableObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                 }
             }.takeUntil(Completable.complete())
@@ -192,7 +192,7 @@ public void otherErrorLate() {
             .takeUntil(new Completable() {
                 @Override
                 protected void subscribeActual(CompletableObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     ref.set(observer);
                 }
             })
@@ -219,7 +219,7 @@ public void otherCompleteLate() {
             .takeUntil(new Completable() {
                 @Override
                 protected void subscribeActual(CompletableObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     ref.set(observer);
                 }
             })
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java
index 9261562220..0b176d808e 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java
@@ -156,7 +156,7 @@ public void run() {
     @Test
     public void ambRace() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         CompositeDisposable cd = new CompositeDisposable();
         AtomicBoolean once = new AtomicBoolean();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java
index d94a4ba097..4d5b99cb90 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java
@@ -43,7 +43,7 @@ public void fusion() throws Exception {
 
         ObserverCompletableObserver co = new ObserverCompletableObserver(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         co.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUnsafeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUnsafeTest.java
index 7cad5802b6..4581a8944d 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUnsafeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUnsafeTest.java
@@ -17,10 +17,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.plugins.RxJavaPlugins;
 import io.reactivex.rxjava3.testsupport.TestHelper;
 
@@ -42,7 +42,7 @@ public void wrapCustomCompletable() {
         Completable.wrap(new CompletableSource() {
             @Override
             public void subscribe(CompletableObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
             }
         })
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsingTest.java
index ca4634c7f5..6a7ddd564b 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsingTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsingTest.java
@@ -411,11 +411,11 @@ public CompletableSource apply(Object v) throws Exception {
                     return Completable.wrap(new CompletableSource() {
                         @Override
                         public void subscribe(CompletableObserver observer) {
-                            Disposable d1 = Disposables.empty();
+                            Disposable d1 = Disposable.empty();
 
                             observer.onSubscribe(d1);
 
-                            Disposable d2 = Disposables.empty();
+                            Disposable d2 = Disposable.empty();
 
                             observer.onSubscribe(d2);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java
index eddc091c77..a0b14ead5a 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java
@@ -163,7 +163,7 @@ public void nestedAsyncConcat() throws InterruptedException {
 
             @Override
             public void subscribe(final Subscriber> subscriber) {
-                final Disposable d = Disposables.empty();
+                final Disposable d = Disposable.empty();
                 subscriber.onSubscribe(new Subscription() {
                     @Override
                     public void request(long n) {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java
index 023f5b5a25..e6470fe6c9 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java
@@ -48,7 +48,7 @@ public void subscribe(FlowableEmitter s) throws Exception { }
     public void basic() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Flowable.create(new FlowableOnSubscribe() {
                 @Override
@@ -81,8 +81,8 @@ public void subscribe(FlowableEmitter e) throws Exception {
     public void basicWithCancellable() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d1 = Disposables.empty();
-            final Disposable d2 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
+            final Disposable d2 = Disposable.empty();
 
             Flowable.create(new FlowableOnSubscribe() {
                 @Override
@@ -122,7 +122,7 @@ public void cancel() throws Exception {
     public void basicWithError() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Flowable.create(new FlowableOnSubscribe() {
                 @Override
@@ -153,7 +153,7 @@ public void subscribe(FlowableEmitter e) throws Exception {
     public void basicSerialized() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Flowable.create(new FlowableOnSubscribe() {
                 @Override
@@ -188,7 +188,7 @@ public void subscribe(FlowableEmitter e) throws Exception {
     public void basicWithErrorSerialized() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Flowable.create(new FlowableOnSubscribe() {
                 @Override
@@ -778,7 +778,7 @@ public void onErrorCrash() {
             Flowable.create(new FlowableOnSubscribe() {
                 @Override
                 public void subscribe(FlowableEmitter e) throws Exception {
-                    Disposable d = Disposables.empty();
+                    Disposable d = Disposable.empty();
                     e.setDisposable(d);
                     try {
                         e.onError(new IOException());
@@ -816,7 +816,7 @@ public void onCompleteCrash() {
             Flowable.create(new FlowableOnSubscribe() {
                 @Override
                 public void subscribe(FlowableEmitter e) throws Exception {
-                    Disposable d = Disposables.empty();
+                    Disposable d = Disposable.empty();
                     e.setDisposable(d);
                     try {
                         e.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java
index 340b3f22d9..0a37faae1e 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java
@@ -72,7 +72,7 @@ public void run() {
 
         for (int i = 0; i < subCount; ++i) {
             TestSubscriber subscriber = new TestSubscriber();
-            subscriptions.add(Disposables.fromSubscription(subscriber));
+            subscriptions.add(Disposable.fromSubscription(subscriber));
             longs.subscribe(subscriber);
             subscribers.add(subscriber);
         }
@@ -134,7 +134,7 @@ public void run() {
         for (int i = 0; i < subCount; ++i) {
             TestSubscriber subscriber = new TestSubscriber();
             longs.subscribe(subscriber);
-            subscriptions.add(Disposables.fromSubscription(subscriber));
+            subscriptions.add(Disposable.fromSubscription(subscriber));
             subscribers.add(subscriber);
         }
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java
index f4701665cb..596e7b3867 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java
@@ -17,13 +17,13 @@
 
 import java.util.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.*;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription;
@@ -344,7 +344,7 @@ public void badSourceObservable() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onNext(2);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableTest.java
index 592756d85e..1c87d6f3b3 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableTest.java
@@ -455,7 +455,7 @@ public CompletableSource apply(Integer v) throws Exception {
                 return new Completable() {
                     @Override
                     protected void subscribeActual(CompletableObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
@@ -494,7 +494,7 @@ public CompletableSource apply(Integer v) throws Exception {
                 return new Completable() {
                     @Override
                     protected void subscribeActual(CompletableObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java
index 332d906952..d425ce8372 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java
@@ -442,7 +442,7 @@ public void badInnerSource() {
             .flatMapMaybe(Functions.justFunction(new Maybe() {
                 @Override
                 protected void subscribeActual(MaybeObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -531,7 +531,7 @@ public MaybeSource apply(Integer v) throws Exception {
                 return new Maybe() {
                     @Override
                     protected void subscribeActual(MaybeObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java
index e207dc57a1..2d3699e8e2 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java
@@ -361,7 +361,7 @@ public void badInnerSource() {
             .flatMapSingle(Functions.justFunction(new Single() {
                 @Override
                 protected void subscribeActual(SingleObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -416,7 +416,7 @@ public SingleSource apply(Integer v) throws Exception {
                 return new Single() {
                     @Override
                     protected void subscribeActual(SingleObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java
index 624ecee97a..994926b237 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java
@@ -1806,7 +1806,7 @@ public boolean isDisposed() {
             public Disposable schedule(Runnable run, long delay,
                     TimeUnit unit) {
                 run.run();
-                return Disposables.empty();
+                return Disposable.empty();
             }
         }
     }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java
index c70429b381..975740cfe3 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java
@@ -805,7 +805,7 @@ static final class BadFlowableSubscribe extends ConnectableFlowable {
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -827,7 +827,7 @@ static final class BadFlowableDispose extends ConnectableFlowable {
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -922,7 +922,7 @@ static final class BadFlowableSubscribe2 extends ConnectableFlowable {
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -969,7 +969,7 @@ static final class BadFlowableConnect2 extends ConnectableFlowable {
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -1214,7 +1214,7 @@ static final class BadFlowableDoubleOnX extends ConnectableFlowable
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -1392,7 +1392,7 @@ public void timeoutResetsSource() {
         FlowableRefCount o = (FlowableRefCount)tcf.refCount();
 
         RefConnection rc = new RefConnection(o);
-        rc.set(Disposables.empty());
+        rc.set(Disposable.empty());
         o.connection = rc;
 
         o.timeout(rc);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java
index 0422a24db2..37d9195175 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java
@@ -210,7 +210,7 @@ private void performTestUsingWithFlowableFactoryError(boolean disposeEagerly) {
         Supplier resourceFactory = new Supplier() {
             @Override
             public Disposable get() {
-                return Disposables.fromRunnable(unsubscribe);
+                return Disposable.fromRunnable(unsubscribe);
             }
         };
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/NotificationLiteTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/NotificationLiteTest.java
index 1df91fbe4d..3a5ff0dc57 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/NotificationLiteTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/NotificationLiteTest.java
@@ -15,10 +15,10 @@
 
 import static org.junit.Assert.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.RxJavaTest;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription;
 import io.reactivex.rxjava3.internal.util.NotificationLite;
@@ -80,7 +80,7 @@ public void completeNotification() {
 
     @Test
     public void disposableNotification() {
-        Object o = NotificationLite.disposable(Disposables.empty());
+        Object o = NotificationLite.disposable(Disposable.empty());
 
         assertEquals("NotificationLite.Disposable[RunnableDisposable(disposed=false, EmptyRunnable)]", o.toString());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java
index 36addaab95..50b76aa424 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java
@@ -19,10 +19,10 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -124,7 +124,7 @@ public void disposeNoFurtherSignals() {
             @Override
             protected void subscribeActual(
                     MaybeObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onSuccess(1);
                 observer.onSuccess(2);
                 observer.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java
index 4419d5e8c3..9ebf939b52 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java
@@ -33,7 +33,7 @@ public class MaybeCallbackObserverTest extends RxJavaTest {
     public void dispose() {
         MaybeCallbackObserver mo = new MaybeCallbackObserver(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         mo.onSubscribe(d);
 
@@ -60,7 +60,7 @@ public void accept(Object v) throws Exception {
                     Functions.emptyConsumer(),
                     Functions.EMPTY_ACTION);
 
-            mo.onSubscribe(Disposables.empty());
+            mo.onSubscribe(Disposable.empty());
 
             mo.onSuccess(1);
 
@@ -84,7 +84,7 @@ public void accept(Object v) throws Exception {
                     },
                     Functions.EMPTY_ACTION);
 
-            mo.onSubscribe(Disposables.empty());
+            mo.onSubscribe(Disposable.empty());
 
             mo.onError(new TestException("Outer"));
 
@@ -113,7 +113,7 @@ public void run() throws Exception {
                         }
                     });
 
-            mo.onSubscribe(Disposables.empty());
+            mo.onSubscribe(Disposable.empty());
 
             mo.onComplete();
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java
index 68ccd8434b..3fa9235fbd 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java
@@ -18,10 +18,10 @@
 import java.io.IOException;
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.plugins.RxJavaPlugins;
 import io.reactivex.rxjava3.subscribers.TestSubscriber;
@@ -142,7 +142,7 @@ public void errorAfterTermination() {
             new Maybe() {
                 @Override
                 protected void subscribeActual(MaybeObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onSuccess(2);
                     o[0] = observer;
                 }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreateTest.java
index b9ffc12ee8..b0857c432f 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreateTest.java
@@ -79,7 +79,7 @@ public void onSuccessThrows() {
         Maybe.create(new MaybeOnSubscribe() {
             @Override
             public void subscribe(MaybeEmitter e) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 e.setDisposable(d);
 
                 try {
@@ -121,7 +121,7 @@ public void onErrorThrows() {
         Maybe.create(new MaybeOnSubscribe() {
             @Override
             public void subscribe(MaybeEmitter e) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 e.setDisposable(d);
 
                 try {
@@ -163,7 +163,7 @@ public void onCompleteThrows() {
         Maybe.create(new MaybeOnSubscribe() {
             @Override
             public void subscribe(MaybeEmitter e) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 e.setDisposable(d);
 
                 try {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java
index 25c2ce742a..ba720a02f9 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java
@@ -64,7 +64,7 @@ public void onComplete() {
 
     @Test
     public void cancelDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Maybe() {
@@ -90,7 +90,7 @@ protected void subscribeActual(MaybeObserver observer) {
 
     @Test
     public void completeDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Maybe() {
@@ -116,7 +116,7 @@ protected void subscribeActual(MaybeObserver observer) {
 
     @Test
     public void errorDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Maybe() {
@@ -142,7 +142,7 @@ protected void subscribeActual(MaybeObserver observer) {
 
     @Test
     public void successDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Maybe() {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEventTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEventTest.java
index 989599f2c6..abe91714e7 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEventTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEventTest.java
@@ -58,7 +58,7 @@ public void accept(Integer v, Throwable e) throws Exception {
     public void onSubscribeCrash() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable bs = Disposables.empty();
+            final Disposable bs = Disposable.empty();
 
             new Maybe() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java
index 5d74906294..2212d50eed 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java
@@ -17,11 +17,11 @@
 
 import java.util.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.Subscription;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.internal.fuseable.*;
 import io.reactivex.rxjava3.internal.operators.maybe.MaybeMergeArray.MergeMaybeObserver;
@@ -178,7 +178,7 @@ public void mergeBadSource() {
         Maybe.mergeArray(new Maybe() {
             @Override
             protected void subscribeActual(MaybeObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onSuccess(1);
                 observer.onSuccess(2);
                 observer.onSuccess(3);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeekTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeekTest.java
index 2dce4c4524..cbf4d60b34 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeekTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeekTest.java
@@ -17,10 +17,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -56,7 +56,7 @@ public void doubleError() {
             TestObserverEx to = new Maybe() {
                 @Override
                 protected void subscribeActual(MaybeObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -87,7 +87,7 @@ public void doubleComplete() {
         TestObserver to = new Maybe() {
             @Override
             protected void subscribeActual(MaybeObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
                 observer.onComplete();
             }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsingTest.java
index 79c81bf9a5..c9ff65e09a 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsingTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUsingTest.java
@@ -411,11 +411,11 @@ public MaybeSource apply(Object v) throws Exception {
                     return Maybe.wrap(new MaybeSource() {
                         @Override
                         public void subscribe(MaybeObserver observer) {
-                            Disposable d1 = Disposables.empty();
+                            Disposable d1 = Disposable.empty();
 
                             observer.onSubscribe(d1);
 
-                            Disposable d2 = Disposables.empty();
+                            Disposable d2 = Disposable.empty();
 
                             observer.onSubscribe(d2);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java
index 488a803172..47dd3d1284 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java
@@ -19,11 +19,11 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.Subscriber;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -302,7 +302,7 @@ public MaybeSource apply(Integer v)
                                     @Override
                                     protected void subscribeActual(
                                             MaybeObserver observer) {
-                                        observer.onSubscribe(Disposables.empty());
+                                        observer.onSubscribe(Disposable.empty());
                                         obs.set(observer);
                                     }
                             };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java
index 359053cdb5..5da39ff395 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java
@@ -18,11 +18,11 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.Subscriber;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -220,7 +220,7 @@ public SingleSource apply(Integer v)
                                     @Override
                                     protected void subscribeActual(
                                             SingleObserver observer) {
-                                        observer.onSubscribe(Disposables.empty());
+                                        observer.onSubscribe(Disposable.empty());
                                         obs.set(observer);
                                     }
                             };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java
index 1eea34f3fa..b7b66c3ca4 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java
@@ -18,11 +18,11 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.*;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -422,7 +422,7 @@ public MaybeSource apply(Integer v)
                         @Override
                         protected void subscribeActual(
                                 MaybeObserver observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             moRef.set(observer);
                         }
                     };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java
index a834f327f9..ca1cb6d919 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java
@@ -18,11 +18,11 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.*;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -370,7 +370,7 @@ public SingleSource apply(Integer v)
                         @Override
                         protected void subscribeActual(
                                 SingleObserver observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             moRef.set(observer);
                         }
                     };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java
index 30f78c41a9..21b9936292 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java
@@ -19,10 +19,10 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -239,7 +239,7 @@ public void mainErrorAfterInnerError() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException("outer"));
                 }
@@ -273,7 +273,7 @@ public MaybeSource apply(Integer v)
                                     @Override
                                     protected void subscribeActual(
                                             MaybeObserver observer) {
-                                        observer.onSubscribe(Disposables.empty());
+                                        observer.onSubscribe(Disposable.empty());
                                         obs.set(observer);
                                     }
                             };
@@ -378,7 +378,7 @@ public void cancelNoConcurrentClean() {
                 new ConcatMapMaybeMainObserver(
                         to, Functions.justFunction(Maybe.never()), 16, ErrorMode.IMMEDIATE);
 
-        operator.onSubscribe(Disposables.empty());
+        operator.onSubscribe(Disposable.empty());
 
         operator.queue.offer(1);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java
index 2d4b46f187..8f24b87bbb 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java
@@ -18,10 +18,10 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -157,7 +157,7 @@ public void mainErrorAfterInnerError() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException("outer"));
                 }
@@ -191,7 +191,7 @@ public SingleSource apply(Integer v)
                                     @Override
                                     protected void subscribeActual(
                                             SingleObserver observer) {
-                                        observer.onSubscribe(Disposables.empty());
+                                        observer.onSubscribe(Disposable.empty());
                                         obs.set(observer);
                                     }
                             };
@@ -318,7 +318,7 @@ public void cancelNoConcurrentClean() {
                 new ConcatMapSingleMainObserver(
                         to, Functions.justFunction(Single.never()), 16, ErrorMode.IMMEDIATE);
 
-        operator.onSubscribe(Disposables.empty());
+        operator.onSubscribe(Disposable.empty());
 
         operator.queue.offer(1);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java
index b308119424..b060b9d03d 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java
@@ -17,10 +17,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -314,7 +314,7 @@ public void innerErrorThenMainError() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException("main"));
                 }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java
index c9a1f9a0dd..7f877089b5 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java
@@ -18,10 +18,10 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -355,7 +355,7 @@ public void mainErrorAfterTermination() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException("outer"));
                 }
@@ -385,7 +385,7 @@ public void innerErrorAfterTermination() {
             TestObserverEx to = new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException("outer"));
                 }
@@ -398,7 +398,7 @@ public MaybeSource apply(Integer v)
                         @Override
                         protected void subscribeActual(
                                 MaybeObserver observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             moRef.set(observer);
                         }
                     };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java
index a59e9954ec..957c20ba0e 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java
@@ -18,10 +18,10 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -324,7 +324,7 @@ public void mainErrorAfterTermination() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException("outer"));
                 }
@@ -354,7 +354,7 @@ public void innerErrorAfterTermination() {
             TestObserverEx to = new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException("outer"));
                 }
@@ -367,7 +367,7 @@ public SingleSource apply(Integer v)
                         @Override
                         protected void subscribeActual(
                                 SingleObserver observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             moRef.set(observer);
                         }
                     };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java
index 194619b8bc..0fcec3a172 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java
@@ -247,7 +247,7 @@ public void noBufferingOrBlockingOfSequence() throws Throwable {
 
                     @Override
                     public void subscribe(final Observer o) {
-                        o.onSubscribe(Disposables.empty());
+                        o.onSubscribe(Disposable.empty());
                         task.replace(Schedulers.single().scheduleDirect(new Runnable() {
 
                             @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToFutureTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToFutureTest.java
index 756f0b4eb3..bca24503c1 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToFutureTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToFutureTest.java
@@ -20,12 +20,12 @@
 import java.util.*;
 import java.util.concurrent.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 
 public class BlockingObservableToFutureTest extends RxJavaTest {
@@ -65,7 +65,7 @@ public void toFutureWithException() {
 
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext("one");
                 observer.onError(new TestException());
             }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java
index 3a98208d70..3658c24b97 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java
@@ -56,7 +56,7 @@ public void toIteratorWithException() {
 
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext("one");
                 observer.onError(new TestException());
             }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java
index a7aca59ada..7e3dfbf8bf 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java
@@ -16,7 +16,7 @@
 
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
+import io.reactivex.rxjava3.disposables.Disposable;
 
 /**
  * Creates {@link Observable} of a number of items followed by either an error or
@@ -36,7 +36,7 @@ public final class Burst extends Observable {
 
     @Override
     protected void subscribeActual(final Observer observer) {
-        observer.onSubscribe(Disposables.empty());
+        observer.onSubscribe(Disposable.empty());
         for (T item: items) {
             observer.onNext(item);
         }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java
index 5934706a4d..38be122378 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java
@@ -311,7 +311,7 @@ public void predicateThrowsObservable() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onNext(2);
@@ -342,7 +342,7 @@ public void predicateThrows() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onNext(2);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java
index a79479067a..6ce9410340 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java
@@ -21,10 +21,10 @@
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -517,7 +517,7 @@ public void predicateThrowsSuppressOthers() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onNext(2);
@@ -548,7 +548,7 @@ public void badSourceSingle() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
 
                     observer.onNext(1);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java
index d1c6c6fc51..c8454452b4 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java
@@ -257,7 +257,7 @@ public void run() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 s[0] = observer;
             }
         }.blockingSubscribe(to);
@@ -294,7 +294,7 @@ public void blockingCancelUpfront() {
         o.dispose();
         assertTrue(o.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         o.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java
index 03c437dad9..34ae5ed780 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java
@@ -70,7 +70,7 @@ public void skipAndCountOverlappingBuffers() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext("one");
                 observer.onNext("two");
                 observer.onNext("three");
@@ -126,7 +126,7 @@ public void timedAndCount() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 push(observer, "one", 10);
                 push(observer, "two", 90);
                 push(observer, "three", 110);
@@ -158,7 +158,7 @@ public void timed() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 push(observer, "one", 97);
                 push(observer, "two", 98);
                 /**
@@ -192,7 +192,7 @@ public void observableBasedOpenerAndCloser() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 push(observer, "one", 10);
                 push(observer, "two", 60);
                 push(observer, "three", 110);
@@ -205,7 +205,7 @@ public void subscribe(Observer observer) {
         Observable openings = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 push(observer, new Object(), 50);
                 push(observer, new Object(), 200);
                 complete(observer, 250);
@@ -218,7 +218,7 @@ public Observable apply(Object opening) {
                 return Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         push(observer, new Object(), 100);
                         complete(observer, 101);
                     }
@@ -1360,8 +1360,8 @@ public void openClosebadSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    Disposable bs1 = Disposables.empty();
-                    Disposable bs2 = Disposables.empty();
+                    Disposable bs1 = Disposable.empty();
+                    Disposable bs2 = Disposable.empty();
 
                     observer.onSubscribe(bs1);
 
@@ -1484,8 +1484,8 @@ protected void subscribeActual(Observer observer) {
 
                     assertFalse(((Disposable)observer).isDisposed());
 
-                    Disposable bs1 = Disposables.empty();
-                    Disposable bs2 = Disposables.empty();
+                    Disposable bs1 = Disposable.empty();
+                    Disposable bs2 = Disposable.empty();
 
                     observer.onSubscribe(bs1);
 
@@ -1529,8 +1529,8 @@ protected void subscribeActual(Observer observer) {
 
                     assertFalse(((Disposable)observer).isDisposed());
 
-                    Disposable bs1 = Disposables.empty();
-                    Disposable bs2 = Disposables.empty();
+                    Disposable bs1 = Disposable.empty();
+                    Disposable bs2 = Disposable.empty();
 
                     observer.onSubscribe(bs1);
 
@@ -1602,7 +1602,7 @@ public void bufferExactBoundaryBadSource() {
         Observable ps = new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
                 observer.onNext(1);
                 observer.onComplete();
@@ -1613,7 +1613,7 @@ protected void subscribeActual(Observer observer) {
         Observable b = new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 ref.set(observer);
             }
         };
@@ -1658,7 +1658,7 @@ public void timedInternalState() {
         BufferExactUnboundedObserver> sub = new BufferExactUnboundedObserver>(
                 to, Functions.justSupplier((List)new ArrayList()), 1, TimeUnit.SECONDS, sch);
 
-        sub.onSubscribe(Disposables.empty());
+        sub.onSubscribe(Disposable.empty());
 
         assertFalse(sub.isDisposed());
 
@@ -1708,7 +1708,7 @@ public void timedSkipInternalState() {
         BufferSkipBoundedObserver> sub = new BufferSkipBoundedObserver>(
                 to, Functions.justSupplier((List)new ArrayList()), 1, 1, TimeUnit.SECONDS, sch.createWorker());
 
-        sub.onSubscribe(Disposables.empty());
+        sub.onSubscribe(Disposable.empty());
 
         sub.enter();
         sub.onComplete();
@@ -1736,7 +1736,7 @@ public List get() throws Exception {
                     }
                 }, 1, 1, TimeUnit.SECONDS, sch.createWorker());
 
-        sub.onSubscribe(Disposables.empty());
+        sub.onSubscribe(Disposable.empty());
 
         sub.run();
 
@@ -1755,7 +1755,7 @@ public void timedSizeBufferAlreadyCleared() {
                         1, TimeUnit.SECONDS, 1, false, sch.createWorker())
         ;
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
 
         sub.onSubscribe(bs);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java
index c3695b24d6..20c4fde308 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java
@@ -20,12 +20,12 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.observers.TestObserver;
@@ -64,7 +64,7 @@ public void cache() throws InterruptedException {
 
             @Override
             public void subscribe(final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 new Thread(new Runnable() {
 
                     @Override
@@ -198,7 +198,7 @@ public void noMissingBackpressureException() {
         Observable firehose = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer t) {
-                t.onSubscribe(Disposables.empty());
+                t.onSubscribe(Disposable.empty());
                 for (int i = 0; i < m; i++) {
                     t.onNext(i);
                 }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapCompletableTest.java
index a3cd0dcdb5..55b880a9f9 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapCompletableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapCompletableTest.java
@@ -85,7 +85,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onComplete();
@@ -248,4 +248,4 @@ public CompletableSource apply(Integer v) throws Exception {
             }
         };
     }
-}
\ No newline at end of file
+}
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java
index ace3f55493..fa72be7ac6 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java
@@ -20,12 +20,12 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -663,7 +663,7 @@ public void badInnerSource() {
             @Override
             protected void subscribeActual(Observer o) {
                 ts0[0] = o;
-                o.onSubscribe(Disposables.empty());
+                o.onSubscribe(Disposable.empty());
                 o.onError(new TestException("First"));
             }
         }), 2, ImmediateThinScheduler.INSTANCE)
@@ -689,7 +689,7 @@ public void badInnerSourceDelayError() {
             @Override
             protected void subscribeActual(Observer o) {
                 ts0[0] = o;
-                o.onSubscribe(Disposables.empty());
+                o.onSubscribe(Disposable.empty());
                 o.onError(new TestException("First"));
             }
         }), true, 2, ImmediateThinScheduler.INSTANCE)
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java
index 61b1f1579b..769f42f984 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java
@@ -150,7 +150,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onComplete();
@@ -181,7 +181,7 @@ public void badSourceDelayError() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onComplete();
@@ -355,7 +355,7 @@ public ObservableSource apply(Integer v) throws Exception {
                         @Override
                         protected void subscribeActual(Observer observer) {
                             o[0] = observer;
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             observer.onComplete();
                         }
                     };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java
index 0e8c64b92d..354c2afc27 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java
@@ -86,7 +86,7 @@ public void concatObservableOfObservables() {
 
             @Override
             public void subscribe(Observer> observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 // simulate what would happen in an Observable
                 observer.onNext(odds);
                 observer.onNext(even);
@@ -161,7 +161,7 @@ public void nestedAsyncConcat() throws InterruptedException {
 
             @Override
             public void subscribe(final Observer> observer) {
-                final Disposable d = Disposables.empty();
+                final Disposable d = Disposable.empty();
                 observer.onSubscribe(d);
                 parent.set(new Thread(new Runnable() {
 
@@ -348,7 +348,7 @@ public void concatNonBlockingObservables() {
 
             @Override
             public void subscribe(Observer> observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 // simulate what would happen in an Observable
                 observer.onNext(Observable.unsafeCreate(w1));
                 observer.onNext(Observable.unsafeCreate(w2));
@@ -666,7 +666,7 @@ public void concatWithNonCompliantSourceDoubleOnComplete() {
 
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext("hello");
                 observer.onComplete();
                 observer.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java
index 65d85f20d5..4ca6d1bb7b 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java
@@ -106,10 +106,10 @@ public void badSource() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
-                Disposable bs2 = Disposables.empty();
+                Disposable bs2 = Disposable.empty();
                 observer.onSubscribe(bs2);
 
                 assertFalse(bs1.isDisposed());
@@ -127,7 +127,7 @@ public void consumerDisposed() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
                 assertFalse(((Disposable)observer).isDisposed());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java
index 4d4f94954a..cd35a91ad1 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java
@@ -117,7 +117,7 @@ public void consumerDisposed() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
                 assertFalse(((Disposable)observer).isDisposed());
@@ -138,10 +138,10 @@ public void badSource() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
-                Disposable bs2 = Disposables.empty();
+                Disposable bs2 = Disposable.empty();
                 observer.onSubscribe(bs2);
 
                 assertFalse(bs1.isDisposed());
@@ -159,10 +159,10 @@ public void badSource2() {
         Flowable.empty().concatWith(new Maybe() {
             @Override
             protected void subscribeActual(MaybeObserver observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
-                Disposable bs2 = Disposables.empty();
+                Disposable bs2 = Disposable.empty();
                 observer.onSubscribe(bs2);
 
                 assertFalse(bs1.isDisposed());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java
index 73a0ff3c0d..45c0b6f269 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java
@@ -90,7 +90,7 @@ public void consumerDisposed() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
                 assertFalse(((Disposable)observer).isDisposed());
@@ -111,10 +111,10 @@ public void badSource() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
-                Disposable bs2 = Disposables.empty();
+                Disposable bs2 = Disposable.empty();
                 observer.onSubscribe(bs2);
 
                 assertFalse(bs1.isDisposed());
@@ -132,10 +132,10 @@ public void badSource2() {
         Flowable.empty().concatWith(new Single() {
             @Override
             protected void subscribeActual(SingleObserver observer) {
-                Disposable bs1 = Disposables.empty();
+                Disposable bs1 = Disposable.empty();
                 observer.onSubscribe(bs1);
 
-                Disposable bs2 = Disposables.empty();
+                Disposable bs2 = Disposable.empty();
                 observer.onSubscribe(bs2);
 
                 assertFalse(bs1.isDisposed());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreateTest.java
index 4a22746c54..f5be343b40 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreateTest.java
@@ -36,7 +36,7 @@ public void nullArgument() {
 
     @Test
     public void basic() {
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Observable.create(new ObservableOnSubscribe() {
             @Override
@@ -61,8 +61,8 @@ public void subscribe(ObservableEmitter e) throws Exception {
 
     @Test
     public void basicWithCancellable() {
-        final Disposable d1 = Disposables.empty();
-        final Disposable d2 = Disposables.empty();
+        final Disposable d1 = Disposable.empty();
+        final Disposable d2 = Disposable.empty();
 
         Observable.create(new ObservableOnSubscribe() {
             @Override
@@ -94,7 +94,7 @@ public void cancel() throws Exception {
 
     @Test
     public void basicWithError() {
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Observable.create(new ObservableOnSubscribe() {
             @Override
@@ -118,7 +118,7 @@ public void subscribe(ObservableEmitter e) throws Exception {
 
     @Test
     public void basicSerialized() {
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Observable.create(new ObservableOnSubscribe() {
             @Override
@@ -145,7 +145,7 @@ public void subscribe(ObservableEmitter e) throws Exception {
 
     @Test
     public void basicWithErrorSerialized() {
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Observable.create(new ObservableOnSubscribe() {
             @Override
@@ -174,7 +174,7 @@ public void wrap() {
         Observable.wrap(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext(1);
                 observer.onNext(2);
                 observer.onNext(3);
@@ -192,7 +192,7 @@ public void unsafe() {
         Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext(1);
                 observer.onNext(2);
                 observer.onNext(3);
@@ -322,7 +322,7 @@ public void onErrorCrash() {
         Observable.create(new ObservableOnSubscribe() {
             @Override
             public void subscribe(ObservableEmitter e) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 e.setDisposable(d);
                 try {
                     e.onError(new IOException());
@@ -358,7 +358,7 @@ public void onCompleteCrash() {
         Observable.create(new ObservableOnSubscribe() {
             @Override
             public void subscribe(ObservableEmitter e) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 e.setDisposable(d);
                 try {
                     e.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java
index d66de923f8..af60b93018 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java
@@ -55,7 +55,7 @@ public void debounceWithCompleted() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 publishNext(observer, 100, "one");    // Should be skipped since "two" will arrive before the timeout expires.
                 publishNext(observer, 400, "two");    // Should be published since "three" will arrive after the timeout expires.
                 publishNext(observer, 900, "three");   // Should be skipped since onComplete will arrive before the timeout expires.
@@ -81,7 +81,7 @@ public void debounceNeverEmits() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 // all should be skipped since they are happening faster than the 200ms timeout
                 publishNext(observer, 100, "a");    // Should be skipped
                 publishNext(observer, 200, "b");    // Should be skipped
@@ -111,7 +111,7 @@ public void debounceWithError() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 Exception error = new TestException();
                 publishNext(observer, 100, "one");    // Should be published since "two" will arrive after the timeout expires.
                 publishNext(observer, 600, "two");    // Should be skipped since onError will arrive before the timeout expires.
@@ -332,7 +332,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                     observer.onNext(1);
                     observer.onError(new TestException());
@@ -418,7 +418,7 @@ public void disposedInOnComplete() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 to.dispose();
                 observer.onComplete();
             }
@@ -442,7 +442,7 @@ public ObservableSource apply(Integer o) throws Exception {
                 return new Observable() {
                     @Override
                     protected void subscribeActual(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         ref.set(observer);
                     }
                 };
@@ -475,7 +475,7 @@ public void timedDisposedIgnoredBySource() {
             @Override
             protected void subscribeActual(
                     Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 to.dispose();
                 observer.onNext(1);
                 observer.onComplete();
@@ -491,7 +491,7 @@ public void timedLateEmit() {
         DebounceTimedObserver sub = new DebounceTimedObserver(
                 to, 1, TimeUnit.SECONDS, new TestScheduler().createWorker());
 
-        sub.onSubscribe(Disposables.empty());
+        sub.onSubscribe(Disposable.empty());
 
         DebounceEmitter de = new DebounceEmitter(1, 50, sub);
         de.run();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java
index 84a6b06b06..5d3e2b6be1 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java
@@ -18,10 +18,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -202,7 +202,7 @@ public void eventsAfterDematerializedTerminal() {
             new Observable>() {
                 @Override
                 protected void subscribeActual(Observer> observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(Notification.createOnComplete());
                     observer.onNext(Notification.createOnNext(1));
                     observer.onNext(Notification.createOnError(new TestException("First")));
@@ -225,7 +225,7 @@ public void nonNotificationInstanceAfterDispose() {
         new Observable>() {
             @Override
             protected void subscribeActual(Observer> observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext(Notification.createOnComplete());
                 observer.onNext(Notification.createOnNext(1));
             }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java
index 2a9d889ba2..888a39cb89 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java
@@ -205,7 +205,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java
index f28a9fb18b..13122988cf 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java
@@ -19,11 +19,11 @@
 import java.io.IOException;
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 import org.mockito.InOrder;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.fuseable.QueueFuseable;
@@ -196,7 +196,7 @@ public void ignoreCancel() {
             Observable.wrap(new ObservableSource() {
                 @Override
                 public void subscribe(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onNext(2);
                     observer.onNext(3);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java
index 4205fad5e2..30f0cb2d6f 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java
@@ -21,10 +21,10 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -232,7 +232,7 @@ public void ignoreCancel() {
             Observable.wrap(new ObservableSource() {
                 @Override
                 public void subscribe(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onNext(2);
                     observer.onError(new IOException());
@@ -262,7 +262,7 @@ public void onErrorAfterCrash() {
             Observable.wrap(new ObservableSource() {
                 @Override
                 public void subscribe(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException());
                 }
             })
@@ -289,7 +289,7 @@ public void onCompleteAfterCrash() {
             Observable.wrap(new ObservableSource() {
                 @Override
                 public void subscribe(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                 }
             })
@@ -313,7 +313,7 @@ public void onCompleteCrash() {
         Observable.wrap(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
             }
         })
@@ -335,7 +335,7 @@ public void ignoreCancelConditional() {
             Observable.wrap(new ObservableSource() {
                 @Override
                 public void subscribe(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onNext(2);
                     observer.onError(new IOException());
@@ -366,7 +366,7 @@ public void onErrorAfterCrashConditional() {
             Observable.wrap(new ObservableSource() {
                 @Override
                 public void subscribe(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException());
                 }
             })
@@ -410,7 +410,7 @@ public void onCompleteAfterCrashConditional() {
             Observable.wrap(new ObservableSource() {
                 @Override
                 public void subscribe(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                 }
             })
@@ -435,7 +435,7 @@ public void onCompleteCrashConditional() {
         Observable.wrap(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
             }
         })
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java
index ef57c50225..26cd2b340b 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java
@@ -74,7 +74,7 @@ public void doOnUnSubscribeWorksWithRefCount() throws Exception {
 
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 onSubscribed.incrementAndGet();
                 sref.set(observer);
             }
@@ -111,7 +111,7 @@ public void accept(Disposable d) {
     public void onSubscribeCrash() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable bs = Disposables.empty();
+            final Disposable bs = Disposable.empty();
 
             new Observable() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtTest.java
index a11255abf2..679915dc51 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtTest.java
@@ -17,12 +17,12 @@
 
 import java.util.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.plugins.RxJavaPlugins;
@@ -233,7 +233,7 @@ public void badSourceObservable() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onNext(2);
@@ -259,7 +259,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onNext(2);
@@ -284,7 +284,7 @@ public void badSource2() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onNext(2);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java
index b07a13a6de..f5bf13b231 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java
@@ -372,7 +372,7 @@ public CompletableSource apply(Integer v) throws Exception {
                 return new Completable() {
                     @Override
                     protected void subscribeActual(CompletableObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
@@ -447,7 +447,7 @@ public CompletableSource apply(Integer v) throws Exception {
                 return new Completable() {
                     @Override
                     protected void subscribeActual(CompletableObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java
index fbd55ae32a..e1133ee311 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java
@@ -325,7 +325,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -348,7 +348,7 @@ public void badInnerSource() {
             .flatMapMaybe(Functions.justFunction(new Maybe() {
                 @Override
                 protected void subscribeActual(MaybeObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -437,7 +437,7 @@ public MaybeSource apply(Integer v) throws Exception {
                 return new Maybe() {
                     @Override
                     protected void subscribeActual(MaybeObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java
index 77db13d5a1..b8c7b351be 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java
@@ -274,7 +274,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -297,7 +297,7 @@ public void badInnerSource() {
             .flatMapSingle(Functions.justFunction(new Single() {
                 @Override
                 protected void subscribeActual(SingleObserver observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -352,7 +352,7 @@ public SingleSource apply(Integer v) throws Exception {
                 return new Single() {
                     @Override
                     protected void subscribeActual(SingleObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
 
                         assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java
index cf31f26c3a..38defaf049 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java
@@ -992,7 +992,7 @@ public Integer apply(Integer v)
                     }
                 }, true, Integer.MAX_VALUE, 128);
 
-        merger.onSubscribe(Disposables.empty());
+        merger.onSubscribe(Disposable.empty());
         merger.getAndIncrement();
 
         merger.onNext(0);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java
index 14ff56a665..1eaf76cda8 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java
@@ -188,7 +188,7 @@ public void groupedEventStream() throws Throwable {
 
             @Override
             public void subscribe(final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 System.out.println("*** Subscribing to EventStream ***");
                 subscribeCounter.incrementAndGet();
                 new Thread(new Runnable() {
@@ -602,7 +602,7 @@ public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsAndThenComplete
 
             @Override
             public void subscribe(Observer sub) {
-                sub.onSubscribe(Disposables.empty());
+                sub.onSubscribe(Disposable.empty());
                 sub.onNext(1);
                 sub.onNext(2);
                 sub.onNext(1);
@@ -681,7 +681,7 @@ public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsWhichThenSubscr
 
             @Override
             public void subscribe(Observer sub) {
-                sub.onSubscribe(Disposables.empty());
+                sub.onSubscribe(Disposable.empty());
                 sub.onNext(1);
                 sub.onNext(2);
                 sub.onNext(1);
@@ -773,7 +773,7 @@ public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsWhichThenObserv
 
             @Override
             public void subscribe(Observer sub) {
-                sub.onSubscribe(Disposables.empty());
+                sub.onSubscribe(Disposable.empty());
                 sub.onNext(1);
                 sub.onNext(2);
                 sub.onNext(1);
@@ -850,7 +850,7 @@ public void groupsWithNestedSubscribeOn() throws InterruptedException {
 
             @Override
             public void subscribe(Observer sub) {
-                sub.onSubscribe(Disposables.empty());
+                sub.onSubscribe(Disposable.empty());
                 sub.onNext(1);
                 sub.onNext(2);
                 sub.onNext(1);
@@ -907,7 +907,7 @@ public void groupsWithNestedObserveOn() throws InterruptedException {
 
             @Override
             public void subscribe(Observer sub) {
-                sub.onSubscribe(Disposables.empty());
+                sub.onSubscribe(Disposable.empty());
                 sub.onNext(1);
                 sub.onNext(2);
                 sub.onNext(1);
@@ -968,7 +968,7 @@ Observable SYNC_INFINITE_OBSERVABLE_OF_EVENT(final int numGroups, final A
 
             @Override
             public void subscribe(final Observer op) {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 op.onSubscribe(d);
                 subscribeCounter.incrementAndGet();
                 int i = 0;
@@ -1427,7 +1427,7 @@ public void onNext(GroupedObservable o) {
                 new ObservableSource() {
                     @Override
                     public void subscribe(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         observer.onNext(0);
                         observer.onNext(1);
                         observer.onError(e);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoinTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoinTest.java
index f10e62016d..aa17361026 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoinTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoinTest.java
@@ -20,11 +20,11 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 import org.mockito.MockitoAnnotations;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -387,7 +387,7 @@ public void badOuterSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onError(new TestException("Second"));
                 }
@@ -424,7 +424,7 @@ public void badEndSource() {
                         @Override
                         protected void subscribeActual(Observer observer) {
                             o[0] = observer;
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             observer.onError(new TestException("First"));
                         }
                     }),
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java
index ee8931853f..7a84d0c981 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java
@@ -13,10 +13,10 @@
 
 package io.reactivex.rxjava3.internal.operators.observable;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -67,7 +67,7 @@ protected void subscribeActual(Observer observer) {
                         Functions.justFunction(Observable.just(2)),
                         Functions.justSupplier(Observable.just(3))
                 );
-                mn.onSubscribe(Disposables.empty());
+                mn.onSubscribe(Disposable.empty());
             }
         });
     }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java
index 4549f6c514..daf82d84ee 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java
@@ -18,12 +18,12 @@
 import java.util.*;
 import java.util.concurrent.ExecutionException;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.observers.DefaultObserver;
 import io.reactivex.rxjava3.testsupport.*;
@@ -150,7 +150,7 @@ private static class TestAsyncErrorObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             t = new Thread(new Runnable() {
 
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java
index 82fd2e28fa..877b75b70a 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java
@@ -20,12 +20,12 @@
 import java.util.*;
 import java.util.concurrent.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.*;
 import io.reactivex.rxjava3.observers.DefaultObserver;
 import io.reactivex.rxjava3.testsupport.*;
@@ -219,7 +219,7 @@ public void mergeObservableOfObservables() {
 
             @Override
             public void subscribe(Observer> observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 // simulate what would happen in an Observable
                 observer.onNext(o1);
                 observer.onNext(o2);
@@ -317,7 +317,7 @@ private static class TestSynchronousObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             observer.onNext("hello");
             observer.onComplete();
         }
@@ -328,7 +328,7 @@ private static class TestASynchronousObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             t = new Thread(new Runnable() {
 
                 @Override
@@ -352,7 +352,7 @@ private static class TestErrorObservable implements ObservableSource {
 
         @Override
         public void subscribe(Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             boolean errorThrown = false;
             for (String s : valuesToReturn) {
                 if (s == null) {
@@ -383,7 +383,7 @@ private static class TestAsyncErrorObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             t = new Thread(new Runnable() {
 
                 @Override
@@ -453,7 +453,7 @@ public void errorInParentObservableDelayed() throws Exception {
             Observable> parentObservable = Observable.unsafeCreate(new ObservableSource>() {
                 @Override
                 public void subscribe(Observer> op) {
-                    op.onSubscribe(Disposables.empty());
+                    op.onSubscribe(Disposable.empty());
                     op.onNext(Observable.unsafeCreate(o1));
                     op.onNext(Observable.unsafeCreate(o2));
                     op.onError(new NullPointerException("throwing exception in parent"));
@@ -480,7 +480,7 @@ private static class TestASynchronous1sDelayedObservable implements ObservableSo
 
         @Override
         public void subscribe(final Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             t = new Thread(new Runnable() {
 
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java
index 8132fbe229..ac69ffbaa7 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java
@@ -19,12 +19,12 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.internal.schedulers.IoScheduler;
 import io.reactivex.rxjava3.observers.TestObserver;
 import io.reactivex.rxjava3.schedulers.Schedulers;
@@ -99,7 +99,7 @@ private static class SubscriptionCheckObservable implements ObservableSource t1) {
-            t1.onSubscribe(Disposables.empty());
+            t1.onSubscribe(Disposable.empty());
             new Thread(new Runnable() {
 
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java
index 8736bd4ae3..c198ff83f0 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java
@@ -79,7 +79,7 @@ public void mergeObservableOfObservables() {
 
             @Override
             public void subscribe(Observer> observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 // simulate what would happen in an Observable
                 observer.onNext(o1);
                 observer.onNext(o2);
@@ -135,7 +135,7 @@ public void unSubscribeObservableOfObservables() throws InterruptedException {
             @Override
             public void subscribe(final Observer> observer) {
                 // verbose on purpose so I can track the inside of it
-                final Disposable upstream = Disposables.fromRunnable(new Runnable() {
+                final Disposable upstream = Disposable.fromRunnable(new Runnable() {
                     @Override
                     public void run() {
                         System.out.println("*** unsubscribed");
@@ -342,7 +342,7 @@ private static class TestSynchronousObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             observer.onNext("hello");
             observer.onComplete();
         }
@@ -354,7 +354,7 @@ private static class TestASynchronousObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             t = new Thread(new Runnable() {
 
                 @Override
@@ -385,7 +385,7 @@ private static class TestErrorObservable implements ObservableSource {
 
         @Override
         public void subscribe(Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             for (String s : valuesToReturn) {
                 if (s == null) {
                     System.out.println("throwing exception");
@@ -486,7 +486,7 @@ public void subscribe(final Observer child) {
                 .subscribe(new Observer() {
                     @Override
                     public void onSubscribe(final Disposable d) {
-                        child.onSubscribe(Disposables.fromRunnable(new Runnable() {
+                        child.onSubscribe(Disposable.fromRunnable(new Runnable() {
                             @Override
                             public void run() {
                                 unsubscribed.set(true);
@@ -545,7 +545,7 @@ public void concurrencyWithSleeping() {
             public void subscribe(final Observer observer) {
                 Worker inner = Schedulers.newThread().createWorker();
                 final CompositeDisposable as = new CompositeDisposable();
-                as.add(Disposables.empty());
+                as.add(Disposable.empty());
                 as.add(inner);
 
                 observer.onSubscribe(as);
@@ -595,7 +595,7 @@ public void concurrencyWithBrokenOnCompleteContract() {
             public void subscribe(final Observer observer) {
                 Worker inner = Schedulers.newThread().createWorker();
                 final CompositeDisposable as = new CompositeDisposable();
-                as.add(Disposables.empty());
+                as.add(Disposable.empty());
                 as.add(inner);
 
                 observer.onSubscribe(as);
@@ -966,7 +966,7 @@ public Observable apply(final Integer i) {
 
                     @Override
                     public void subscribe(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         if (i < 500) {
                             try {
                                 Thread.sleep(1);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java
index 8cdb78df76..e5414dfe7f 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java
@@ -123,7 +123,7 @@ public void isDisposed() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
 
                 assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java
index 5fd7763825..4949ad2081 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java
@@ -182,7 +182,7 @@ public void onErrorMainOverflow() {
             TestObserver to = new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observerRef.set(observer);
                 }
             }
@@ -234,7 +234,7 @@ public void isDisposed() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
 
                 assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java
index a4e38c0936..28c6099d4c 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java
@@ -174,7 +174,7 @@ public void onErrorMainOverflow() {
             TestObserver to = new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observerRef.set(observer);
                 }
             }
@@ -226,7 +226,7 @@ public void isDisposed() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
 
                 assertFalse(((Disposable)observer).isDisposed());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java
index 0602249ceb..286d115edf 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java
@@ -495,7 +495,7 @@ public void badSource() {
             TestObserver to = new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                     observer.onNext(1);
                     observer.onError(new TestException());
@@ -675,7 +675,7 @@ public void nonFusedPollThrows() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
 
                 @SuppressWarnings("unchecked")
                 ObserveOnObserver oo = (ObserveOnObserver)observer;
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java
index 2761a6c14b..69a9b0f7cd 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java
@@ -21,12 +21,12 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.reactivestreams.Subscription;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.internal.functions.Functions;
 import io.reactivex.rxjava3.observers.TestObserver;
@@ -42,7 +42,7 @@ public void resumeNextWithSynchronousExecution() {
 
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext("one");
                 observer.onError(new Throwable("injected failure"));
                 observer.onNext("two");
@@ -199,7 +199,7 @@ static class TestObservable implements ObservableSource {
         @Override
         public void subscribe(final Observer observer) {
             System.out.println("TestObservable subscribed to ...");
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             t = new Thread(new Runnable() {
 
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java
index 3f20a8a7db..d9520fe6d2 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java
@@ -20,11 +20,11 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.observers.TestObserver;
@@ -188,7 +188,7 @@ private static class TestObservable implements ObservableSource {
 
         @Override
         public void subscribe(final Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             System.out.println("TestObservable subscribed to ...");
             t = new Thread(new Runnable() {
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java
index 38f0140e5d..ee80dc2cab 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java
@@ -45,7 +45,7 @@ public void publish() throws InterruptedException {
 
             @Override
             public void subscribe(final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 new Thread(new Runnable() {
 
                     @Override
@@ -354,7 +354,7 @@ public void connectIsIdempotent() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer t) {
-                t.onSubscribe(Disposables.empty());
+                t.onSubscribe(Disposable.empty());
                 calls.getAndIncrement();
             }
         });
@@ -589,7 +589,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onComplete();
                     observer.onNext(2);
@@ -715,7 +715,7 @@ protected void subscribeActual(Observer observer) {
         .connect()
         .dispose();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
 
         sub[0].onSubscribe(bs);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceTest.java
index f05e007068..d997e9899f 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceTest.java
@@ -19,10 +19,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.plugins.RxJavaPlugins;
@@ -291,7 +291,7 @@ public void reduceMaybeBadSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                     observer.onNext(1);
                     observer.onError(new TestException());
@@ -345,7 +345,7 @@ public void seedBadSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                     observer.onNext(1);
                     observer.onError(new TestException());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java
index d5bcf0440e..7e862e1ad2 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java
@@ -310,7 +310,7 @@ private Observable synchronousInterval() {
             @Override
             public void subscribe(Observer observer) {
                 final AtomicBoolean cancel = new AtomicBoolean();
-                observer.onSubscribe(Disposables.fromRunnable(new Runnable() {
+                observer.onSubscribe(Disposable.fromRunnable(new Runnable() {
                     @Override
                     public void run() {
                         cancel.set(true);
@@ -338,7 +338,7 @@ public void onlyFirstShouldSubscribeAndLastUnsubscribe() {
             @Override
             public void subscribe(Observer observer) {
                 subscriptionCount.incrementAndGet();
-                observer.onSubscribe(Disposables.fromRunnable(new Runnable() {
+                observer.onSubscribe(Disposable.fromRunnable(new Runnable() {
                     @Override
                     public void run() {
                             unsubscriptionCount.incrementAndGet();
@@ -623,7 +623,7 @@ public void reset() {
 
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.disposed());
+                observer.onSubscribe(Disposable.disposed());
             }
         }.refCount();
 
@@ -779,7 +779,7 @@ static final class BadObservableSubscribe extends ConnectableObservable
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -806,7 +806,7 @@ public void reset() {
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -814,7 +814,7 @@ public void connect(Consumer connection) {
 
         @Override
         protected void subscribeActual(Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
         }
     }
 
@@ -832,7 +832,7 @@ public void reset() {
 
         @Override
         protected void subscribeActual(Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
         }
     }
 
@@ -882,7 +882,7 @@ static final class BadObservableSubscribe2 extends ConnectableObservable
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -896,7 +896,7 @@ public void reset() {
         @Override
         protected void subscribeActual(Observer observer) {
             if (++count == 1) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
             } else {
                 throw new TestException("subscribeActual");
             }
@@ -922,7 +922,7 @@ static final class BadObservableConnect2 extends ConnectableObservable {
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -935,7 +935,7 @@ public void reset() {
 
         @Override
         protected void subscribeActual(Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             observer.onComplete();
         }
     }
@@ -1163,7 +1163,7 @@ static final class BadObservableDoubleOnX extends ConnectableObservable
         @Override
         public void connect(Consumer connection) {
             try {
-                connection.accept(Disposables.empty());
+                connection.accept(Disposable.empty());
             } catch (Throwable ex) {
                 throw ExceptionHelper.wrapOrThrow(ex);
             }
@@ -1176,8 +1176,8 @@ public void reset() {
 
         @Override
         protected void subscribeActual(Observer observer) {
-            observer.onSubscribe(Disposables.empty());
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
+            observer.onSubscribe(Disposable.empty());
             observer.onComplete();
             observer.onComplete();
             observer.onError(new TestException());
@@ -1341,7 +1341,7 @@ public void timeoutResetsSource() {
         ObservableRefCount o = (ObservableRefCount)tco.refCount();
 
         RefConnection rc = new RefConnection(o);
-        rc.set(Disposables.empty());
+        rc.set(Disposable.empty());
         o.connection = rc;
 
         o.timeout(rc);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java
index 55906d1214..f7b1d93b0b 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java
@@ -74,7 +74,7 @@ public void repeatTakeWithSubscribeOn() throws InterruptedException {
 
             @Override
             public void subscribe(Observer sub) {
-                sub.onSubscribe(Disposables.empty());
+                sub.onSubscribe(Disposable.empty());
                 counter.incrementAndGet();
                 sub.onNext(1);
                 sub.onNext(2);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java
index e1c0bf6f8c..ba3a7d6974 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java
@@ -892,7 +892,7 @@ public void cache() throws InterruptedException {
 
             @Override
             public void subscribe(final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 new Thread(new Runnable() {
 
                     @Override
@@ -1028,7 +1028,7 @@ public void noMissingBackpressureException() {
         Observable firehose = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer t) {
-                t.onSubscribe(Disposables.empty());
+                t.onSubscribe(Disposable.empty());
                 for (int i = 0; i < m; i++) {
                     t.onNext(i);
                 }
@@ -1232,7 +1232,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onNext(1);
                     observer.onError(new TestException("Second"));
@@ -1438,7 +1438,7 @@ protected void subscribeActual(Observer observer) {
         .connect()
         .dispose();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
 
         sub[0].onSubscribe(bs);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java
index 5ef40f340c..e55d5ff716 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java
@@ -892,7 +892,7 @@ public void cache() throws InterruptedException {
 
             @Override
             public void subscribe(final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 new Thread(new Runnable() {
 
                     @Override
@@ -1028,7 +1028,7 @@ public void noMissingBackpressureException() {
         Observable firehose = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer t) {
-                t.onSubscribe(Disposables.empty());
+                t.onSubscribe(Disposable.empty());
                 for (int i = 0; i < m; i++) {
                     t.onNext(i);
                 }
@@ -1232,7 +1232,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onNext(1);
                     observer.onError(new TestException("Second"));
@@ -1438,7 +1438,7 @@ protected void subscribeActual(Observer observer) {
         .connect()
         .dispose();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
 
         sub[0].onSubscribe(bs);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java
index 9143762a5a..d5a70e0e47 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java
@@ -30,7 +30,7 @@ public void disposed() {
         TestObserver to = new TestObserver();
         ObserverResourceWrapper orw = new ObserverResourceWrapper(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         orw.onSubscribe(d);
 
@@ -54,8 +54,8 @@ public void onErrorDisposes() {
         TestObserver to = new TestObserver();
         ObserverResourceWrapper orw = new ObserverResourceWrapper(to);
 
-        Disposable d = Disposables.empty();
-        Disposable d1 = Disposables.empty();
+        Disposable d = Disposable.empty();
+        Disposable d1 = Disposable.empty();
 
         orw.setResource(d1);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java
index 26377c742c..82d96dc5d7 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java
@@ -52,7 +52,7 @@ public void iterativeBackoff() {
 
             @Override
             public void subscribe(Observer t1) {
-                t1.onSubscribe(Disposables.empty());
+                t1.onSubscribe(Disposable.empty());
                 System.out.println(count.get() + " @ " + String.valueOf(last - System.currentTimeMillis()));
                 last = System.currentTimeMillis();
                 if (count.getAndDecrement() == 0) {
@@ -248,7 +248,7 @@ public void singleSubscriptionOnFirst() throws Exception {
         ObservableSource onSubscribe = new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 final int emit = inc.incrementAndGet();
                 observer.onNext(emit);
                 observer.onComplete();
@@ -397,7 +397,7 @@ public static class FuncWithErrors implements ObservableSource {
 
         @Override
         public void subscribe(final Observer o) {
-            o.onSubscribe(Disposables.empty());
+            o.onSubscribe(Disposable.empty());
             o.onNext("beginningEveryTime");
             int i = count.getAndIncrement();
             if (i < numFailures) {
@@ -432,7 +432,7 @@ public void retryAllowsSubscriptionAfterAllSubscriptionsUnsubscribed() throws In
             @Override
             public void subscribe(Observer observer) {
                 subsCount.incrementAndGet();
-                observer.onSubscribe(Disposables.fromRunnable(new Runnable() {
+                observer.onSubscribe(Disposable.fromRunnable(new Runnable() {
                     @Override
                     public void run() {
                             subsCount.decrementAndGet();
@@ -491,7 +491,7 @@ public void sourceObservableRetry1() throws InterruptedException {
         ObservableSource onSubscribe = new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 subsCount.incrementAndGet();
                 observer.onError(new RuntimeException("failed"));
             }
@@ -510,7 +510,7 @@ public void sourceObservableRetry0() throws InterruptedException {
         ObservableSource onSubscribe = new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 subsCount.incrementAndGet();
                 observer.onError(new RuntimeException("failed"));
             }
@@ -539,7 +539,7 @@ static final class SlowObservable implements ObservableSource {
         @Override
         public void subscribe(final Observer observer) {
             final AtomicBoolean terminate = new AtomicBoolean(false);
-            observer.onSubscribe(Disposables.fromRunnable(new Runnable() {
+            observer.onSubscribe(Disposable.fromRunnable(new Runnable() {
                 @Override
                 public void run() {
                         terminate.set(true);
@@ -840,7 +840,7 @@ public void issue1900SourceNotSupportingBackpressure() {
 
             @Override
             public void subscribe(Observer o) {
-                o.onSubscribe(Disposables.empty());
+                o.onSubscribe(Disposable.empty());
                 for (int i = 0; i < NUM_MSG; i++) {
                     o.onNext("msg:" + count.incrementAndGet());
                 }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java
index 38f293cb4a..d7d20c1347 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java
@@ -77,7 +77,7 @@ public void retryTwice() {
             int count;
             @Override
             public void subscribe(Observer t1) {
-                t1.onSubscribe(Disposables.empty());
+                t1.onSubscribe(Disposable.empty());
                 count++;
                 t1.onNext(0);
                 t1.onNext(1);
@@ -112,7 +112,7 @@ public void retryTwiceAndGiveUp() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer t1) {
-                t1.onSubscribe(Disposables.empty());
+                t1.onSubscribe(Disposable.empty());
                 t1.onNext(0);
                 t1.onNext(1);
                 t1.onError(new TestException());
@@ -141,7 +141,7 @@ public void retryOnSpecificException() {
             int count;
             @Override
             public void subscribe(Observer t1) {
-                t1.onSubscribe(Disposables.empty());
+                t1.onSubscribe(Disposable.empty());
                 count++;
                 t1.onNext(0);
                 t1.onNext(1);
@@ -178,7 +178,7 @@ public void retryOnSpecificExceptionAndNotOther() {
             int count;
             @Override
             public void subscribe(Observer t1) {
-                t1.onSubscribe(Disposables.empty());
+                t1.onSubscribe(Disposable.empty());
                 count++;
                 t1.onNext(0);
                 t1.onNext(1);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTest.java
index 5335a75475..0861fc2596 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTest.java
@@ -50,7 +50,7 @@ public void sample() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(final Observer observer1) {
-                observer1.onSubscribe(Disposables.empty());
+                observer1.onSubscribe(Disposable.empty());
                 innerScheduler.schedule(new Runnable() {
                     @Override
                     public void run() {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java
index 5a7d312ede..590273ce60 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java
@@ -321,7 +321,7 @@ public void accept(Throwable t) throws Exception {
             Observable.unsafeCreate(new ObservableSource() {
                 @Override
                 public void subscribe(Observer o) {
-                    Disposable d = Disposables.empty();
+                    Disposable d = Disposable.empty();
                     o.onSubscribe(d);
                     o.onNext(1);
                     o.onNext(2);
@@ -346,7 +346,7 @@ public void scanFunctionThrowsAndUpstreamCompletesDoesNotResultInTwoTerminalEven
         Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer o) {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 o.onSubscribe(d);
                 o.onNext(1);
                 o.onNext(2);
@@ -369,7 +369,7 @@ public void scanFunctionThrowsAndUpstreamEmitsOnNextResultsInScanFunctionBeingCa
         Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer o) {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 o.onSubscribe(d);
                 o.onNext(1);
                 o.onNext(2);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerializeTest.java
index b0023e2989..2a2b7dff0d 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerializeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerializeTest.java
@@ -20,10 +20,10 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.observers.DefaultObserver;
 import io.reactivex.rxjava3.testsupport.TestHelper;
 
@@ -221,7 +221,7 @@ static class TestSingleThreadedObservable implements ObservableSource {
 
         @Override
         public void subscribe(final Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             System.out.println("TestSingleThreadedObservable subscribed to ...");
             t = new Thread(new Runnable() {
 
@@ -272,7 +272,7 @@ private static class TestMultiThreadedObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             System.out.println("TestMultiThreadedObservable subscribed to ...");
             final NullPointerException npe = new NullPointerException();
             t = new Thread(new Runnable() {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java
index 8d2a1706ba..26ffe95a71 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java
@@ -43,7 +43,7 @@ public void issue813() throws InterruptedException {
             @Override
             public void subscribe(
                     final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 scheduled.countDown();
                 try {
                     try {
@@ -79,7 +79,7 @@ public void onError() {
 
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onError(new RuntimeException("fail"));
             }
 
@@ -153,7 +153,7 @@ public void unsubscribeInfiniteStream() throws InterruptedException {
 
             @Override
             public void subscribe(Observer sub) {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 sub.onSubscribe(d);
                 for (int i = 1; !d.isDisposed(); i++) {
                     count.incrementAndGet();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmptyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmptyTest.java
index 00948a6e7f..34e24d81cb 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmptyTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmptyTest.java
@@ -54,7 +54,7 @@ public void switchWhenEmpty() throws Exception {
     @Test
     public void switchTriggerUnsubscribe() throws Exception {
 
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Observable withProducer = Observable.unsafeCreate(new ObservableSource() {
             @Override
@@ -96,7 +96,7 @@ public void onNext(Long aLong) {
 
     @Test
     public void switchShouldTriggerUnsubscribe() {
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Observable.unsafeCreate(new ObservableSource() {
             @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java
index dadabeeea3..5eb40104fc 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java
@@ -57,11 +57,11 @@ public void switchWhenOuterCompleteBeforeInner() {
         Observable> source = Observable.unsafeCreate(new ObservableSource>() {
             @Override
             public void subscribe(Observer> outerObserver) {
-                outerObserver.onSubscribe(Disposables.empty());
+                outerObserver.onSubscribe(Disposable.empty());
                 publishNext(outerObserver, 50, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 70, "one");
                         publishNext(innerObserver, 100, "two");
                         publishCompleted(innerObserver, 200);
@@ -86,11 +86,11 @@ public void switchWhenInnerCompleteBeforeOuter() {
         Observable> source = Observable.unsafeCreate(new ObservableSource>() {
             @Override
             public void subscribe(Observer> outerObserver) {
-                outerObserver.onSubscribe(Disposables.empty());
+                outerObserver.onSubscribe(Disposable.empty());
                 publishNext(outerObserver, 10, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 0, "one");
                         publishNext(innerObserver, 10, "two");
                         publishCompleted(innerObserver, 20);
@@ -100,7 +100,7 @@ public void subscribe(Observer innerObserver) {
                 publishNext(outerObserver, 100, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 0, "three");
                         publishNext(innerObserver, 10, "four");
                         publishCompleted(innerObserver, 20);
@@ -132,11 +132,11 @@ public void switchWithComplete() {
         Observable> source = Observable.unsafeCreate(new ObservableSource>() {
             @Override
             public void subscribe(Observer> outerObserver) {
-                outerObserver.onSubscribe(Disposables.empty());
+                outerObserver.onSubscribe(Disposable.empty());
                 publishNext(outerObserver, 50, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(final Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 60, "one");
                         publishNext(innerObserver, 100, "two");
                     }
@@ -145,7 +145,7 @@ public void subscribe(final Observer innerObserver) {
                 publishNext(outerObserver, 200, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(final Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 0, "three");
                         publishNext(innerObserver, 100, "four");
                     }
@@ -191,11 +191,11 @@ public void switchWithError() {
         Observable> source = Observable.unsafeCreate(new ObservableSource>() {
             @Override
             public void subscribe(Observer> outerObserver) {
-                outerObserver.onSubscribe(Disposables.empty());
+                outerObserver.onSubscribe(Disposable.empty());
                 publishNext(outerObserver, 50, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(final Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 50, "one");
                         publishNext(innerObserver, 100, "two");
                     }
@@ -204,7 +204,7 @@ public void subscribe(final Observer innerObserver) {
                 publishNext(outerObserver, 200, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 0, "three");
                         publishNext(innerObserver, 100, "four");
                     }
@@ -250,11 +250,11 @@ public void switchWithSubsequenceComplete() {
         Observable> source = Observable.unsafeCreate(new ObservableSource>() {
             @Override
             public void subscribe(Observer> outerObserver) {
-                outerObserver.onSubscribe(Disposables.empty());
+                outerObserver.onSubscribe(Disposable.empty());
                 publishNext(outerObserver, 50, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 50, "one");
                         publishNext(innerObserver, 100, "two");
                     }
@@ -263,7 +263,7 @@ public void subscribe(Observer innerObserver) {
                 publishNext(outerObserver, 130, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishCompleted(innerObserver, 0);
                     }
                 }));
@@ -271,7 +271,7 @@ public void subscribe(Observer innerObserver) {
                 publishNext(outerObserver, 150, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 50, "three");
                     }
                 }));
@@ -304,11 +304,11 @@ public void switchWithSubsequenceError() {
         Observable> source = Observable.unsafeCreate(new ObservableSource>() {
             @Override
             public void subscribe(Observer> observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 publishNext(observer, 50, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         publishNext(observer, 50, "one");
                         publishNext(observer, 100, "two");
                     }
@@ -317,7 +317,7 @@ public void subscribe(Observer observer) {
                 publishNext(observer, 130, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         publishError(observer, 0, new TestException());
                     }
                 }));
@@ -325,7 +325,7 @@ public void subscribe(Observer observer) {
                 publishNext(observer, 150, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         publishNext(observer, 50, "three");
                     }
                 }));
@@ -387,11 +387,11 @@ public void switchIssue737() {
         Observable> source = Observable.unsafeCreate(new ObservableSource>() {
             @Override
             public void subscribe(Observer> outerObserver) {
-                outerObserver.onSubscribe(Disposables.empty());
+                outerObserver.onSubscribe(Disposable.empty());
                 publishNext(outerObserver, 0, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 10, "1-one");
                         publishNext(innerObserver, 20, "1-two");
                         // The following events will be ignored
@@ -402,7 +402,7 @@ public void subscribe(Observer innerObserver) {
                 publishNext(outerObserver, 25, Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         publishNext(innerObserver, 10, "2-one");
                         publishNext(innerObserver, 20, "2-two");
                         publishNext(innerObserver, 30, "2-three");
@@ -435,7 +435,7 @@ public void unsubscribe() {
                 Observable.unsafeCreate(new ObservableSource>() {
                     @Override
                     public void subscribe(final Observer> observer) {
-                        Disposable bs = Disposables.empty();
+                        Disposable bs = Disposable.empty();
                         observer.onSubscribe(bs);
                         observer.onNext(Observable.just(1));
                         isUnsubscribed.set(bs.isDisposed());
@@ -666,7 +666,7 @@ public SingleSource apply(Object v) throws Exception {
                 return new SingleSource() {
                     @Override
                     public void subscribe(SingleObserver observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         observer.onSuccess(1);
                     }
                 };
@@ -870,7 +870,7 @@ public void badMainSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                     observer.onError(new TestException());
                     observer.onComplete();
@@ -910,7 +910,7 @@ public void badInnerSource() {
             .switchMap(Functions.justFunction(new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException());
                     observer.onComplete();
                     observer.onError(new TestException());
@@ -1023,10 +1023,10 @@ public ObservableSource apply(Integer v) throws Exception {
                 })
                 .test();
 
-                obs1.get().onSubscribe(Disposables.empty());
+                obs1.get().onSubscribe(Disposable.empty());
                 obs1.get().onNext(1);
 
-                obs2.get().onSubscribe(Disposables.empty());
+                obs2.get().onSubscribe(Disposable.empty());
 
                 final TestException ex1 = new TestException();
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java
index 73e2cc33c7..2225f3022b 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java
@@ -115,7 +115,7 @@ public void takeDoesntLeakErrors() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext("one");
                 observer.onError(new Throwable("test failed"));
             }
@@ -181,7 +181,7 @@ public void multiTake() {
 
             @Override
             public void subscribe(Observer observer) {
-                Disposable bs = Disposables.empty();
+                Disposable bs = Disposable.empty();
                 observer.onSubscribe(bs);
                 for (int i = 0; !bs.isDisposed(); i++) {
                     System.out.println("Emit: " + i);
@@ -214,7 +214,7 @@ static class TestObservableFunc implements ObservableSource {
 
         @Override
         public void subscribe(final Observer observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             System.out.println("TestObservable subscribed to ...");
             t = new Thread(new Runnable() {
 
@@ -243,7 +243,7 @@ public void run() {
 
         @Override
         public void subscribe(Observer op) {
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
             op.onSubscribe(d);
             long l = 1;
             while (!d.isDisposed()) {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java
index b197d0bd89..9566d64c9a 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java
@@ -18,10 +18,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -180,7 +180,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onComplete();
                     observer.onNext(1);
                     observer.onError(new TestException());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java
index 7c30166bb3..b348cda2ce 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java
@@ -104,7 +104,7 @@ public void takeWhileDoesntLeakErrors() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext("one");
                 observer.onError(new Throwable("test failed"));
             }
@@ -271,7 +271,7 @@ public void badSource() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onComplete();
                 observer.onComplete();
             }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTest.java
index 416c248f6a..c8eedc72bd 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTest.java
@@ -19,11 +19,11 @@
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 import org.mockito.InOrder;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.plugins.RxJavaPlugins;
 import io.reactivex.rxjava3.schedulers.TestScheduler;
@@ -48,7 +48,7 @@ public void throttlingWithCompleted() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer innerObserver) {
-                innerObserver.onSubscribe(Disposables.empty());
+                innerObserver.onSubscribe(Disposable.empty());
                 publishNext(innerObserver, 100, "one");    // publish as it's first
                 publishNext(innerObserver, 300, "two");    // skip as it's last within the first 400
                 publishNext(innerObserver, 900, "three");   // publish
@@ -76,7 +76,7 @@ public void throttlingWithError() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer innerObserver) {
-                innerObserver.onSubscribe(Disposables.empty());
+                innerObserver.onSubscribe(Disposable.empty());
                 Exception error = new TestException();
                 publishNext(innerObserver, 100, "one");    // Should be published since it is first
                 publishNext(innerObserver, 200, "two");    // Should be skipped since onError will arrive before the timeout expires
@@ -173,7 +173,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onNext(2);
                     observer.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java
index f1bb9e7bfe..f0001ab276 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java
@@ -241,7 +241,7 @@ public void run() {
 
                     @Override
                     public void subscribe(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         try {
                             timeoutSetuped.countDown();
                             exit.await();
@@ -360,7 +360,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onComplete();
@@ -386,7 +386,7 @@ public void badSourceOther() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     observer.onNext(1);
                     observer.onComplete();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java
index e590ec1663..9e6540e0b3 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java
@@ -283,7 +283,7 @@ public Observable apply(Integer t1) {
                     return Observable.unsafeCreate(new ObservableSource() {
                         @Override
                         public void subscribe(Observer observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             enteredTimeoutOne.countDown();
                             // force the timeout message be sent after observer.onNext(2)
                             while (true) {
@@ -436,7 +436,7 @@ public void badInnerSource() {
             .timeout(Functions.justFunction(new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onNext(2);
                     observer.onError(new TestException("Second"));
@@ -465,7 +465,7 @@ public void badInnerSourceOther() {
             .timeout(Functions.justFunction(new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onNext(2);
                     observer.onError(new TestException("Second"));
@@ -497,7 +497,7 @@ public void badSourceTimeout() {
         new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onNext(1);
                 observer.onNext(2);
                 observer.onError(new TestException("First"));
@@ -564,7 +564,7 @@ public void lateOnTimeoutError() {
                     @Override
                     protected void subscribeActual(
                             Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         sub[count++] = observer;
                     }
                 };
@@ -619,7 +619,7 @@ public void lateOnTimeoutFallbackRace() {
                     protected void subscribeActual(
                             Observer observer) {
                         assertFalse(((Disposable)observer).isDisposed());
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         sub[count++] = observer;
                     }
                 };
@@ -674,7 +674,7 @@ public void onErrorOnTimeoutRace() {
                     protected void subscribeActual(
                             Observer observer) {
                         assertFalse(((Disposable)observer).isDisposed());
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         sub[count++] = observer;
                     }
                 };
@@ -729,7 +729,7 @@ public void onCompleteOnTimeoutRace() {
                     protected void subscribeActual(
                             Observer observer) {
                         assertFalse(((Disposable)observer).isDisposed());
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         sub[count++] = observer;
                     }
                 };
@@ -782,7 +782,7 @@ public void onCompleteOnTimeoutRaceFallback() {
                     protected void subscribeActual(
                             Observer observer) {
                         assertFalse(((Disposable)observer).isDisposed());
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         sub[count++] = observer;
                     }
                 };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java
index ae1d856b0b..d77ee456db 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java
@@ -21,11 +21,11 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 import org.mockito.*;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.Function;
 import io.reactivex.rxjava3.internal.operators.observable.ObservableTimer.TimerObserver;
@@ -349,7 +349,7 @@ public Long apply(Long v) throws Exception {
     @Test
     public void cancelledAndRun() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
         TimerObserver tm = new TimerObserver(to);
 
         tm.dispose();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java
index ce8eee9b50..8a4ac039b7 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java
@@ -248,7 +248,7 @@ public void signalAfterDispose() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onNext(2);
                     observer.onError(new TestException());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java
index 62e8986292..6312ed7d42 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java
@@ -209,7 +209,7 @@ private void performTestUsingWithObservableFactoryError(boolean disposeEagerly)
         Supplier resourceFactory = new Supplier() {
             @Override
             public Disposable get() {
-                return Disposables.fromRunnable(unsubscribe);
+                return Disposable.fromRunnable(unsubscribe);
             }
         };
 
@@ -538,7 +538,7 @@ public void eagerDisposedOnComplete() {
         Observable.using(Functions.justSupplier(1), Functions.justFunction(new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 to.dispose();
                 observer.onComplete();
             }
@@ -553,7 +553,7 @@ public void eagerDisposedOnError() {
         Observable.using(Functions.justSupplier(1), Functions.justFunction(new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 to.dispose();
                 observer.onError(new TestException());
             }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java
index e6804ddcab..63bb37eafa 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java
@@ -374,7 +374,7 @@ public void mainAndBoundaryBothError() {
             .window(new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     ref.set(observer);
                 }
             })
@@ -413,14 +413,14 @@ public void mainCompleteBoundaryErrorRace() {
                 TestObserverEx> to = new Observable() {
                     @Override
                     protected void subscribeActual(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         refMain.set(observer);
                     }
                 }
                 .window(new Observable() {
                     @Override
                     protected void subscribeActual(Observer observer) {
-                        observer.onSubscribe(Disposables.empty());
+                        observer.onSubscribe(Disposable.empty());
                         ref.set(observer);
                     }
                 })
@@ -463,14 +463,14 @@ public void mainNextBoundaryNextRace() {
             TestObserver> to = new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     refMain.set(observer);
                 }
             }
             .window(new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     ref.set(observer);
                 }
             })
@@ -506,14 +506,14 @@ public void takeOneAnotherBoundary() {
         TestObserverEx> to = new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 refMain.set(observer);
             }
         }
         .window(new Observable() {
             @Override
             protected void subscribeActual(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 ref.set(observer);
             }
         })
@@ -538,7 +538,7 @@ public void disposeMainBoundaryCompleteRace() {
             final TestObserver> to = new Observable() {
                  @Override
                  protected void subscribeActual(Observer observer) {
-                     observer.onSubscribe(Disposables.empty());
+                     observer.onSubscribe(Disposable.empty());
                      refMain.set(observer);
                  }
              }
@@ -596,7 +596,7 @@ public void disposeMainBoundaryErrorRace() {
            final TestObserver> to = new Observable() {
                @Override
                protected void subscribeActual(Observer observer) {
-                   observer.onSubscribe(Disposables.empty());
+                   observer.onSubscribe(Disposable.empty());
                    refMain.set(observer);
                }
            }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java
index b7ef436399..274c9712a7 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java
@@ -219,7 +219,7 @@ public static Observable hotStream() {
         return Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 observer.onSubscribe(d);
                 while (!d.isDisposed()) {
                     // burst some number of items
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java
index da02116cd9..d3466c7eb4 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java
@@ -19,12 +19,12 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -53,7 +53,7 @@ public void observableBasedOpenerAndCloser() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer innerObserver) {
-                innerObserver.onSubscribe(Disposables.empty());
+                innerObserver.onSubscribe(Disposable.empty());
                 push(innerObserver, "one", 10);
                 push(innerObserver, "two", 60);
                 push(innerObserver, "three", 110);
@@ -66,7 +66,7 @@ public void subscribe(Observer innerObserver) {
         Observable openings = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer innerObserver) {
-                innerObserver.onSubscribe(Disposables.empty());
+                innerObserver.onSubscribe(Disposable.empty());
                 push(innerObserver, new Object(), 50);
                 push(innerObserver, new Object(), 200);
                 complete(innerObserver, 250);
@@ -79,7 +79,7 @@ public Observable apply(Object opening) {
                 return Observable.unsafeCreate(new ObservableSource() {
                     @Override
                     public void subscribe(Observer innerObserver) {
-                        innerObserver.onSubscribe(Disposables.empty());
+                        innerObserver.onSubscribe(Disposable.empty());
                         push(innerObserver, new Object(), 100);
                         complete(innerObserver, 101);
                     }
@@ -371,7 +371,7 @@ public Observable apply(Integer f) throws Exception {
                         @Override
                         protected void subscribeActual(
                                 Observer observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             observer.onNext(1);
                             observer.onNext(2);
                             observer.onError(new TestException());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java
index 8eaebf5991..51591516a6 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java
@@ -19,12 +19,12 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -52,7 +52,7 @@ public void timedAndCount() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 push(observer, "one", 10);
                 push(observer, "two", 90);
                 push(observer, "three", 110);
@@ -88,7 +88,7 @@ public void timed() {
         Observable source = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 push(observer, "one", 98);
                 push(observer, "two", 99);
                 push(observer, "three", 99); // FIXME happens after the window is open
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java
index f87491e0e0..00b4e73f18 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java
@@ -19,13 +19,13 @@
 
 import java.util.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.mockito.InOrder;
 
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
 import io.reactivex.rxjava3.core.RxJavaTest;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.Functions;
@@ -545,7 +545,7 @@ public void manyErrors() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onError(new TestException("First"));
                     observer.onNext(1);
                     observer.onError(new TestException("Second"));
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterableTest.java
index 53fd62d50d..a4ce20c33c 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipIterableTest.java
@@ -20,13 +20,13 @@
 import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 import org.mockito.InOrder;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.util.CrashingIterable;
@@ -409,7 +409,7 @@ public void badSource() {
             new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onComplete();
                     observer.onNext(2);
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java
index e185dea68d..b52b90183d 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java
@@ -624,7 +624,7 @@ private static class TestObservable implements ObservableSource {
         public void subscribe(Observer observer) {
             // just store the variable where it can be accessed so we can manually trigger it
             this.observer = observer;
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
         }
 
     }
@@ -1024,7 +1024,7 @@ Observable OBSERVABLE_OF_5_INTEGERS(final AtomicInteger numEmitted) {
 
             @Override
             public void subscribe(final Observer o) {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 o.onSubscribe(d);
                 for (int i = 1; i <= 5; i++) {
                     if (d.isDisposed()) {
@@ -1045,7 +1045,7 @@ Observable ASYNC_OBSERVABLE_OF_INFINITE_INTEGERS(final CountDownLatch l
 
             @Override
             public void subscribe(final Observer o) {
-                final Disposable d = Disposables.empty();
+                final Disposable d = Disposable.empty();
                 o.onSubscribe(d);
                 Thread t = new Thread(new Runnable() {
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleCreateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleCreateTest.java
index 98c0a67903..489744b708 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleCreateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleCreateTest.java
@@ -36,7 +36,7 @@ public void nullArgument() {
 
     @Test
     public void basic() {
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Single.create(new SingleOnSubscribe() {
             @Override
@@ -57,8 +57,8 @@ public void subscribe(SingleEmitter e) throws Exception {
 
     @Test
     public void basicWithCancellable() {
-        final Disposable d1 = Disposables.empty();
-        final Disposable d2 = Disposables.empty();
+        final Disposable d1 = Disposable.empty();
+        final Disposable d2 = Disposable.empty();
 
         Single.create(new SingleOnSubscribe() {
             @Override
@@ -86,7 +86,7 @@ public void cancel() throws Exception {
 
     @Test
     public void basicWithError() {
-        final Disposable d = Disposables.empty();
+        final Disposable d = Disposable.empty();
 
         Single.create(new SingleOnSubscribe() {
             @Override
@@ -193,7 +193,7 @@ public void createConsumerThrowsResource() {
         Single.create(new SingleOnSubscribe() {
             @Override
             public void subscribe(SingleEmitter s) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 s.setDisposable(d);
                 try {
                     s.onSuccess(1);
@@ -257,7 +257,7 @@ public void createConsumerThrowsResourceOnError() {
         Single.create(new SingleOnSubscribe() {
             @Override
             public void subscribe(SingleEmitter s) throws Exception {
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
                 s.setDisposable(d);
                 try {
                     s.onError(new IOException());
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java
index 4f97e5c6bf..1eaf426511 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java
@@ -19,11 +19,11 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 import org.reactivestreams.Subscriber;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription;
@@ -215,7 +215,7 @@ public void withObservableError2() {
             .delaySubscription(new Observable() {
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     observer.onNext(1);
                     observer.onError(new TestException());
                 }
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java
index 1b7f5e7a34..d582339cd0 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java
@@ -64,7 +64,7 @@ public void onSuccess() {
 
     @Test
     public void cancelDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Single() {
@@ -90,7 +90,7 @@ protected void subscribeActual(SingleObserver observer) {
 
     @Test
     public void errorDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Single() {
@@ -116,7 +116,7 @@ protected void subscribeActual(SingleObserver observer) {
 
     @Test
     public void successDetaches() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         final WeakReference wr = new WeakReference(d);
 
         TestObserver to = new Single() {
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnTest.java
index 9bea6e178c..cf67202641 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnTest.java
@@ -332,7 +332,7 @@ public void accept(Integer v) throws Exception {
     public void onSubscribeCrash() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable bs = Disposables.empty();
+            final Disposable bs = Disposable.empty();
 
             new Single() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMiscTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMiscTest.java
index bf6e4fa8cf..ca2ccacdc2 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMiscTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMiscTest.java
@@ -18,10 +18,10 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.schedulers.Schedulers;
@@ -51,7 +51,7 @@ public void wrap() {
         Single.wrap(new SingleSource() {
             @Override
             public void subscribe(SingleObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onSuccess(1);
             }
         })
diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleUsingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleUsingTest.java
index c1463923cf..28f268f423 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleUsingTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleUsingTest.java
@@ -101,7 +101,7 @@ public void errorNonEager() {
 
     @Test
     public void eagerMapperThrowsDisposerThrows() {
-        TestObserverEx to = Single.using(Functions.justSupplier(Disposables.empty()), mapperThrows, disposerThrows)
+        TestObserverEx to = Single.using(Functions.justSupplier(Disposable.empty()), mapperThrows, disposerThrows)
         .to(TestHelper.testConsumer())
         .assertFailure(CompositeException.class);
 
@@ -116,7 +116,7 @@ public void noneagerMapperThrowsDisposerThrows() {
         List errors = TestHelper.trackPluginErrors();
 
         try {
-            Single.using(Functions.justSupplier(Disposables.empty()), mapperThrows, disposerThrows, false)
+            Single.using(Functions.justSupplier(Disposable.empty()), mapperThrows, disposerThrows, false)
             .to(TestHelper.testConsumer())
             .assertFailureAndMessage(TestException.class, "Mapper");
 
@@ -128,7 +128,7 @@ public void noneagerMapperThrowsDisposerThrows() {
 
     @Test
     public void resourceDisposedIfMapperCrashes() {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         Single.using(Functions.justSupplier(d), mapperThrows, disposer)
         .test()
@@ -139,7 +139,7 @@ public void resourceDisposedIfMapperCrashes() {
 
     @Test
     public void resourceDisposedIfMapperCrashesNonEager() {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         Single.using(Functions.justSupplier(d), mapperThrows, disposer, false)
         .test()
@@ -150,7 +150,7 @@ public void resourceDisposedIfMapperCrashesNonEager() {
 
     @Test
     public void dispose() {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         Single.using(Functions.justSupplier(d), mapper, disposer, false)
         .test(true);
@@ -160,7 +160,7 @@ public void dispose() {
 
     @Test
     public void disposerThrowsEager() {
-        Single.using(Functions.justSupplier(Disposables.empty()), mapper, disposerThrows)
+        Single.using(Functions.justSupplier(Disposable.empty()), mapper, disposerThrows)
         .test()
         .assertFailure(TestException.class);
     }
@@ -171,7 +171,7 @@ public void disposerThrowsNonEager() {
         List errors = TestHelper.trackPluginErrors();
 
         try {
-            Single.using(Functions.justSupplier(Disposables.empty()), mapper, disposerThrows, false)
+            Single.using(Functions.justSupplier(Disposable.empty()), mapper, disposerThrows, false)
             .test()
             .assertResult(1);
             TestHelper.assertUndeliverable(errors, 0, TestException.class, "Disposer");
@@ -182,7 +182,7 @@ public void disposerThrowsNonEager() {
 
     @Test
     public void errorAndDisposerThrowsEager() {
-        TestObserverEx to = Single.using(Functions.justSupplier(Disposables.empty()),
+        TestObserverEx to = Single.using(Functions.justSupplier(Disposable.empty()),
         new Function>() {
             @Override
             public SingleSource apply(Disposable v) throws Exception {
@@ -202,7 +202,7 @@ public void errorAndDisposerThrowsNonEager() {
         List errors = TestHelper.trackPluginErrors();
 
         try {
-            Single.using(Functions.justSupplier(Disposables.empty()),
+            Single.using(Functions.justSupplier(Disposable.empty()),
             new Function>() {
                 @Override
                 public SingleSource apply(Disposable v) throws Exception {
@@ -222,7 +222,7 @@ public void successDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final PublishProcessor pp = PublishProcessor.create();
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             final TestObserver to = Single.using(Functions.justSupplier(d), new Function>() {
                 @Override
@@ -264,11 +264,11 @@ public SingleSource apply(Integer v) throws Exception {
                     return new Single() {
                         @Override
                         protected void subscribeActual(SingleObserver observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
 
                             assertFalse(((Disposable)observer).isDisposed());
 
-                            Disposable d = Disposables.empty();
+                            Disposable d = Disposable.empty();
                             observer.onSubscribe(d);
 
                             assertTrue(d.isDisposed());
@@ -297,7 +297,7 @@ public void errorDisposeRace() {
         for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) {
             final PublishProcessor pp = PublishProcessor.create();
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             final TestObserver to = Single.using(Functions.justSupplier(d), new Function>() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancelTest.java b/src/test/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancelTest.java
index 19a68d0818..8018806a37 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancelTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancelTest.java
@@ -26,7 +26,7 @@ public class DisposeOnCancelTest extends RxJavaTest {
 
     @Test
     public void basicCoverage() throws Exception {
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         DisposeOnCancel doc = new DisposeOnCancel(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhenTest.java b/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhenTest.java
index 8225fd4e06..d48228d20c 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhenTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerWhenTest.java
@@ -268,7 +268,7 @@ public void scheduledActiondisposedSetRace() {
                 @Override
                 protected Disposable callActual(Worker actualWorker,
                         CompletableObserver actionCompletable) {
-                    return Disposables.empty();
+                    return Disposable.empty();
                 }
 
             };
@@ -299,7 +299,7 @@ public void scheduledActionStates() {
             protected Disposable callActual(Worker actualWorker,
                     CompletableObserver actionCompletable) {
                 count.incrementAndGet();
-                return Disposables.empty();
+                return Disposable.empty();
             }
 
         };
@@ -320,7 +320,7 @@ protected Disposable callActual(Worker actualWorker,
         assertEquals(0, count.get());
 
         // should not run when already scheduled
-        sa.set(Disposables.empty());
+        sa.set(Disposable.empty());
 
         sa.call(Schedulers.single().createWorker(), null);
 
@@ -336,7 +336,7 @@ protected Disposable callActual(Worker actualWorker,
                     CompletableObserver actionCompletable) {
                 count.incrementAndGet();
                 dispose();
-                return Disposables.empty();
+                return Disposable.empty();
             }
 
         };
diff --git a/src/test/java/io/reactivex/rxjava3/internal/schedulers/SingleSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/internal/schedulers/SingleSchedulerTest.java
index 82f66ceff6..f6dfe5e455 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/schedulers/SingleSchedulerTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/schedulers/SingleSchedulerTest.java
@@ -43,20 +43,20 @@ public void run() {
         Scheduler s = new SingleScheduler();
         s.shutdown();
 
-        assertEquals(Disposables.disposed(), s.scheduleDirect(r));
+        assertEquals(Disposable.disposed(), s.scheduleDirect(r));
 
-        assertEquals(Disposables.disposed(), s.scheduleDirect(r, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), s.scheduleDirect(r, 1, TimeUnit.SECONDS));
 
-        assertEquals(Disposables.disposed(), s.schedulePeriodicallyDirect(r, 1, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), s.schedulePeriodicallyDirect(r, 1, 1, TimeUnit.SECONDS));
 
         Worker w = s.createWorker();
         ((ScheduledWorker)w).executor.shutdownNow();
 
-        assertEquals(Disposables.disposed(), w.schedule(r));
+        assertEquals(Disposable.disposed(), w.schedule(r));
 
-        assertEquals(Disposables.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
 
-        assertEquals(Disposables.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
 
         assertEquals(0, calls[0]);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/EmptyComponentTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/EmptyComponentTest.java
index d2d9b91163..60da2b7a29 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/EmptyComponentTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/EmptyComponentTest.java
@@ -44,7 +44,7 @@ public void normal() {
 
             c.request(-10);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             c.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java
index 74bfb0c626..41d3178384 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java
@@ -146,7 +146,7 @@ public boolean accept(Subscriber a, Integer v) {
     @Test
     public void unorderedFastPathNoRequest() {
         TestSubscriber ts = new TestSubscriber(0);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createUnordered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -160,7 +160,7 @@ public void unorderedFastPathNoRequest() {
     @Test
     public void orderedFastPathNoRequest() {
         TestSubscriber ts = new TestSubscriber(0);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createOrdered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -174,7 +174,7 @@ public void orderedFastPathNoRequest() {
     @Test
     public void acceptBadRequest() {
         TestSubscriber ts = new TestSubscriber(0);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createUnordered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -192,7 +192,7 @@ public void acceptBadRequest() {
     @Test
     public void unorderedFastPathRequest1() {
         TestSubscriber ts = new TestSubscriber(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createUnordered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -206,7 +206,7 @@ public void unorderedFastPathRequest1() {
     @Test
     public void orderedFastPathRequest1() {
         TestSubscriber ts = new TestSubscriber(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createOrdered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -220,7 +220,7 @@ public void orderedFastPathRequest1() {
     @Test
     public void unorderedSlowPath() {
         TestSubscriber ts = new TestSubscriber(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createUnordered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -233,7 +233,7 @@ public void unorderedSlowPath() {
     @Test
     public void orderedSlowPath() {
         TestSubscriber ts = new TestSubscriber(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createOrdered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -246,7 +246,7 @@ public void orderedSlowPath() {
     @Test
     public void orderedSlowPathNonEmptyQueue() {
         TestSubscriber ts = new TestSubscriber(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createOrdered(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -262,7 +262,7 @@ public void unorderedOnNextRace() {
         for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) {
 
             TestSubscriber ts = new TestSubscriber(1);
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
             final QueueDrainSubscriber qd = createUnordered(ts, d);
             ts.onSubscribe(new BooleanSubscription());
 
@@ -285,7 +285,7 @@ public void orderedOnNextRace() {
         for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) {
 
             TestSubscriber ts = new TestSubscriber(1);
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
             final QueueDrainSubscriber qd = createOrdered(ts, d);
             ts.onSubscribe(new BooleanSubscription());
 
@@ -306,7 +306,7 @@ public void run() {
     @Test
     public void unorderedFastPathReject() {
         TestSubscriber ts = new TestSubscriber(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createUnorderedReject(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
@@ -322,7 +322,7 @@ public void unorderedFastPathReject() {
     @Test
     public void orderedFastPathReject() {
         TestSubscriber ts = new TestSubscriber(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
         QueueDrainSubscriber qd = createOrderedReject(ts, d);
         ts.onSubscribe(new BooleanSubscription());
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java
index 76ed844842..dc7c82893d 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java
@@ -35,7 +35,7 @@ public class SubscriberResourceWrapperTest extends RxJavaTest {
     @Test
     public void cancel() {
         BooleanSubscription bs = new BooleanSubscription();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         s.setResource(d);
 
@@ -54,7 +54,7 @@ public void cancel() {
     @Test
     public void error() {
         BooleanSubscription bs = new BooleanSubscription();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         s.setResource(d);
 
@@ -71,7 +71,7 @@ public void error() {
     @Test
     public void complete() {
         BooleanSubscription bs = new BooleanSubscription();
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         s.setResource(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java
index 86f823e22e..9b033bffdd 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java
@@ -17,10 +17,10 @@
 
 import java.util.List;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.exceptions.TestException;
 import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription;
 import io.reactivex.rxjava3.observers.TestObserver;
@@ -124,7 +124,7 @@ public void tryTerminateConsumerSubscriberTerminated() {
     @Test
     public void tryTerminateConsumerObserverNoError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.tryTerminateConsumer((Observer)to);
@@ -134,7 +134,7 @@ public void tryTerminateConsumerObserverNoError() {
     @Test
     public void tryTerminateConsumerObserverError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.set(new TestException());
@@ -145,7 +145,7 @@ public void tryTerminateConsumerObserverError() {
     @Test
     public void tryTerminateConsumerObserverTerminated() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.terminate();
@@ -156,7 +156,7 @@ public void tryTerminateConsumerObserverTerminated() {
     @Test
     public void tryTerminateConsumerMaybeObserverNoError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.tryTerminateConsumer((MaybeObserver)to);
@@ -166,7 +166,7 @@ public void tryTerminateConsumerMaybeObserverNoError() {
     @Test
     public void tryTerminateConsumerMaybeObserverError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.set(new TestException());
@@ -177,7 +177,7 @@ public void tryTerminateConsumerMaybeObserverError() {
     @Test
     public void tryTerminateConsumerMaybeObserverTerminated() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.terminate();
@@ -188,7 +188,7 @@ public void tryTerminateConsumerMaybeObserverTerminated() {
     @Test
     public void tryTerminateConsumerSingleNoError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.tryTerminateConsumer((SingleObserver)to);
@@ -198,7 +198,7 @@ public void tryTerminateConsumerSingleNoError() {
     @Test
     public void tryTerminateConsumerSingleError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.set(new TestException());
@@ -209,7 +209,7 @@ public void tryTerminateConsumerSingleError() {
     @Test
     public void tryTerminateConsumerSingleTerminated() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.terminate();
@@ -220,7 +220,7 @@ public void tryTerminateConsumerSingleTerminated() {
     @Test
     public void tryTerminateConsumerCompletableObserverNoError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.tryTerminateConsumer((CompletableObserver)to);
@@ -230,7 +230,7 @@ public void tryTerminateConsumerCompletableObserverNoError() {
     @Test
     public void tryTerminateConsumerCompletableObserverError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.set(new TestException());
@@ -241,7 +241,7 @@ public void tryTerminateConsumerCompletableObserverError() {
     @Test
     public void tryTerminateConsumerCompletableObserverTerminated() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.terminate();
@@ -271,7 +271,7 @@ public void onComplete() {
     @Test
     public void tryTerminateConsumerEmitterNoError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.tryTerminateConsumer(wrapToEmitter(to));
@@ -281,7 +281,7 @@ public void tryTerminateConsumerEmitterNoError() {
     @Test
     public void tryTerminateConsumerEmitterError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.set(new TestException());
@@ -292,7 +292,7 @@ public void tryTerminateConsumerEmitterError() {
     @Test
     public void tryTerminateConsumerEmitterTerminated() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         AtomicThrowable ex = new AtomicThrowable();
         ex.terminate();
diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/BlockingHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/BlockingHelperTest.java
index 7d4e07dab4..419febcaee 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/util/BlockingHelperTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/util/BlockingHelperTest.java
@@ -34,7 +34,7 @@ public void emptyEnum() {
     @Test
     public void interrupted() {
         CountDownLatch cdl = new CountDownLatch(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         Thread.currentThread().interrupt();
 
@@ -50,7 +50,7 @@ public void interrupted() {
     @Test
     public void unblock() {
         final CountDownLatch cdl = new CountDownLatch(1);
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         Schedulers.computation().scheduleDirect(new Runnable() {
             @Override
diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/EndConsumerHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/EndConsumerHelperTest.java
index 0ff03f24c2..1cb96e732f 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/util/EndConsumerHelperTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/util/EndConsumerHelperTest.java
@@ -209,13 +209,13 @@ public void onComplete() {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -244,13 +244,13 @@ public void onComplete() {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -279,13 +279,13 @@ public void onComplete() {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -310,13 +310,13 @@ public void onError(Throwable t) {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -341,13 +341,13 @@ public void onError(Throwable t) {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -376,13 +376,13 @@ public void onComplete() {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -411,13 +411,13 @@ public void onComplete() {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -442,13 +442,13 @@ public void onComplete() {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -473,13 +473,13 @@ public void onComplete() {
             }
         };
 
-        Disposable sub1 = Disposables.empty();
+        Disposable sub1 = Disposable.empty();
 
         consumer.onSubscribe(sub1);
 
         assertFalse(sub1.isDisposed());
 
-        Disposable sub2 = Disposables.empty();
+        Disposable sub2 = Disposable.empty();
 
         consumer.onSubscribe(sub2);
 
@@ -494,7 +494,7 @@ public void onComplete() {
 
     @Test
     public void validateDisposable() {
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         assertFalse(EndConsumerHelper.validate(DisposableHelper.DISPOSED, d1, getClass()));
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java
index 2171fadc2d..050e58dc64 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java
@@ -64,7 +64,7 @@ public void onComplete() {
 
         a[0] = observer;
 
-        observer.onSubscribe(Disposables.empty());
+        observer.onSubscribe(Disposable.empty());
 
         HalfSerializer.onNext(observer, 1, wip, error);
 
@@ -108,7 +108,7 @@ public void onComplete() {
 
         a[0] = observer;
 
-        observer.onSubscribe(Disposables.empty());
+        observer.onSubscribe(Disposable.empty());
 
         HalfSerializer.onNext(observer, 1, wip, error);
 
@@ -152,7 +152,7 @@ public void onComplete() {
 
         a[0] = observer;
 
-        observer.onSubscribe(Disposables.empty());
+        observer.onSubscribe(Disposable.empty());
 
         HalfSerializer.onNext(observer, 1, wip, error);
 
@@ -194,7 +194,7 @@ public void onComplete() {
 
         a[0] = observer;
 
-        observer.onSubscribe(Disposables.empty());
+        observer.onSubscribe(Disposable.empty());
 
         HalfSerializer.onError(observer, new TestException(), wip, error);
 
@@ -209,7 +209,7 @@ public void onNextOnCompleteRace() {
             final AtomicThrowable error = new AtomicThrowable();
 
             final TestObserver to = new TestObserver();
-            to.onSubscribe(Disposables.empty());
+            to.onSubscribe(Disposable.empty());
 
             Runnable r1 = new Runnable() {
                 @Override
@@ -242,7 +242,7 @@ public void onErrorOnCompleteRace() {
 
             final TestObserverEx to = new TestObserverEx();
 
-            to.onSubscribe(Disposables.empty());
+            to.onSubscribe(Disposable.empty());
 
             final TestException ex = new TestException();
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java
index 0149b14f4e..6fafc2d5db 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java
@@ -28,7 +28,7 @@ public class NotificationLiteTest extends RxJavaTest {
     public void acceptFullObserver() {
         TestObserverEx to = new TestObserverEx();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         assertFalse(NotificationLite.acceptFull(NotificationLite.disposable(d), to));
 
diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java
index 9060e85c45..f2ace8d235 100644
--- a/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java
+++ b/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java
@@ -317,7 +317,7 @@ public boolean accept(Subscriber a, Integer v) {
         SpscArrayQueue q = new SpscArrayQueue(32);
         q.offer(1);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         QueueDrainHelper.drainMaxLoop(q, ts, false, d, qd);
 
@@ -600,7 +600,7 @@ public boolean accept(Subscriber a, Integer v) {
     @Test
     public void observerCheckTerminatedDelayErrorEmpty() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         ObservableQueueDrain qd = new ObservableQueueDrain() {
             @Override
@@ -643,7 +643,7 @@ public void accept(Observer a, Integer v) {
     @Test
     public void observerCheckTerminatedDelayErrorEmptyResource() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         ObservableQueueDrain qd = new ObservableQueueDrain() {
             @Override
@@ -678,7 +678,7 @@ public void accept(Observer a, Integer v) {
 
         SpscArrayQueue q = new SpscArrayQueue(32);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         QueueDrainHelper.checkTerminated(true, true, to, true, q, d, qd);
 
@@ -690,7 +690,7 @@ public void accept(Observer a, Integer v) {
     @Test
     public void observerCheckTerminatedDelayErrorNonEmpty() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         ObservableQueueDrain qd = new ObservableQueueDrain() {
             @Override
@@ -733,7 +733,7 @@ public void accept(Observer a, Integer v) {
     @Test
     public void observerCheckTerminatedDelayErrorEmptyError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         ObservableQueueDrain qd = new ObservableQueueDrain() {
             @Override
@@ -776,7 +776,7 @@ public void accept(Observer a, Integer v) {
     @Test
     public void observerCheckTerminatedNonDelayErrorError() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         ObservableQueueDrain qd = new ObservableQueueDrain() {
             @Override
@@ -819,7 +819,7 @@ public void accept(Observer a, Integer v) {
     @Test
     public void observerCheckTerminatedNonDelayErrorErrorResource() {
         TestObserver to = new TestObserver();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         ObservableQueueDrain qd = new ObservableQueueDrain() {
             @Override
@@ -854,7 +854,7 @@ public void accept(Observer a, Integer v) {
 
         SpscArrayQueue q = new SpscArrayQueue(32);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         QueueDrainHelper.checkTerminated(true, false, to, false, q, d, qd);
 
diff --git a/src/test/java/io/reactivex/rxjava3/maybe/MaybeCreateTest.java b/src/test/java/io/reactivex/rxjava3/maybe/MaybeCreateTest.java
index d681c85c0c..ee97e25015 100644
--- a/src/test/java/io/reactivex/rxjava3/maybe/MaybeCreateTest.java
+++ b/src/test/java/io/reactivex/rxjava3/maybe/MaybeCreateTest.java
@@ -36,7 +36,7 @@ public void nullArgument() {
     public void basic() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Maybe.create(new MaybeOnSubscribe() {
                 @Override
@@ -61,8 +61,8 @@ public void subscribe(MaybeEmitter e) throws Exception {
     public void basicWithCancellable() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d1 = Disposables.empty();
-            final Disposable d2 = Disposables.empty();
+            final Disposable d1 = Disposable.empty();
+            final Disposable d2 = Disposable.empty();
 
             Maybe.create(new MaybeOnSubscribe() {
                 @Override
@@ -95,7 +95,7 @@ public void cancel() throws Exception {
     public void basicWithError() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Maybe.create(new MaybeOnSubscribe() {
                 @Override
@@ -120,7 +120,7 @@ public void subscribe(MaybeEmitter e) throws Exception {
     public void basicWithCompletion() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Maybe.create(new MaybeOnSubscribe() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java
index 3796f365c0..69ea889363 100644
--- a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java
+++ b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java
@@ -230,7 +230,7 @@ public void wrapCustom() {
         Maybe.wrap(new MaybeSource() {
             @Override
             public void subscribe(MaybeObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onSuccess(1);
             }
         })
@@ -356,7 +356,7 @@ public void unsafeCreate() {
         Maybe.unsafeCreate(new MaybeSource() {
             @Override
             public void subscribe(MaybeObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onSuccess(1);
             }
         })
@@ -1503,7 +1503,7 @@ public void nullArgument() {
     public void basic() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Maybe.create(new MaybeOnSubscribe() {
                 @Override
@@ -1533,7 +1533,7 @@ public void subscribe(MaybeEmitter e) throws Exception {
     public void basicWithError() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Maybe.create(new MaybeOnSubscribe() {
                 @Override
@@ -1561,7 +1561,7 @@ public void subscribe(MaybeEmitter e) throws Exception {
     public void basicWithComplete() {
         List errors = TestHelper.trackPluginErrors();
         try {
-            final Disposable d = Disposables.empty();
+            final Disposable d = Disposable.empty();
 
             Maybe.create(new MaybeOnSubscribe() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java
index 4d26afe20d..a3d79a426c 100644
--- a/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java
@@ -500,7 +500,7 @@ public void publishLast() throws InterruptedException {
         ConnectableObservable connectable = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 count.incrementAndGet();
                 new Thread(new Runnable() {
                     @Override
@@ -538,7 +538,7 @@ public void replay() throws InterruptedException {
         ConnectableObservable o = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(final Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     new Thread(new Runnable() {
 
                         @Override
@@ -591,7 +591,7 @@ public void cache() throws InterruptedException {
         Observable o = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(final Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
                     new Thread(new Runnable() {
                         @Override
                         public void run() {
@@ -636,7 +636,7 @@ public void cacheWithCapacity() throws InterruptedException {
         Observable o = Observable.unsafeCreate(new ObservableSource() {
             @Override
             public void subscribe(final Observer observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 new Thread(new Runnable() {
                     @Override
                     public void run() {
diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java
index 45d4b82172..84db3819ae 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java
@@ -79,9 +79,9 @@ public void startOnce() {
         try {
             TestCompletable tc = new TestCompletable();
 
-            tc.onSubscribe(Disposables.empty());
+            tc.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             tc.onSubscribe(d);
 
@@ -102,7 +102,7 @@ public void dispose() {
 
         assertTrue(tc.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         tc.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java
index 812e40ca3a..42cbd25c5d 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java
@@ -87,9 +87,9 @@ public void startOnce() {
         try {
             TestMaybe tc = new TestMaybe();
 
-            tc.onSubscribe(Disposables.empty());
+            tc.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             tc.onSubscribe(d);
 
@@ -110,7 +110,7 @@ public void dispose() {
 
         assertTrue(tc.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         tc.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java
index 6a2b332293..b223e47b48 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java
@@ -86,9 +86,9 @@ public void startOnce() {
         try {
             TestDisposableObserver tc = new TestDisposableObserver();
 
-            tc.onSubscribe(Disposables.empty());
+            tc.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             tc.onSubscribe(d);
 
@@ -112,7 +112,7 @@ public void dispose() {
 
         assertTrue(tc.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         tc.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java
index 983a8dc847..6bf84e21d3 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java
@@ -79,9 +79,9 @@ public void startOnce() {
         try {
             TestSingle tc = new TestSingle();
 
-            tc.onSubscribe(Disposables.empty());
+            tc.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             tc.onSubscribe(d);
 
@@ -102,7 +102,7 @@ public void dispose() {
 
         assertTrue(tc.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         tc.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java
index 487f42352a..6564ae4d16 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java
@@ -68,7 +68,7 @@ public void addResources() {
 
         assertFalse(rco.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rco.add(d);
 
@@ -93,7 +93,7 @@ public void onCompleteCleansUp() {
 
         assertFalse(rco.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rco.add(d);
 
@@ -112,7 +112,7 @@ public void onErrorCleansUp() {
 
         assertFalse(rco.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rco.add(d);
 
@@ -167,9 +167,9 @@ public void startOnce() {
         try {
             TestResourceCompletableObserver rco = new TestResourceCompletableObserver();
 
-            rco.onSubscribe(Disposables.empty());
+            rco.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             rco.onSubscribe(d);
 
@@ -188,7 +188,7 @@ public void dispose() {
         TestResourceCompletableObserver rco = new TestResourceCompletableObserver();
         rco.dispose();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rco.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java
index 92370a3203..04a093d753 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java
@@ -77,7 +77,7 @@ public void addResources() {
 
         assertFalse(rmo.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rmo.add(d);
 
@@ -102,7 +102,7 @@ public void onCompleteCleansUp() {
 
         assertFalse(rmo.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rmo.add(d);
 
@@ -121,7 +121,7 @@ public void onSuccessCleansUp() {
 
         assertFalse(rmo.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rmo.add(d);
 
@@ -140,7 +140,7 @@ public void onErrorCleansUp() {
 
         assertFalse(rmo.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rmo.add(d);
 
@@ -217,9 +217,9 @@ public void startOnce() {
         try {
             TestResourceMaybeObserver rmo = new TestResourceMaybeObserver();
 
-            rmo.onSubscribe(Disposables.empty());
+            rmo.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             rmo.onSubscribe(d);
 
@@ -238,7 +238,7 @@ public void dispose() {
         TestResourceMaybeObserver rmo = new TestResourceMaybeObserver();
         rmo.dispose();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rmo.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java
index d81b7fbe88..e0ff866e75 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java
@@ -77,7 +77,7 @@ public void addResources() {
 
         assertFalse(ro.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ro.add(d);
 
@@ -102,7 +102,7 @@ public void onCompleteCleansUp() {
 
         assertFalse(ro.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ro.add(d);
 
@@ -121,7 +121,7 @@ public void onErrorCleansUp() {
 
         assertFalse(ro.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ro.add(d);
 
@@ -178,9 +178,9 @@ public void startOnce() {
         try {
             TestResourceObserver tc = new TestResourceObserver();
 
-            tc.onSubscribe(Disposables.empty());
+            tc.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             tc.onSubscribe(d);
 
@@ -199,7 +199,7 @@ public void dispose() {
         TestResourceObserver tc = new TestResourceObserver();
         tc.dispose();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         tc.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java
index 1add001b63..054f189bc5 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java
@@ -68,7 +68,7 @@ public void addResources() {
 
         assertFalse(rso.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rso.add(d);
 
@@ -93,7 +93,7 @@ public void onSuccessCleansUp() {
 
         assertFalse(rso.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rso.add(d);
 
@@ -112,7 +112,7 @@ public void onErrorCleansUp() {
 
         assertFalse(rso.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rso.add(d);
 
@@ -169,9 +169,9 @@ public void startOnce() {
         try {
             TestResourceSingleObserver rso = new TestResourceSingleObserver();
 
-            rso.onSubscribe(Disposables.empty());
+            rso.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             rso.onSubscribe(d);
 
@@ -190,7 +190,7 @@ public void dispose() {
         TestResourceSingleObserver rso = new TestResourceSingleObserver();
         rso.dispose();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         rso.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java
index 876bcffbe4..446dfe20df 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java
@@ -46,7 +46,7 @@ public void onNextFailureSafe() {
         AtomicReference onError = new AtomicReference();
         try {
             SafeObserver safeObserver = new SafeObserver(OBSERVER_ONNEXT_FAIL(onError));
-            safeObserver.onSubscribe(Disposables.empty());
+            safeObserver.onSubscribe(Disposable.empty());
             safeObserver.onNext("one");
             assertNotNull(onError.get());
             assertTrue(onError.get() instanceof SafeObserverTestException);
@@ -209,7 +209,7 @@ public void dispose() {
 
         SafeObserver so = new SafeObserver(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         so.onSubscribe(d);
 
@@ -226,7 +226,7 @@ public void onNextAfterComplete() {
 
         SafeObserver so = new SafeObserver(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         so.onSubscribe(d);
 
@@ -247,7 +247,7 @@ public void onNextNull() {
 
         SafeObserver so = new SafeObserver(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         so.onSubscribe(d);
 
@@ -298,7 +298,7 @@ public void onNextNormal() {
 
         SafeObserver so = new SafeObserver(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         so.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java
index 1d467242f5..84fa434879 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java
@@ -220,7 +220,7 @@ public void runConcurrencyTest() {
             TestConcurrencySubscriber tw = new TestConcurrencySubscriber();
             // we need Synchronized + SafeObserver to handle synchronization plus life-cycle
             Observer w = serializedObserver(new SafeObserver(tw));
-            w.onSubscribe(Disposables.empty());
+            w.onSubscribe(Disposable.empty());
 
             Future f1 = tp.submit(new OnNextThread(w, 12000));
             Future f2 = tp.submit(new OnNextThread(w, 5000));
@@ -383,7 +383,7 @@ public void onNext(String t) {
         AtomicInteger p1 = new AtomicInteger();
         AtomicInteger p2 = new AtomicInteger();
 
-        o.onSubscribe(Disposables.empty());
+        o.onSubscribe(Disposable.empty());
         DisposableObserver as1 = new DisposableObserver() {
             @Override
             public void onNext(String t) {
@@ -446,7 +446,7 @@ private static Observable infinite(final AtomicInteger produced) {
 
             @Override
             public void subscribe(Observer observer) {
-                Disposable bs = Disposables.empty();
+                Disposable bs = Disposable.empty();
                 observer.onSubscribe(bs);
                 while (!bs.isDisposed()) {
                     observer.onNext("onNext");
@@ -651,7 +651,7 @@ private static class TestSingleThreadedObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             System.out.println("TestSingleThreadedObservable subscribed to ...");
             t = new Thread(new Runnable() {
 
@@ -703,7 +703,7 @@ private static class TestMultiThreadedObservable implements ObservableSource observer) {
-            observer.onSubscribe(Disposables.empty());
+            observer.onSubscribe(Disposable.empty());
             final NullPointerException npe = new NullPointerException();
             System.out.println("TestMultiThreadedObservable subscribed to ...");
             t = new Thread(new Runnable() {
@@ -859,7 +859,7 @@ public void onNext(Integer v) {
                 }
             };
             SerializedObserver sobs = new SerializedObserver(to);
-            sobs.onSubscribe(Disposables.empty());
+            sobs.onSubscribe(Disposable.empty());
             serial.set(sobs);
 
             sobs.onNext(1);
@@ -886,7 +886,7 @@ public void onNext(Integer v) {
             }
         };
         SerializedObserver sobs = new SerializedObserver(to);
-        sobs.onSubscribe(Disposables.empty());
+        sobs.onSubscribe(Disposable.empty());
         serial.set(sobs);
 
         sobs.onNext(1);
@@ -902,7 +902,7 @@ public void dispose() {
 
         SerializedObserver so = new SerializedObserver(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         so.onSubscribe(d);
 
@@ -922,7 +922,7 @@ public void onCompleteRace() {
 
             final SerializedObserver so = new SerializedObserver(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             so.onSubscribe(d);
 
@@ -948,7 +948,7 @@ public void onNextOnCompleteRace() {
 
             final SerializedObserver so = new SerializedObserver(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             so.onSubscribe(d);
 
@@ -984,7 +984,7 @@ public void onNextOnErrorRace() {
 
             final SerializedObserver so = new SerializedObserver(to);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             so.onSubscribe(d);
 
@@ -1022,7 +1022,7 @@ public void onNextOnErrorRaceDelayError() {
 
             final SerializedObserver so = new SerializedObserver(to, true);
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             so.onSubscribe(d);
 
@@ -1063,9 +1063,9 @@ public void startOnce() {
 
             final SerializedObserver so = new SerializedObserver(to);
 
-            so.onSubscribe(Disposables.empty());
+            so.onSubscribe(Disposable.empty());
 
-            Disposable d = Disposables.empty();
+            Disposable d = Disposable.empty();
 
             so.onSubscribe(d);
 
@@ -1087,7 +1087,7 @@ public void onCompleteOnErrorRace() {
 
                 final SerializedObserver so = new SerializedObserver(to);
 
-                Disposable d = Disposables.empty();
+                Disposable d = Disposable.empty();
 
                 so.onSubscribe(d);
 
@@ -1134,7 +1134,7 @@ public void nullOnNext() {
 
         final SerializedObserver so = new SerializedObserver(to);
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         so.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java
index 1468dc9cb5..581aa0b0b1 100644
--- a/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java
+++ b/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java
@@ -171,7 +171,7 @@ public void createDelegate() {
 
         assertFalse(to.hasSubscription());
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         assertTrue(to.hasSubscription());
 
@@ -243,7 +243,7 @@ public void assertError() {
             // expected
         }
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertSubscribed();
 
@@ -315,7 +315,7 @@ public void valueAndClass() {
     public void assertFailure() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onError(new TestException("Forced failure"));
 
@@ -330,7 +330,7 @@ public void assertFailure() {
     public void assertFuseable() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to = TestObserver.create();
 
@@ -341,7 +341,7 @@ public void assertFuseable() {
     public void assertResult() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onComplete();
 
@@ -378,7 +378,7 @@ public void assertResult() {
     public void await() throws Exception {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         assertFalse(to.await(100, TimeUnit.MILLISECONDS));
 
@@ -406,7 +406,7 @@ public void await() throws Exception {
 
         final TestObserver to1 = TestObserver.create();
 
-        to1.onSubscribe(Disposables.empty());
+        to1.onSubscribe(Disposable.empty());
 
         Schedulers.single().scheduleDirect(new Runnable() {
             @Override
@@ -422,7 +422,7 @@ public void run() {
     public void onNext() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         assertEquals(0, to.values().size());
 
@@ -445,7 +445,7 @@ public void onNext() {
     public void multipleTerminals() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertNotComplete();
 
@@ -479,7 +479,7 @@ public void multipleTerminals() {
     public void assertValue() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.assertValue(1);
@@ -519,7 +519,7 @@ public void onNextMisbehave() {
 
         to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onNext(null);
 
@@ -530,7 +530,7 @@ public void onNextMisbehave() {
     public void awaitTerminalEventInterrupt() {
         final TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         Thread.currentThread().interrupt();
 
@@ -559,7 +559,7 @@ public void awaitTerminalEventInterrupt() {
     public void assertTerminated2() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onError(new TestException());
         to.onError(new IOException());
@@ -573,7 +573,7 @@ public void assertTerminated2() {
 
         to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onError(new TestException());
         to.onComplete();
@@ -589,9 +589,9 @@ public void onSubscribe() {
 
         to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         to.onSubscribe(d1);
 
@@ -602,7 +602,7 @@ public void onSubscribe() {
         to = TestObserver.create();
         to.dispose();
 
-        d1 = Disposables.empty();
+        d1 = Disposable.empty();
 
         to.onSubscribe(d1);
 
@@ -614,7 +614,7 @@ public void onSubscribe() {
     public void assertValueSequence() {
         TestObserver to = TestObserver.create();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onNext(1);
         to.onNext(2);
@@ -654,7 +654,7 @@ public void assertEmpty() {
             // expected
         }
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertEmpty();
 
@@ -731,7 +731,7 @@ public boolean test(Throwable throwable) throws Exception {
     public void assertComplete() {
         TestObserver to = new TestObserver();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.assertComplete();
@@ -789,7 +789,7 @@ public void onComplete() {
 
         });
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.onComplete();
@@ -825,7 +825,7 @@ public void onComplete() {
 
         });
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.onError(new IOException());
@@ -1028,7 +1028,7 @@ public void withTag() {
     @Test
     public void assertValuesOnly() {
         TestObserver to = TestObserver.create();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
         to.assertValuesOnly();
 
         to.onNext(5);
@@ -1041,7 +1041,7 @@ public void assertValuesOnly() {
     @Test
     public void assertValuesOnlyThrowsOnUnexpectedValue() {
         TestObserver to = TestObserver.create();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
         to.assertValuesOnly();
 
         to.onNext(5);
@@ -1060,7 +1060,7 @@ public void assertValuesOnlyThrowsOnUnexpectedValue() {
     @Test
     public void assertValuesOnlyThrowsWhenCompleted() {
         TestObserver to = TestObserver.create();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onComplete();
 
@@ -1075,7 +1075,7 @@ public void assertValuesOnlyThrowsWhenCompleted() {
     @Test
     public void assertValuesOnlyThrowsWhenErrored() {
         TestObserver to = TestObserver.create();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onError(new TestException());
 
diff --git a/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java b/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java
index 5e0b5ab207..090ff6b766 100644
--- a/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java
+++ b/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java
@@ -1261,7 +1261,7 @@ public void reset() {
                         @SuppressWarnings("unchecked")
                         @Override
                         protected void subscribeActual(Observer observer) {
-                            observer.onSubscribe(Disposables.empty());
+                            observer.onSubscribe(Disposable.empty());
                             observer.onNext(10);
                             observer.onComplete();
                         }
diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/CachedThreadSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/CachedThreadSchedulerTest.java
index 35be875a9a..8782142035 100644
--- a/src/test/java/io/reactivex/rxjava3/schedulers/CachedThreadSchedulerTest.java
+++ b/src/test/java/io/reactivex/rxjava3/schedulers/CachedThreadSchedulerTest.java
@@ -114,11 +114,11 @@ public void run() {
         Worker w = s.createWorker();
         w.dispose();
 
-        assertEquals(Disposables.disposed(), w.schedule(r));
+        assertEquals(Disposable.disposed(), w.schedule(r));
 
-        assertEquals(Disposables.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
 
-        assertEquals(Disposables.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
 
         assertEquals(0, calls[0]);
     }
diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java b/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java
index 6d2aec07c9..a963f24665 100644
--- a/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java
+++ b/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java
@@ -18,11 +18,11 @@
 import java.util.HashMap;
 import java.util.concurrent.*;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.Test;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Scheduler.Worker;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.schedulers.ComputationScheduler;
 
@@ -173,22 +173,22 @@ public void run() {
         s.shutdown();
         s.shutdown();
 
-        assertEquals(Disposables.disposed(), s.scheduleDirect(r));
+        assertEquals(Disposable.disposed(), s.scheduleDirect(r));
 
-        assertEquals(Disposables.disposed(), s.scheduleDirect(r, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), s.scheduleDirect(r, 1, TimeUnit.SECONDS));
 
-        assertEquals(Disposables.disposed(), s.schedulePeriodicallyDirect(r, 1, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), s.schedulePeriodicallyDirect(r, 1, 1, TimeUnit.SECONDS));
 
         Worker w = s.createWorker();
         w.dispose();
 
         assertTrue(w.isDisposed());
 
-        assertEquals(Disposables.disposed(), w.schedule(r));
+        assertEquals(Disposable.disposed(), w.schedule(r));
 
-        assertEquals(Disposables.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
 
-        assertEquals(Disposables.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
 
         assertEquals(0, calls[0]);
     }
diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/NewThreadSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/NewThreadSchedulerTest.java
index df114d23e8..352e13333c 100644
--- a/src/test/java/io/reactivex/rxjava3/schedulers/NewThreadSchedulerTest.java
+++ b/src/test/java/io/reactivex/rxjava3/schedulers/NewThreadSchedulerTest.java
@@ -53,11 +53,11 @@ public void run() {
 
         assertTrue(w.isDisposed());
 
-        assertEquals(Disposables.disposed(), w.schedule(r));
+        assertEquals(Disposable.disposed(), w.schedule(r));
 
-        assertEquals(Disposables.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedule(r, 1, TimeUnit.SECONDS));
 
-        assertEquals(Disposables.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
+        assertEquals(Disposable.disposed(), w.schedulePeriodically(r, 1, 1, TimeUnit.SECONDS));
 
         NewThreadWorker actual = (NewThreadWorker)w;
 
diff --git a/src/test/java/io/reactivex/rxjava3/single/SingleTest.java b/src/test/java/io/reactivex/rxjava3/single/SingleTest.java
index 0e99b0b26d..81b5038719 100644
--- a/src/test/java/io/reactivex/rxjava3/single/SingleTest.java
+++ b/src/test/java/io/reactivex/rxjava3/single/SingleTest.java
@@ -133,7 +133,7 @@ public void createSuccess() {
         Single.unsafeCreate(new SingleSource() {
             @Override
             public void subscribe(SingleObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onSuccess("Hello");
             }
         }).toFlowable().subscribe(ts);
@@ -147,7 +147,7 @@ public void createError() {
         Single.unsafeCreate(new SingleSource() {
             @Override
             public void subscribe(SingleObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 observer.onError(new RuntimeException("fail"));
             }
         }).toFlowable().subscribe(ts);
@@ -204,7 +204,7 @@ public void timeout() {
         Single s1 = Single.unsafeCreate(new SingleSource() {
             @Override
             public void subscribe(SingleObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                 try {
                     Thread.sleep(5000);
                 } catch (InterruptedException e) {
@@ -226,7 +226,7 @@ public void timeoutWithFallback() {
         Single s1 = Single.unsafeCreate(new SingleSource() {
             @Override
             public void subscribe(SingleObserver observer) {
-                observer.onSubscribe(Disposables.empty());
+                observer.onSubscribe(Disposable.empty());
                     try {
                         Thread.sleep(5000);
                     } catch (InterruptedException e) {
@@ -269,7 +269,7 @@ public void run() {
                     }
 
                 });
-                sd.replace(Disposables.fromRunnable(new Runnable() {
+                sd.replace(Disposable.fromRunnable(new Runnable() {
                     @Override
                     public void run() {
                         unsubscribed.set(true);
@@ -343,7 +343,7 @@ public void run() {
                     }
 
                 });
-                sd.replace(Disposables.fromRunnable(new Runnable() {
+                sd.replace(Disposable.fromRunnable(new Runnable() {
                     @Override
                     public void run() {
                         unsubscribed.set(true);
@@ -399,7 +399,7 @@ public void run() {
                     }
 
                 });
-                sd.replace(Disposables.fromRunnable(new Runnable() {
+                sd.replace(Disposable.fromRunnable(new Runnable() {
                     @Override
                     public void run() {
                         unsubscribed.set(true);
@@ -431,7 +431,7 @@ public void backpressureAsObservable() {
         Single s = Single.unsafeCreate(new SingleSource() {
             @Override
             public void subscribe(SingleObserver t) {
-                t.onSubscribe(Disposables.empty());
+                t.onSubscribe(Disposable.empty());
                 t.onSuccess("hello");
             }
         });
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java
index 7a7e381bf9..e6f8245d22 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java
@@ -362,14 +362,14 @@ public void fusionOfflie() {
     public void onSubscribeAfterDone() {
         AsyncSubject p = AsyncSubject.create();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
         p.onSubscribe(bs);
 
         assertFalse(bs.isDisposed());
 
         p.onComplete();
 
-        bs = Disposables.empty();
+        bs = Disposable.empty();
         p.onSubscribe(bs);
 
         assertTrue(bs.isDisposed());
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java
index af73a522aa..0d14802814 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java
@@ -542,7 +542,7 @@ public void cancelOnArrival() {
     public void onSubscribe() {
         BehaviorSubject p = BehaviorSubject.create();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
 
         p.onSubscribe(bs);
 
@@ -550,7 +550,7 @@ public void onSubscribe() {
 
         p.onComplete();
 
-        bs = Disposables.empty();
+        bs = Disposable.empty();
 
         p.onSubscribe(bs);
 
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/CompletableSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/CompletableSubjectTest.java
index 34aaff3831..02ac92b064 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/CompletableSubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/CompletableSubjectTest.java
@@ -189,7 +189,7 @@ public void onComplete() {
     public void onSubscribeDispose() {
         CompletableSubject cs = CompletableSubject.create();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         cs.onSubscribe(d);
 
@@ -197,7 +197,7 @@ public void onSubscribeDispose() {
 
         cs.onComplete();
 
-        d = Disposables.empty();
+        d = Disposable.empty();
 
         cs.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/MaybeSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/MaybeSubjectTest.java
index 8ff3f7b1b5..31742e10a9 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/MaybeSubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/MaybeSubjectTest.java
@@ -234,7 +234,7 @@ public void onComplete() {
     public void onSubscribeDispose() {
         MaybeSubject ms = MaybeSubject.create();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ms.onSubscribe(d);
 
@@ -242,7 +242,7 @@ public void onSubscribeDispose() {
 
         ms.onComplete();
 
-        d = Disposables.empty();
+        d = Disposable.empty();
 
         ms.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java
index 45520ebffa..c1cef763fe 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java
@@ -19,12 +19,12 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.functions.Consumer;
 import io.reactivex.rxjava3.observers.DefaultObserver;
 import io.reactivex.rxjava3.schedulers.Schedulers;
@@ -43,7 +43,7 @@ public void run() {
 
                     @Override
                     public void subscribe(Observer o) {
-                        o.onSubscribe(Disposables.empty());
+                        o.onSubscribe(Disposable.empty());
                         System.out.println("********* Start Source Data ***********");
                         for (long l = 1; l <= 10000; l++) {
                             o.onNext(l);
@@ -153,7 +153,7 @@ public void run() {
 
                     @Override
                     public void subscribe(Observer o) {
-                        o.onSubscribe(Disposables.empty());
+                        o.onSubscribe(Disposable.empty());
                         System.out.println("********* Start Source Data ***********");
                         for (long l = 1; l <= 10000; l++) {
                             o.onNext(l);
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java
index b0d2e55aad..c608875071 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java
@@ -19,12 +19,12 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicReference;
 
+import io.reactivex.rxjava3.disposables.Disposable;
 import org.junit.*;
 
 import io.reactivex.rxjava3.core.*;
 import io.reactivex.rxjava3.core.Observable;
 import io.reactivex.rxjava3.core.Observer;
-import io.reactivex.rxjava3.disposables.Disposables;
 import io.reactivex.rxjava3.functions.Consumer;
 import io.reactivex.rxjava3.observers.DefaultObserver;
 import io.reactivex.rxjava3.schedulers.Schedulers;
@@ -43,7 +43,7 @@ public void run() {
 
                     @Override
                     public void subscribe(Observer o) {
-                        o.onSubscribe(Disposables.empty());
+                        o.onSubscribe(Disposable.empty());
                         System.out.println("********* Start Source Data ***********");
                         for (long l = 1; l <= 10000; l++) {
                             o.onNext(l);
@@ -153,7 +153,7 @@ public void run() {
 
                     @Override
                     public void subscribe(Observer o) {
-                        o.onSubscribe(Disposables.empty());
+                        o.onSubscribe(Disposable.empty());
                         System.out.println("********* Start Source Data ***********");
                         for (long l = 1; l <= 10000; l++) {
                             o.onNext(l);
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java
index 1993112789..e8d36a8c62 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java
@@ -967,7 +967,7 @@ public void subscribeAfterDone() {
         ReplaySubject rp = ReplaySubject.create();
         rp.onComplete();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
 
         rp.onSubscribe(bs);
 
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java
index 9355d77d29..2d9999380b 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java
@@ -410,7 +410,7 @@ public void normal() {
         }
         s.onComplete();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
         s.onSubscribe(bs);
         assertTrue(bs.isDisposed());
     }
@@ -522,7 +522,7 @@ public void onNextOnSubscribeRace() {
 
             TestObserver to = s.test();
 
-            final Disposable bs = Disposables.empty();
+            final Disposable bs = Disposable.empty();
 
             Runnable r1 = new Runnable() {
                 @Override
@@ -551,7 +551,7 @@ public void onCompleteOnSubscribeRace() {
 
             TestObserver to = s.test();
 
-            final Disposable bs = Disposables.empty();
+            final Disposable bs = Disposable.empty();
 
             Runnable r1 = new Runnable() {
                 @Override
@@ -643,8 +643,8 @@ public void onSubscribeOnSubscribeRace() {
 
             TestObserver to = s.test();
 
-            final Disposable bs1 = Disposables.empty();
-            final Disposable bs2 = Disposables.empty();
+            final Disposable bs1 = Disposable.empty();
+            final Disposable bs2 = Disposable.empty();
 
             Runnable r1 = new Runnable() {
                 @Override
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/SingleSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/SingleSubjectTest.java
index 7c07b63f0a..521fb77fe4 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/SingleSubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/SingleSubjectTest.java
@@ -209,7 +209,7 @@ public void onError(Throwable e) {
     public void onSubscribeDispose() {
         SingleSubject ss = SingleSubject.create();
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ss.onSubscribe(d);
 
@@ -217,7 +217,7 @@ public void onSubscribeDispose() {
 
         ss.onSuccess(1);
 
-        d = Disposables.empty();
+        d = Disposable.empty();
 
         ss.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java
index ba2deb2c91..a65544532b 100644
--- a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java
@@ -262,7 +262,7 @@ public void afterDone() {
         UnicastSubject p = UnicastSubject.create();
         p.onComplete();
 
-        Disposable bs = Disposables.empty();
+        Disposable bs = Disposable.empty();
         p.onSubscribe(bs);
 
         p.onNext(1);
@@ -381,7 +381,7 @@ public void dispose() {
             RxJavaPlugins.reset();
         }
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         us.onSubscribe(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java
index 190fe0e836..5a444ee9a9 100644
--- a/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java
+++ b/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java
@@ -81,7 +81,7 @@ public void addResources() {
 
         assertFalse(ro.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ro.add(d);
 
@@ -106,7 +106,7 @@ public void onCompleteCleansUp() {
 
         assertFalse(ro.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ro.add(d);
 
@@ -125,7 +125,7 @@ public void onErrorCleansUp() {
 
         assertFalse(ro.isDisposed());
 
-        Disposable d = Disposables.empty();
+        Disposable d = Disposable.empty();
 
         ro.add(d);
 
diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java
index f31475ae58..3faefd13a5 100644
--- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java
+++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java
@@ -520,11 +520,11 @@ public static void doubleOnSubscribe(Subscriber subscriber) {
     public static void doubleOnSubscribe(Observer observer) {
         List errors = trackPluginErrors();
         try {
-            Disposable d1 = Disposables.empty();
+            Disposable d1 = Disposable.empty();
 
             observer.onSubscribe(d1);
 
-            Disposable d2 = Disposables.empty();
+            Disposable d2 = Disposable.empty();
 
             observer.onSubscribe(d2);
 
@@ -546,11 +546,11 @@ public static void doubleOnSubscribe(Observer observer) {
     public static void doubleOnSubscribe(SingleObserver observer) {
         List errors = trackPluginErrors();
         try {
-            Disposable d1 = Disposables.empty();
+            Disposable d1 = Disposable.empty();
 
             observer.onSubscribe(d1);
 
-            Disposable d2 = Disposables.empty();
+            Disposable d2 = Disposable.empty();
 
             observer.onSubscribe(d2);
 
@@ -572,11 +572,11 @@ public static void doubleOnSubscribe(SingleObserver observer) {
     public static void doubleOnSubscribe(CompletableObserver observer) {
         List errors = trackPluginErrors();
         try {
-            Disposable d1 = Disposables.empty();
+            Disposable d1 = Disposable.empty();
 
             observer.onSubscribe(d1);
 
-            Disposable d2 = Disposables.empty();
+            Disposable d2 = Disposable.empty();
 
             observer.onSubscribe(d2);
 
@@ -598,11 +598,11 @@ public static void doubleOnSubscribe(CompletableObserver observer) {
     public static void doubleOnSubscribe(MaybeObserver observer) {
         List errors = trackPluginErrors();
         try {
-            Disposable d1 = Disposables.empty();
+            Disposable d1 = Disposable.empty();
 
             observer.onSubscribe(d1);
 
-            Disposable d2 = Disposables.empty();
+            Disposable d2 = Disposable.empty();
 
             observer.onSubscribe(d2);
 
@@ -899,11 +899,11 @@ public static  void checkDoubleOnSubscribeMaybe(Function, ? exten
                 @Override
                 protected void subscribeActual(MaybeObserver observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -953,11 +953,11 @@ public static  void checkDoubleOnSubscribeMaybeToSingle(Function,
                 @Override
                 protected void subscribeActual(MaybeObserver observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1007,11 +1007,11 @@ public static  void checkDoubleOnSubscribeMaybeToObservable(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1061,11 +1061,11 @@ public static  void checkDoubleOnSubscribeMaybeToFlowable(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1115,11 +1115,11 @@ public static  void checkDoubleOnSubscribeSingleToMaybe(Function
                 @Override
                 protected void subscribeActual(SingleObserver observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1169,11 +1169,11 @@ public static  void checkDoubleOnSubscribeSingleToObservable(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1223,11 +1223,11 @@ public static  void checkDoubleOnSubscribeSingleToFlowable(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1276,11 +1276,11 @@ public static  void checkDoubleOnSubscribeMaybeToCompletable(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1330,11 +1330,11 @@ public static  void checkDoubleOnSubscribeSingle(Function, ? ext
                 @Override
                 protected void subscribeActual(SingleObserver observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1438,11 +1438,11 @@ public static  void checkDoubleOnSubscribeObservable(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1492,11 +1492,11 @@ public static  void checkDoubleOnSubscribeObservableToSingle(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1546,11 +1546,11 @@ public static  void checkDoubleOnSubscribeObservableToMaybe(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1599,11 +1599,11 @@ public static  void checkDoubleOnSubscribeObservableToCompletable(Function observer) {
                     try {
-                        Disposable d1 = Disposables.empty();
+                        Disposable d1 = Disposable.empty();
 
                         observer.onSubscribe(d1);
 
-                        Disposable d2 = Disposables.empty();
+                        Disposable d2 = Disposable.empty();
 
                         observer.onSubscribe(d2);
 
@@ -1866,11 +1866,11 @@ public static void checkDoubleOnSubscribeCompletable(Function void checkDoubleOnSubscribeCompletableToMaybe(Function void checkDoubleOnSubscribeCompletableToSingle(Function void checkBadSourceObservable(Function, Object>
                 boolean once;
                 @Override
                 protected void subscribeActual(Observer observer) {
-                    observer.onSubscribe(Disposables.empty());
+                    observer.onSubscribe(Disposable.empty());
 
                     if (once) {
                         return;
@@ -3336,7 +3336,7 @@ public Integer apply(Integer v) throws Throwable {
                 to.assertEmpty();
             } else if (result instanceof Publisher) {
                 TestSubscriberEx ts = new TestSubscriberEx<>();
-                disposable.set(Disposables.fromSubscription(ts));
+                disposable.set(Disposable.fromSubscription(ts));
 
                 ((Publisher)result)
                 .subscribe(ts);
@@ -3404,7 +3404,7 @@ public Integer apply(Integer v) throws Throwable {
                 to.assertEmpty();
             } else if (result instanceof Publisher) {
                 TestSubscriberEx ts = new TestSubscriberEx<>();
-                disposable.set(Disposables.fromSubscription(ts));
+                disposable.set(Disposable.fromSubscription(ts));
 
                 ((Publisher)result)
                 .subscribe(ts);
diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java
index 6f8d3e25cc..e374aa67a2 100644
--- a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java
+++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java
@@ -275,7 +275,7 @@ public void createDelegate() {
 
         assertFalse(to.hasSubscription());
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.assertNotSubscribed();
@@ -370,7 +370,7 @@ public void assertError() {
             // expected
         }
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertSubscribed();
 
@@ -453,7 +453,7 @@ public void valueAndClass() {
     public void assertFailure() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onError(new TestException("Forced failure"));
 
@@ -476,7 +476,7 @@ public void assertFailure() {
     public void assertFuseable() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertNotFuseable();
 
@@ -539,7 +539,7 @@ public void assertTerminated() {
     public void assertResult() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onComplete();
 
@@ -576,7 +576,7 @@ public void assertResult() {
     public void await() throws Exception {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         assertFalse(to.await(100, TimeUnit.MILLISECONDS));
 
@@ -598,7 +598,7 @@ public void await() throws Exception {
 
         final TestObserverEx to1 = new TestObserverEx();
 
-        to1.onSubscribe(Disposables.empty());
+        to1.onSubscribe(Disposable.empty());
 
         Schedulers.single().scheduleDirect(new Runnable() {
             @Override
@@ -614,7 +614,7 @@ public void run() {
     public void errors() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         assertEquals(0, to.errors().size());
 
@@ -629,7 +629,7 @@ public void errors() {
     public void onNext() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertValueCount(0);
 
@@ -660,7 +660,7 @@ public void fusionModeToString() {
     public void multipleTerminals() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertNotComplete();
 
@@ -703,7 +703,7 @@ public void multipleTerminals() {
     public void assertValue() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.assertValue(1);
@@ -743,7 +743,7 @@ public void onNextMisbehave() {
 
         to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onNext(null);
 
@@ -754,7 +754,7 @@ public void onNextMisbehave() {
     public void assertTerminated2() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertNotTerminated();
 
@@ -780,7 +780,7 @@ public void assertTerminated2() {
 
         to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onError(new TestException());
         to.onComplete();
@@ -803,9 +803,9 @@ public void onSubscribe() {
 
         to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
-        Disposable d1 = Disposables.empty();
+        Disposable d1 = Disposable.empty();
 
         to.onSubscribe(d1);
 
@@ -816,7 +816,7 @@ public void onSubscribe() {
         to = new TestObserverEx();
         to.dispose();
 
-        d1 = Disposables.empty();
+        d1 = Disposable.empty();
 
         to.onSubscribe(d1);
 
@@ -828,7 +828,7 @@ public void onSubscribe() {
     public void assertValueSequence() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onNext(1);
         to.onNext(2);
@@ -868,7 +868,7 @@ public void assertEmpty() {
             // expected
         }
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.assertEmpty();
 
@@ -967,7 +967,7 @@ public boolean test(Throwable throwable) throws Exception {
     public void assertComplete() {
         TestObserverEx to = new TestObserverEx();
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.assertComplete();
@@ -1025,7 +1025,7 @@ public void onComplete() {
 
         });
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.onComplete();
@@ -1061,7 +1061,7 @@ public void onComplete() {
 
         });
 
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         try {
             to.onError(new IOException());
@@ -1323,7 +1323,7 @@ public void withTag() {
     @Test
     public void assertValuesOnly() {
         TestObserverEx to = new TestObserverEx();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
         to.assertValuesOnly();
 
         to.onNext(5);
@@ -1336,7 +1336,7 @@ public void assertValuesOnly() {
     @Test
     public void assertValuesOnlyThrowsOnUnexpectedValue() {
         TestObserverEx to = new TestObserverEx();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
         to.assertValuesOnly();
 
         to.onNext(5);
@@ -1355,7 +1355,7 @@ public void assertValuesOnlyThrowsOnUnexpectedValue() {
     @Test
     public void assertValuesOnlyThrowsWhenCompleted() {
         TestObserverEx to = new TestObserverEx();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onComplete();
 
@@ -1370,7 +1370,7 @@ public void assertValuesOnlyThrowsWhenCompleted() {
     @Test
     public void assertValuesOnlyThrowsWhenErrored() {
         TestObserverEx to = new TestObserverEx();
-        to.onSubscribe(Disposables.empty());
+        to.onSubscribe(Disposable.empty());
 
         to.onError(new TestException());
 
diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TimesteppingScheduler.java b/src/test/java/io/reactivex/rxjava3/testsupport/TimesteppingScheduler.java
index cb9a81eca0..2615a3fbe5 100644
--- a/src/test/java/io/reactivex/rxjava3/testsupport/TimesteppingScheduler.java
+++ b/src/test/java/io/reactivex/rxjava3/testsupport/TimesteppingScheduler.java
@@ -37,7 +37,7 @@ public boolean isDisposed() {
         @Override
         public Disposable schedule(Runnable run, long delay, TimeUnit unit) {
             run.run();
-            return Disposables.disposed();
+            return Disposable.disposed();
         }
 
         @Override

From 747f59eaaa66af6ce8eb06d779cd72909ec9ed05 Mon Sep 17 00:00:00 2001
From: David Karnok 
Date: Sat, 21 Dec 2019 11:08:18 +0100
Subject: [PATCH 014/665] 3.x: [Java 8] Add fromOpt/Stage, mapOptional,
 toCompletionStage to M/S/C (#6783)

---
 .../reactivex/rxjava3/core/Completable.java   |  70 ++++++-
 .../java/io/reactivex/rxjava3/core/Maybe.java | 162 +++++++++++++++-
 .../io/reactivex/rxjava3/core/Single.java     |  91 ++++++++-
 .../jdk8/CompletableFromCompletionStage.java  |  78 ++++++++
 .../jdk8/CompletionStageConsumer.java         |  99 ++++++++++
 .../jdk8/MaybeFromCompletionStage.java        |  81 ++++++++
 .../internal/jdk8/MaybeMapOptional.java       | 110 +++++++++++
 .../jdk8/SingleFromCompletionStage.java       |  81 ++++++++
 .../internal/jdk8/SingleMapOptional.java      | 105 +++++++++++
 .../CompletableFromCompletionStageTest.java   |  78 ++++++++
 .../CompletableToCompletionStageTest.java     | 154 ++++++++++++++++
 .../jdk8/MaybeFromCompletionStageTest.java    |  71 +++++++
 .../internal/jdk8/MaybeFromOptionalTest.java  |  38 ++++
 .../internal/jdk8/MaybeMapOptionalTest.java   |  89 +++++++++
 .../jdk8/MaybeToCompletionStageTest.java      | 174 ++++++++++++++++++
 .../jdk8/SingleFromCompletionStageTest.java   |  71 +++++++
 .../internal/jdk8/SingleMapOptionalTest.java  |  76 ++++++++
 .../jdk8/SingleToCompletionStageTest.java     | 153 +++++++++++++++
 .../ParamValidationCheckerTest.java           |   3 +
 19 files changed, 1779 insertions(+), 5 deletions(-)
 create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStage.java
 create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletionStageConsumer.java
 create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStage.java
 create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptional.java
 create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStage.java
 create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptional.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStageTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableToCompletionStageTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStageTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromOptionalTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptionalTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeToCompletionStageTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStageTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptionalTest.java
 create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleToCompletionStageTest.java

diff --git a/src/main/java/io/reactivex/rxjava3/core/Completable.java b/src/main/java/io/reactivex/rxjava3/core/Completable.java
index f25320d66b..d9991a913e 100644
--- a/src/main/java/io/reactivex/rxjava3/core/Completable.java
+++ b/src/main/java/io/reactivex/rxjava3/core/Completable.java
@@ -12,7 +12,7 @@
  */
 package io.reactivex.rxjava3.core;
 
-import java.util.Objects;
+import java.util.*;
 import java.util.concurrent.*;
 
 import org.reactivestreams.Publisher;
@@ -23,6 +23,7 @@
 import io.reactivex.rxjava3.functions.*;
 import io.reactivex.rxjava3.internal.functions.*;
 import io.reactivex.rxjava3.internal.fuseable.*;
+import io.reactivex.rxjava3.internal.jdk8.*;
 import io.reactivex.rxjava3.internal.observers.*;
 import io.reactivex.rxjava3.internal.operators.completable.*;
 import io.reactivex.rxjava3.internal.operators.maybe.*;
@@ -2753,4 +2754,71 @@ public final TestObserver test(boolean dispose) {
         subscribe(to);
         return to;
     }
+
+    // -------------------------------------------------------------------------
+    // JDK 8 Support
+    // -------------------------------------------------------------------------
+
+    /**
+     * Signals completion (or error) when the {@link CompletionStage} terminates.
+     * 

+ * + *

+ * Note that the operator takes an already instantiated, running or terminated {@code CompletionStage}. + * If the optional is to be created per consumer upon subscription, use {@link #defer(Supplier)} + * around {@code fromCompletionStage}: + *


+     * Maybe.defer(() -> Completable.fromCompletionStage(createCompletionStage()));
+     * 
+ *

+ * Canceling the flow can't cancel the execution of the {@code CompletionStage} because {@code CompletionStage} + * itself doesn't support cancellation. Instead, the operator detaches from the {@code CompletionStage}. + *

+ *
Scheduler:
+ *
{@code fromCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param stage the CompletionStage to convert to Maybe and signal its terminal value or error + * @return the new Completable instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Completable fromCompletionStage(@NonNull CompletionStage stage) { + Objects.requireNonNull(stage, "stage is null"); + return RxJavaPlugins.onAssembly(new CompletableFromCompletionStage<>(stage)); + } + + /** + * Signals the given default item when the upstream completes or signals the upstream error via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).toCompletionStage(Optional.empty());
+     * 
+ *
+ *
Scheduler:
+ *
{@code toCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the type of the default item to signal upon completion + * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage toCompletionStage(@Nullable T defaultItem) { + return subscribeWith(new CompletionStageConsumer<>(true, defaultItem)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/core/Maybe.java b/src/main/java/io/reactivex/rxjava3/core/Maybe.java index e3f4641ad3..35484ebef4 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Maybe.java +++ b/src/main/java/io/reactivex/rxjava3/core/Maybe.java @@ -13,8 +13,7 @@ package io.reactivex.rxjava3.core; -import java.util.NoSuchElementException; -import java.util.Objects; +import java.util.*; import java.util.concurrent.*; import org.reactivestreams.*; @@ -25,6 +24,7 @@ import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.functions.*; import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.internal.jdk8.*; import io.reactivex.rxjava3.internal.observers.BlockingMultiObserver; import io.reactivex.rxjava3.internal.operators.flowable.*; import io.reactivex.rxjava3.internal.operators.maybe.*; @@ -4794,4 +4794,162 @@ public final TestObserver test(boolean dispose) { subscribe(to); return to; } + + // ------------------------------------------------------------------------- + // JDK 8 Support + // ------------------------------------------------------------------------- + + /** + * Converts the existing value of the provided optional into a {@link #just(Object)} + * or an empty optional into an {@link #empty()} {@code Maybe} instance. + *

+ * + *

+ * Note that the operator takes an already instantiated optional reference and does not + * by any means create this original optional. If the optional is to be created per + * consumer upon subscription, use {@link #defer(Supplier)} around {@code fromOptional}: + *


+     * Maybe.defer(() -> Maybe.fromOptional(createOptional()));
+     * 
+ *
+ *
Scheduler:
+ *
{@code fromOptional} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the optional value + * @param optional the optional value to convert into a {@code Maybe} + * @return the new Maybe instance + * @see #just(Object) + * @see #empty() + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Maybe<@NonNull T> fromOptional(@NonNull Optional optional) { + Objects.requireNonNull(optional, "optional is null"); + return optional.map(Maybe::just).orElseGet(Maybe::empty); + } + + /** + * Signals the completion value or error of the given (hot) {@link CompletionStage}-based asynchronous calculation. + *

+ * + *

+ * Note that the operator takes an already instantiated, running or terminated {@code CompletionStage}. + * If the optional is to be created per consumer upon subscription, use {@link #defer(Supplier)} + * around {@code fromCompletionStage}: + *


+     * Maybe.defer(() -> Maybe.fromCompletionStage(createCompletionStage()));
+     * 
+ *

+ * If the {@code CompletionStage} completes with {@code null}, the resulting {@code Maybe} is completed via {@code onComplete}. + *

+ * Canceling the flow can't cancel the execution of the {@code CompletionStage} because {@code CompletionStage} + * itself doesn't support cancellation. Instead, the operator detaches from the {@code CompletionStage}. + *

+ *
Scheduler:
+ *
{@code fromCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the CompletionStage + * @param stage the CompletionStage to convert to Maybe and signal its terminal value or error + * @return the new Maybe instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Maybe<@NonNull T> fromCompletionStage(@NonNull CompletionStage stage) { + Objects.requireNonNull(stage, "stage is null"); + return RxJavaPlugins.onAssembly(new MaybeFromCompletionStage<>(stage)); + } + + /** + * Maps the upstream success value into an {@link Optional} and emits the contained item if not empty. + *

+ * + * + *

+ *
Scheduler:
+ *
{@code mapOptional} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the non-null output type + * @param mapper the function that receives the upstream success iteem and should return a non-empty {@code Optional} + * to emit as the success output or an empty {@code Optional} to complete the {@code Maybe} + * @return the new Maybe instance + * @since 3.0.0 + * @see #map(Function) + * @see #filter(Predicate) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Maybe mapOptional(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new MaybeMapOptional<>(this, mapper)); + } + + /** + * Signals the upstream success item (or a {@link NoSuchElementException} if the upstream is empty) via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@link #toCompletionStage(Object)} with {@code null} or turn the upstrea into a sequence of {@link Optional}s and + * default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).toCompletionStage(Optional.empty());
+     * 
+ *
+ *
Scheduler:
+ *
{@code toCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + * @see #toCompletionStage(Object) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage toCompletionStage() { + return subscribeWith(new CompletionStageConsumer<>(false, null)); + } + + /** + * Signals the upstream success item (or the default item if the upstream is empty) via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).toCompletionStage(Optional.empty());
+     * 
+ *
+ *
Scheduler:
+ *
{@code toCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage toCompletionStage(@Nullable T defaultItem) { + return subscribeWith(new CompletionStageConsumer<>(true, defaultItem)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/core/Single.java b/src/main/java/io/reactivex/rxjava3/core/Single.java index 27fad6d53e..ae2b04fc7d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Single.java +++ b/src/main/java/io/reactivex/rxjava3/core/Single.java @@ -13,8 +13,7 @@ package io.reactivex.rxjava3.core; -import java.util.NoSuchElementException; -import java.util.Objects; +import java.util.*; import java.util.concurrent.*; import org.reactivestreams.Publisher; @@ -25,6 +24,7 @@ import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.functions.*; import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.internal.jdk8.*; import io.reactivex.rxjava3.internal.observers.*; import io.reactivex.rxjava3.internal.operators.completable.*; import io.reactivex.rxjava3.internal.operators.flowable.*; @@ -4181,4 +4181,91 @@ public final TestObserver test(boolean dispose) { private static Single toSingle(Flowable source) { return RxJavaPlugins.onAssembly(new FlowableSingleSingle(source, null)); } + + // ------------------------------------------------------------------------- + // JDK 8 Support + // ------------------------------------------------------------------------- + + /** + * Signals the completion value or error of the given (hot) {@link CompletionStage}-based asynchronous calculation. + *

+ * + *

+ * Note that the operator takes an already instantiated, running or terminated {@code CompletionStage}. + * If the optional is to be created per consumer upon subscription, use {@link #defer(Supplier)} + * around {@code fromCompletionStage}: + *


+     * Single.defer(() -> Single.fromCompletionStage(createCompletionStage()));
+     * 
+ *

+ * If the {@code CompletionStage} completes with {@code null}, the resulting {@code Single} is terminated with + * a {@link NullPointerException}. + *

+ * Canceling the flow can't cancel the execution of the {@code CompletionStage} because {@code CompletionStage} + * itself doesn't support cancellation. Instead, the operator detaches from the {@code CompletionStage}. + *

+ *
Scheduler:
+ *
{@code fromCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the CompletionStage + * @param stage the CompletionStage to convert to Single and signal its success value or error + * @return the new Single instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Single<@NonNull T> fromCompletionStage(@NonNull CompletionStage stage) { + Objects.requireNonNull(stage, "stage is null"); + return RxJavaPlugins.onAssembly(new SingleFromCompletionStage<>(stage)); + } + + /** + * Maps the upstream success value into an {@link Optional} and emits the contained item if not empty. + *

+ * + * + *

+ *
Scheduler:
+ *
{@code mapOptional} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the non-null output type + * @param mapper the function that receives the upstream success iteem and should return a non-empty {@code Optional} + * to emit as the success output or an empty {@code Optional} to complete the {@code Maybe} + * @return the new Maybe instance + * @since 3.0.0 + * @see #map(Function) + * @see #filter(Predicate) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Maybe mapOptional(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new SingleMapOptional<>(this, mapper)); + } + + /** + * Signals the upstream success item (or error) via a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ *
Scheduler:
+ *
{@code toCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage toCompletionStage() { + return subscribeWith(new CompletionStageConsumer<>(false, null)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStage.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStage.java new file mode 100644 index 0000000000..44f539bf4b --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStage.java @@ -0,0 +1,78 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletionStage; +import java.util.function.BiConsumer; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.internal.jdk8.FlowableFromCompletionStage.BiConsumerAtomicReference; + +/** + * Wrap a CompletionStage and signal its outcome. + * @param the element type of the CompletionsStage + * @since 3.0.0 + */ +public final class CompletableFromCompletionStage extends Completable { + + final CompletionStage stage; + + public CompletableFromCompletionStage(CompletionStage stage) { + this.stage = stage; + } + + @Override + protected void subscribeActual(CompletableObserver observer) { + // We need an indirection because one can't detach from a whenComplete + // and cancellation should not hold onto the stage. + BiConsumerAtomicReference whenReference = new BiConsumerAtomicReference<>(); + CompletionStageHandler handler = new CompletionStageHandler<>(observer, whenReference); + whenReference.lazySet(handler); + + observer.onSubscribe(handler); + stage.whenComplete(whenReference); + } + + static final class CompletionStageHandler + implements Disposable, BiConsumer { + + final CompletableObserver downstream; + + final BiConsumerAtomicReference whenReference; + + CompletionStageHandler(CompletableObserver downstream, BiConsumerAtomicReference whenReference) { + this.downstream = downstream; + this.whenReference = whenReference; + } + + @Override + public void accept(T item, Throwable error) { + if (error != null) { + downstream.onError(error); + } else { + downstream.onComplete(); + } + } + + @Override + public void dispose() { + whenReference.set(null); + } + + @Override + public boolean isDisposed() { + return whenReference.get() == null; + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletionStageConsumer.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletionStageConsumer.java new file mode 100644 index 0000000000..d92e56041e --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/CompletionStageConsumer.java @@ -0,0 +1,99 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.internal.disposables.DisposableHelper; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Class that extends CompletableFuture and converts multiple types of reactive consumers + * and their signals into completion signals. + * @param the element type + * @since 3.0.0 + */ +public final class CompletionStageConsumer extends CompletableFuture +implements MaybeObserver, SingleObserver, CompletableObserver { + + final AtomicReference upstream; + + final boolean hasDefault; + + final T defaultItem; + + public CompletionStageConsumer(boolean hasDefault, T defaultItem) { + this.hasDefault = hasDefault; + this.defaultItem = defaultItem; + this.upstream = new AtomicReference<>(); + } + + @Override + public void onSubscribe(@NonNull Disposable d) { + DisposableHelper.setOnce(upstream, d); + } + + @Override + public void onSuccess(@NonNull T t) { + clear(); + complete(t); + } + + @Override + public void onError(Throwable t) { + clear(); + if (!completeExceptionally(t)) { + RxJavaPlugins.onError(t); + } + } + + @Override + public void onComplete() { + if (hasDefault) { + complete(defaultItem); + } else { + completeExceptionally(new NoSuchElementException("The source was empty")); + } + } + + void cancelUpstream() { + DisposableHelper.dispose(upstream); + } + + void clear() { + upstream.lazySet(DisposableHelper.DISPOSED); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + cancelUpstream(); + return super.cancel(mayInterruptIfRunning); + } + + @Override + public boolean complete(T value) { + cancelUpstream(); + return super.complete(value); + } + + @Override + public boolean completeExceptionally(Throwable ex) { + cancelUpstream(); + return super.completeExceptionally(ex); + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStage.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStage.java new file mode 100644 index 0000000000..d8523b8ead --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStage.java @@ -0,0 +1,81 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletionStage; +import java.util.function.BiConsumer; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.internal.jdk8.FlowableFromCompletionStage.BiConsumerAtomicReference; + +/** + * Wrap a CompletionStage and signal its outcome. + * @param the element type + * @since 3.0.0 + */ +public final class MaybeFromCompletionStage extends Maybe { + + final CompletionStage stage; + + public MaybeFromCompletionStage(CompletionStage stage) { + this.stage = stage; + } + + @Override + protected void subscribeActual(MaybeObserver observer) { + // We need an indirection because one can't detach from a whenComplete + // and cancellation should not hold onto the stage. + BiConsumerAtomicReference whenReference = new BiConsumerAtomicReference<>(); + CompletionStageHandler handler = new CompletionStageHandler<>(observer, whenReference); + whenReference.lazySet(handler); + + observer.onSubscribe(handler); + stage.whenComplete(whenReference); + } + + static final class CompletionStageHandler + implements Disposable, BiConsumer { + + final MaybeObserver downstream; + + final BiConsumerAtomicReference whenReference; + + CompletionStageHandler(MaybeObserver downstream, BiConsumerAtomicReference whenReference) { + this.downstream = downstream; + this.whenReference = whenReference; + } + + @Override + public void accept(T item, Throwable error) { + if (error != null) { + downstream.onError(error); + } + else if (item != null) { + downstream.onSuccess(item); + } else { + downstream.onComplete(); + } + } + + @Override + public void dispose() { + whenReference.set(null); + } + + @Override + public boolean isDisposed() { + return whenReference.get() == null; + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptional.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptional.java new file mode 100644 index 0000000000..8330d5249a --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptional.java @@ -0,0 +1,110 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.disposables.DisposableHelper; + +/** + * Maps the success value to an {@link Optional} and emits its non-empty value or completes. + * + * @param the upstream success value type + * @param the result value type + * @since 3.0.0 + */ +public final class MaybeMapOptional extends Maybe { + + final Maybe source; + + final Function> mapper; + + public MaybeMapOptional(Maybe source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(MaybeObserver observer) { + source.subscribe(new MapOptionalMaybeObserver<>(observer, mapper)); + } + + static final class MapOptionalMaybeObserver implements MaybeObserver, Disposable { + + final MaybeObserver downstream; + + final Function> mapper; + + Disposable upstream; + + MapOptionalMaybeObserver(MaybeObserver downstream, Function> mapper) { + this.downstream = downstream; + this.mapper = mapper; + } + + @Override + public void dispose() { + Disposable d = this.upstream; + this.upstream = DisposableHelper.DISPOSED; + d.dispose(); + } + + @Override + public boolean isDisposed() { + return upstream.isDisposed(); + } + + @Override + public void onSubscribe(Disposable d) { + if (DisposableHelper.validate(this.upstream, d)) { + this.upstream = d; + + downstream.onSubscribe(this); + } + } + + @Override + public void onSuccess(T value) { + Optional v; + + try { + v = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null item"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + + if (v.isPresent()) { + downstream.onSuccess(v.get()); + } else { + downstream.onComplete(); + } + } + + @Override + public void onError(Throwable e) { + downstream.onError(e); + } + + @Override + public void onComplete() { + downstream.onComplete(); + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStage.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStage.java new file mode 100644 index 0000000000..f1be64375e --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStage.java @@ -0,0 +1,81 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletionStage; +import java.util.function.BiConsumer; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.internal.jdk8.FlowableFromCompletionStage.BiConsumerAtomicReference; + +/** + * Wrap a CompletionStage and signal its outcome. + * @param the element type + * @since 3.0.0 + */ +public final class SingleFromCompletionStage extends Single { + + final CompletionStage stage; + + public SingleFromCompletionStage(CompletionStage stage) { + this.stage = stage; + } + + @Override + protected void subscribeActual(SingleObserver observer) { + // We need an indirection because one can't detach from a whenComplete + // and cancellation should not hold onto the stage. + BiConsumerAtomicReference whenReference = new BiConsumerAtomicReference<>(); + CompletionStageHandler handler = new CompletionStageHandler<>(observer, whenReference); + whenReference.lazySet(handler); + + observer.onSubscribe(handler); + stage.whenComplete(whenReference); + } + + static final class CompletionStageHandler + implements Disposable, BiConsumer { + + final SingleObserver downstream; + + final BiConsumerAtomicReference whenReference; + + CompletionStageHandler(SingleObserver downstream, BiConsumerAtomicReference whenReference) { + this.downstream = downstream; + this.whenReference = whenReference; + } + + @Override + public void accept(T item, Throwable error) { + if (error != null) { + downstream.onError(error); + } + else if (item != null) { + downstream.onSuccess(item); + } else { + downstream.onError(new NullPointerException("The CompletionStage terminated with null.")); + } + } + + @Override + public void dispose() { + whenReference.set(null); + } + + @Override + public boolean isDisposed() { + return whenReference.get() == null; + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptional.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptional.java new file mode 100644 index 0000000000..9ee9a6f803 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptional.java @@ -0,0 +1,105 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.disposables.DisposableHelper; + +/** + * Maps the success value to an {@link Optional} and emits its non-empty value or completes. + * + * @param the upstream success value type + * @param the result value type + * @since 3.0.0 + */ +public final class SingleMapOptional extends Maybe { + + final Single source; + + final Function> mapper; + + public SingleMapOptional(Single source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(MaybeObserver observer) { + source.subscribe(new MapOptionalSingleObserver<>(observer, mapper)); + } + + static final class MapOptionalSingleObserver implements SingleObserver, Disposable { + + final MaybeObserver downstream; + + final Function> mapper; + + Disposable upstream; + + MapOptionalSingleObserver(MaybeObserver downstream, Function> mapper) { + this.downstream = downstream; + this.mapper = mapper; + } + + @Override + public void dispose() { + Disposable d = this.upstream; + this.upstream = DisposableHelper.DISPOSED; + d.dispose(); + } + + @Override + public boolean isDisposed() { + return upstream.isDisposed(); + } + + @Override + public void onSubscribe(Disposable d) { + if (DisposableHelper.validate(this.upstream, d)) { + this.upstream = d; + + downstream.onSubscribe(this); + } + } + + @Override + public void onSuccess(T value) { + Optional v; + + try { + v = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null item"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + + if (v.isPresent()) { + downstream.onSuccess(v.get()); + } else { + downstream.onComplete(); + } + } + + @Override + public void onError(Throwable e) { + downstream.onError(e); + } + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStageTest.java new file mode 100644 index 0000000000..3519cc5454 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableFromCompletionStageTest.java @@ -0,0 +1,78 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class CompletableFromCompletionStageTest extends RxJavaTest { + + @Test + public void syncSuccess() { + Completable.fromCompletionStage(CompletableFuture.completedFuture(1)) + .test() + .assertResult(); + } + + @Test + public void syncSuccessNull() { + Completable.fromCompletionStage(CompletableFuture.completedFuture(null)) + .test() + .assertResult(); + } + + @Test + public void syncFailure() { + CompletableFuture cf = new CompletableFuture<>(); + cf.completeExceptionally(new TestException()); + + Completable.fromCompletionStage(cf) + .test() + .assertFailure(TestException.class); + } + + @Test + public void syncNull() { + Completable.fromCompletionStage(CompletableFuture.completedFuture(null)) + .test() + .assertResult(); + } + + @Test + public void dispose() { + CompletableFuture cf = new CompletableFuture<>(); + + TestObserver to = Completable.fromCompletionStage(cf) + .test(); + + to.assertEmpty(); + + to.dispose(); + + cf.complete(1); + + to.assertEmpty(); + } + + @Test + public void dispose2() { + TestHelper.checkDisposed(Completable.fromCompletionStage(new CompletableFuture<>())); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableToCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableToCompletionStageTest.java new file mode 100644 index 0000000000..91fdfb9cde --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/CompletableToCompletionStageTest.java @@ -0,0 +1,154 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.subjects.CompletableSubject; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class CompletableToCompletionStageTest extends RxJavaTest { + + @Test + public void complete() throws Exception { + Object v = Completable.complete() + .toCompletionStage(null) + .toCompletableFuture() + .get(); + + assertNull(v); + } + + @Test + public void completableFutureCancels() throws Exception { + CompletableSubject source = CompletableSubject.create(); + + CompletableFuture cf = source + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void completableManualCompleteCancels() throws Exception { + CompletableSubject source = CompletableSubject.create(); + + CompletableFuture cf = source + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals(1, cf.get()); + } + + @Test + public void completableManualCompleteExceptionallyCancels() throws Exception { + CompletableSubject source = CompletableSubject.create(); + + CompletableFuture cf = source + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void error() throws Exception { + CompletableFuture cf = Completable.error(new TestException()) + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void sourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Object v = new Completable() { + @Override + protected void subscribeActual(CompletableObserver observer) { + observer.onSubscribe(Disposable.empty()); + observer.onComplete(); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .toCompletionStage(null) + .toCompletableFuture() + .get(); + + assertNull(v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void doubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Object v = new Completable() { + @Override + protected void subscribeActual(CompletableObserver observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onComplete(); + } + } + .toCompletionStage(null) + .toCompletableFuture() + .get(); + + assertNull(v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStageTest.java new file mode 100644 index 0000000000..90ad2bc626 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromCompletionStageTest.java @@ -0,0 +1,71 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class MaybeFromCompletionStageTest extends RxJavaTest { + + @Test + public void syncSuccess() { + Maybe.fromCompletionStage(CompletableFuture.completedFuture(1)) + .test() + .assertResult(1); + } + + @Test + public void syncFailure() { + CompletableFuture cf = new CompletableFuture<>(); + cf.completeExceptionally(new TestException()); + + Maybe.fromCompletionStage(cf) + .test() + .assertFailure(TestException.class); + } + + @Test + public void syncNull() { + Maybe.fromCompletionStage(CompletableFuture.completedFuture(null)) + .test() + .assertResult(); + } + + @Test + public void dispose() { + CompletableFuture cf = new CompletableFuture<>(); + + TestObserver to = Maybe.fromCompletionStage(cf) + .test(); + + to.assertEmpty(); + + to.dispose(); + + cf.complete(1); + + to.assertEmpty(); + } + + @Test + public void dispose2() { + TestHelper.checkDisposed(Maybe.fromCompletionStage(new CompletableFuture<>())); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromOptionalTest.java new file mode 100644 index 0000000000..4e09fea0b1 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFromOptionalTest.java @@ -0,0 +1,38 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Optional; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; + +public class MaybeFromOptionalTest extends RxJavaTest { + + @Test + public void hasValue() { + Maybe.fromOptional(Optional.of(1)) + .test() + .assertResult(1); + } + + @Test + public void empty() { + Maybe.fromOptional(Optional.empty()) + .test() + .assertResult(); + } + +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptionalTest.java new file mode 100644 index 0000000000..e8b9f2aac3 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeMapOptionalTest.java @@ -0,0 +1,89 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +import java.util.Optional; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class MaybeMapOptionalTest extends RxJavaTest { + + @Test + public void successSuccess() { + Maybe.just(1) + .mapOptional(Optional::of) + .test() + .assertResult(1); + } + + @Test + public void successEmpty() { + Maybe.just(1) + .mapOptional(v -> Optional.empty()) + .test() + .assertResult(); + } + + @Test + public void empty() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Maybe.empty() + .mapOptional(f) + .test() + .assertResult(); + + verify(f, never()).apply(any()); + } + + @Test + public void error() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Maybe.error(new TestException()) + .mapOptional(f) + .test() + .assertFailure(TestException.class); + + verify(f, never()).apply(any()); + } + + @Test + public void mapperCrash() { + Maybe.just(1) + .mapOptional(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(Maybe.never().mapOptional(Optional::of)); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeMaybe(m -> m.mapOptional(Optional::of)); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeToCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeToCompletionStageTest.java new file mode 100644 index 0000000000..c6c83e68a3 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeToCompletionStageTest.java @@ -0,0 +1,174 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.subjects.MaybeSubject; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class MaybeToCompletionStageTest extends RxJavaTest { + + @Test + public void just() throws Exception { + Integer v = Maybe.just(1) + .toCompletionStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void empty() throws Exception { + Integer v = Maybe.empty() + .toCompletionStage(2) + .toCompletableFuture() + .get(); + + assertEquals((Integer)2, v); + } + + @Test + public void emptyError() throws Exception { + CompletableFuture cf = Maybe.empty() + .toCompletionStage() + .toCompletableFuture(); + + TestHelper.assertError(cf, NoSuchElementException.class); + } + + @Test + public void completableFutureCancels() throws Exception { + MaybeSubject source = MaybeSubject.create(); + + CompletableFuture cf = source + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void completableManualCompleteCancels() throws Exception { + MaybeSubject source = MaybeSubject.create(); + + CompletableFuture cf = source + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void completableManualCompleteExceptionallyCancels() throws Exception { + MaybeSubject source = MaybeSubject.create(); + + CompletableFuture cf = source + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void error() throws Exception { + CompletableFuture cf = Maybe.error(new TestException()) + .toCompletionStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void sourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Maybe() { + @Override + protected void subscribeActual(MaybeObserver observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSuccess(1); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .toCompletionStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void doubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Maybe() { + @Override + protected void subscribeActual(MaybeObserver observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onSuccess(1); + } + } + .toCompletionStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStageTest.java new file mode 100644 index 0000000000..7adb3664be --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFromCompletionStageTest.java @@ -0,0 +1,71 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class SingleFromCompletionStageTest extends RxJavaTest { + + @Test + public void syncSuccess() { + Single.fromCompletionStage(CompletableFuture.completedFuture(1)) + .test() + .assertResult(1); + } + + @Test + public void syncFailure() { + CompletableFuture cf = new CompletableFuture<>(); + cf.completeExceptionally(new TestException()); + + Single.fromCompletionStage(cf) + .test() + .assertFailure(TestException.class); + } + + @Test + public void syncNull() { + Single.fromCompletionStage(CompletableFuture.completedFuture(null)) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void dispose() { + CompletableFuture cf = new CompletableFuture<>(); + + TestObserver to = Single.fromCompletionStage(cf) + .test(); + + to.assertEmpty(); + + to.dispose(); + + cf.complete(1); + + to.assertEmpty(); + } + + @Test + public void dispose2() { + TestHelper.checkDisposed(Single.fromCompletionStage(new CompletableFuture<>())); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptionalTest.java new file mode 100644 index 0000000000..b8ab17a17c --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleMapOptionalTest.java @@ -0,0 +1,76 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +import java.util.Optional; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class SingleMapOptionalTest extends RxJavaTest { + + @Test + public void successSuccess() { + Single.just(1) + .mapOptional(Optional::of) + .test() + .assertResult(1); + } + + @Test + public void successEmpty() { + Single.just(1) + .mapOptional(v -> Optional.empty()) + .test() + .assertResult(); + } + + @Test + public void error() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Single.error(new TestException()) + .mapOptional(f) + .test() + .assertFailure(TestException.class); + + verify(f, never()).apply(any()); + } + + @Test + public void mapperCrash() { + Single.just(1) + .mapOptional(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(Single.never().mapOptional(Optional::of)); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeSingleToMaybe(m -> m.mapOptional(Optional::of)); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleToCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleToCompletionStageTest.java new file mode 100644 index 0000000000..2560c68ed8 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleToCompletionStageTest.java @@ -0,0 +1,153 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.subjects.SingleSubject; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class SingleToCompletionStageTest extends RxJavaTest { + + @Test + public void just() throws Exception { + Integer v = Single.just(1) + .toCompletionStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void completableFutureCancels() throws Exception { + SingleSubject source = SingleSubject.create(); + + CompletableFuture cf = source + .toCompletionStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void completableManualCompleteCancels() throws Exception { + SingleSubject source = SingleSubject.create(); + + CompletableFuture cf = source + .toCompletionStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void completableManualCompleteExceptionallyCancels() throws Exception { + SingleSubject source = SingleSubject.create(); + + CompletableFuture cf = source + .toCompletionStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void error() throws Exception { + CompletableFuture cf = Single.error(new TestException()) + .toCompletionStage() + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void sourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Single() { + @Override + protected void subscribeActual(SingleObserver observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSuccess(1); + observer.onError(new TestException()); + } + } + .toCompletionStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void doubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Single() { + @Override + protected void subscribeActual(SingleObserver observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onSuccess(1); + } + } + .toCompletionStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java index 99f6783b4a..b796904cdb 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java +++ b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java @@ -509,6 +509,9 @@ public void checkParallelFlowable() { addOverride(new ParamOverride(Flowable.class, 0, ParamMode.ANY, "singleStage", Object.class)); addOverride(new ParamOverride(Flowable.class, 0, ParamMode.ANY, "lastStage", Object.class)); + addOverride(new ParamOverride(Maybe.class, 0, ParamMode.ANY, "toCompletionStage", Object.class)); + addOverride(new ParamOverride(Completable.class, 0, ParamMode.ANY, "toCompletionStage", Object.class)); + // ----------------------------------------------------------------------------------- ignores = new HashMap>(); From dba670554c69d6c09d28643505880c1a468af3db Mon Sep 17 00:00:00 2001 From: David Karnok Date: Sat, 21 Dec 2019 11:08:34 +0100 Subject: [PATCH 015/665] 3.x: Cleanup and prettify Javadocs, widen XOperator throws (#6785) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * 3.x: Cleanup and prettify Javadocs, widen XOperator throws * Fix encoding of § in javadoc * Fix a wrong tag name --- .../rxjava3/annotations/BackpressureKind.java | 6 +- .../core/BackpressureOverflowStrategy.java | 5 +- .../rxjava3/core/BackpressureStrategy.java | 11 +- .../rxjava3/core/CompletableEmitter.java | 6 +- .../rxjava3/core/CompletableObserver.java | 6 +- .../rxjava3/core/CompletableOnSubscribe.java | 4 +- .../rxjava3/core/CompletableOperator.java | 10 +- .../rxjava3/core/CompletableSource.java | 8 +- .../rxjava3/core/CompletableTransformer.java | 10 +- .../io/reactivex/rxjava3/core/Emitter.java | 6 +- .../io/reactivex/rxjava3/core/Flowable.java | 72 ++++---- .../rxjava3/core/FlowableConverter.java | 6 +- .../rxjava3/core/FlowableEmitter.java | 20 +- .../rxjava3/core/FlowableOnSubscribe.java | 4 +- .../rxjava3/core/FlowableOperator.java | 12 +- .../rxjava3/core/FlowableSubscriber.java | 2 +- .../rxjava3/core/FlowableTransformer.java | 8 +- .../rxjava3/core/MaybeConverter.java | 6 +- .../reactivex/rxjava3/core/MaybeEmitter.java | 22 ++- .../reactivex/rxjava3/core/MaybeObserver.java | 14 +- .../rxjava3/core/MaybeOnSubscribe.java | 4 +- .../reactivex/rxjava3/core/MaybeOperator.java | 12 +- .../reactivex/rxjava3/core/MaybeSource.java | 6 +- .../rxjava3/core/MaybeTransformer.java | 8 +- .../reactivex/rxjava3/core/Notification.java | 42 ++--- .../io/reactivex/rxjava3/core/Observable.java | 12 +- .../rxjava3/core/ObservableConverter.java | 6 +- .../rxjava3/core/ObservableEmitter.java | 24 +-- .../rxjava3/core/ObservableOnSubscribe.java | 4 +- .../rxjava3/core/ObservableOperator.java | 12 +- .../rxjava3/core/ObservableSource.java | 6 +- .../rxjava3/core/ObservableTransformer.java | 8 +- .../io/reactivex/rxjava3/core/Observer.java | 16 +- .../io/reactivex/rxjava3/core/Single.java | 2 +- .../rxjava3/core/SingleConverter.java | 6 +- .../reactivex/rxjava3/core/SingleEmitter.java | 18 +- .../rxjava3/core/SingleObserver.java | 14 +- .../rxjava3/core/SingleOnSubscribe.java | 4 +- .../rxjava3/core/SingleOperator.java | 12 +- .../reactivex/rxjava3/core/SingleSource.java | 6 +- .../rxjava3/core/SingleTransformer.java | 8 +- .../disposables/CompositeDisposable.java | 10 +- .../exceptions/UndeliverableException.java | 2 +- .../internal/util/BackpressureHelper.java | 35 ++-- .../internal/util/QueueDrainHelper.java | 4 +- .../rxjava3/observers/BaseTestConsumer.java | 171 +++++++++++------- .../DisposableCompletableObserver.java | 4 +- .../observers/DisposableMaybeObserver.java | 6 +- .../rxjava3/observers/DisposableObserver.java | 4 +- .../observers/DisposableSingleObserver.java | 6 +- .../LambdaConsumerIntrospection.java | 4 +- .../ResourceCompletableObserver.java | 18 +- .../observers/ResourceMaybeObserver.java | 18 +- .../rxjava3/observers/ResourceObserver.java | 18 +- .../observers/ResourceSingleObserver.java | 18 +- .../rxjava3/observers/SafeObserver.java | 6 +- .../rxjava3/observers/SerializedObserver.java | 23 +-- .../rxjava3/observers/TestObserver.java | 56 +++--- .../rxjava3/processors/AsyncProcessor.java | 2 +- .../rxjava3/processors/BehaviorProcessor.java | 4 +- .../rxjava3/processors/PublishProcessor.java | 2 +- .../rxjava3/processors/ReplayProcessor.java | 4 +- .../rxjava3/processors/UnicastProcessor.java | 4 +- .../reactivex/rxjava3/schedulers/Timed.java | 6 +- .../rxjava3/subjects/BehaviorSubject.java | 2 +- .../subscribers/DefaultSubscriber.java | 6 +- .../subscribers/DisposableSubscriber.java | 20 +- .../subscribers/ResourceSubscriber.java | 24 +-- .../rxjava3/subscribers/SafeSubscriber.java | 15 +- .../subscribers/SerializedSubscriber.java | 30 +-- .../rxjava3/subscribers/TestSubscriber.java | 77 ++++---- .../flowable/FlowableSubscriberTest.java | 6 +- .../io/reactivex/rxjava3/tck/BaseTck.java | 4 +- .../rxjava3/testsupport/TestHelper.java | 2 +- .../rxjava3/testsupport/TestSubscriberEx.java | 2 +- 75 files changed, 570 insertions(+), 511 deletions(-) diff --git a/src/main/java/io/reactivex/rxjava3/annotations/BackpressureKind.java b/src/main/java/io/reactivex/rxjava3/annotations/BackpressureKind.java index aa20e5f78e..fd53c196b3 100644 --- a/src/main/java/io/reactivex/rxjava3/annotations/BackpressureKind.java +++ b/src/main/java/io/reactivex/rxjava3/annotations/BackpressureKind.java @@ -32,13 +32,13 @@ public enum BackpressureKind { */ SPECIAL, /** - * The operator requests Long.MAX_VALUE from upstream but respects the backpressure + * The operator requests {@link Long#MAX_VALUE} from upstream but respects the backpressure * of the downstream. */ UNBOUNDED_IN, /** - * The operator will emit a MissingBackpressureException if the downstream didn't request - * enough or in time. + * The operator will emit a {@link io.reactivex.rxjava3.exceptions.MissingBackpressureException MissingBackpressureException} + * if the downstream didn't request enough or in time. */ ERROR, /** diff --git a/src/main/java/io/reactivex/rxjava3/core/BackpressureOverflowStrategy.java b/src/main/java/io/reactivex/rxjava3/core/BackpressureOverflowStrategy.java index 5ee8ecca1d..750a1308f2 100644 --- a/src/main/java/io/reactivex/rxjava3/core/BackpressureOverflowStrategy.java +++ b/src/main/java/io/reactivex/rxjava3/core/BackpressureOverflowStrategy.java @@ -19,7 +19,10 @@ * Options to deal with buffer overflow when using onBackpressureBuffer. */ public enum BackpressureOverflowStrategy { - /** Signal a MissingBackpressureException and terminate the sequence. */ + /** + * Signal a {@link io.reactivex.rxjava3.exceptions.MissingBackpressureException MissingBackpressureException} + * and terminate the sequence. + */ ERROR, /** Drop the oldest value from the buffer. */ DROP_OLDEST, diff --git a/src/main/java/io/reactivex/rxjava3/core/BackpressureStrategy.java b/src/main/java/io/reactivex/rxjava3/core/BackpressureStrategy.java index c0831d7759..4a26072650 100644 --- a/src/main/java/io/reactivex/rxjava3/core/BackpressureStrategy.java +++ b/src/main/java/io/reactivex/rxjava3/core/BackpressureStrategy.java @@ -18,25 +18,26 @@ */ public enum BackpressureStrategy { /** - * OnNext events are written without any buffering or dropping. + * The {@code onNext} events are written without any buffering or dropping. * Downstream has to deal with any overflow. *

Useful when one applies one of the custom-parameter onBackpressureXXX operators. */ MISSING, /** - * Signals a MissingBackpressureException in case the downstream can't keep up. + * Signals a {@link io.reactivex.rxjava3.exceptions.MissingBackpressureException MissingBackpressureException} + * in case the downstream can't keep up. */ ERROR, /** - * Buffers all onNext values until the downstream consumes it. + * Buffers all {@code onNext} values until the downstream consumes it. */ BUFFER, /** - * Drops the most recent onNext value if the downstream can't keep up. + * Drops the most recent {@code onNext} value if the downstream can't keep up. */ DROP, /** - * Keeps only the latest onNext value, overwriting any previous value if the + * Keeps only the latest {@code onNext} value, overwriting any previous value if the * downstream can't keep up. */ LATEST diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableEmitter.java b/src/main/java/io/reactivex/rxjava3/core/CompletableEmitter.java index 51c9780f73..5ebb4e1877 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableEmitter.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableEmitter.java @@ -82,12 +82,12 @@ public interface CompletableEmitter { boolean isDisposed(); /** - * Attempts to emit the specified {@code Throwable} error if the downstream + * Attempts to emit the specified {@link Throwable} error if the downstream * hasn't cancelled the sequence or is otherwise terminated, returning false * if the emission is not allowed to happen due to lifecycle restrictions. *

- * Unlike {@link #onError(Throwable)}, the {@code RxJavaPlugins.onError} is not called - * if the error could not be delivered. + * Unlike {@link #onError(Throwable)}, the {@link io.reactivex.rxjava3.plugins.RxJavaPlugins#onError(Throwable) RxjavaPlugins.onError} + * is not called if the error could not be delivered. *

History: 2.1.1 - experimental * @param t the throwable error to signal if possible * @return true if successful, false if the downstream is not able to accept further diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableObserver.java b/src/main/java/io/reactivex/rxjava3/core/CompletableObserver.java index af0a017d66..7c01ebfee1 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableObserver.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableObserver.java @@ -49,9 +49,9 @@ */ public interface CompletableObserver { /** - * Called once by the Completable to set a Disposable on this instance which + * Called once by the {@link Completable} to set a {@link Disposable} on this instance which * then can be used to cancel the subscription at any time. - * @param d the Disposable instance to call dispose on for cancellation, not null + * @param d the {@code Disposable} instance to call dispose on for cancellation, not null */ void onSubscribe(@NonNull Disposable d); @@ -62,7 +62,7 @@ public interface CompletableObserver { /** * Called once if the deferred computation 'throws' an exception. - * @param e the exception, not null. + * @param e the exception, not {@code null}. */ void onError(@NonNull Throwable e); } diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java index 6aea056be7..70d79e62b6 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableOnSubscribe.java @@ -23,8 +23,8 @@ public interface CompletableOnSubscribe { /** - * Called for each CompletableObserver that subscribes. - * @param emitter the safe emitter instance, never null + * Called for each {@link CompletableObserver} that subscribes. + * @param emitter the safe emitter instance, never {@code null} * @throws Throwable on error */ void subscribe(@NonNull CompletableEmitter emitter) throws Throwable; diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java b/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java index 763cd670c6..e9b1df83cb 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableOperator.java @@ -21,11 +21,11 @@ @FunctionalInterface public interface CompletableOperator { /** - * Applies a function to the child CompletableObserver and returns a new parent CompletableObserver. - * @param observer the child CompletableObservable instance - * @return the parent CompletableObserver instance - * @throws Exception on failure + * Applies a function to the child {@link CompletableObserver} and returns a new parent {@code CompletableObserver}. + * @param observer the child {@code CompletableObserver} instance + * @return the parent {@code CompletableObserver} instance + * @throws Throwable on failure */ @NonNull - CompletableObserver apply(@NonNull CompletableObserver observer) throws Exception; + CompletableObserver apply(@NonNull CompletableObserver observer) throws Throwable; } diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java b/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java index 9b754a698a..58edf9471c 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableSource.java @@ -24,9 +24,9 @@ public interface CompletableSource { /** - * Subscribes the given CompletableObserver to this CompletableSource instance. - * @param co the CompletableObserver, not null - * @throws NullPointerException if {@code co} is null + * Subscribes the given {@link CompletableObserver} to this {@code CompletableSource} instance. + * @param observer the {@code CompletableObserver}, not {@code null} + * @throws NullPointerException if {@code observer} is {@code null} */ - void subscribe(@NonNull CompletableObserver co); + void subscribe(@NonNull CompletableObserver observer); } diff --git a/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java b/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java index 91e1a361ba..98a9e2aa19 100644 --- a/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/CompletableTransformer.java @@ -16,15 +16,15 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Convenience interface and callback used by the compose operator to turn a Completable into another - * Completable fluently. + * Convenience interface and callback used by the compose operator to turn a {@link Completable} into another + * {@code Completable} fluently. */ @FunctionalInterface public interface CompletableTransformer { /** - * Applies a function to the upstream Completable and returns a CompletableSource. - * @param upstream the upstream Completable instance - * @return the transformed CompletableSource instance + * Applies a function to the upstream {@link Completable} and returns a {@link CompletableSource}. + * @param upstream the upstream {@code Completable} instance + * @return the transformed {@code CompletableSource} instance */ @NonNull CompletableSource apply(@NonNull Completable upstream); diff --git a/src/main/java/io/reactivex/rxjava3/core/Emitter.java b/src/main/java/io/reactivex/rxjava3/core/Emitter.java index e222f9c44f..0fc2b47c0d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Emitter.java +++ b/src/main/java/io/reactivex/rxjava3/core/Emitter.java @@ -29,13 +29,13 @@ public interface Emitter { /** * Signal a normal value. - * @param value the value to signal, not null + * @param value the value to signal, not {@code null} */ void onNext(@NonNull T value); /** - * Signal a Throwable exception. - * @param error the Throwable to signal, not null + * Signal a {@link Throwable} exception. + * @param error the {@code Throwable} to signal, not {@code null} */ void onError(@NonNull Throwable error); diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 09cb859b83..0b2e9197cb 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -1434,7 +1434,7 @@ public static Flowable concatArrayEager(Publisher... sources * * @param the value type * @param sources an array of Publishers that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrent subscriptions at a time, Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrent subscriptions at a time, {@link Integer#MAX_VALUE} * is interpreted as an indication to subscribe to all sources at once * @param prefetch the number of elements to prefetch from each Publisher source * @return the new Publisher instance with the specified concatenation behavior @@ -1503,7 +1503,7 @@ public static Flowable concatArrayEagerDelayError(Publisher. * * @param the value type * @param sources an array of {@code Publisher}s that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrent subscriptions at a time, Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrent subscriptions at a time, {@link Integer#MAX_VALUE} * is interpreted as indication to subscribe to all sources at once * @param prefetch the number of elements to prefetch from each {@code Publisher} source * @return the new Flowable instance with the specified concatenation behavior @@ -1635,7 +1635,7 @@ public static Flowable concatEager(Publisher * @param the value type * @param sources a sequence of Publishers that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrently running inner Publishers; Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrently running inner Publishers; {@link Integer#MAX_VALUE} * is interpreted as all inner Publishers can be active at the same time * @param prefetch the number of elements to prefetch from each inner Publisher source * @return the new Publisher instance with the specified concatenation behavior @@ -1695,7 +1695,7 @@ public static Flowable concatEager(Iterable * @param the value type * @param sources a sequence of Publishers that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrently running inner Publishers; Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrently running inner Publishers; {@link Integer#MAX_VALUE} * is interpreted as all inner Publishers can be active at the same time * @param prefetch the number of elements to prefetch from each inner Publisher source * @return the new Publisher instance with the specified concatenation behavior @@ -3976,7 +3976,7 @@ public static Flowable never() { * @return a Flowable that emits a range of sequential Integers * @throws IllegalArgumentException * if {@code count} is less than zero, or if {@code start} + {@code count} − 1 exceeds - * {@code Integer.MAX_VALUE} + * {@link Integer#MAX_VALUE} * @see ReactiveX operators documentation: Range */ @CheckReturnValue @@ -4016,7 +4016,7 @@ public static Flowable range(int start, int count) { * @return a Flowable that emits a range of sequential Longs * @throws IllegalArgumentException * if {@code count} is less than zero, or if {@code start} + {@code count} − 1 exceeds - * {@code Long.MAX_VALUE} + * {@link Long#MAX_VALUE} * @see ReactiveX operators documentation: Range */ @CheckReturnValue @@ -6258,7 +6258,7 @@ public final > Flowable buffer(int count, Sup * *

*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
This version of {@code buffer} operates by default on the {@code computation} {@link Scheduler}.
@@ -6293,7 +6293,7 @@ public final Flowable> buffer(long timespan, long timeskip, TimeUnit uni * *
*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -6330,7 +6330,7 @@ public final Flowable> buffer(long timespan, long timeskip, TimeUnit uni * *
*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -6375,7 +6375,7 @@ public final > Flowable buffer(long timespan, * *
*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
This version of {@code buffer} operates by default on the {@code computation} {@link Scheduler}.
@@ -6408,7 +6408,7 @@ public final Flowable> buffer(long timespan, TimeUnit unit) { * *
*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
This version of {@code buffer} operates by default on the {@code computation} {@link Scheduler}.
@@ -6445,7 +6445,7 @@ public final Flowable> buffer(long timespan, TimeUnit unit, int count) { * *
*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -6484,7 +6484,7 @@ public final Flowable> buffer(long timespan, TimeUnit unit, Scheduler sc * *
*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -6536,7 +6536,7 @@ public final > Flowable buffer( * *
*
Backpressure:
- *
This operator does not support backpressure as it uses time. It requests {@code Long.MAX_VALUE} + *
This operator does not support backpressure as it uses time. It requests {@link Long#MAX_VALUE} * upstream and does not obey downstream requests.
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -6571,7 +6571,7 @@ public final Flowable> buffer(long timespan, TimeUnit unit, Scheduler sc *
*
Backpressure:
*
This operator does not support backpressure as it is instead controlled by the given Publishers and - * buffers data. It requests {@code Long.MAX_VALUE} upstream and does not obey downstream requests.
+ * buffers data. It requests {@link Long#MAX_VALUE} upstream and does not obey downstream requests. *
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
*
@@ -6607,7 +6607,7 @@ public final Flowable> buffer( *
*
Backpressure:
*
This operator does not support backpressure as it is instead controlled by the given Publishers and - * buffers data. It requests {@code Long.MAX_VALUE} upstream and does not obey downstream requests.
+ * buffers data. It requests {@link Long#MAX_VALUE} upstream and does not obey downstream requests. *
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
*
@@ -6652,7 +6652,7 @@ public final > Flowable b *
*
Backpressure:
*
This operator does not support backpressure as it is instead controlled by the {@code Publisher} - * {@code boundary} and buffers data. It requests {@code Long.MAX_VALUE} upstream and does not obey + * {@code boundary} and buffers data. It requests {@link Long#MAX_VALUE} upstream and does not obey * downstream requests.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
@@ -6686,7 +6686,7 @@ public final Flowable> buffer(Publisher boundaryIndicator) { *
*
Backpressure:
*
This operator does not support backpressure as it is instead controlled by the {@code Publisher} - * {@code boundary} and buffers data. It requests {@code Long.MAX_VALUE} upstream and does not obey + * {@code boundary} and buffers data. It requests {@link Long#MAX_VALUE} upstream and does not obey * downstream requests.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
@@ -6723,7 +6723,7 @@ public final Flowable> buffer(Publisher boundaryIndicator, final *
*
Backpressure:
*
This operator does not support backpressure as it is instead controlled by the {@code Publisher} - * {@code boundary} and buffers data. It requests {@code Long.MAX_VALUE} upstream and does not obey + * {@code boundary} and buffers data. It requests {@link Long#MAX_VALUE} upstream and does not obey * downstream requests.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
@@ -8503,7 +8503,7 @@ public final Flowable delay(Publisher subscriptionIndicator, *
*
Backpressure:
*
The operator forwards the backpressure requests to this Publisher once - * the subscription happens and requests Long.MAX_VALUE from the other Publisher
+ * the subscription happens and requests {@link Long#MAX_VALUE} from the other Publisher *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
@@ -10004,9 +10004,9 @@ public final Completable flatMapCompletable(Function *
Backpressure:
- *
If {@code maxConcurrency == Integer.MAX_VALUE} the operator consumes the upstream in an unbounded manner. + *
If {@code maxConcurrency == }{@link Integer#MAX_VALUE} the operator consumes the upstream in an unbounded manner. * Otherwise, the operator expects the upstream to honor backpressure. If the upstream doesn't support backpressure - * the operator behaves as if {@code maxConcurrency == Integer.MAX_VALUE} was used.
+ * the operator behaves as if {@code maxConcurrency == }{@link Integer#MAX_VALUE} was used. *
Scheduler:
*
{@code flatMapCompletable} does not operate by default on a particular {@link Scheduler}.
*
@@ -10198,9 +10198,9 @@ public final Flowable flatMapMaybe(Function *
Backpressure:
- *
If {@code maxConcurrency == Integer.MAX_VALUE} the operator consumes the upstream in an unbounded manner. + *
If {@code maxConcurrency == }{@link Integer#MAX_VALUE} the operator consumes the upstream in an unbounded manner. * Otherwise, the operator expects the upstream to honor backpressure. If the upstream doesn't support backpressure - * the operator behaves as if {@code maxConcurrency == Integer.MAX_VALUE} was used.
+ * the operator behaves as if {@code maxConcurrency == }{@link Integer#MAX_VALUE} was used. *
Scheduler:
*
{@code flatMapMaybe} does not operate by default on a particular {@link Scheduler}.
*
@@ -10247,9 +10247,9 @@ public final Flowable flatMapSingle(Function *
Backpressure:
- *
If {@code maxConcurrency == Integer.MAX_VALUE} the operator consumes the upstream in an unbounded manner. + *
If {@code maxConcurrency == }{@link Integer#MAX_VALUE} the operator consumes the upstream in an unbounded manner. * Otherwise, the operator expects the upstream to honor backpressure. If the upstream doesn't support backpressure - * the operator behaves as if {@code maxConcurrency == Integer.MAX_VALUE} was used.
+ * the operator behaves as if {@code maxConcurrency == }{@link Integer#MAX_VALUE} was used. *
Scheduler:
*
{@code flatMapSingle} does not operate by default on a particular {@link Scheduler}.
*
@@ -10415,7 +10415,7 @@ public final Disposable forEachWhile(final Predicate onNext, final Co * coordination of the {@code groupBy} operator. Such hangs can be usually avoided by using * {@link #flatMap(Function, int)} or {@link #concatMapEager(Function, int, int)} and overriding the default maximum concurrency * value to be greater or equal to the expected number of groups, possibly using - * {@code Integer.MAX_VALUE} if the number of expected groups is unknown. + * {@link Integer#MAX_VALUE} if the number of expected groups is unknown. *

* Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be @@ -10473,7 +10473,7 @@ public final Flowable> groupBy(Function * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be @@ -10532,7 +10532,7 @@ public final Flowable> groupBy(Function * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be @@ -10596,7 +10596,7 @@ public final Flowable> groupBy(Function * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be @@ -10661,7 +10661,7 @@ public final Flowable> groupBy(Function * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be @@ -10774,7 +10774,7 @@ public final Flowable> groupBy(Function * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be @@ -12337,7 +12337,7 @@ public final ConnectableFlowable publish(int bufferSize) { * Requests {@code n} initially from the upstream and then 75% of {@code n} subsequently * after 75% of {@code n} values have been emitted to the downstream. * - *

This operator allows preventing the downstream to trigger unbounded mode via {@code request(Long.MAX_VALUE)} + *

This operator allows preventing the downstream to trigger unbounded mode via {@code request(}{@link Long#MAX_VALUE}{@code )} * or compensate for the per-item overhead of small and frequent requests. * *

@@ -17224,7 +17224,7 @@ public final Single>> toMultimap( *
*
Backpressure:
*
Observables don't support backpressure thus the current Flowable is consumed in an unbounded - * manner (by requesting Long.MAX_VALUE).
+ * manner (by requesting {@link Long#MAX_VALUE}). *
Scheduler:
*
{@code toObservable} does not operate by default on a particular {@link Scheduler}.
*
@@ -18552,8 +18552,8 @@ public final Flowable zipWith(Publisher other, // Fluent test support, super handy and reduces test preparation boilerplate // ------------------------------------------------------------------------- /** - * Creates a TestSubscriber that requests Long.MAX_VALUE and subscribes - * it to this Flowable. + * Creates a {@link TestSubscriber} that requests {@link Long#MAX_VALUE} and subscribes + * it to this {@code Flowable}. *
*
Backpressure:
*
The returned TestSubscriber consumes this Flowable in an unbounded fashion.
diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java b/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java index abc89eec4a..92733c3c1e 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableConverter.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Convenience interface and callback used by the {@link Flowable#to} operator to turn a Flowable into another + * Convenience interface and callback used by the {@link Flowable#to} operator to turn a {@link Flowable} into another * value fluently. *

History: 2.1.7 - experimental * @param the upstream type @@ -26,9 +26,9 @@ @FunctionalInterface public interface FlowableConverter { /** - * Applies a function to the upstream Flowable and returns a converted value of type {@code R}. + * Applies a function to the upstream {@link Flowable} and returns a converted value of type {@code R}. * - * @param upstream the upstream Flowable instance + * @param upstream the upstream {@code Flowable} instance * @return the converted value */ @NonNull diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableEmitter.java b/src/main/java/io/reactivex/rxjava3/core/FlowableEmitter.java index 8f66b7bbee..1221721f29 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableEmitter.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableEmitter.java @@ -53,14 +53,16 @@ public interface FlowableEmitter extends Emitter { /** * Sets a Disposable on this emitter; any previous {@link Disposable} * or {@link Cancellable} will be disposed/cancelled. - * @param d the disposable, null is allowed + *

This method is thread-safe. + * @param d the disposable, {@code null} is allowed */ void setDisposable(@Nullable Disposable d); /** - * Sets a Cancellable on this emitter; any previous {@link Disposable} - * or {@link Cancellable} will be disposed/cancelled. - * @param c the cancellable resource, null is allowed + * Sets a {@link Cancellable} on this emitter; any previous {@link Disposable} + * or {@code Cancellable} will be disposed/cancelled. + *

This method is thread-safe. + * @param c the {@code Cancellable} resource, {@code null} is allowed */ void setCancellable(@Nullable Cancellable c); @@ -81,19 +83,19 @@ public interface FlowableEmitter extends Emitter { boolean isCancelled(); /** - * Ensures that calls to onNext, onError and onComplete are properly serialized. - * @return the serialized FlowableEmitter + * Ensures that calls to {@code onNext}, {@code onError} and {@code onComplete} are properly serialized. + * @return the serialized {@link FlowableEmitter} */ @NonNull FlowableEmitter serialize(); /** - * Attempts to emit the specified {@code Throwable} error if the downstream + * Attempts to emit the specified {@link Throwable} error if the downstream * hasn't cancelled the sequence or is otherwise terminated, returning false * if the emission is not allowed to happen due to lifecycle restrictions. *

- * Unlike {@link #onError(Throwable)}, the {@code RxJavaPlugins.onError} is not called - * if the error could not be delivered. + * Unlike {@link #onError(Throwable)}, the {@link io.reactivex.rxjava3.plugins.RxJavaPlugins#onError(Throwable) RxjavaPlugins.onError} + * is not called if the error could not be delivered. *

History: 2.1.1 - experimental * @param t the throwable error to signal if possible * @return true if successful, false if the downstream is not able to accept further diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java index 2bbc8b57a3..156b9f2dc7 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableOnSubscribe.java @@ -25,8 +25,8 @@ public interface FlowableOnSubscribe { /** - * Called for each Subscriber that subscribes. - * @param emitter the safe emitter instance, never null + * Called for each {@link org.reactivestreams.Subscriber Subscriber} that subscribes. + * @param emitter the safe emitter instance, never {@code null} * @throws Throwable on error */ void subscribe(@NonNull FlowableEmitter emitter) throws Throwable; diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java b/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java index 6d241f7498..5b5855ea2f 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableOperator.java @@ -18,7 +18,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to map/wrap a downstream subscriber to an upstream subscriber. + * Interface to map/wrap a downstream {@link Subscriber} to an upstream {@code Subscriber}. * * @param the value type of the downstream * @param the value type of the upstream @@ -26,11 +26,11 @@ @FunctionalInterface public interface FlowableOperator { /** - * Applies a function to the child Subscriber and returns a new parent Subscriber. - * @param subscriber the child Subscriber instance - * @return the parent Subscriber instance - * @throws Exception on failure + * Applies a function to the child {@link Subscriber} and returns a new parent {@code Subscriber}. + * @param subscriber the child {@code Subscriber} instance + * @return the parent {@code Subscriber} instance + * @throws Throwable on failure */ @NonNull - Subscriber apply(@NonNull Subscriber subscriber) throws Exception; + Subscriber apply(@NonNull Subscriber subscriber) throws Throwable; } diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableSubscriber.java b/src/main/java/io/reactivex/rxjava3/core/FlowableSubscriber.java index 3b0a80cf9a..23449fbedc 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableSubscriber.java @@ -18,7 +18,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Represents a Reactive-Streams inspired Subscriber that is RxJava 2 only + * Represents a Reactive-Streams inspired {@link Subscriber} that is RxJava 2 only * and weakens rules §1.3 and §3.9 of the specification for gaining performance. * *

History: 2.0.7 - experimental; 2.1 - beta diff --git a/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java b/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java index 877ad1916a..3b98a97dad 100644 --- a/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/FlowableTransformer.java @@ -18,7 +18,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to compose Flowables. + * Interface to compose {@link Flowable}s. * * @param the upstream value type * @param the downstream value type @@ -26,10 +26,10 @@ @FunctionalInterface public interface FlowableTransformer { /** - * Applies a function to the upstream Flowable and returns a Publisher with + * Applies a function to the upstream {@link Flowable} and returns a {@link Publisher} with * optionally different element type. - * @param upstream the upstream Flowable instance - * @return the transformed Publisher instance + * @param upstream the upstream {@code Flowable} instance + * @return the transformed {@code Publisher} instance */ @NonNull Publisher apply(@NonNull Flowable upstream); diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java b/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java index a8e6790e5f..eefe8c0973 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeConverter.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Convenience interface and callback used by the {@link Maybe#to} operator to turn a Maybe into another + * Convenience interface and callback used by the {@link Maybe#to} operator to turn a {@link Maybe} into another * value fluently. *

History: 2.1.7 - experimental * @param the upstream type @@ -26,9 +26,9 @@ @FunctionalInterface public interface MaybeConverter { /** - * Applies a function to the upstream Maybe and returns a converted value of type {@code R}. + * Applies a function to the upstream {@link Maybe} and returns a converted value of type {@code R}. * - * @param upstream the upstream Maybe instance + * @param upstream the upstream {@code Maybe} instance * @return the converted value */ @NonNull diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeEmitter.java b/src/main/java/io/reactivex/rxjava3/core/MaybeEmitter.java index dfc8671073..7757e4b72a 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeEmitter.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeEmitter.java @@ -57,7 +57,7 @@ public interface MaybeEmitter { /** * Signal an exception. - * @param t the exception, not null + * @param t the exception, not {@code null} */ void onError(@NonNull Throwable t); @@ -67,16 +67,18 @@ public interface MaybeEmitter { void onComplete(); /** - * Sets a Disposable on this emitter; any previous {@link Disposable} + * Sets a {@link Disposable} on this emitter; any previous {@code Disposable} * or {@link Cancellable} will be disposed/cancelled. - * @param d the disposable, null is allowed + *

This method is thread-safe. + * @param d the disposable, {@code null} is allowed */ void setDisposable(@Nullable Disposable d); /** - * Sets a Cancellable on this emitter; any previous {@link Disposable} - * or {@link Cancellable} will be disposed/cancelled. - * @param c the cancellable resource, null is allowed + * Sets a {@link Cancellable} on this emitter; any previous {@link Disposable} + * or {@code Cancellable} will be disposed/cancelled. + *

This method is thread-safe. + * @param c the {@code Cancellable} resource, {@code null} is allowed */ void setCancellable(@Nullable Cancellable c); @@ -91,14 +93,14 @@ public interface MaybeEmitter { boolean isDisposed(); /** - * Attempts to emit the specified {@code Throwable} error if the downstream + * Attempts to emit the specified {@link Throwable} error if the downstream * hasn't cancelled the sequence or is otherwise terminated, returning false * if the emission is not allowed to happen due to lifecycle restrictions. *

- * Unlike {@link #onError(Throwable)}, the {@code RxJavaPlugins.onError} is not called - * if the error could not be delivered. + * Unlike {@link #onError(Throwable)}, the {@link io.reactivex.rxjava3.plugins.RxJavaPlugins#onError(Throwable) RxjavaPlugins.onError} + * is not called if the error could not be delivered. *

History: 2.1.1 - experimental - * @param t the throwable error to signal if possible + * @param t the {@code Throwable} error to signal if possible * @return true if successful, false if the downstream is not able to accept further * events * @since 2.2 diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeObserver.java b/src/main/java/io/reactivex/rxjava3/core/MaybeObserver.java index 7b7eb1a0d4..52bee5923a 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeObserver.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeObserver.java @@ -56,32 +56,32 @@ public interface MaybeObserver { /** - * Provides the MaybeObserver with the means of cancelling (disposing) the - * connection (channel) with the Maybe in both + * Provides the {@link MaybeObserver} with the means of cancelling (disposing) the + * connection (channel) with the {@link Maybe} in both * synchronous (from within {@code onSubscribe(Disposable)} itself) and asynchronous manner. - * @param d the Disposable instance whose {@link Disposable#dispose()} can + * @param d the {@link Disposable} instance whose {@link Disposable#dispose()} can * be called anytime to cancel the connection */ void onSubscribe(@NonNull Disposable d); /** - * Notifies the MaybeObserver with one item and that the {@link Maybe} has finished sending + * Notifies the {@link MaybeObserver} with one item and that the {@link Maybe} has finished sending * push-based notifications. *

* The {@link Maybe} will not call this method if it calls {@link #onError}. * * @param t - * the item emitted by the Maybe + * the item emitted by the {@code Maybe} */ void onSuccess(@NonNull T t); /** - * Notifies the MaybeObserver that the {@link Maybe} has experienced an error condition. + * Notifies the {@link MaybeObserver} that the {@link Maybe} has experienced an error condition. *

* If the {@link Maybe} calls this method, it will not thereafter call {@link #onSuccess}. * * @param e - * the exception encountered by the Maybe + * the exception encountered by the {@code Maybe} */ void onError(@NonNull Throwable e); diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java index 2dde118757..350d17af5d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeOnSubscribe.java @@ -25,8 +25,8 @@ public interface MaybeOnSubscribe { /** - * Called for each MaybeObserver that subscribes. - * @param emitter the safe emitter instance, never null + * Called for each {@link MaybeObserver} that subscribes. + * @param emitter the safe emitter instance, never {@code null} * @throws Throwable on error */ void subscribe(@NonNull MaybeEmitter emitter) throws Throwable; diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java b/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java index 48039a9bd0..30da3286b6 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeOperator.java @@ -15,7 +15,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to map/wrap a downstream observer to an upstream observer. + * Interface to map/wrap a downstream {@link MaybeObserver} to an upstream {@code MaybeObserver}. * * @param the value type of the downstream * @param the value type of the upstream @@ -23,11 +23,11 @@ @FunctionalInterface public interface MaybeOperator { /** - * Applies a function to the child MaybeObserver and returns a new parent MaybeObserver. - * @param observer the child MaybeObserver instance - * @return the parent MaybeObserver instance - * @throws Exception on failure + * Applies a function to the child {@link MaybeObserver} and returns a new parent {@code MaybeObserver}. + * @param observer the child {@code MaybeObserver} instance + * @return the parent {@code MaybeObserver} instance + * @throws Throwable on failure */ @NonNull - MaybeObserver apply(@NonNull MaybeObserver observer) throws Exception; + MaybeObserver apply(@NonNull MaybeObserver observer) throws Throwable; } diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java b/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java index 17e2b7f694..f066e5d32e 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeSource.java @@ -28,9 +28,9 @@ public interface MaybeSource { /** - * Subscribes the given MaybeObserver to this MaybeSource instance. - * @param observer the MaybeObserver, not null - * @throws NullPointerException if {@code observer} is null + * Subscribes the given {@link MaybeObserver} to this {@link MaybeSource} instance. + * @param observer the {@code MaybeObserver}, not {@code null} + * @throws NullPointerException if {@code observer} is {@code null} */ void subscribe(@NonNull MaybeObserver observer); } diff --git a/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java b/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java index b5a94bce79..aebcb8353f 100644 --- a/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/MaybeTransformer.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to compose Maybes. + * Interface to compose {@link Maybe}s. * * @param the upstream value type * @param the downstream value type @@ -24,10 +24,10 @@ @FunctionalInterface public interface MaybeTransformer { /** - * Applies a function to the upstream Maybe and returns a MaybeSource with + * Applies a function to the upstream {@link Maybe} and returns a {@link MaybeSource} with * optionally different element type. - * @param upstream the upstream Maybe instance - * @return the transformed MaybeSource instance + * @param upstream the upstream {@code Maybe} instance + * @return the transformed {@code MaybeSource} instance */ @NonNull MaybeSource apply(@NonNull Maybe upstream); diff --git a/src/main/java/io/reactivex/rxjava3/core/Notification.java b/src/main/java/io/reactivex/rxjava3/core/Notification.java index 8f8ae5c4d8..bb1c35c3f1 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Notification.java +++ b/src/main/java/io/reactivex/rxjava3/core/Notification.java @@ -18,8 +18,8 @@ import java.util.Objects; /** - * Represents the reactive signal types: onNext, onError and onComplete and - * holds their parameter values (a value, a Throwable, nothing). + * Represents the reactive signal types: {@code onNext}, {@code onError} and {@code onComplete} and + * holds their parameter values (a value, a {@link Throwable}, nothing). * @param the value type */ public final class Notification { @@ -32,17 +32,17 @@ private Notification(Object value) { } /** - * Returns true if this notification is an onComplete signal. - * @return true if this notification is an onComplete signal + * Returns true if this notification is an {@code onComplete} signal. + * @return true if this notification is an {@code onComplete} signal */ public boolean isOnComplete() { return value == null; } /** - * Returns true if this notification is an onError signal and - * {@link #getError()} returns the contained Throwable. - * @return true if this notification is an onError signal + * Returns true if this notification is an {@code onError} signal and + * {@link #getError()} returns the contained {@link Throwable}. + * @return true if this notification is an {@code onError} signal * @see #getError() */ public boolean isOnError() { @@ -50,9 +50,9 @@ public boolean isOnError() { } /** - * Returns true if this notification is an onNext signal and + * Returns true if this notification is an {@code onNext} signal and * {@link #getValue()} returns the contained value. - * @return true if this notification is an onNext signal + * @return true if this notification is an {@code onNext} signal * @see #getValue() */ public boolean isOnNext() { @@ -61,7 +61,7 @@ public boolean isOnNext() { } /** - * Returns the contained value if this notification is an onNext + * Returns the contained value if this notification is an {@code onNext} * signal, null otherwise. * @return the value contained or null * @see #isOnNext() @@ -77,9 +77,9 @@ public T getValue() { } /** - * Returns the container Throwable error if this notification is an onError + * Returns the container {@link Throwable} error if this notification is an {@code onError} * signal, null otherwise. - * @return the Throwable error contained or null + * @return the {@code Throwable} error contained or {@code null} * @see #isOnError() */ @Nullable @@ -121,14 +121,14 @@ public String toString() { /** * Constructs an onNext notification containing the given value. * @param the value type - * @param value the value to carry around in the notification, not null + * @param value the value to carry around in the notification, not {@code null} * @return the new Notification instance - * @throws NullPointerException if value is null + * @throws NullPointerException if value is {@code null} */ @NonNull - public static Notification createOnNext(@NonNull T value) { + public static <@NonNull T> Notification createOnNext(T value) { Objects.requireNonNull(value, "value is null"); - return new Notification(value); + return new Notification<>(value); } /** @@ -136,19 +136,19 @@ public static Notification createOnNext(@NonNull T value) { * @param the value type * @param error the error Throwable to carry around in the notification, not null * @return the new Notification instance - * @throws NullPointerException if error is null + * @throws NullPointerException if error is {@code null} */ @NonNull public static Notification createOnError(@NonNull Throwable error) { Objects.requireNonNull(error, "error is null"); - return new Notification(NotificationLite.error(error)); + return new Notification<>(NotificationLite.error(error)); } /** * Returns the empty and stateless shared instance of a notification representing - * an onComplete signal. + * an {@code onComplete} signal. * @param the target value type - * @return the shared Notification instance representing an onComplete signal + * @return the shared Notification instance representing an {@code onComplete} signal */ @SuppressWarnings("unchecked") @NonNull @@ -157,5 +157,5 @@ public static Notification createOnComplete() { } /** The singleton instance for createOnComplete. */ - static final Notification COMPLETE = new Notification(null); + static final Notification COMPLETE = new Notification<>(null); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index 63406824ba..0e2ff9dec1 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -1261,7 +1261,7 @@ public static Observable concatArrayEager(ObservableSource.. * * @param the value type * @param sources an array of ObservableSources that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrent subscriptions at a time, Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrent subscriptions at a time, {@link Integer#MAX_VALUE} * is interpreted as indication to subscribe to all sources at once * @param prefetch the number of elements to prefetch from each ObservableSource source * @return the new ObservableSource instance with the specified concatenation behavior @@ -1313,7 +1313,7 @@ public static Observable concatArrayEagerDelayError(ObservableSource * @param the value type * @param sources an array of {@code ObservableSource}s that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrent subscriptions at a time, Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrent subscriptions at a time, {@link Integer#MAX_VALUE} * is interpreted as indication to subscribe to all sources at once * @param prefetch the number of elements to prefetch from each {@code ObservableSource} source * @return the new Observable instance with the specified concatenation behavior @@ -1432,7 +1432,7 @@ public static Observable concatEager(ObservableSource * @param the value type * @param sources a sequence of ObservableSources that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrently running inner ObservableSources; Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrently running inner ObservableSources; {@link Integer#MAX_VALUE} * is interpreted as all inner ObservableSources can be active at the same time * @param prefetch the number of elements to prefetch from each inner ObservableSource source * @return the new ObservableSource instance with the specified concatenation behavior @@ -1482,7 +1482,7 @@ public static Observable concatEager(Iterable * @param the value type * @param sources a sequence of ObservableSources that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrently running inner ObservableSources; Integer.MAX_VALUE + * @param maxConcurrency the maximum number of concurrently running inner ObservableSources; {@link Integer#MAX_VALUE} * is interpreted as all inner ObservableSources can be active at the same time * @param prefetch the number of elements to prefetch from each inner ObservableSource source * @return the new ObservableSource instance with the specified concatenation behavior @@ -3562,7 +3562,7 @@ public static Observable never() { * @return an Observable that emits a range of sequential Integers * @throws IllegalArgumentException * if {@code count} is less than zero, or if {@code start} + {@code count} − 1 exceeds - * {@code Integer.MAX_VALUE} + * {@link Integer#MAX_VALUE} * @see ReactiveX operators documentation: Range */ @CheckReturnValue @@ -3599,7 +3599,7 @@ public static Observable range(final int start, final int count) { * @return an Observable that emits a range of sequential Longs * @throws IllegalArgumentException * if {@code count} is less than zero, or if {@code start} + {@code count} − 1 exceeds - * {@code Long.MAX_VALUE} + * {@link Long#MAX_VALUE} * @see ReactiveX operators documentation: Range */ @CheckReturnValue diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java b/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java index 40b432cfea..acdde63b29 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableConverter.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Convenience interface and callback used by the {@link Observable#to} operator to turn an Observable into another + * Convenience interface and callback used by the {@link Observable#to} operator to turn an {@link Observable} into another * value fluently. *

History: 2.1.7 - experimental * @param the upstream type @@ -26,9 +26,9 @@ @FunctionalInterface public interface ObservableConverter { /** - * Applies a function to the upstream Observable and returns a converted value of type {@code R}. + * Applies a function to the upstream {@link Observable} and returns a converted value of type {@code R}. * - * @param upstream the upstream Observable instance + * @param upstream the upstream {@code Observable} instance * @return the converted value */ @NonNull diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableEmitter.java b/src/main/java/io/reactivex/rxjava3/core/ObservableEmitter.java index f4f7ef6bb0..776d6963d9 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableEmitter.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableEmitter.java @@ -50,16 +50,18 @@ public interface ObservableEmitter extends Emitter { /** - * Sets a Disposable on this emitter; any previous {@link Disposable} + * Sets a {@link Disposable} on this emitter; any previous {@code Disposable} * or {@link Cancellable} will be disposed/cancelled. - * @param d the disposable, null is allowed + *

This method is thread-safe. + * @param d the {@code Disposable}, {@code null} is allowed */ void setDisposable(@Nullable Disposable d); /** - * Sets a Cancellable on this emitter; any previous {@link Disposable} - * or {@link Cancellable} will be disposed/cancelled. - * @param c the cancellable resource, null is allowed + * Sets a {@link Cancellable} on this emitter; any previous {@link Disposable} + * or {@code Cancellable} will be disposed/cancelled. + *

This method is thread-safe. + * @param c the {@code Cancellable} resource, {@code null} is allowed */ void setCancellable(@Nullable Cancellable c); @@ -73,21 +75,21 @@ public interface ObservableEmitter extends Emitter { boolean isDisposed(); /** - * Ensures that calls to onNext, onError and onComplete are properly serialized. - * @return the serialized ObservableEmitter + * Ensures that calls to {@code onNext}, {@code onError} and {@code onComplete} are properly serialized. + * @return the serialized {@link ObservableEmitter} */ @NonNull ObservableEmitter serialize(); /** - * Attempts to emit the specified {@code Throwable} error if the downstream + * Attempts to emit the specified {@link Throwable} error if the downstream * hasn't cancelled the sequence or is otherwise terminated, returning false * if the emission is not allowed to happen due to lifecycle restrictions. *

- * Unlike {@link #onError(Throwable)}, the {@code RxJavaPlugins.onError} is not called - * if the error could not be delivered. + * Unlike {@link #onError(Throwable)}, the {@link io.reactivex.rxjava3.plugins.RxJavaPlugins#onError(Throwable) RxjavaPlugins.onError} + * is not called if the error could not be delivered. *

History: 2.1.1 - experimental - * @param t the throwable error to signal if possible + * @param t the {@code Throwable} error to signal if possible * @return true if successful, false if the downstream is not able to accept further * events * @since 2.2 diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java index 43d6252849..bcde496462 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableOnSubscribe.java @@ -25,8 +25,8 @@ public interface ObservableOnSubscribe { /** - * Called for each Observer that subscribes. - * @param emitter the safe emitter instance, never null + * Called for each {@link Observer} that subscribes. + * @param emitter the safe emitter instance, never {@code null} * @throws Throwable on error */ void subscribe(@NonNull ObservableEmitter emitter) throws Throwable; diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java b/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java index 76ec06e295..ff9dbc5852 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableOperator.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to map/wrap a downstream observer to an upstream observer. + * Interface to map/wrap a downstream {@link Observer} to an upstream {@code Observer}. * * @param the value type of the downstream * @param the value type of the upstream @@ -24,11 +24,11 @@ @FunctionalInterface public interface ObservableOperator { /** - * Applies a function to the child Observer and returns a new parent Observer. - * @param observer the child Observer instance - * @return the parent Observer instance - * @throws Exception on failure + * Applies a function to the child {@link Observer} and returns a new parent {@code Observer}. + * @param observer the child {@code Observer} instance + * @return the parent {@code Observer} instance + * @throws Throwable on failure */ @NonNull - Observer apply(@NonNull Observer observer) throws Exception; + Observer apply(@NonNull Observer observer) throws Throwable; } diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java b/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java index ad5d5c79c3..c2309e5b8d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableSource.java @@ -25,9 +25,9 @@ public interface ObservableSource { /** - * Subscribes the given Observer to this ObservableSource instance. - * @param observer the Observer, not null - * @throws NullPointerException if {@code observer} is null + * Subscribes the given {@link Observer} to this {@link ObservableSource} instance. + * @param observer the {@code Observer}, not {@code null} + * @throws NullPointerException if {@code observer} is {@code null} */ void subscribe(@NonNull Observer observer); } diff --git a/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java b/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java index 259deefa85..18ce5e02c9 100644 --- a/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/ObservableTransformer.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to compose Observables. + * Interface to compose {@link Observable}s. * * @param the upstream value type * @param the downstream value type @@ -24,10 +24,10 @@ @FunctionalInterface public interface ObservableTransformer { /** - * Applies a function to the upstream Observable and returns an ObservableSource with + * Applies a function to the upstream {@link Observable} and returns an {@link ObservableSource} with * optionally different element type. - * @param upstream the upstream Observable instance - * @return the transformed ObservableSource instance + * @param upstream the upstream {@code Observable} instance + * @return the transformed {@code ObservableSource} instance */ @NonNull ObservableSource apply(@NonNull Observable upstream); diff --git a/src/main/java/io/reactivex/rxjava3/core/Observer.java b/src/main/java/io/reactivex/rxjava3/core/Observer.java index ef95e18034..de02f8ac88 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observer.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observer.java @@ -76,17 +76,17 @@ public interface Observer { /** - * Provides the Observer with the means of cancelling (disposing) the - * connection (channel) with the Observable in both + * Provides the {@link Observer} with the means of cancelling (disposing) the + * connection (channel) with the {@link Observable} in both * synchronous (from within {@link #onNext(Object)}) and asynchronous manner. - * @param d the Disposable instance whose {@link Disposable#dispose()} can + * @param d the {@link Disposable} instance whose {@link Disposable#dispose()} can * be called anytime to cancel the connection * @since 2.0 */ void onSubscribe(@NonNull Disposable d); /** - * Provides the Observer with a new item to observe. + * Provides the {@link Observer} with a new item to observe. *

* The {@link Observable} may call this method 0 or more times. *

@@ -99,9 +99,9 @@ public interface Observer { void onNext(@NonNull T t); /** - * Notifies the Observer that the {@link Observable} has experienced an error condition. + * Notifies the {@link Observer} that the {@link Observable} has experienced an error condition. *

- * If the {@link Observable} calls this method, it will not thereafter call {@link #onNext} or + * If the {@code Observable} calls this method, it will not thereafter call {@link #onNext} or * {@link #onComplete}. * * @param e @@ -110,9 +110,9 @@ public interface Observer { void onError(@NonNull Throwable e); /** - * Notifies the Observer that the {@link Observable} has finished sending push-based notifications. + * Notifies the {@link Observer} that the {@link Observable} has finished sending push-based notifications. *

- * The {@link Observable} will not call this method if it calls {@link #onError}. + * The {@code Observable} will not call this method if it calls {@link #onError}. */ void onComplete(); diff --git a/src/main/java/io/reactivex/rxjava3/core/Single.java b/src/main/java/io/reactivex/rxjava3/core/Single.java index ae2b04fc7d..bb742d4a03 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Single.java +++ b/src/main/java/io/reactivex/rxjava3/core/Single.java @@ -2346,7 +2346,7 @@ public final Single delaySubscription(ObservableSource other) { * *

If the delaying source signals an error, that error is re-emitted and no subscription * to the current Single happens. - *

The other source is consumed in an unbounded manner (requesting Long.MAX_VALUE from it). + *

The other source is consumed in an unbounded manner (requesting {@link Long#MAX_VALUE} from it). *

*
Backpressure:
*
The {@code other} publisher is consumed in an unbounded fashion but will be diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java b/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java index 4adc22b8c5..eb100b6c87 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleConverter.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Convenience interface and callback used by the {@link Single#to} operator to turn a Single into another + * Convenience interface and callback used by the {@link Single#to} operator to turn a {@link Single} into another * value fluently. *

History: 2.1.7 - experimental * @param the upstream type @@ -26,9 +26,9 @@ @FunctionalInterface public interface SingleConverter { /** - * Applies a function to the upstream Single and returns a converted value of type {@code R}. + * Applies a function to the upstream {@link Single} and returns a converted value of type {@code R}. * - * @param upstream the upstream Single instance + * @param upstream the upstream {@code Single} instance * @return the converted value */ @NonNull diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleEmitter.java b/src/main/java/io/reactivex/rxjava3/core/SingleEmitter.java index a2302c5f4c..bb3768243d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleEmitter.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleEmitter.java @@ -57,21 +57,23 @@ public interface SingleEmitter { /** * Signal an exception. - * @param t the exception, not null + * @param t the exception, not {@code null} */ void onError(@NonNull Throwable t); /** - * Sets a Disposable on this emitter; any previous Disposable - * or Cancellable will be disposed/cancelled. - * @param d the disposable, null is allowed + * Sets a {@link Disposable} on this emitter; any previous {@code Disposable} + * or {@link Cancellable} will be disposed/cancelled. + *

This method is thread-safe. + * @param d the {@code Disposable}, {@code null} is allowed */ void setDisposable(@Nullable Disposable d); /** * Sets a Cancellable on this emitter; any previous {@link Disposable} * or {@link Cancellable} will be disposed/cancelled. - * @param c the cancellable resource, null is allowed + *

This method is thread-safe. + * @param c the {@code Cancellable} resource, {@code null} is allowed */ void setCancellable(@Nullable Cancellable c); @@ -85,12 +87,12 @@ public interface SingleEmitter { boolean isDisposed(); /** - * Attempts to emit the specified {@code Throwable} error if the downstream + * Attempts to emit the specified {@link Throwable} error if the downstream * hasn't cancelled the sequence or is otherwise terminated, returning false * if the emission is not allowed to happen due to lifecycle restrictions. *

- * Unlike {@link #onError(Throwable)}, the {@code RxJavaPlugins.onError} is not called - * if the error could not be delivered. + * Unlike {@link #onError(Throwable)}, the {@link io.reactivex.rxjava3.plugins.RxJavaPlugins#onError(Throwable) RxjavaPlugins.onError} + * is not called if the error could not be delivered. *

History: 2.1.1 - experimental * @param t the throwable error to signal if possible * @return true if successful, false if the downstream is not able to accept further diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleObserver.java b/src/main/java/io/reactivex/rxjava3/core/SingleObserver.java index 778d6feabc..d2d788d38f 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleObserver.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleObserver.java @@ -53,7 +53,7 @@ public interface SingleObserver { /** - * Provides the SingleObserver with the means of cancelling (disposing) the + * Provides the {@link SingleObserver} with the means of cancelling (disposing) the * connection (channel) with the Single in both * synchronous (from within {@code onSubscribe(Disposable)} itself) and asynchronous manner. * @param d the Disposable instance whose {@link Disposable#dispose()} can @@ -63,23 +63,23 @@ public interface SingleObserver { void onSubscribe(@NonNull Disposable d); /** - * Notifies the SingleObserver with a single item and that the {@link Single} has finished sending + * Notifies the {@link SingleObserver} with a single item and that the {@link Single} has finished sending * push-based notifications. *

- * The {@link Single} will not call this method if it calls {@link #onError}. + * The {@code Single} will not call this method if it calls {@link #onError}. * * @param t - * the item emitted by the Single + * the item emitted by the {@code Single} */ void onSuccess(@NonNull T t); /** - * Notifies the SingleObserver that the {@link Single} has experienced an error condition. + * Notifies the {@link SingleObserver} that the {@link Single} has experienced an error condition. *

- * If the {@link Single} calls this method, it will not thereafter call {@link #onSuccess}. + * If the {@code Single} calls this method, it will not thereafter call {@link #onSuccess}. * * @param e - * the exception encountered by the Single + * the exception encountered by the {@code Single} */ void onError(@NonNull Throwable e); } diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java index 46cc6a6f56..7a6a5adf68 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleOnSubscribe.java @@ -25,8 +25,8 @@ public interface SingleOnSubscribe { /** - * Called for each SingleObserver that subscribes. - * @param emitter the safe emitter instance, never null + * Called for each {@link SingleObserver} that subscribes. + * @param emitter the safe emitter instance, never {@code null} * @throws Throwable on error */ void subscribe(@NonNull SingleEmitter emitter) throws Throwable; diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java b/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java index 2fc2d80ab6..de2f38f23d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleOperator.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to map/wrap a downstream observer to an upstream observer. + * Interface to map/wrap a downstream {@link SingleObserver} to an upstream {@code SingleObserver}. * * @param the value type of the downstream * @param the value type of the upstream @@ -24,11 +24,11 @@ @FunctionalInterface public interface SingleOperator { /** - * Applies a function to the child SingleObserver and returns a new parent SingleObserver. - * @param observer the child SingleObserver instance - * @return the parent SingleObserver instance - * @throws Exception on failure + * Applies a function to the child {@link SingleObserver} and returns a new parent {@code SingleObserver}. + * @param observer the child {@code SingleObserver} instance + * @return the parent {@code SingleObserver} instance + * @throws Throwable on failure */ @NonNull - SingleObserver apply(@NonNull SingleObserver observer) throws Exception; + SingleObserver apply(@NonNull SingleObserver observer) throws Throwable; } diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleSource.java b/src/main/java/io/reactivex/rxjava3/core/SingleSource.java index 496462d0e1..b4c923905d 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleSource.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleSource.java @@ -28,9 +28,9 @@ public interface SingleSource { /** - * Subscribes the given SingleObserver to this SingleSource instance. - * @param observer the SingleObserver, not null - * @throws NullPointerException if {@code observer} is null + * Subscribes the given {@link SingleObserver} to this {@link SingleSource} instance. + * @param observer the {@code SingleObserver}, not {@code null} + * @throws NullPointerException if {@code observer} is {@code null} */ void subscribe(@NonNull SingleObserver observer); } diff --git a/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java b/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java index 08f9408cdb..1f0d478a0e 100644 --- a/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java +++ b/src/main/java/io/reactivex/rxjava3/core/SingleTransformer.java @@ -16,7 +16,7 @@ import io.reactivex.rxjava3.annotations.NonNull; /** - * Interface to compose Singles. + * Interface to compose {@link Single}s. * * @param the upstream value type * @param the downstream value type @@ -24,10 +24,10 @@ @FunctionalInterface public interface SingleTransformer { /** - * Applies a function to the upstream Single and returns a SingleSource with + * Applies a function to the upstream {@link Single} and returns a {@link SingleSource} with * optionally different element type. - * @param upstream the upstream Single instance - * @return the transformed SingleSource instance + * @param upstream the upstream {@code Single} instance + * @return the transformed {@code SingleSource} instance */ @NonNull SingleSource apply(@NonNull Single upstream); diff --git a/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java index c9c0a8988c..3a75ac5f3e 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java @@ -37,7 +37,7 @@ public CompositeDisposable() { /** * Creates a CompositeDisposables with the given array of initial elements. * @param disposables the array of Disposables to start with - * @throws NullPointerException if {@code disposables} or any of its array items is null + * @throws NullPointerException if {@code disposables} or any of its array items is {@code null} */ public CompositeDisposable(@NonNull Disposable... disposables) { Objects.requireNonNull(disposables, "disposables is null"); @@ -51,7 +51,7 @@ public CompositeDisposable(@NonNull Disposable... disposables) { /** * Creates a CompositeDisposables with the given Iterable sequence of initial elements. * @param disposables the Iterable sequence of Disposables to start with - * @throws NullPointerException if {@code disposables} or any of its items is null + * @throws NullPointerException if {@code disposables} or any of its items is {@code null} */ public CompositeDisposable(@NonNull Iterable disposables) { Objects.requireNonNull(disposables, "disposables is null"); @@ -90,7 +90,7 @@ public boolean isDisposed() { * container has been disposed. * @param disposable the disposable to add, not null * @return true if successful, false if this container has been disposed - * @throws NullPointerException if {@code disposable} is null + * @throws NullPointerException if {@code disposable} is {@code null} */ @Override public boolean add(@NonNull Disposable disposable) { @@ -117,7 +117,7 @@ public boolean add(@NonNull Disposable disposable) { * disposes them all if the container has been disposed. * @param disposables the array of Disposables * @return true if the operation was successful, false if the container has been disposed - * @throws NullPointerException if {@code disposables} or any of its array items is null + * @throws NullPointerException if {@code disposables} or any of its array items is {@code null} */ public boolean addAll(@NonNull Disposable... disposables) { Objects.requireNonNull(disposables, "disposables is null"); @@ -163,7 +163,7 @@ public boolean remove(@NonNull Disposable disposable) { * container. * @param disposable the disposable to remove, not null * @return true if the operation was successful - * @throws NullPointerException if {@code disposable} is null + * @throws NullPointerException if {@code disposable} is {@code null} */ @Override public boolean delete(@NonNull Disposable disposable) { diff --git a/src/main/java/io/reactivex/rxjava3/exceptions/UndeliverableException.java b/src/main/java/io/reactivex/rxjava3/exceptions/UndeliverableException.java index 6c84920e74..08f926e9b3 100644 --- a/src/main/java/io/reactivex/rxjava3/exceptions/UndeliverableException.java +++ b/src/main/java/io/reactivex/rxjava3/exceptions/UndeliverableException.java @@ -14,7 +14,7 @@ package io.reactivex.rxjava3.exceptions; /** - * Wrapper for Throwable errors that are sent to `RxJavaPlugins.onError`. + * Wrapper for Throwable errors that are sent to {@link io.reactivex.rxjava3.plugins.RxJavaPlugins#onError(Throwable) RxJavaPlugins.onError}. *

History: 2.0.6 - experimental; 2.1 - beta * @since 2.2 */ diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/BackpressureHelper.java b/src/main/java/io/reactivex/rxjava3/internal/util/BackpressureHelper.java index 9237c11a18..4ccfe67845 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/BackpressureHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/BackpressureHelper.java @@ -14,6 +14,7 @@ import java.util.concurrent.atomic.AtomicLong; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** @@ -26,10 +27,10 @@ private BackpressureHelper() { } /** - * Adds two long values and caps the sum at Long.MAX_VALUE. + * Adds two long values and caps the sum at {@link Long#MAX_VALUE}. * @param a the first value * @param b the second value - * @return the sum capped at Long.MAX_VALUE + * @return the sum capped at {@link Long#MAX_VALUE} */ public static long addCap(long a, long b) { long u = a + b; @@ -40,10 +41,10 @@ public static long addCap(long a, long b) { } /** - * Multiplies two long values and caps the product at Long.MAX_VALUE. + * Multiplies two long values and caps the product at {@link Long#MAX_VALUE}. * @param a the first value * @param b the second value - * @return the product capped at Long.MAX_VALUE + * @return the product capped at {@link Long#MAX_VALUE} */ public static long multiplyCap(long a, long b) { long u = a * b; @@ -56,13 +57,13 @@ public static long multiplyCap(long a, long b) { } /** - * Atomically adds the positive value n to the requested value in the AtomicLong and - * caps the result at Long.MAX_VALUE and returns the previous value. - * @param requested the AtomicLong holding the current requested value + * Atomically adds the positive value n to the requested value in the {@link AtomicLong} and + * caps the result at {@link Long#MAX_VALUE} and returns the previous value. + * @param requested the {@code AtomicLong} holding the current requested value * @param n the value to add, must be positive (not verified) * @return the original value before the add */ - public static long add(AtomicLong requested, long n) { + public static long add(@NonNull AtomicLong requested, long n) { for (;;) { long r = requested.get(); if (r == Long.MAX_VALUE) { @@ -76,14 +77,14 @@ public static long add(AtomicLong requested, long n) { } /** - * Atomically adds the positive value n to the requested value in the AtomicLong and - * caps the result at Long.MAX_VALUE and returns the previous value and - * considers Long.MIN_VALUE as a cancel indication (no addition then). - * @param requested the AtomicLong holding the current requested value + * Atomically adds the positive value n to the requested value in the {@link AtomicLong} and + * caps the result at {@link Long#MAX_VALUE} and returns the previous value and + * considers {@link Long#MIN_VALUE} as a cancel indication (no addition then). + * @param requested the {@code AtomicLong} holding the current requested value * @param n the value to add, must be positive (not verified) * @return the original value before the add */ - public static long addCancel(AtomicLong requested, long n) { + public static long addCancel(@NonNull AtomicLong requested, long n) { for (;;) { long r = requested.get(); if (r == Long.MIN_VALUE) { @@ -100,12 +101,12 @@ public static long addCancel(AtomicLong requested, long n) { } /** - * Atomically subtract the given number (positive, not validated) from the target field unless it contains Long.MAX_VALUE. + * Atomically subtract the given number (positive, not validated) from the target field unless it contains {@link Long#MAX_VALUE}. * @param requested the target field holding the current requested amount * @param n the produced element count, positive (not validated) * @return the new amount */ - public static long produced(AtomicLong requested, long n) { + public static long produced(@NonNull AtomicLong requested, long n) { for (;;) { long current = requested.get(); if (current == Long.MAX_VALUE) { @@ -124,12 +125,12 @@ public static long produced(AtomicLong requested, long n) { /** * Atomically subtract the given number (positive, not validated) from the target field if - * it doesn't contain Long.MIN_VALUE (indicating some cancelled state) or Long.MAX_VALUE (unbounded mode). + * it doesn't contain {@link Long#MIN_VALUE} (indicating some cancelled state) or {@link Long#MAX_VALUE} (unbounded mode). * @param requested the target field holding the current requested amount * @param n the produced element count, positive (not validated) * @return the new amount */ - public static long producedCancel(AtomicLong requested, long n) { + public static long producedCancel(@NonNull AtomicLong requested, long n) { for (;;) { long current = requested.get(); if (current == Long.MIN_VALUE) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java b/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java index 41906b9cf7..9cdfab07ce 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java @@ -218,7 +218,7 @@ public static SimpleQueue createQueue(int capacityHint) { } /** - * Requests Long.MAX_VALUE if prefetch is negative or the exact + * Requests {@link Long#MAX_VALUE} if prefetch is negative or the exact * amount if prefetch is positive. * @param s the Subscription to request from * @param prefetch the prefetch value @@ -383,7 +383,7 @@ static boolean postCompleteDrain(long n, * in completed mode, requests no-longer reach the upstream but help in draining the queue. *

* The algorithm utilizes the most significant bit (bit 63) of a long value (AtomicLong) since - * request amount only goes up to Long.MAX_VALUE (bits 0-62) and negative values aren't + * request amount only goes up to {@link Long#MAX_VALUE} (bits 0-62) and negative values aren't * allowed. * * @param the value type emitted diff --git a/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java b/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java index f8a0356a75..458325c280 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java +++ b/src/main/java/io/reactivex/rxjava3/observers/BaseTestConsumer.java @@ -16,15 +16,17 @@ import java.util.*; import java.util.concurrent.*; +import io.reactivex.rxjava3.annotations.*; import io.reactivex.rxjava3.exceptions.CompositeException; import io.reactivex.rxjava3.functions.Predicate; -import io.reactivex.rxjava3.internal.functions.*; +import io.reactivex.rxjava3.internal.functions.Functions; import io.reactivex.rxjava3.internal.util.*; /** - * Base class with shared infrastructure to support TestSubscriber and TestObserver. + * Base class with shared infrastructure to support + * {@link io.reactivex.rxjava3.subscribers.TestSubscriber TestSubscriber} and {@link TestObserver}. * @param the value type consumed - * @param the subclass of this BaseTestConsumer + * @param the subclass of this {@code BaseTestConsumer} */ public abstract class BaseTestConsumer> { /** The latch that indicates an onError or onComplete has been called. */ @@ -47,26 +49,26 @@ public abstract class BaseTestConsumer> { protected CharSequence tag; /** - * Indicates that one of the awaitX method has timed out. + * Indicates that one of the {@code awaitX} method has timed out. * @since 2.0.7 */ protected boolean timeout; public BaseTestConsumer() { - this.values = new VolatileSizeArrayList(); - this.errors = new VolatileSizeArrayList(); + this.values = new VolatileSizeArrayList<>(); + this.errors = new VolatileSizeArrayList<>(); this.done = new CountDownLatch(1); } /** - * Returns a shared list of received onNext values. + * Returns a shared list of received {@code onNext} values or the single {@code onSuccess} value. *

* Note that accessing the items via certain methods of the {@link List} * interface while the upstream is still actively emitting * more items may result in a {@code ConcurrentModificationException}. *

* The {@link List#size()} method will return the number of items - * already received by this TestObserver/TestSubscriber in a thread-safe + * already received by this {@code TestObserver}/{@code TestSubscriber} in a thread-safe * manner that can be read via {@link List#get(int)}) method * (index range of 0 to {@code List.size() - 1}). *

@@ -76,6 +78,7 @@ public BaseTestConsumer() { * {@code ConcurrentModificationException}. * @return a list of received onNext values */ + @NonNull public final List values() { return values; } @@ -89,7 +92,8 @@ public final List values() { * @param message the message to use * @return AssertionError the prepared AssertionError instance */ - protected final AssertionError fail(String message) { + @NonNull + protected final AssertionError fail(@NonNull String message) { StringBuilder b = new StringBuilder(64 + message.length()); b.append(message); @@ -131,11 +135,12 @@ protected final AssertionError fail(String message) { } /** - * Awaits until this TestObserver/TestSubscriber receives an onError or onComplete events. + * Awaits until this {@code TestObserver}/{@code TestSubscriber} receives an {@code onError} or {@code onComplete} events. * @return this * @throws InterruptedException if the current thread is interrupted while waiting */ @SuppressWarnings("unchecked") + @NonNull public final U await() throws InterruptedException { if (done.getCount() == 0) { return (U)this; @@ -146,14 +151,14 @@ public final U await() throws InterruptedException { } /** - * Awaits the specified amount of time or until this TestObserver/TestSubscriber - * receives an onError or onComplete events, whichever happens first. + * Awaits the specified amount of time or until this {@code TestObserver}/{@code TestSubscriber} + * receives an {@code onError} or {@code onComplete} events, whichever happens first. * @param time the waiting time * @param unit the time unit of the waiting time - * @return true if the TestObserver/TestSubscriber terminated, false if timeout happened + * @return true if the {@code TestObserver}/{@code TestSubscriber} terminated, false if timeout happened * @throws InterruptedException if the current thread is interrupted while waiting */ - public final boolean await(long time, TimeUnit unit) throws InterruptedException { + public final boolean await(long time, @NonNull TimeUnit unit) throws InterruptedException { boolean d = done.getCount() == 0 || (done.await(time, unit)); timeout = !d; return d; @@ -162,10 +167,11 @@ public final boolean await(long time, TimeUnit unit) throws InterruptedException // assertion methods /** - * Assert that this TestObserver/TestSubscriber received exactly one onComplete event. + * Assert that this {@code TestObserver}/{@code TestSubscriber} received exactly one {@code onComplete} event. * @return this */ @SuppressWarnings("unchecked") + @NonNull public final U assertComplete() { long c = completions; if (c == 0) { @@ -178,10 +184,11 @@ public final U assertComplete() { } /** - * Assert that this TestObserver/TestSubscriber has not received any onComplete event. + * Assert that this {@code TestObserver}/{@code TestSubscriber} has not received an {@code onComplete} event. * @return this */ @SuppressWarnings("unchecked") + @NonNull public final U assertNotComplete() { long c = completions; if (c == 1) { @@ -194,10 +201,11 @@ public final U assertNotComplete() { } /** - * Assert that this TestObserver/TestSubscriber has not received any onError event. + * Assert that this {@code TestObserver}/{@code TestSubscriber} has not received an {@code onError} event. * @return this */ @SuppressWarnings("unchecked") + @NonNull public final U assertNoErrors() { int s = errors.size(); if (s != 0) { @@ -207,9 +215,9 @@ public final U assertNoErrors() { } /** - * Assert that this TestObserver/TestSubscriber received exactly the specified onError event value. + * Assert that this {@code TestObserver}/{@code TestSubscriber} received exactly the specified {@code onError} event value. * - *

The comparison is performed via Objects.equals(); since most exceptions don't + *

The comparison is performed via {@link Objects#equals(Object, Object)}; since most exceptions don't * implement equals(), this assertion may fail. Use the {@link #assertError(Class)} * overload to test against the class of an error instead of an instance of an error * or {@link #assertError(Predicate)} to test with different condition. @@ -218,31 +226,34 @@ public final U assertNoErrors() { * @see #assertError(Class) * @see #assertError(Predicate) */ - public final U assertError(Throwable error) { + @NonNull + public final U assertError(@NonNull Throwable error) { return assertError(Functions.equalsWith(error)); } /** - * Asserts that this TestObserver/TestSubscriber received exactly one onError event which is an - * instance of the specified errorClass class. - * @param errorClass the error class to expect + * Asserts that this {@code TestObserver}/{@code TestSubscriber} received exactly one {@code onError} event which is an + * instance of the specified {@code errorClass} {@link Class}. + * @param errorClass the error {@code Class} to expect * @return this */ @SuppressWarnings({ "unchecked", "rawtypes" }) - public final U assertError(Class errorClass) { + @NonNull + public final U assertError(@NonNull Class errorClass) { return (U)assertError((Predicate)Functions.isInstanceOf(errorClass)); } /** - * Asserts that this TestObserver/TestSubscriber received exactly one onError event for which - * the provided predicate returns true. + * Asserts that this {@code TestObserver}/{@code TestSubscriber} received exactly one {@code onError} event for which + * the provided predicate returns {@code true}. * @param errorPredicate - * the predicate that receives the error Throwable - * and should return true for expected errors. + * the predicate that receives the error {@link Throwable} + * and should return {@code true} for expected errors. * @return this */ @SuppressWarnings("unchecked") - public final U assertError(Predicate errorPredicate) { + @NonNull + public final U assertError(@NonNull Predicate errorPredicate) { int s = errors.size(); if (s == 0) { throw fail("No errors"); @@ -272,13 +283,14 @@ public final U assertError(Predicate errorPredicate) { } /** - * Assert that this TestObserver/TestSubscriber received exactly one onNext value which is equal to - * the given value with respect to Objects.equals. + * Assert that this {@code TestObserver}/{@code TestSubscriber} received exactly one {@code onNext} value which is equal to + * the given value with respect to {@link Objects#equals(Object, Object)}. * @param value the value to expect * @return this */ @SuppressWarnings("unchecked") - public final U assertValue(T value) { + @NonNull + public final U assertValue(@NonNull T value) { int s = values.size(); if (s != 1) { throw fail("expected: " + valueAndClass(value) + " but was: " + values); @@ -291,15 +303,16 @@ public final U assertValue(T value) { } /** - * Asserts that this TestObserver/TestSubscriber received exactly one onNext value for which - * the provided predicate returns true. + * Asserts that this {@code TestObserver}/{@code TestSubscriber} received exactly one {@code onNext} value for which + * the provided predicate returns {@code true}. * @param valuePredicate - * the predicate that receives the onNext value - * and should return true for the expected value. + * the predicate that receives the {@code onNext} value + * and should return {@code true} for the expected value. * @return this */ @SuppressWarnings("unchecked") - public final U assertValue(Predicate valuePredicate) { + @NonNull + public final U assertValue(@NonNull Predicate valuePredicate) { assertValueAt(0, valuePredicate); if (values.size() > 1) { @@ -310,8 +323,8 @@ public final U assertValue(Predicate valuePredicate) { } /** - * Asserts that this TestObserver/TestSubscriber received an onNext value at the given index - * which is equal to the given value with respect to null-safe Object.equals. + * Asserts that this {@code TestObserver}/{@code TestSubscriber} received an {@code onNext} value at the given index + * which is equal to the given value with respect to {@code null}-safe {@link Objects#equals(Object, Object)}. *

History: 2.1.3 - experimental * @param index the position to assert on * @param value the value to expect @@ -319,7 +332,8 @@ public final U assertValue(Predicate valuePredicate) { * @since 2.2 */ @SuppressWarnings("unchecked") - public final U assertValueAt(int index, T value) { + @NonNull + public final U assertValueAt(int index, @NonNull T value) { int s = values.size(); if (s == 0) { throw fail("No values"); @@ -337,16 +351,17 @@ public final U assertValueAt(int index, T value) { } /** - * Asserts that this TestObserver/TestSubscriber received an onNext value at the given index - * for the provided predicate returns true. + * Asserts that this {@code TestObserver}/{@code TestSubscriber} received an {@code onNext} value at the given index + * for the provided predicate returns {@code true}. * @param index the position to assert on * @param valuePredicate - * the predicate that receives the onNext value - * and should return true for the expected value. + * the predicate that receives the {@code onNext} value + * and should return {@code true} for the expected value. * @return this */ @SuppressWarnings("unchecked") - public final U assertValueAt(int index, Predicate valuePredicate) { + @NonNull + public final U assertValueAt(int index, @NonNull Predicate valuePredicate) { int s = values.size(); if (s == 0) { throw fail("No values"); @@ -373,11 +388,12 @@ public final U assertValueAt(int index, Predicate valuePredicate) { } /** - * Appends the class name to a non-null value. + * Appends the class name to a non-{@code null} value or returns {@code "null"}. * @param o the object * @return the string representation */ - public static String valueAndClass(Object o) { + @NonNull + public static String valueAndClass(@Nullable Object o) { if (o != null) { return o + " (class: " + o.getClass().getSimpleName() + ")"; } @@ -385,11 +401,12 @@ public static String valueAndClass(Object o) { } /** - * Assert that this TestObserver/TestSubscriber received the specified number onNext events. - * @param count the expected number of onNext events + * Assert that this {@code TestObserver}/{@code TestSubscriber} received the specified number {@code onNext} events. + * @param count the expected number of {@code onNext} events * @return this */ @SuppressWarnings("unchecked") + @NonNull public final U assertValueCount(int count) { int s = values.size(); if (s != count) { @@ -399,20 +416,23 @@ public final U assertValueCount(int count) { } /** - * Assert that this TestObserver/TestSubscriber has not received any onNext events. + * Assert that this {@code TestObserver}/{@code TestSubscriber} has not received any {@code onNext} events. * @return this */ + @NonNull public final U assertNoValues() { return assertValueCount(0); } /** - * Assert that the TestObserver/TestSubscriber received only the specified values in the specified order. + * Assert that the {@code TestObserver}/{@code TestSubscriber} received only the specified values in the specified order. * @param values the values expected * @return this */ @SuppressWarnings("unchecked") - public final U assertValues(T... values) { + @SafeVarargs + @NonNull + public final U assertValues(@NonNull T... values) { int s = this.values.size(); if (s != values.length) { throw fail("Value count differs; expected: " + values.length + " " + Arrays.toString(values) @@ -429,14 +449,15 @@ public final U assertValues(T... values) { } /** - * Assert that the TestObserver/TestSubscriber received only the specified values in the specified order without terminating. + * Assert that the {@code TestObserver}/{@code TestSubscriber} received only the specified values in the specified order without terminating. *

History: 2.1.4 - experimental * @param values the values expected * @return this * @since 2.2 */ @SafeVarargs - public final U assertValuesOnly(T... values) { + @NonNull + public final U assertValuesOnly(@NonNull T... values) { return assertSubscribed() .assertValues(values) .assertNoErrors() @@ -444,12 +465,13 @@ public final U assertValuesOnly(T... values) { } /** - * Assert that the TestObserver/TestSubscriber received only the specified sequence of values in the same order. + * Assert that the {@code TestObserver}/{@code TestSubscriber} received only the specified sequence of values in the same order. * @param sequence the sequence of expected values in order * @return this */ @SuppressWarnings("unchecked") - public final U assertValueSequence(Iterable sequence) { + @NonNull + public final U assertValueSequence(@NonNull Iterable sequence) { int i = 0; Iterator actualIterator = values.iterator(); Iterator expectedIterator = sequence.iterator(); @@ -482,20 +504,22 @@ public final U assertValueSequence(Iterable sequence) { } /** - * Assert that the onSubscribe method was called exactly once. + * Assert that the {@code onSubscribe} method was called exactly once. * @return this */ + @NonNull protected abstract U assertSubscribed(); /** - * Assert that the upstream signalled the specified values in order and + * Assert that the upstream signaled the specified values in order and * completed normally. * @param values the expected values, asserted in order * @return this * @see #assertFailure(Class, Object...) */ @SafeVarargs - public final U assertResult(T... values) { + @NonNull + public final U assertResult(@NonNull T... values) { return assertSubscribed() .assertValues(values) .assertNoErrors() @@ -503,14 +527,15 @@ public final U assertResult(T... values) { } /** - * Assert that the upstream signalled the specified values in order - * and then failed with a specific class or subclass of Throwable. - * @param error the expected exception (parent) class + * Assert that the upstream signaled the specified values in order + * and then failed with a specific class or subclass of {@link Throwable}. + * @param error the expected exception (parent) {@link Class} * @param values the expected values, asserted in order * @return this */ @SafeVarargs - public final U assertFailure(Class error, T... values) { + @NonNull + public final U assertFailure(@NonNull Class error, @NonNull T... values) { return assertSubscribed() .assertValues(values) .assertError(error) @@ -519,14 +544,15 @@ public final U assertFailure(Class error, T... values) { /** * Awaits until the internal latch is counted down. - *

If the wait times out or gets interrupted, the TestObserver/TestSubscriber is cancelled. + *

If the wait times out or gets interrupted, the {@code TestObserver}/{@code TestSubscriber} is cancelled. * @param time the waiting time * @param unit the time unit of the waiting time * @return this - * @throws RuntimeException wrapping an InterruptedException if the wait is interrupted + * @throws RuntimeException wrapping an {@link InterruptedException} if the wait is interrupted */ @SuppressWarnings("unchecked") - public final U awaitDone(long time, TimeUnit unit) { + @NonNull + public final U awaitDone(long time, @NonNull TimeUnit unit) { try { if (!done.await(time, unit)) { timeout = true; @@ -540,9 +566,12 @@ public final U awaitDone(long time, TimeUnit unit) { } /** - * Assert that the TestObserver/TestSubscriber has received a Disposable but no other events. + * Assert that the {@code TestObserver}/{@code TestSubscriber} has received a + * {@link io.reactivex.rxjava3.disposables.Disposable Disposable}/{@link org.reactivestreams.Subscription Subscription} + * via {@code onSubscribe} but no other events. * @return this */ + @NonNull public final U assertEmpty() { return assertSubscribed() .assertNoValues() @@ -554,18 +583,19 @@ public final U assertEmpty() { * Set the tag displayed along with an assertion failure's * other state information. *

History: 2.0.7 - experimental - * @param tag the string to display (null won't print any tag) + * @param tag the string to display ({@code null} won't print any tag) * @return this * @since 2.1 */ @SuppressWarnings("unchecked") - public final U withTag(CharSequence tag) { + @NonNull + public final U withTag(@Nullable CharSequence tag) { this.tag = tag; return (U)this; } /** - * Await until the TestObserver/TestSubscriber receives the given + * Await until the {@code TestObserver}/{@code TestSubscriber} receives the given * number of items or terminates by sleeping 10 milliseconds at a time * up to 5000 milliseconds of timeout. *

History: 2.0.7 - experimental @@ -574,6 +604,7 @@ public final U withTag(CharSequence tag) { * @since 2.1 */ @SuppressWarnings("unchecked") + @NonNull public final U awaitCount(int atLeast) { long start = System.currentTimeMillis(); long timeoutMillis = 5000; diff --git a/src/main/java/io/reactivex/rxjava3/observers/DisposableCompletableObserver.java b/src/main/java/io/reactivex/rxjava3/observers/DisposableCompletableObserver.java index 25926663d5..940654b622 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/DisposableCompletableObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/DisposableCompletableObserver.java @@ -53,7 +53,7 @@ */ public abstract class DisposableCompletableObserver implements CompletableObserver, Disposable { - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); @Override public final void onSubscribe(@NonNull Disposable d) { @@ -63,7 +63,7 @@ public final void onSubscribe(@NonNull Disposable d) { } /** - * Called once the single upstream Disposable is set via onSubscribe. + * Called once the single upstream {@link Disposable} is set via {@link #onSubscribe(Disposable)}. */ protected void onStart() { } diff --git a/src/main/java/io/reactivex/rxjava3/observers/DisposableMaybeObserver.java b/src/main/java/io/reactivex/rxjava3/observers/DisposableMaybeObserver.java index 66ac6a8620..092994e818 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/DisposableMaybeObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/DisposableMaybeObserver.java @@ -22,7 +22,7 @@ import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** - * An abstract {@link MaybeObserver} that allows asynchronous cancellation by implementing Disposable. + * An abstract {@link MaybeObserver} that allows asynchronous cancellation by implementing {@link Disposable}. * *

All pre-implemented final methods are thread-safe. * @@ -62,7 +62,7 @@ */ public abstract class DisposableMaybeObserver implements MaybeObserver, Disposable { - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); @Override public final void onSubscribe(@NonNull Disposable d) { @@ -72,7 +72,7 @@ public final void onSubscribe(@NonNull Disposable d) { } /** - * Called once the single upstream Disposable is set via onSubscribe. + * Called once the single upstream {@link Disposable} is set via {@link #onSubscribe(Disposable)}. */ protected void onStart() { } diff --git a/src/main/java/io/reactivex/rxjava3/observers/DisposableObserver.java b/src/main/java/io/reactivex/rxjava3/observers/DisposableObserver.java index a15587317c..ff2eba7758 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/DisposableObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/DisposableObserver.java @@ -22,7 +22,7 @@ import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** - * An abstract {@link Observer} that allows asynchronous cancellation by implementing Disposable. + * An abstract {@link Observer} that allows asynchronous cancellation by implementing {@link Disposable}. * *

All pre-implemented final methods are thread-safe. * @@ -66,7 +66,7 @@ */ public abstract class DisposableObserver implements Observer, Disposable { - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); @Override public final void onSubscribe(@NonNull Disposable d) { diff --git a/src/main/java/io/reactivex/rxjava3/observers/DisposableSingleObserver.java b/src/main/java/io/reactivex/rxjava3/observers/DisposableSingleObserver.java index 6d8ce73fb0..56e5c543b7 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/DisposableSingleObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/DisposableSingleObserver.java @@ -22,7 +22,7 @@ import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** - * An abstract {@link SingleObserver} that allows asynchronous cancellation by implementing Disposable. + * An abstract {@link SingleObserver} that allows asynchronous cancellation by implementing {@link Disposable}. * *

All pre-implemented final methods are thread-safe. * @@ -55,7 +55,7 @@ */ public abstract class DisposableSingleObserver implements SingleObserver, Disposable { - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); @Override public final void onSubscribe(@NonNull Disposable d) { @@ -65,7 +65,7 @@ public final void onSubscribe(@NonNull Disposable d) { } /** - * Called once the single upstream Disposable is set via onSubscribe. + * Called once the single upstream {@link Disposable} is set via {@link #onSubscribe(Disposable)}. */ protected void onStart() { } diff --git a/src/main/java/io/reactivex/rxjava3/observers/LambdaConsumerIntrospection.java b/src/main/java/io/reactivex/rxjava3/observers/LambdaConsumerIntrospection.java index 5d74ac5e8c..e3f5c46cfe 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/LambdaConsumerIntrospection.java +++ b/src/main/java/io/reactivex/rxjava3/observers/LambdaConsumerIntrospection.java @@ -24,8 +24,8 @@ public interface LambdaConsumerIntrospection { /** - * Returns true or false if a custom onError consumer has been provided. - * @return {@code true} if a custom onError consumer implementation was supplied. Returns {@code false} if the + * Returns {@code true} or {@code false} if a custom {@code onError} consumer has been provided. + * @return {@code true} if a custom {@code onError} consumer implementation was supplied. Returns {@code false} if the * implementation is missing an error consumer and thus using a throwing default implementation. */ boolean hasCustomOnError(); diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java index b57d77f6f5..df05fe0169 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceCompletableObserver.java @@ -74,17 +74,17 @@ */ public abstract class ResourceCompletableObserver implements CompletableObserver, Disposable { /** The active subscription. */ - private final AtomicReference upstream = new AtomicReference(); + private final AtomicReference upstream = new AtomicReference<>(); /** The resource composite, can never be null. */ private final ListCompositeDisposable resources = new ListCompositeDisposable(); /** - * Adds a resource to this ResourceObserver. + * Adds a resource to this {@code ResourceCompletableObserver}. * * @param resource the resource to add * - * @throws NullPointerException if resource is null + * @throws NullPointerException if resource is {@code null} */ public final void add(@NonNull Disposable resource) { Objects.requireNonNull(resource, "resource is null"); @@ -99,7 +99,7 @@ public final void onSubscribe(@NonNull Disposable d) { } /** - * Called once the upstream sets a Subscription on this ResourceObserver. + * Called once the upstream sets a {@link Disposable} on this {@code ResourceCompletableObserver}. * *

You can perform initialization at this moment. The default * implementation does nothing. @@ -109,10 +109,10 @@ protected void onStart() { /** * Cancels the main disposable (if any) and disposes the resources associated with - * this ResourceObserver (if any). + * this {@code ResourceCompletableObserver} (if any). * - *

This method can be called before the upstream calls onSubscribe at which - * case the main Disposable will be immediately disposed. + *

This method can be called before the upstream calls {@link #onSubscribe(Disposable)} at which + * case the main {@link Disposable} will be immediately disposed. */ @Override public final void dispose() { @@ -122,8 +122,8 @@ public final void dispose() { } /** - * Returns true if this ResourceObserver has been disposed/cancelled. - * @return true if this ResourceObserver has been disposed/cancelled + * Returns true if this {@code ResourceCompletableObserver} has been disposed/cancelled. + * @return true if this {@code ResourceCompletableObserver} has been disposed/cancelled */ @Override public final boolean isDisposed() { diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java index cb42a70fbd..22345e60ce 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceMaybeObserver.java @@ -84,17 +84,17 @@ */ public abstract class ResourceMaybeObserver implements MaybeObserver, Disposable { /** The active subscription. */ - private final AtomicReference upstream = new AtomicReference(); + private final AtomicReference upstream = new AtomicReference<>(); /** The resource composite, can never be null. */ private final ListCompositeDisposable resources = new ListCompositeDisposable(); /** - * Adds a resource to this ResourceObserver. + * Adds a resource to this {@code ResourceMaybeObserver}. * * @param resource the resource to add * - * @throws NullPointerException if resource is null + * @throws NullPointerException if resource is {@code null} */ public final void add(@NonNull Disposable resource) { Objects.requireNonNull(resource, "resource is null"); @@ -109,7 +109,7 @@ public final void onSubscribe(@NonNull Disposable d) { } /** - * Called once the upstream sets a Subscription on this ResourceObserver. + * Called once the upstream sets a {@link Disposable} on this {@code ResourceMaybeObserver}. * *

You can perform initialization at this moment. The default * implementation does nothing. @@ -119,10 +119,10 @@ protected void onStart() { /** * Cancels the main disposable (if any) and disposes the resources associated with - * this ResourceObserver (if any). + * this {@code ResourceMaybeObserver} (if any). * - *

This method can be called before the upstream calls onSubscribe at which - * case the main Disposable will be immediately disposed. + *

This method can be called before the upstream calls {@link #onSubscribe(Disposable)} at which + * case the main {@link Disposable} will be immediately disposed. */ @Override public final void dispose() { @@ -132,8 +132,8 @@ public final void dispose() { } /** - * Returns true if this ResourceObserver has been disposed/cancelled. - * @return true if this ResourceObserver has been disposed/cancelled + * Returns true if this {@code ResourceMaybeObserver} has been disposed/cancelled. + * @return true if this {@code ResourceMaybeObserver} has been disposed/cancelled */ @Override public final boolean isDisposed() { diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java index b6d77a0d5f..f0810f4c5e 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceObserver.java @@ -82,17 +82,17 @@ */ public abstract class ResourceObserver implements Observer, Disposable { /** The active subscription. */ - private final AtomicReference upstream = new AtomicReference(); + private final AtomicReference upstream = new AtomicReference<>(); /** The resource composite, can never be null. */ private final ListCompositeDisposable resources = new ListCompositeDisposable(); /** - * Adds a resource to this ResourceObserver. + * Adds a resource to this {@code ResourceObserver}. * * @param resource the resource to add * - * @throws NullPointerException if resource is null + * @throws NullPointerException if resource is {@code null} */ public final void add(@NonNull Disposable resource) { Objects.requireNonNull(resource, "resource is null"); @@ -107,7 +107,7 @@ public final void onSubscribe(Disposable d) { } /** - * Called once the upstream sets a Subscription on this ResourceObserver. + * Called once the upstream sets a {@link Disposable} on this {@code ResourceObserver}. * *

You can perform initialization at this moment. The default * implementation does nothing. @@ -117,10 +117,10 @@ protected void onStart() { /** * Cancels the main disposable (if any) and disposes the resources associated with - * this ResourceObserver (if any). + * this {@code ResourceObserver} (if any). * - *

This method can be called before the upstream calls onSubscribe at which - * case the main Disposable will be immediately disposed. + *

This method can be called before the upstream calls {@link #onSubscribe(Disposable)} at which + * case the main {@link Disposable} will be immediately disposed. */ @Override public final void dispose() { @@ -130,8 +130,8 @@ public final void dispose() { } /** - * Returns true if this ResourceObserver has been disposed/cancelled. - * @return true if this ResourceObserver has been disposed/cancelled + * Returns true if this {@code ResourceObserver} has been disposed/cancelled. + * @return true if this {@code ResourceObserver} has been disposed/cancelled */ @Override public final boolean isDisposed() { diff --git a/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java b/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java index af723bf789..efebfaa85c 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/ResourceSingleObserver.java @@ -77,17 +77,17 @@ */ public abstract class ResourceSingleObserver implements SingleObserver, Disposable { /** The active subscription. */ - private final AtomicReference upstream = new AtomicReference(); + private final AtomicReference upstream = new AtomicReference<>(); /** The resource composite, can never be null. */ private final ListCompositeDisposable resources = new ListCompositeDisposable(); /** - * Adds a resource to this ResourceObserver. + * Adds a resource to this {@code ResourceSingleObserver}. * * @param resource the resource to add * - * @throws NullPointerException if resource is null + * @throws NullPointerException if resource is {@code null} */ public final void add(@NonNull Disposable resource) { Objects.requireNonNull(resource, "resource is null"); @@ -102,7 +102,7 @@ public final void onSubscribe(@NonNull Disposable d) { } /** - * Called once the upstream sets a Subscription on this ResourceObserver. + * Called once the upstream sets a {@link Disposable} on this {@code ResourceSingleObserver}. * *

You can perform initialization at this moment. The default * implementation does nothing. @@ -112,10 +112,10 @@ protected void onStart() { /** * Cancels the main disposable (if any) and disposes the resources associated with - * this ResourceObserver (if any). + * this {@code ResourceSingleObserver} (if any). * - *

This method can be called before the upstream calls onSubscribe at which - * case the main Disposable will be immediately disposed. + *

This method can be called before the upstream calls {@link #onSubscribe(Disposable)} at which + * case the main {@link Disposable} will be immediately disposed. */ @Override public final void dispose() { @@ -125,8 +125,8 @@ public final void dispose() { } /** - * Returns true if this ResourceObserver has been disposed/cancelled. - * @return true if this ResourceObserver has been disposed/cancelled + * Returns true if this {@code ResourceSingleObserver} has been disposed/cancelled. + * @return true if this {@code ResourceSingleObserver} has been disposed/cancelled */ @Override public final boolean isDisposed() { diff --git a/src/main/java/io/reactivex/rxjava3/observers/SafeObserver.java b/src/main/java/io/reactivex/rxjava3/observers/SafeObserver.java index f7612378e2..c3935e79dc 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/SafeObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/SafeObserver.java @@ -21,7 +21,7 @@ import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** - * Wraps another Subscriber and ensures all onXXX methods conform the protocol + * Wraps another {@link Observer} and ensures all {@code onXXX} methods conform the protocol * (except the requirement for serialized access). * * @param the value type @@ -35,8 +35,8 @@ public final class SafeObserver implements Observer, Disposable { boolean done; /** - * Constructs a SafeObserver by wrapping the given actual Observer. - * @param downstream the actual Observer to wrap, not null (not validated) + * Constructs a {@code SafeObserver} by wrapping the given actual {@link Observer}. + * @param downstream the actual {@code Observer} to wrap, not {@code null} (not validated) */ public SafeObserver(@NonNull Observer downstream) { this.downstream = downstream; diff --git a/src/main/java/io/reactivex/rxjava3/observers/SerializedObserver.java b/src/main/java/io/reactivex/rxjava3/observers/SerializedObserver.java index a0ffe75cce..e0d32fbd7a 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/SerializedObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/SerializedObserver.java @@ -20,13 +20,14 @@ import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** - * Serializes access to the onNext, onError and onComplete methods of another Observer. + * Serializes access to the {@link Observer#onNext(Object)}, {@link Observer#onError(Throwable)} and + * {@link Observer#onComplete()} methods of another {@link Observer}. * *

Note that {@link #onSubscribe(Disposable)} is not serialized in respect of the other methods so - * make sure the {@code onSubscribe()} is called with a non-null {@code Disposable} + * make sure the {@code onSubscribe()} is called with a non-null {@link Disposable} * before any of the other methods are called. * - *

The implementation assumes that the actual Observer's methods don't throw. + *

The implementation assumes that the actual {@code Observer}'s methods don't throw. * * @param the value type */ @@ -44,19 +45,19 @@ public final class SerializedObserver implements Observer, Disposable { volatile boolean done; /** - * Construct a SerializedObserver by wrapping the given actual Observer. - * @param downstream the actual Observer, not null (not verified) + * Construct a {@code SerializedObserver} by wrapping the given actual {@link Observer}. + * @param downstream the actual {@code Observer}, not {@code null} (not verified) */ public SerializedObserver(@NonNull Observer downstream) { this(downstream, false); } /** - * Construct a SerializedObserver by wrapping the given actual Observer and + * Construct a SerializedObserver by wrapping the given actual {@link Observer} and * optionally delaying the errors till all regular values have been emitted * from the internal buffer. - * @param actual the actual Observer, not null (not verified) - * @param delayError if true, errors are emitted after regular values have been emitted + * @param actual the actual {@code Observer}, not {@code null} (not verified) + * @param delayError if {@code true}, errors are emitted after regular values have been emitted */ public SerializedObserver(@NonNull Observer actual, boolean delayError) { this.downstream = actual; @@ -100,7 +101,7 @@ public void onNext(@NonNull T t) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(QUEUE_LINK_SIZE); + q = new AppendOnlyLinkedArrayList<>(QUEUE_LINK_SIZE); queue = q; } q.add(NotificationLite.next(t)); @@ -129,7 +130,7 @@ public void onError(@NonNull Throwable t) { done = true; AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(QUEUE_LINK_SIZE); + q = new AppendOnlyLinkedArrayList<>(QUEUE_LINK_SIZE); queue = q; } Object err = NotificationLite.error(t); @@ -167,7 +168,7 @@ public void onComplete() { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(QUEUE_LINK_SIZE); + q = new AppendOnlyLinkedArrayList<>(QUEUE_LINK_SIZE); queue = q; } q.add(NotificationLite.complete()); diff --git a/src/main/java/io/reactivex/rxjava3/observers/TestObserver.java b/src/main/java/io/reactivex/rxjava3/observers/TestObserver.java index eae4fe113e..44b129fdab 100644 --- a/src/main/java/io/reactivex/rxjava3/observers/TestObserver.java +++ b/src/main/java/io/reactivex/rxjava3/observers/TestObserver.java @@ -14,19 +14,24 @@ import java.util.concurrent.atomic.AtomicReference; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; /** - * An Observer that records events and allows making assertions about them. + * An {@link Observer}, {@link MaybeObserver}, {@link SingleObserver} and + * {@link CompletableObserver} composite that can record events from + * {@link Observable}s, {@link Maybe}s, {@link Single}s and {@link Completable}s + * and allows making assertions about them. * - *

You can override the onSubscribe, onNext, onError, onComplete, onSuccess and - * cancel methods but not the others (this is by design). + *

You can override the {@link #onSubscribe(Disposable)}, {@link #onNext(Object)}, {@link #onError(Throwable)}, + * {@link #onComplete()} and {@link #onSuccess(Object)} methods but not the others (this is by design). * - *

The TestObserver implements Disposable for convenience where dispose calls cancel. + *

The {@code TestObserver} implements {@link Disposable} for convenience where dispose calls cancel. * * @param the value type + * @see io.reactivex.rxjava3.subscribers.TestSubscriber */ public class TestObserver extends BaseTestConsumer> @@ -35,25 +40,27 @@ public class TestObserver private final Observer downstream; /** Holds the current subscription if any. */ - private final AtomicReference upstream = new AtomicReference(); + private final AtomicReference upstream = new AtomicReference<>(); /** - * Constructs a non-forwarding TestObserver. + * Constructs a non-forwarding {@code TestObserver}. * @param the value type received - * @return the new TestObserver instance + * @return the new {@code TestObserver} instance */ + @NonNull public static TestObserver create() { - return new TestObserver(); + return new TestObserver<>(); } /** - * Constructs a forwarding TestObserver. + * Constructs a forwarding {@code TestObserver}. * @param the value type received - * @param delegate the actual Observer to forward events to - * @return the new TestObserver instance + * @param delegate the actual {@link Observer} to forward events to + * @return the new {@code TestObserver} instance */ - public static TestObserver create(Observer delegate) { - return new TestObserver(delegate); + @NonNull + public static TestObserver create(@NonNull Observer delegate) { + return new TestObserver<>(delegate); } /** @@ -64,15 +71,15 @@ public TestObserver() { } /** - * Constructs a forwarding TestObserver. - * @param downstream the actual Observer to forward events to + * Constructs a forwarding {@code TestObserver}. + * @param downstream the actual {@link Observer} to forward events to */ - public TestObserver(Observer downstream) { + public TestObserver(@NonNull Observer downstream) { this.downstream = downstream; } @Override - public void onSubscribe(Disposable d) { + public void onSubscribe(@NonNull Disposable d) { lastThread = Thread.currentThread(); if (d == null) { @@ -91,7 +98,7 @@ public void onSubscribe(Disposable d) { } @Override - public void onNext(T t) { + public void onNext(@NonNull T t) { if (!checkSubscriptionOnce) { checkSubscriptionOnce = true; if (upstream.get() == null) { @@ -111,7 +118,7 @@ public void onNext(T t) { } @Override - public void onError(Throwable t) { + public void onError(@NonNull Throwable t) { if (!checkSubscriptionOnce) { checkSubscriptionOnce = true; if (upstream.get() == null) { @@ -164,18 +171,19 @@ public final boolean isDisposed() { // state retrieval methods /** - * Returns true if this TestObserver received a subscription. - * @return true if this TestObserver received a subscription + * Returns true if this {@code TestObserver} received a subscription. + * @return true if this {@code TestObserver} received a subscription */ public final boolean hasSubscription() { return upstream.get() != null; } /** - * Assert that the onSubscribe method was called exactly once. - * @return this; + * Assert that the {@link #onSubscribe(Disposable)} method was called exactly once. + * @return this */ @Override + @NonNull protected final TestObserver assertSubscribed() { if (upstream.get() == null) { throw fail("Not subscribed!"); @@ -184,7 +192,7 @@ protected final TestObserver assertSubscribed() { } @Override - public void onSuccess(T value) { + public void onSuccess(@NonNull T value) { onNext(value); onComplete(); } diff --git a/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java index 9a75a1f4ca..9491773080 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java @@ -68,7 +68,7 @@ *

The {@code AsyncProcessor} honors the backpressure of the downstream {@code Subscriber}s and won't emit * its single value to a particular {@code Subscriber} until that {@code Subscriber} has requested an item. * When the {@code AsyncProcessor} is subscribed to a {@link io.reactivex.rxjava3.core.Flowable}, the processor consumes this - * {@code Flowable} in an unbounded manner (requesting `Long.MAX_VALUE`) as only the very last upstream item is + * {@code Flowable} in an unbounded manner (requesting {@link Long#MAX_VALUE}) as only the very last upstream item is * retained by it. *
*
Scheduler:
diff --git a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java index eaf47ecae3..4b85ff5ab0 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java @@ -108,7 +108,7 @@ * that returns true if any of the {@code Subscriber}s is not ready to receive {@code onNext} events. If * there are no {@code Subscriber}s to the processor, {@code offer()} always succeeds. * If the {@code BehaviorProcessor} is (optionally) subscribed to another {@code Publisher}, this upstream - * {@code Publisher} is consumed in an unbounded fashion (requesting {@code Long.MAX_VALUE}). + * {@code Publisher} is consumed in an unbounded fashion (requesting {@link Long#MAX_VALUE}). *
Scheduler:
*
{@code BehaviorProcessor} does not operate by default on a particular {@link io.reactivex.rxjava3.core.Scheduler} and * the {@code Subscriber}s get notified on the thread the respective {@code onXXX} methods were invoked.
@@ -229,7 +229,7 @@ public static BehaviorProcessor createDefault(T defaultValue) { /** * Constructs a BehaviorProcessor with the given initial value. * @param defaultValue the initial value, not null (verified) - * @throws NullPointerException if {@code defaultValue} is null + * @throws NullPointerException if {@code defaultValue} is {@code null} * @since 2.0 */ BehaviorProcessor(T defaultValue) { diff --git a/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java index 2d44f60f05..4dc9bc9371 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java @@ -71,7 +71,7 @@ *
*
Backpressure:
*
The processor does not coordinate backpressure for its subscribers and implements a weaker {@code onSubscribe} which - * calls requests Long.MAX_VALUE from the incoming Subscriptions. This makes it possible to subscribe the {@code PublishProcessor} + * calls requests {@link Long#MAX_VALUE} from the incoming Subscriptions. This makes it possible to subscribe the {@code PublishProcessor} * to multiple sources (note on serialization though) unlike the standard {@code Subscriber} contract. Child subscribers, however, are not overflown but receive an * {@link IllegalStateException} in case their requested amount is zero.
*
Scheduler:
diff --git a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java index fcd2cb913f..04de41a77f 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java @@ -66,7 +66,7 @@ *

* This {@code ReplayProcessor} respects the individual backpressure behavior of its {@code Subscriber}s but * does not coordinate their request amounts towards the upstream (because there might not be any) and - * consumes the upstream in an unbounded manner (requesting {@code Long.MAX_VALUE}). + * consumes the upstream in an unbounded manner (requesting {@link Long#MAX_VALUE}). * Note that {@code Subscriber}s receive a continuous sequence of values after they subscribed even * if an individual item gets delayed due to backpressure. * Due to concurrency requirements, a size-bounded {@code ReplayProcessor} may hold strong references to more source @@ -104,7 +104,7 @@ *

Backpressure:
*
This {@code ReplayProcessor} respects the individual backpressure behavior of its {@code Subscriber}s but * does not coordinate their request amounts towards the upstream (because there might not be any) and - * consumes the upstream in an unbounded manner (requesting {@code Long.MAX_VALUE}). + * consumes the upstream in an unbounded manner (requesting {@link Long#MAX_VALUE}). * Note that {@code Subscriber}s receive a continuous sequence of values after they subscribed even * if an individual item gets delayed due to backpressure.
*
Scheduler:
diff --git a/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java index 4a1862a95f..ee93a5ac9c 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java @@ -67,7 +67,7 @@ * {@link NullPointerException} being thrown and the processor's state is not changed. *

* Since a {@code UnicastProcessor} is a {@link io.reactivex.rxjava3.core.Flowable} as well as a {@link FlowableProcessor}, it - * honors the downstream backpressure but consumes an upstream source in an unbounded manner (requesting {@code Long.MAX_VALUE}). + * honors the downstream backpressure but consumes an upstream source in an unbounded manner (requesting {@link Long#MAX_VALUE}). *

* When this {@code UnicastProcessor} is terminated via {@link #onError(Throwable)} the current or late single {@code Subscriber} * may receive the {@code Throwable} before any available items could be emitted. To make sure an {@code onError} event is delivered @@ -91,7 +91,7 @@ *

*
Backpressure:
*
{@code UnicastProcessor} honors the downstream backpressure but consumes an upstream source - * (if any) in an unbounded manner (requesting {@code Long.MAX_VALUE}).
+ * (if any) in an unbounded manner (requesting {@link Long#MAX_VALUE}). *
Scheduler:
*
{@code UnicastProcessor} does not operate by default on a particular {@link io.reactivex.rxjava3.core.Scheduler} and * the single {@code Subscriber} gets notified on the thread the respective {@code onXXX} methods were invoked.
diff --git a/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java b/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java index 845273ff22..814b223dcf 100644 --- a/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java +++ b/src/main/java/io/reactivex/rxjava3/schedulers/Timed.java @@ -29,11 +29,11 @@ public final class Timed { final TimeUnit unit; /** - * Constructs a Timed instance with the given value and time information. + * Constructs a {@code Timed} instance with the given value and time information. * @param value the value to hold * @param time the time to hold * @param unit the time unit, not null - * @throws NullPointerException if unit is null + * @throws NullPointerException if unit is {@code null} */ public Timed(@NonNull T value, long time, @NonNull TimeUnit unit) { this.value = value; @@ -69,7 +69,7 @@ public long time() { /** * Returns the contained time value in the time unit specified. - * @param unit the time unt + * @param unit the time unit * @return the converted time */ public long time(@NonNull TimeUnit unit) { diff --git a/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java index 68efd5bf8f..b3b212eea4 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java @@ -213,7 +213,7 @@ public static BehaviorSubject createDefault(T defaultValue) { /** * Constructs a BehaviorSubject with the given initial value. * @param defaultValue the initial value, not null (verified) - * @throws NullPointerException if {@code defaultValue} is null + * @throws NullPointerException if {@code defaultValue} is {@code null} * @since 2.0 */ BehaviorSubject(T defaultValue) { diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/DefaultSubscriber.java b/src/main/java/io/reactivex/rxjava3/subscribers/DefaultSubscriber.java index 3e674a16b0..6a936eb8d9 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/DefaultSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/DefaultSubscriber.java @@ -27,7 +27,7 @@ * *

All pre-implemented final methods are thread-safe. * - *

The default {@link #onStart()} requests Long.MAX_VALUE by default. Override + *

The default {@link #onStart()} requests {@link Long#MAX_VALUE} by default. Override * the method to request a custom positive amount. * *

Note that calling {@link #request(long)} from {@link #onStart()} may trigger @@ -85,7 +85,7 @@ public final void onSubscribe(Subscription s) { } /** - * Requests from the upstream Subscription. + * Requests from the upstream {@link Subscription}. * @param n the request amount, positive */ protected final void request(long n) { @@ -96,7 +96,7 @@ protected final void request(long n) { } /** - * Cancels the upstream's Subscription. + * Cancels the upstream's {@link Subscription}. */ protected final void cancel() { Subscription s = this.upstream; diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/DisposableSubscriber.java b/src/main/java/io/reactivex/rxjava3/subscribers/DisposableSubscriber.java index 2c6c126f8d..566b0097f7 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/DisposableSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/DisposableSubscriber.java @@ -23,11 +23,11 @@ import io.reactivex.rxjava3.internal.util.EndConsumerHelper; /** - * An abstract Subscriber that allows asynchronous, external cancellation by implementing Disposable. + * An abstract Subscriber that allows asynchronous, external cancellation by implementing {@link Disposable}. * *

All pre-implemented final methods are thread-safe. * - *

The default {@link #onStart()} requests Long.MAX_VALUE by default. Override + *

The default {@link #onStart()} requests {@link Long#MAX_VALUE} by default. Override * the method to request a custom positive amount. Use the protected {@link #request(long)} * to request more items and {@link #cancel()} to cancel the sequence from within an * {@code onNext} implementation. @@ -74,7 +74,7 @@ * @param the received value type. */ public abstract class DisposableSubscriber implements FlowableSubscriber, Disposable { - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); @Override public final void onSubscribe(Subscription s) { @@ -84,18 +84,18 @@ public final void onSubscribe(Subscription s) { } /** - * Called once the single upstream Subscription is set via onSubscribe. + * Called once the single upstream {@link Subscription} is set via {@link #onSubscribe(Subscription)}. */ protected void onStart() { upstream.get().request(Long.MAX_VALUE); } /** - * Requests the specified amount from the upstream if its Subscription is set via + * Requests the specified amount from the upstream if its {@link Subscription} is set via * onSubscribe already. - *

Note that calling this method before a Subscription is set via onSubscribe - * leads to NullPointerException and meant to be called from inside onStart or - * onNext. + *

Note that calling this method before a {@link Subscription} is set via {@link #onSubscribe(Subscription)} + * leads to {@link NullPointerException} and meant to be called from inside {@link #onStart()} or + * {@link #onNext(Object)}. * @param n the request amount, positive */ protected final void request(long n) { @@ -103,8 +103,8 @@ protected final void request(long n) { } /** - * Cancels the Subscription set via onSubscribe or makes sure a - * Subscription set asynchronously (later) is cancelled immediately. + * Cancels the Subscription set via {@link #onSubscribe(Subscription)} or makes sure a + * {@link Subscription} set asynchronously (later) is cancelled immediately. *

This method is thread-safe and can be exposed as a public API. */ protected final void cancel() { diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java b/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java index 1990939311..7d9411048a 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/ResourceSubscriber.java @@ -40,7 +40,7 @@ * {@code ResourceSubscriber} and then add/remove resources to/from the {@code CompositeDisposable} * freely. * - *

The default {@link #onStart()} requests Long.MAX_VALUE by default. Override + *

The default {@link #onStart()} requests {@link Long#MAX_VALUE} by default. Override * the method to request a custom positive amount. Use the protected {@link #request(long)} * to request more items and {@link #dispose()} to cancel the sequence from within an * {@code onNext} implementation. @@ -94,7 +94,7 @@ */ public abstract class ResourceSubscriber implements FlowableSubscriber, Disposable { /** The active subscription. */ - private final AtomicReference upstream = new AtomicReference(); + private final AtomicReference upstream = new AtomicReference<>(); /** The resource composite, can never be null. */ private final ListCompositeDisposable resources = new ListCompositeDisposable(); @@ -103,11 +103,11 @@ public abstract class ResourceSubscriber implements FlowableSubscriber, Di private final AtomicLong missedRequested = new AtomicLong(); /** - * Adds a resource to this AsyncObserver. + * Adds a resource to this {@code ResourceSubscriber}. * * @param resource the resource to add * - * @throws NullPointerException if resource is null + * @throws NullPointerException if {@code resource} is {@code null} */ public final void add(Disposable resource) { Objects.requireNonNull(resource, "resource is null"); @@ -126,10 +126,10 @@ public final void onSubscribe(Subscription s) { } /** - * Called once the upstream sets a Subscription on this AsyncObserver. + * Called once the upstream sets a {@link Subscription} on this {@code ResourceSubscriber}. * *

You can perform initialization at this moment. The default - * implementation requests Long.MAX_VALUE from upstream. + * implementation requests {@link Long#MAX_VALUE} from upstream. */ protected void onStart() { request(Long.MAX_VALUE); @@ -138,7 +138,7 @@ protected void onStart() { /** * Request the specified amount of elements from upstream. * - *

This method can be called before the upstream calls onSubscribe(). + *

This method can be called before the upstream calls {@link #onSubscribe(Subscription)}. * When the subscription happens, all missed requests are requested. * * @param n the request amount, must be positive @@ -149,10 +149,10 @@ protected final void request(long n) { /** * Cancels the subscription (if any) and disposes the resources associated with - * this AsyncObserver (if any). + * this {@code ResourceSubscriber} (if any). * - *

This method can be called before the upstream calls onSubscribe at which - * case the Subscription will be immediately cancelled. + *

This method can be called before the upstream calls {@link #onSubscribe(Subscription)} at which + * case the {@link Subscription} will be immediately cancelled. */ @Override public final void dispose() { @@ -162,8 +162,8 @@ public final void dispose() { } /** - * Returns true if this AsyncObserver has been disposed/cancelled. - * @return true if this AsyncObserver has been disposed/cancelled + * Returns true if this {@code ResourceSubscriber} has been disposed/cancelled. + * @return true if this {@code ResourceSubscriber} has been disposed/cancelled */ @Override public final boolean isDisposed() { diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/SafeSubscriber.java b/src/main/java/io/reactivex/rxjava3/subscribers/SafeSubscriber.java index 096a99e623..0db19796fa 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/SafeSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/SafeSubscriber.java @@ -14,6 +14,7 @@ import org.reactivestreams.*; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.FlowableSubscriber; import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -21,7 +22,7 @@ import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** - * Wraps another Subscriber and ensures all onXXX methods conform the protocol + * Wraps another {@link Subscriber} and ensures all {@code onXXX} methods conform the protocol * (except the requirement for serialized access). * * @param the value type @@ -35,15 +36,15 @@ public final class SafeSubscriber implements FlowableSubscriber, Subscript boolean done; /** - * Constructs a SafeSubscriber by wrapping the given actual Subscriber. - * @param downstream the actual Subscriber to wrap, not null (not validated) + * Constructs a {@code SafeSubscriber} by wrapping the given actual {@link Subscriber}. + * @param downstream the actual {@code Subscriber} to wrap, not {@code null} (not validated) */ - public SafeSubscriber(Subscriber downstream) { + public SafeSubscriber(@NonNull Subscriber downstream) { this.downstream = downstream; } @Override - public void onSubscribe(Subscription s) { + public void onSubscribe(@NonNull Subscription s) { if (SubscriptionHelper.validate(this.upstream, s)) { this.upstream = s; try { @@ -65,7 +66,7 @@ public void onSubscribe(Subscription s) { } @Override - public void onNext(T t) { + public void onNext(@NonNull T t) { if (done) { return; } @@ -124,7 +125,7 @@ void onNextNoSubscription() { } @Override - public void onError(Throwable t) { + public void onError(@NonNull Throwable t) { if (done) { RxJavaPlugins.onError(t); return; diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/SerializedSubscriber.java b/src/main/java/io/reactivex/rxjava3/subscribers/SerializedSubscriber.java index 92ecbbe6d0..9698d97e21 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/SerializedSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/SerializedSubscriber.java @@ -14,19 +14,21 @@ import org.reactivestreams.*; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.FlowableSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** - * Serializes access to the onNext, onError and onComplete methods of another Subscriber. + * Serializes access to the {@link Subscriber#onNext(Object)}, {@link Subscriber#onError(Throwable)} and + * {@link Subscriber#onComplete()} methods of another {@link Subscriber}. * *

Note that {@link #onSubscribe(Subscription)} is not serialized in respect of the other methods so - * make sure the {@code onSubscribe} is called with a non-null {@code Subscription} + * make sure the {@code onSubscribe} is called with a non-{@code null} {@link Subscription} * before any of the other methods are called. * - *

The implementation assumes that the actual Subscriber's methods don't throw. + *

The implementation assumes that the actual {@code Subscriber}'s methods don't throw. * * @param the value type */ @@ -44,27 +46,27 @@ public final class SerializedSubscriber implements FlowableSubscriber, Sub volatile boolean done; /** - * Construct a SerializedSubscriber by wrapping the given actual Subscriber. - * @param downstream the actual Subscriber, not null (not verified) + * Construct a {@code SerializedSubscriber} by wrapping the given actual {@link Subscriber}. + * @param downstream the actual {@code Subscriber}, not null (not verified) */ public SerializedSubscriber(Subscriber downstream) { this(downstream, false); } /** - * Construct a SerializedSubscriber by wrapping the given actual Observer and + * Construct a {@code SerializedSubscriber} by wrapping the given actual {@link Subscriber} and * optionally delaying the errors till all regular values have been emitted * from the internal buffer. - * @param actual the actual Subscriber, not null (not verified) - * @param delayError if true, errors are emitted after regular values have been emitted + * @param actual the actual {@code Subscriber}, not {@code null} (not verified) + * @param delayError if {@code true}, errors are emitted after regular values have been emitted */ - public SerializedSubscriber(Subscriber actual, boolean delayError) { + public SerializedSubscriber(@NonNull Subscriber actual, boolean delayError) { this.downstream = actual; this.delayError = delayError; } @Override - public void onSubscribe(Subscription s) { + public void onSubscribe(@NonNull Subscription s) { if (SubscriptionHelper.validate(this.upstream, s)) { this.upstream = s; downstream.onSubscribe(this); @@ -72,7 +74,7 @@ public void onSubscribe(Subscription s) { } @Override - public void onNext(T t) { + public void onNext(@NonNull T t) { if (done) { return; } @@ -88,7 +90,7 @@ public void onNext(T t) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(QUEUE_LINK_SIZE); + q = new AppendOnlyLinkedArrayList<>(QUEUE_LINK_SIZE); queue = q; } q.add(NotificationLite.next(t)); @@ -117,7 +119,7 @@ public void onError(Throwable t) { done = true; AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(QUEUE_LINK_SIZE); + q = new AppendOnlyLinkedArrayList<>(QUEUE_LINK_SIZE); queue = q; } Object err = NotificationLite.error(t); @@ -155,7 +157,7 @@ public void onComplete() { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(QUEUE_LINK_SIZE); + q = new AppendOnlyLinkedArrayList<>(QUEUE_LINK_SIZE); queue = q; } q.add(NotificationLite.complete()); diff --git a/src/main/java/io/reactivex/rxjava3/subscribers/TestSubscriber.java b/src/main/java/io/reactivex/rxjava3/subscribers/TestSubscriber.java index 3607337b36..cbc9d10320 100644 --- a/src/main/java/io/reactivex/rxjava3/subscribers/TestSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/subscribers/TestSubscriber.java @@ -16,20 +16,19 @@ import org.reactivestreams.*; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.FlowableSubscriber; import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; import io.reactivex.rxjava3.observers.BaseTestConsumer; /** - * A subscriber that records events and allows making assertions about them. + * A {@link Subscriber} implementation that records events and allows making assertions about them. * - *

You can override the onSubscribe, onNext, onError, onComplete, request and - * cancel methods but not the others (this is by design). - * - *

The TestSubscriber implements Disposable for convenience where dispose calls cancel. + *

You can override the {@link #onSubscribe(Subscription)}, {@link #onNext(Object)}, {@link #onError(Throwable)} and + * {@link #onComplete()} methods but not the others (this is by design). * *

When calling the default request method, you are requesting on behalf of the - * wrapped actual subscriber. + * wrapped actual {@link Subscriber} if any. * * @param the value type */ @@ -49,78 +48,82 @@ public class TestSubscriber private final AtomicLong missedRequested; /** - * Creates a TestSubscriber with Long.MAX_VALUE initial request. + * Creates a {@code TestSubscriber} with {@link Long#MAX_VALUE} initial request amount. * @param the value type - * @return the new TestSubscriber instance. + * @return the new {@code TestSubscriber} instance. + * @see #create(long) */ + @NonNull public static TestSubscriber create() { - return new TestSubscriber(); + return new TestSubscriber<>(); } /** - * Creates a TestSubscriber with the given initial request. + * Creates a {@code TestSubscriber} with the given initial request amount. * @param the value type * @param initialRequested the initial requested amount - * @return the new TestSubscriber instance. + * @return the new {@code TestSubscriber} instance. */ + @NonNull public static TestSubscriber create(long initialRequested) { - return new TestSubscriber(initialRequested); + return new TestSubscriber<>(initialRequested); } /** - * Constructs a forwarding TestSubscriber. + * Constructs a forwarding {@code TestSubscriber}. * @param the value type received - * @param delegate the actual Subscriber to forward events to + * @param delegate the actual {@link Subscriber} to forward events to * @return the new TestObserver instance */ - public static TestSubscriber create(Subscriber delegate) { - return new TestSubscriber(delegate); + public static TestSubscriber create(@NonNull Subscriber delegate) { + return new TestSubscriber<>(delegate); } /** - * Constructs a non-forwarding TestSubscriber with an initial request value of Long.MAX_VALUE. + * Constructs a non-forwarding {@code TestSubscriber} with an initial request value of {@link Long#MAX_VALUE}. */ public TestSubscriber() { this(EmptySubscriber.INSTANCE, Long.MAX_VALUE); } /** - * Constructs a non-forwarding TestSubscriber with the specified initial request value. - *

The TestSubscriber doesn't validate the initialRequest value so one can + * Constructs a non-forwarding {@code TestSubscriber} with the specified initial request value. + *

The {@code TestSubscriber} doesn't validate the {@code initialRequest} amount so one can * test sources with invalid values as well. - * @param initialRequest the initial request value + * @param initialRequest the initial request amount */ public TestSubscriber(long initialRequest) { this(EmptySubscriber.INSTANCE, initialRequest); } /** - * Constructs a forwarding TestSubscriber but leaves the requesting to the wrapped subscriber. - * @param downstream the actual Subscriber to forward events to + * Constructs a forwarding {@code TestSubscriber} but leaves the requesting to the wrapped {@link Subscriber}. + * @param downstream the actual {@code Subscriber} to forward events to */ - public TestSubscriber(Subscriber downstream) { + public TestSubscriber(@NonNull Subscriber downstream) { this(downstream, Long.MAX_VALUE); } /** - * Constructs a forwarding TestSubscriber with the specified initial request value. - *

The TestSubscriber doesn't validate the initialRequest value so one can + * Constructs a forwarding {@code TestSubscriber} with the specified initial request amount + * and an actual {@link Subscriber} to forward events to. + *

The {@code TestSubscriber} doesn't validate the initialRequest value so one can * test sources with invalid values as well. - * @param actual the actual Subscriber to forward events to - * @param initialRequest the initial request value + * @param actual the actual {@code Subscriber} to forward events to + * @param initialRequest the initial request amount */ - public TestSubscriber(Subscriber actual, long initialRequest) { + public TestSubscriber(@NonNull Subscriber actual, long initialRequest) { super(); if (initialRequest < 0) { throw new IllegalArgumentException("Negative initial request not allowed"); } this.downstream = actual; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.missedRequested = new AtomicLong(initialRequest); } @Override - public void onSubscribe(Subscription s) { + public void onSubscribe(@NonNull Subscription s) { lastThread = Thread.currentThread(); if (s == null) { @@ -153,7 +156,7 @@ protected void onStart() { } @Override - public void onNext(T t) { + public void onNext(@NonNull T t) { if (!checkSubscriptionOnce) { checkSubscriptionOnce = true; if (upstream.get() == null) { @@ -172,7 +175,7 @@ public void onNext(T t) { } @Override - public void onError(Throwable t) { + public void onError(@NonNull Throwable t) { if (!checkSubscriptionOnce) { checkSubscriptionOnce = true; if (upstream.get() == null) { @@ -225,8 +228,8 @@ public final void cancel() { } /** - * Returns true if this TestSubscriber has been cancelled. - * @return true if this TestSubscriber has been cancelled + * Returns true if this {@code TestSubscriber} has been cancelled. + * @return true if this {@code TestSubscriber} has been cancelled */ public final boolean isCancelled() { return cancelled; @@ -245,8 +248,8 @@ protected final boolean isDisposed() { // state retrieval methods /** - * Returns true if this TestSubscriber received a subscription. - * @return true if this TestSubscriber received a subscription + * Returns true if this {@code TestSubscriber} received a {@link Subscription} via {@link #onSubscribe(Subscription)}. + * @return true if this {@code TestSubscriber} received a {@link Subscription} via {@link #onSubscribe(Subscription)} */ public final boolean hasSubscription() { return upstream.get() != null; @@ -255,7 +258,7 @@ public final boolean hasSubscription() { // assertion methods /** - * Assert that the onSubscribe method was called exactly once. + * Assert that the {@link #onSubscribe(Subscription)} method was called exactly once. * @return this */ @Override diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java index 32c954f42b..2a94f73b8e 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java @@ -83,7 +83,7 @@ public void cancel() { } @Test - public void requestFromChainedOperator() throws Exception { + public void requestFromChainedOperator() throws Throwable { TestSubscriber s = new TestSubscriber(10L); FlowableOperator o = new FlowableOperator() { @Override @@ -135,7 +135,7 @@ public void cancel() { } @Test - public void requestFromDecoupledOperator() throws Exception { + public void requestFromDecoupledOperator() throws Throwable { TestSubscriber s = new TestSubscriber(0L); FlowableOperator o = new FlowableOperator() { @Override @@ -188,7 +188,7 @@ public void cancel() { } @Test - public void requestFromDecoupledOperatorThatRequestsN() throws Exception { + public void requestFromDecoupledOperatorThatRequestsN() throws Throwable { TestSubscriber s = new TestSubscriber(10L); final AtomicLong innerR = new AtomicLong(); FlowableOperator o = new FlowableOperator() { diff --git a/src/test/java/io/reactivex/rxjava3/tck/BaseTck.java b/src/test/java/io/reactivex/rxjava3/tck/BaseTck.java index d14aac229f..82c25e9591 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/BaseTck.java +++ b/src/test/java/io/reactivex/rxjava3/tck/BaseTck.java @@ -51,8 +51,8 @@ public long maxElementsFromPublisher() { /** * Creates an Iterable with the specified number of elements or an infinite one if - * elements > Integer.MAX_VALUE. - * @param elements the number of elements to return, Integer.MAX_VALUE means an infinite sequence + * elements > {@link Integer#MAX_VALUE}. + * @param elements the number of elements to return, {@link Integer#MAX_VALUE} means an infinite sequence * @return the Iterable */ protected Iterable iterate(long elements) { diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index 3faefd13a5..886728709d 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -69,7 +69,7 @@ public enum TestHelper { public static final int RACE_LONG_LOOPS = 10000; /** - * Mocks a subscriber and prepares it to request Long.MAX_VALUE. + * Mocks a subscriber and prepares it to request {@link Long#MAX_VALUE}. * @param the value type * @return the mocked subscriber */ diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java index bdc1a6a1da..23297689b8 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java @@ -51,7 +51,7 @@ public class TestSubscriberEx private QueueSubscription qs; /** - * Constructs a non-forwarding TestSubscriber with an initial request value of Long.MAX_VALUE. + * Constructs a non-forwarding TestSubscriber with an initial request value of {@link Long#MAX_VALUE}. */ public TestSubscriberEx() { this(EmptySubscriber.INSTANCE, Long.MAX_VALUE); From e162d664d86639b97c214054aca346fc2ba37e33 Mon Sep 17 00:00:00 2001 From: Dmitry Volkov <42734936+dvolkovv@users.noreply.github.com> Date: Sun, 22 Dec 2019 13:39:23 +0300 Subject: [PATCH 016/665] diamond operators in unit tests, flowable package (#6787) --- .../flowable/BlockingFlowableNextTest.java | 6 +- .../BlockingFlowableToIteratorTest.java | 8 +- .../operators/flowable/FlowableAllTest.java | 10 +- .../operators/flowable/FlowableAmbTest.java | 20 ++-- .../operators/flowable/FlowableAnyTest.java | 10 +- .../flowable/FlowableBlockingTest.java | 38 ++++---- .../flowable/FlowableBufferTest.java | 89 ++++++++--------- .../operators/flowable/FlowableCacheTest.java | 40 ++++---- .../flowable/FlowableCombineLatestTest.java | 18 ++-- .../FlowableConcatDelayErrorTest.java | 4 +- .../flowable/FlowableConcatMapEagerTest.java | 12 +-- .../FlowableConcatMapSchedulerTest.java | 2 +- .../flowable/FlowableConcatMapTest.java | 4 +- .../flowable/FlowableConcatTest.java | 52 +++++----- .../FlowableConcatWithCompletableTest.java | 8 +- .../flowable/FlowableConcatWithMaybeTest.java | 10 +- .../FlowableConcatWithSingleTest.java | 8 +- .../flowable/FlowableCreateTest.java | 2 +- .../flowable/FlowableDebounceTest.java | 20 ++-- .../flowable/FlowableDefaultIfEmptyTest.java | 4 +- .../FlowableDelaySubscriptionOtherTest.java | 16 ++-- .../operators/flowable/FlowableDelayTest.java | 28 +++--- .../flowable/FlowableDematerializeTest.java | 2 +- .../flowable/FlowableDetachTest.java | 22 ++--- .../flowable/FlowableDoAfterNextTest.java | 2 +- .../FlowableDoAfterTerminateTest.java | 2 +- .../flowable/FlowableDoFinallyTest.java | 4 +- .../flowable/FlowableDoOnEachTest.java | 14 +-- .../flowable/FlowableDoOnRequestTest.java | 2 +- .../flowable/FlowableDoOnSubscribeTest.java | 2 +- .../flowable/FlowableDoOnUnsubscribeTest.java | 12 +-- .../flowable/FlowableElementAtTest.java | 4 +- .../flowable/FlowableFilterTest.java | 2 +- .../flowable/FlowableFlatMapMaybeTest.java | 2 +- .../flowable/FlowableFlatMapSingleTest.java | 2 +- .../flowable/FlowableFlatMapTest.java | 18 ++-- .../flowable/FlowableFlattenIterableTest.java | 58 +++++------ .../flowable/FlowableForEachTest.java | 4 +- .../flowable/FlowableFromArrayTest.java | 2 +- .../flowable/FlowableFromCallableTest.java | 4 +- .../flowable/FlowableFromIterableTest.java | 38 ++++---- .../flowable/FlowableFromSourceTest.java | 2 +- .../flowable/FlowableFromSupplierTest.java | 4 +- .../flowable/FlowableGroupByTest.java | 95 ++++++++++--------- .../flowable/FlowableIgnoreElementsTest.java | 8 +- .../flowable/FlowableIntervalTest.java | 2 +- .../flowable/FlowableMapNotificationTest.java | 6 +- .../operators/flowable/FlowableMapTest.java | 6 +- .../flowable/FlowableMaterializeTest.java | 16 ++-- .../flowable/FlowableMergeDelayErrorTest.java | 14 +-- .../FlowableMergeMaxConcurrentTest.java | 48 +++++----- .../operators/flowable/FlowableMergeTest.java | 68 ++++++------- .../FlowableMergeWithCompletableTest.java | 4 +- .../flowable/FlowableMergeWithMaybeTest.java | 4 +- .../flowable/FlowableMergeWithSingleTest.java | 6 +- .../flowable/FlowableObserveOnTest.java | 30 +++--- ...wableOnBackpressureBufferStrategyTest.java | 2 +- .../FlowableOnBackpressureBufferTest.java | 12 ++- .../FlowableOnBackpressureDropTest.java | 6 +- .../FlowableOnBackpressureErrorTest.java | 4 +- .../FlowableOnBackpressureLatestTest.java | 8 +- ...wableOnErrorResumeNextViaFlowableTest.java | 2 +- ...wableOnErrorResumeNextViaFunctionTest.java | 8 +- .../flowable/FlowableOnErrorReturnTest.java | 8 +- .../flowable/FlowablePublishFunctionTest.java | 12 +-- .../FlowablePublishMulticastTest.java | 38 ++++---- .../flowable/FlowablePublishTest.java | 66 ++++++------- .../flowable/FlowableRangeLongTest.java | 22 ++--- .../operators/flowable/FlowableRangeTest.java | 22 ++--- .../flowable/FlowableRefCountTest.java | 22 ++--- .../flowable/FlowableRepeatTest.java | 6 +- .../FlowableReplayEagerTruncateTest.java | 88 ++++++++--------- .../flowable/FlowableReplayTest.java | 88 ++++++++--------- .../operators/flowable/FlowableRetryTest.java | 30 +++--- .../FlowableRetryWithPredicateTest.java | 14 +-- .../flowable/FlowableScalarXMapTest.java | 24 ++--- .../operators/flowable/FlowableScanTest.java | 16 ++-- .../flowable/FlowableSequenceEqualTest.java | 4 +- .../flowable/FlowableSingleTest.java | 8 +- .../flowable/FlowableSkipLastTest.java | 2 +- .../operators/flowable/FlowableSkipTest.java | 4 +- .../flowable/FlowableSubscribeOnTest.java | 14 +-- .../flowable/FlowableSwitchIfEmptyTest.java | 8 +- .../flowable/FlowableSwitchTest.java | 18 ++-- .../flowable/FlowableTakeLastOneTest.java | 10 +- .../flowable/FlowableTakeLastTest.java | 6 +- .../flowable/FlowableTakeLastTimedTest.java | 2 +- .../operators/flowable/FlowableTakeTest.java | 12 +-- .../operators/flowable/FlowableTakeTest2.java | 2 +- .../FlowableTakeUntilPredicateTest.java | 4 +- .../flowable/FlowableTakeUntilTest.java | 8 +- .../flowable/FlowableTakeWhileTest.java | 6 +- .../flowable/FlowableTimeIntervalTest.java | 6 +- .../flowable/FlowableTimeoutTests.java | 26 ++--- .../FlowableTimeoutWithSelectorTest.java | 2 +- .../operators/flowable/FlowableTimerTest.java | 18 ++-- .../flowable/FlowableTimestampTest.java | 12 +-- .../flowable/FlowableToCompletableTest.java | 2 +- .../flowable/FlowableToFutureTest.java | 20 ++-- .../flowable/FlowableToListTest.java | 2 +- .../operators/flowable/FlowableToMapTest.java | 24 ++--- .../flowable/FlowableToMultimapTest.java | 60 ++++++------ .../flowable/FlowableToSortedListTest.java | 2 +- .../flowable/FlowableUnsubscribeOnTest.java | 8 +- .../operators/flowable/FlowableUsingTest.java | 12 +-- .../FlowableWindowWithFlowableTest.java | 32 +++---- .../flowable/FlowableWindowWithSizeTest.java | 22 ++--- ...lowableWindowWithStartEndFlowableTest.java | 14 +-- .../flowable/FlowableWindowWithTimeTest.java | 26 ++--- .../flowable/FlowableWithLatestFromTest.java | 50 +++++----- .../operators/flowable/FlowableZipTest.java | 30 +++--- 111 files changed, 948 insertions(+), 944 deletions(-) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNextTest.java index 8bc31fa85e..35c1501ae0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNextTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNextTest.java @@ -352,7 +352,7 @@ public void interrupt() { @Test public void nextObserverError() { - NextSubscriber no = new NextSubscriber(); + NextSubscriber no = new NextSubscriber<>(); List errors = TestHelper.trackPluginErrors(); try { @@ -366,7 +366,7 @@ public void nextObserverError() { @Test public void nextObserverOnNext() throws Exception { - NextSubscriber no = new NextSubscriber(); + NextSubscriber no = new NextSubscriber<>(); no.setWaiting(); no.onNext(Notification.createOnNext(1)); @@ -379,7 +379,7 @@ public void nextObserverOnNext() throws Exception { @Test public void nextObserverOnCompleteOnNext() throws Exception { - NextSubscriber no = new NextSubscriber(); + NextSubscriber no = new NextSubscriber<>(); no.setWaiting(); no.onNext(Notification.createOnComplete()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableToIteratorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableToIteratorTest.java index 20bfcf9962..64b22b1785 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableToIteratorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableToIteratorTest.java @@ -117,13 +117,13 @@ public void remove() { @Test(expected = UnsupportedOperationException.class) public void remove() { - BlockingFlowableIterator it = new BlockingFlowableIterator(128); + BlockingFlowableIterator it = new BlockingFlowableIterator<>(128); it.remove(); } @Test public void dispose() { - BlockingFlowableIterator it = new BlockingFlowableIterator(128); + BlockingFlowableIterator it = new BlockingFlowableIterator<>(128); assertFalse(it.isDisposed()); @@ -134,7 +134,7 @@ public void dispose() { @Test public void interruptWait() { - BlockingFlowableIterator it = new BlockingFlowableIterator(128); + BlockingFlowableIterator it = new BlockingFlowableIterator<>(128); try { Thread.currentThread().interrupt(); @@ -147,7 +147,7 @@ public void interruptWait() { @Test(expected = NoSuchElementException.class) public void emptyThrowsNoSuch() { - BlockingFlowableIterator it = new BlockingFlowableIterator(128); + BlockingFlowableIterator it = new BlockingFlowableIterator<>(128); it.onComplete(); it.next(); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllTest.java index 7515b66cb0..387a2017dd 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllTest.java @@ -146,7 +146,7 @@ public Publisher apply(Boolean t1) { @Test public void backpressureIfOneRequestedOneShouldBeDelivered() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Flowable.empty().all(new Predicate() { @Override @@ -164,7 +164,7 @@ public boolean test(Object t) { @Test public void predicateThrowsExceptionAndValueInCauseMessage() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); @@ -306,7 +306,7 @@ public Publisher apply(Boolean t1) { @Test public void backpressureIfNoneRequestedNoneShouldBeDeliveredFlowable() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.empty().all(new Predicate() { @Override public boolean test(Object t1) { @@ -323,7 +323,7 @@ public boolean test(Object t1) { @Test public void backpressureIfOneRequestedOneShouldBeDeliveredFlowable() { - TestSubscriberEx ts = new TestSubscriberEx(1L); + TestSubscriberEx ts = new TestSubscriberEx<>(1L); Flowable.empty().all(new Predicate() { @Override @@ -343,7 +343,7 @@ public boolean test(Object t) { @Test public void predicateThrowsExceptionAndValueInCauseMessageFlowable() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java index 72073f3150..42cba044db 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java @@ -174,7 +174,7 @@ public void amb3() { @SuppressWarnings("unchecked") @Test public void producerRequestThroughAmb() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); ts.request(3); final AtomicLong requested1 = new AtomicLong(); final AtomicLong requested2 = new AtomicLong(); @@ -225,7 +225,7 @@ public void cancel() { @Test public void backpressure() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, Flowable.bufferSize() * 2) .ambWith(Flowable.range(0, Flowable.bufferSize() * 2)) .observeOn(Schedulers.computation()) // observeOn has a backpressured RxRingBuffer @@ -254,7 +254,7 @@ public void accept(Subscription s) { //this stream emits second Flowable f2 = Flowable.just(1).doOnSubscribe(incrementer) .delay(100, TimeUnit.MILLISECONDS).subscribeOn(Schedulers.computation()); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.ambArray(f1, f2).subscribe(ts); ts.request(1); ts.awaitDone(5, TimeUnit.SECONDS); @@ -271,7 +271,7 @@ public void secondaryRequestsPropagatedToChildren() throws InterruptedException //this stream emits second Flowable f2 = Flowable.fromArray(4, 5, 6) .delay(200, TimeUnit.MILLISECONDS).subscribeOn(Schedulers.computation()); - TestSubscriber ts = new TestSubscriber(1L); + TestSubscriber ts = new TestSubscriber<>(1L); Flowable.ambArray(f1, f2).subscribe(ts); // before first emission request 20 more @@ -309,7 +309,7 @@ public void ambCancelsOthers() { PublishProcessor source2 = PublishProcessor.create(); PublishProcessor source3 = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.ambArray(source1, source2, source3).subscribe(ts); @@ -327,8 +327,8 @@ public void ambCancelsOthers() { @Test public void multipleUse() { - TestSubscriber ts1 = new TestSubscriber(); - TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); + TestSubscriber ts2 = new TestSubscriber<>(); Flowable amb = Flowable.timer(100, TimeUnit.MILLISECONDS).ambWith(Flowable.timer(200, TimeUnit.MILLISECONDS)); @@ -541,7 +541,7 @@ public void nullIterableElement() { @Test public void iteratorThrows() { - Flowable.amb(new CrashingMappedIterable>(1, 100, 100, new Function>() { + Flowable.amb(new CrashingMappedIterable<>(1, 100, 100, new Function>() { @Override public Flowable apply(Integer v) throws Exception { return Flowable.never(); @@ -550,7 +550,7 @@ public Flowable apply(Integer v) throws Exception { .to(TestHelper.testConsumer()) .assertFailureAndMessage(TestException.class, "iterator()"); - Flowable.amb(new CrashingMappedIterable>(100, 1, 100, new Function>() { + Flowable.amb(new CrashingMappedIterable<>(100, 1, 100, new Function>() { @Override public Flowable apply(Integer v) throws Exception { return Flowable.never(); @@ -559,7 +559,7 @@ public Flowable apply(Integer v) throws Exception { .to(TestHelper.testConsumer()) .assertFailureAndMessage(TestException.class, "hasNext()"); - Flowable.amb(new CrashingMappedIterable>(100, 100, 1, new Function>() { + Flowable.amb(new CrashingMappedIterable<>(100, 100, 1, new Function>() { @Override public Flowable apply(Integer v) throws Exception { return Flowable.never(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnyTest.java index 936d74f5e0..e121cbc924 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnyTest.java @@ -239,7 +239,7 @@ public Publisher apply(Boolean t1) { @Test public void backpressureIfOneRequestedOneShouldBeDelivered() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Flowable.just(1).any(new Predicate() { @Override public boolean test(Integer v) { @@ -255,7 +255,7 @@ public boolean test(Integer v) { @Test public void predicateThrowsExceptionAndValueInCauseMessage() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); Flowable.just("Boo!").any(new Predicate() { @@ -488,7 +488,7 @@ public Publisher apply(Boolean t1) { @Test public void backpressureIfNoneRequestedNoneShouldBeDeliveredFlowable() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.just(1).any(new Predicate() { @Override @@ -505,7 +505,7 @@ public boolean test(Integer t) { @Test public void backpressureIfOneRequestedOneShouldBeDeliveredFlowable() { - TestSubscriberEx ts = new TestSubscriberEx(1L); + TestSubscriberEx ts = new TestSubscriberEx<>(1L); Flowable.just(1).any(new Predicate() { @Override public boolean test(Integer v) { @@ -521,7 +521,7 @@ public boolean test(Integer v) { @Test public void predicateThrowsExceptionAndValueInCauseMessageFlowable() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); Flowable.just("Boo!").any(new Predicate() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingTest.java index cb767ef718..db560e0327 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingTest.java @@ -49,7 +49,7 @@ public void blockingFirstDefault() { @Test public void blockingSubscribeConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -65,7 +65,7 @@ public void accept(Integer v) throws Exception { @Test public void boundedBlockingSubscribeConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -81,7 +81,7 @@ public void accept(Integer v) throws Exception { @Test public void boundedBlockingSubscribeConsumerBufferExceed() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -97,7 +97,7 @@ public void accept(Integer v) throws Exception { @Test public void blockingSubscribeConsumerConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -113,7 +113,7 @@ public void accept(Integer v) throws Exception { @Test public void boundedBlockingSubscribeConsumerConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -129,7 +129,7 @@ public void accept(Integer v) throws Exception { @Test public void boundedBlockingSubscribeConsumerConsumerBufferExceed() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -145,7 +145,7 @@ public void accept(Integer v) throws Exception { @Test public void blockingSubscribeConsumerConsumerError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); TestException ex = new TestException(); @@ -165,7 +165,7 @@ public void accept(Object v) throws Exception { @Test public void boundedBlockingSubscribeConsumerConsumerError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); TestException ex = new TestException(); @@ -185,7 +185,7 @@ public void accept(Object v) throws Exception { @Test public void blockingSubscribeConsumerConsumerAction() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Consumer cons = new Consumer() { @Override @@ -208,7 +208,7 @@ public void run() throws Exception { @Test public void boundedBlockingSubscribeConsumerConsumerAction() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Consumer cons = new Consumer() { @Override @@ -233,7 +233,7 @@ public void run() throws Exception { @Test public void boundedBlockingSubscribeConsumerConsumerActionBufferExceed() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Consumer cons = new Consumer() { @Override @@ -258,7 +258,7 @@ public void run() throws Exception { @Test public void boundedBlockingSubscribeConsumerConsumerActionBufferExceedMillionItem() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Consumer cons = new Consumer() { @Override @@ -283,7 +283,7 @@ public void run() throws Exception { @Test public void blockingSubscribeObserver() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -316,7 +316,7 @@ public void onComplete() { @Test public void blockingSubscribeObserverError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); final TestException ex = new TestException(); @@ -430,7 +430,7 @@ public void subscribe(Subscriber s) { @Test public void interrupt() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Thread.currentThread().interrupt(); @@ -451,7 +451,7 @@ public void blockingSingleEmpty() { @Test public void onCompleteDelayed() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.empty().delay(100, TimeUnit.MILLISECONDS) .blockingSubscribe(ts); @@ -466,7 +466,7 @@ public void utilityClass() { @Test public void disposeUpFront() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.cancel(); Flowable.just(1).blockingSubscribe(ts); @@ -476,7 +476,7 @@ public void disposeUpFront() { @SuppressWarnings("rawtypes") @Test public void delayed() throws Exception { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final Subscriber[] s = { null }; Schedulers.single().scheduleDirect(new Runnable() { @@ -506,7 +506,7 @@ protected void subscribeActual(Subscriber subscriber) { @Test public void blockinsSubscribeCancelAsync() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final PublishProcessor pp = PublishProcessor.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java index 781de93fb0..707d98e0d4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java @@ -272,7 +272,7 @@ public void accept(List t1) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -302,7 +302,7 @@ public void bufferStopsWhenUnsubscribed1() { Flowable source = Flowable.never(); Subscriber> subscriber = TestHelper.mockSubscriber(); - TestSubscriber> ts = new TestSubscriber>(subscriber, 0L); + TestSubscriber> ts = new TestSubscriber<>(subscriber, 0L); source.buffer(100, 200, TimeUnit.MILLISECONDS, scheduler) .doOnNext(new Consumer>() { @@ -730,7 +730,7 @@ public Flowable apply(Integer t1) { @Test public void producerRequestThroughBufferWithSize1() { - TestSubscriber> ts = new TestSubscriber>(3L); + TestSubscriber> ts = new TestSubscriber<>(3L); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -761,7 +761,7 @@ public void cancel() { @Test public void producerRequestThroughBufferWithSize2() { - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -789,7 +789,7 @@ public void cancel() { @Test public void producerRequestThroughBufferWithSize3() { - TestSubscriber> ts = new TestSubscriber>(3L); + TestSubscriber> ts = new TestSubscriber<>(3L); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -818,7 +818,7 @@ public void cancel() { @Test public void producerRequestThroughBufferWithSize4() { - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -845,7 +845,7 @@ public void cancel() { @Test public void producerRequestOverflowThroughBufferWithSize1() { - TestSubscriber> ts = new TestSubscriber>(Long.MAX_VALUE >> 1); + TestSubscriber> ts = new TestSubscriber<>(Long.MAX_VALUE >> 1); final AtomicLong requested = new AtomicLong(); @@ -874,7 +874,7 @@ public void cancel() { @Test public void producerRequestOverflowThroughBufferWithSize2() { - TestSubscriber> ts = new TestSubscriber>(Long.MAX_VALUE >> 1); + TestSubscriber> ts = new TestSubscriber<>(Long.MAX_VALUE >> 1); final AtomicLong requested = new AtomicLong(); @@ -1249,7 +1249,7 @@ public void bufferBoundaryHint() { } static HashSet set(Integer... values) { - return new HashSet(Arrays.asList(values)); + return new HashSet<>(Arrays.asList(values)); } @SuppressWarnings("unchecked") @@ -1259,7 +1259,7 @@ public void bufferIntoCustomCollection() { .buffer(3, new Supplier>() { @Override public Collection get() throws Exception { - return new HashSet(); + return new HashSet<>(); } }) .test() @@ -1273,7 +1273,7 @@ public void bufferSkipIntoCustomCollection() { .buffer(3, 3, new Supplier>() { @Override public Collection get() throws Exception { - return new HashSet(); + return new HashSet<>(); } }) .test() @@ -1317,7 +1317,7 @@ public Collection get() throws Exception { if (count++ == 1) { return null; } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -1337,7 +1337,7 @@ public Collection get() throws Exception { if (count++ == 1) { return null; } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -1357,7 +1357,7 @@ public Collection get() throws Exception { if (count++ == 1) { return null; } else { - return new ArrayList(); + return new ArrayList<>(); } } }) @@ -1419,7 +1419,7 @@ public Collection get() throws Exception { if (count++ == 1) { throw new TestException(); } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -1439,7 +1439,7 @@ public Collection get() throws Exception { if (count++ == 1) { throw new TestException(); } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -1459,7 +1459,7 @@ public Collection get() throws Exception { if (count++ == 1) { throw new TestException(); } else { - return new ArrayList(); + return new ArrayList<>(); } } }) @@ -1497,7 +1497,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }) .test() @@ -1515,7 +1515,7 @@ public List get() throws Exception { if (++calls == 1) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }) .test() @@ -1533,7 +1533,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }) .test() @@ -1624,7 +1624,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }, true) .test(); @@ -2204,7 +2204,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }).test(); @@ -2226,7 +2226,7 @@ protected void subscribeActual(Subscriber subscriber) { } }; - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); Flowable b = new Flowable() { @Override protected void subscribeActual(Subscriber subscriber) { @@ -2303,10 +2303,10 @@ public void timedCancelledUpfront() { public void timedInternalState() { TestScheduler sch = new TestScheduler(); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); - BufferExactUnboundedSubscriber> sub = new BufferExactUnboundedSubscriber>( - ts, Functions.justSupplier((List)new ArrayList()), 1, TimeUnit.SECONDS, sch); + BufferExactUnboundedSubscriber> sub = new BufferExactUnboundedSubscriber<>( + ts, Functions.justSupplier((List) new ArrayList()), 1, TimeUnit.SECONDS, sch); sub.onSubscribe(new BooleanSubscription()); @@ -2322,7 +2322,7 @@ public void timedInternalState() { assertTrue(sub.isDisposed()); - sub.buffer = new ArrayList(); + sub.buffer = new ArrayList<>(); sub.enter(); sub.onComplete(); } @@ -2353,10 +2353,10 @@ public Publisher> apply(Flowable f) public void timedSkipInternalState() { TestScheduler sch = new TestScheduler(); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); - BufferSkipBoundedSubscriber> sub = new BufferSkipBoundedSubscriber>( - ts, Functions.justSupplier((List)new ArrayList()), 1, 1, TimeUnit.SECONDS, sch.createWorker()); + BufferSkipBoundedSubscriber> sub = new BufferSkipBoundedSubscriber<>( + ts, Functions.justSupplier((List) new ArrayList()), 1, 1, TimeUnit.SECONDS, sch.createWorker()); sub.onSubscribe(new BooleanSubscription()); @@ -2372,19 +2372,20 @@ public void timedSkipInternalState() { public void timedSkipCancelWhenSecondBuffer() { TestScheduler sch = new TestScheduler(); - final TestSubscriber> ts = new TestSubscriber>(); + final TestSubscriber> ts = new TestSubscriber<>(); - BufferSkipBoundedSubscriber> sub = new BufferSkipBoundedSubscriber>( + BufferSkipBoundedSubscriber> sub = new BufferSkipBoundedSubscriber<>( ts, new Supplier>() { - int calls; - @Override - public List get() throws Exception { - if (++calls == 2) { - ts.cancel(); - } - return new ArrayList(); - } - }, 1, 1, TimeUnit.SECONDS, sch.createWorker()); + int calls; + + @Override + public List get() throws Exception { + if (++calls == 2) { + ts.cancel(); + } + return new ArrayList<>(); + } + }, 1, 1, TimeUnit.SECONDS, sch.createWorker()); sub.onSubscribe(new BooleanSubscription()); @@ -2397,11 +2398,11 @@ public List get() throws Exception { public void timedSizeBufferAlreadyCleared() { TestScheduler sch = new TestScheduler(); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); BufferExactBoundedSubscriber> sub = - new BufferExactBoundedSubscriber>( - ts, Functions.justSupplier((List)new ArrayList()), + new BufferExactBoundedSubscriber<>( + ts, Functions.justSupplier((List) new ArrayList()), 1, TimeUnit.SECONDS, 1, false, sch.createWorker()) ; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCacheTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCacheTest.java index e13ef40f0c..a317055889 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCacheTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCacheTest.java @@ -35,11 +35,11 @@ public class FlowableCacheTest extends RxJavaTest { @Test public void coldReplayNoBackpressure() { - FlowableCache source = new FlowableCache(Flowable.range(0, 1000), 16); + FlowableCache source = new FlowableCache<>(Flowable.range(0, 1000), 16); assertFalse("Source is connected!", source.isConnected()); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.subscribe(ts); @@ -58,11 +58,11 @@ public void coldReplayNoBackpressure() { @Test public void coldReplayBackpressure() { - FlowableCache source = new FlowableCache(Flowable.range(0, 1000), 16); + FlowableCache source = new FlowableCache<>(Flowable.range(0, 1000), 16); assertFalse("Source is connected!", source.isConnected()); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); ts.request(10); source.subscribe(ts); @@ -145,9 +145,9 @@ public void unsubscribeSource() throws Throwable { @Test public void take() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - FlowableCache cached = new FlowableCache(Flowable.range(1, 100), 16); + FlowableCache cached = new FlowableCache<>(Flowable.range(1, 100), 16); cached.take(10).subscribe(ts); ts.assertNoErrors(); @@ -160,9 +160,9 @@ public void take() { public void async() { Flowable source = Flowable.range(1, 10000); for (int i = 0; i < 100; i++) { - TestSubscriber ts1 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); - FlowableCache cached = new FlowableCache(source, 16); + FlowableCache cached = new FlowableCache<>(source, 16); cached.observeOn(Schedulers.computation()).subscribe(ts1); @@ -171,7 +171,7 @@ public void async() { ts1.assertComplete(); assertEquals(10000, ts1.values().size()); - TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts2 = new TestSubscriber<>(); cached.observeOn(Schedulers.computation()).subscribe(ts2); ts2.awaitDone(2, TimeUnit.SECONDS); @@ -186,18 +186,18 @@ public void asyncComeAndGo() { Flowable source = Flowable.interval(1, 1, TimeUnit.MILLISECONDS) .take(1000) .subscribeOn(Schedulers.io()); - FlowableCache cached = new FlowableCache(source, 16); + FlowableCache cached = new FlowableCache<>(source, 16); Flowable output = cached.observeOn(Schedulers.computation()); - List> list = new ArrayList>(100); + List> list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); list.add(ts); output.skip(i * 10).take(10).subscribe(ts); } - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 10; i++) { expected.add((long)(i - 10)); } @@ -231,7 +231,7 @@ public void subscribe(Subscriber t) { } }); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); firehose.cache().observeOn(Schedulers.computation()).takeLast(100).subscribe(ts); ts.awaitDone(3, TimeUnit.SECONDS); @@ -247,14 +247,14 @@ public void valuesAndThenError() { .concatWith(Flowable.error(new TestException())) .cache(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.subscribe(ts); ts.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); ts.assertNotComplete(); ts.assertError(TestException.class); - TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts2 = new TestSubscriber<>(); source.subscribe(ts2); ts2.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); @@ -294,7 +294,7 @@ public void subscribeEmitRace() { cache.test(); - final TestSubscriberEx ts = new TestSubscriberEx(); + final TestSubscriberEx ts = new TestSubscriberEx<>(); Runnable r1 = new Runnable() { @Override @@ -432,8 +432,8 @@ public void subscribeSubscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final Flowable cache = Flowable.range(1, 500).cache(); - final TestSubscriberEx ts1 = new TestSubscriberEx(); - final TestSubscriberEx ts2 = new TestSubscriberEx(); + final TestSubscriberEx ts1 = new TestSubscriberEx<>(); + final TestSubscriberEx ts2 = new TestSubscriberEx<>(); Runnable r1 = new Runnable() { @Override @@ -476,7 +476,7 @@ public void subscribeCompleteRace() { cache.test(); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java index 836fd8a4c3..8ed8197138 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java @@ -437,8 +437,8 @@ public List apply(Object[] args) { }; for (int i = 1; i <= n; i++) { System.out.println("test1ToNSources: " + i + " sources"); - List> sources = new ArrayList>(); - List values = new ArrayList(); + List> sources = new ArrayList<>(); + List values = new ArrayList<>(); for (int j = 0; j < i; j++) { sources.add(Flowable.just(j)); values.add(j); @@ -468,8 +468,8 @@ public List apply(Object[] args) { }; for (int i = 1; i <= n; i++) { System.out.println("test1ToNSourcesScheduled: " + i + " sources"); - List> sources = new ArrayList>(); - List values = new ArrayList(); + List> sources = new ArrayList<>(); + List values = new ArrayList<>(); for (int j = 0; j < i; j++) { sources.add(Flowable.just(j).subscribeOn(Schedulers.io())); values.add(j); @@ -749,7 +749,7 @@ public void backpressure() { BiFunction combineLatestFunction = getConcatStringIntegerCombineLatestFunction(); int num = Flowable.bufferSize() * 4; - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.combineLatest( Flowable.just("one", "two"), Flowable.range(2, num), @@ -784,7 +784,7 @@ public void accept(Notification n) { } }).take(SIZE); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.combineLatest(timer, Flowable. never(), new BiFunction() { @Override @@ -889,7 +889,7 @@ public Integer apply(Object[] args) { public void combineMany() { int n = Flowable.bufferSize() * 3; - List> sources = new ArrayList>(); + List> sources = new ArrayList<>(); StringBuilder expected = new StringBuilder(n * 2); @@ -1066,7 +1066,7 @@ public void combineLatestArrayNSources() { for (int i = 1; i < 100; i++) { Flowable[] sources = new Flowable[i]; Arrays.fill(sources, Flowable.just(1)); - List expected = new ArrayList(i); + List expected = new ArrayList<>(i); for (int j = 1; j <= i; j++) { expected.add(1); } @@ -1207,7 +1207,7 @@ public Object apply(Object a, Object b) throws Exception { @Test public void cancelWhileSubscribing() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.combineLatest( Flowable.just(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java index e5ce3766ea..0543eb1b0f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java @@ -232,7 +232,7 @@ public void concatDelayErrorFlowable() { @Test public void concatDelayErrorFlowableError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.concatDelayError( withError(Flowable.just(withError(Flowable.just(1)), withError(Flowable.just(2))))) @@ -269,7 +269,7 @@ public void concatDelayErrorIterable() { @SuppressWarnings("unchecked") @Test public void concatDelayErrorIterableError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.concatDelayError( Arrays.asList(withError(Flowable.just(1)), withError(Flowable.just(2)))) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java index 5f41cce3dc..5c052d616c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java @@ -276,8 +276,8 @@ public Flowable apply(Integer t) { @Before public void before() { - ts = new TestSubscriber(); - tsBp = new TestSubscriber(0L); + ts = new TestSubscriber<>(); + tsBp = new TestSubscriber<>(0L); } @Test @@ -642,7 +642,7 @@ public void accept(Integer t) { @Test public void maxConcurrent5() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.range(1, 100).doOnRequest(new LongConsumer() { @Override public void accept(long reqCount) { @@ -955,7 +955,7 @@ public void run() { @Test public void mapperCancels() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).hide() .concatMapEager(new Function>() { @@ -1097,7 +1097,7 @@ public void drainCancelRaceOnEmpty() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final PublishProcessor pp = PublishProcessor.create(); - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Flowable.just(1) .concatMapEager(Functions.justFunction(pp)) @@ -1157,7 +1157,7 @@ public Flowable apply(Integer i) throws Exception { public void maxConcurrencyOf2() { List[] list = new ArrayList[100]; for (int i = 0; i < 100; i++) { - List lst = new ArrayList(); + List lst = new ArrayList<>(); list[i] = lst; for (int k = 1; k <= 10; k++) { lst.add((i) * 10 + k); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapSchedulerTest.java index fc8d665559..0cd7562199 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapSchedulerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapSchedulerTest.java @@ -366,7 +366,7 @@ public void concatMapRangeAsyncLoopIssue2876() { if (i % 1000 == 0) { System.out.println("concatMapRangeAsyncLoop > " + i); } - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(0, 1000) .concatMap(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapTest.java index 74394b122e..e815c3675e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapTest.java @@ -33,8 +33,8 @@ public class FlowableConcatMapTest extends RxJavaTest { @Test public void weakSubscriptionRequest() { - TestSubscriber ts = new TestSubscriber(0); - WeakScalarSubscription ws = new WeakScalarSubscription(1, ts); + TestSubscriber ts = new TestSubscriber<>(0); + WeakScalarSubscription ws = new WeakScalarSubscription<>(1, ts); ts.onSubscribe(ws); ws.request(0); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java index a0b14ead5a..67451d6097 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java @@ -65,7 +65,7 @@ public void concatWithList() { final Flowable odds = Flowable.fromArray(o); final Flowable even = Flowable.fromArray(e); - final List> list = new ArrayList>(); + final List> list = new ArrayList<>(); list.add(odds); list.add(even); Flowable concat = Flowable.concat(Flowable.fromIterable(list)); @@ -110,8 +110,8 @@ public void subscribe(Subscriber> subscriber) { public void simpleAsyncConcat() { Subscriber subscriber = TestHelper.mockSubscriber(); - TestObservable o1 = new TestObservable("one", "two", "three"); - TestObservable o2 = new TestObservable("four", "five", "six"); + TestObservable o1 = new TestObservable<>("one", "two", "three"); + TestObservable o2 = new TestObservable<>("four", "five", "six"); Flowable.concat(Flowable.unsafeCreate(o1), Flowable.unsafeCreate(o2)).subscribe(subscriber); @@ -150,12 +150,12 @@ public void nestedAsyncConcatLoop() throws Throwable { public void nestedAsyncConcat() throws InterruptedException { Subscriber subscriber = TestHelper.mockSubscriber(); - final TestObservable o1 = new TestObservable("one", "two", "three"); - final TestObservable o2 = new TestObservable("four", "five", "six"); - final TestObservable o3 = new TestObservable("seven", "eight", "nine"); + final TestObservable o1 = new TestObservable<>("one", "two", "three"); + final TestObservable o2 = new TestObservable<>("four", "five", "six"); + final TestObservable o3 = new TestObservable<>("seven", "eight", "nine"); final CountDownLatch allowThird = new CountDownLatch(1); - final AtomicReference parent = new AtomicReference(); + final AtomicReference parent = new AtomicReference<>(); final CountDownLatch parentHasStarted = new CountDownLatch(1); final CountDownLatch parentHasFinished = new CountDownLatch(1); @@ -284,7 +284,7 @@ public void blockedObservableOfObservables() { final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); @SuppressWarnings("unchecked") - TestObservable> observableOfObservables = new TestObservable>(callOnce, okToContinue, odds, even); + TestObservable> observableOfObservables = new TestObservable<>(callOnce, okToContinue, odds, even); Flowable concatF = Flowable.concat(Flowable.unsafeCreate(observableOfObservables)); concatF.subscribe(subscriber); try { @@ -316,14 +316,14 @@ public void blockedObservableOfObservables() { @Test public void concatConcurrentWithInfinity() { - final TestObservable w1 = new TestObservable("one", "two", "three"); + final TestObservable w1 = new TestObservable<>("one", "two", "three"); //This observable will send "hello" MAX_VALUE time. - final TestObservable w2 = new TestObservable("hello", Integer.MAX_VALUE); + final TestObservable w2 = new TestObservable<>("hello", Integer.MAX_VALUE); Subscriber subscriber = TestHelper.mockSubscriber(); @SuppressWarnings("unchecked") - TestObservable> observableOfObservables = new TestObservable>(Flowable.unsafeCreate(w1), Flowable.unsafeCreate(w2)); + TestObservable> observableOfObservables = new TestObservable<>(Flowable.unsafeCreate(w1), Flowable.unsafeCreate(w2)); Flowable concatF = Flowable.concat(Flowable.unsafeCreate(observableOfObservables)); concatF.take(50).subscribe(subscriber); @@ -351,8 +351,8 @@ public void concatNonBlockingObservables() { final CountDownLatch okToContinueW1 = new CountDownLatch(1); final CountDownLatch okToContinueW2 = new CountDownLatch(1); - final TestObservable w1 = new TestObservable(null, okToContinueW1, "one", "two", "three"); - final TestObservable w2 = new TestObservable(null, okToContinueW2, "four", "five", "six"); + final TestObservable w1 = new TestObservable<>(null, okToContinueW1, "one", "two", "three"); + final TestObservable w2 = new TestObservable<>(null, okToContinueW2, "four", "five", "six"); Subscriber subscriber = TestHelper.mockSubscriber(); @@ -402,11 +402,11 @@ public void subscribe(Subscriber> subscriber) { public void concatUnsubscribe() { final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); - final TestObservable w1 = new TestObservable("one", "two", "three"); - final TestObservable w2 = new TestObservable(callOnce, okToContinue, "four", "five", "six"); + final TestObservable w1 = new TestObservable<>("one", "two", "three"); + final TestObservable w2 = new TestObservable<>(callOnce, okToContinue, "four", "five", "six"); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber, 0L); + TestSubscriber ts = new TestSubscriber<>(subscriber, 0L); final Flowable concat = Flowable.concat(Flowable.unsafeCreate(w1), Flowable.unsafeCreate(w2)); @@ -444,14 +444,14 @@ public void concatUnsubscribe() { public void concatUnsubscribeConcurrent() { final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); - final TestObservable w1 = new TestObservable("one", "two", "three"); - final TestObservable w2 = new TestObservable(callOnce, okToContinue, "four", "five", "six"); + final TestObservable w1 = new TestObservable<>("one", "two", "three"); + final TestObservable w2 = new TestObservable<>(callOnce, okToContinue, "four", "five", "six"); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber, 0L); + TestSubscriber ts = new TestSubscriber<>(subscriber, 0L); @SuppressWarnings("unchecked") - TestObservable> observableOfObservables = new TestObservable>(Flowable.unsafeCreate(w1), Flowable.unsafeCreate(w2)); + TestObservable> observableOfObservables = new TestObservable<>(Flowable.unsafeCreate(w1), Flowable.unsafeCreate(w2)); Flowable concatF = Flowable.concat(Flowable.unsafeCreate(observableOfObservables)); concatF.subscribe(ts); @@ -630,7 +630,7 @@ public Flowable apply(Integer v) { result.subscribe(o); - List list = new ArrayList(n); + List list = new ArrayList<>(n); for (int i = 0; i < n; i++) { list.add(i); } @@ -655,7 +655,7 @@ public Flowable apply(Integer v) { result.subscribe(o); - List list = new ArrayList(n); + List list = new ArrayList<>(n); for (int i = 0; i < n / 2; i++) { list.add(i); } @@ -674,7 +674,7 @@ public void concatOuterBackpressure() { @Test public void innerBackpressureWithAlignedBoundaries() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, Flowable.bufferSize() * 2) .concatWith(Flowable.range(0, Flowable.bufferSize() * 2)) .observeOn(Schedulers.computation()) // observeOn has a backpressured RxRingBuffer @@ -693,7 +693,7 @@ public void innerBackpressureWithAlignedBoundaries() { */ @Test public void innerBackpressureWithoutAlignedBoundaries() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, (Flowable.bufferSize() * 2) + 10) .concatWith(Flowable.range(0, (Flowable.bufferSize() * 2) + 10)) .observeOn(Schedulers.computation()) // observeOn has a backpressured RxRingBuffer @@ -719,7 +719,7 @@ public void subscribe(Subscriber s) { }); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.concat(f, f).subscribe(ts); ts.awaitDone(500, TimeUnit.MILLISECONDS); ts.assertTerminated(); @@ -815,7 +815,7 @@ public void concatMapRangeAsyncLoopIssue2876() { if (i % 1000 == 0) { System.out.println("concatMapRangeAsyncLoop > " + i); } - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(0, 1000) .concatMap(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletableTest.java index a89897090a..fcf199dfd0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletableTest.java @@ -33,7 +33,7 @@ public class FlowableConcatWithCompletableTest extends RxJavaTest { @Test public void normal() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Completable.fromAction(new Action() { @@ -49,7 +49,7 @@ public void run() throws Exception { @Test public void mainError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.error(new TestException()) .concatWith(Completable.fromAction(new Action() { @@ -65,7 +65,7 @@ public void run() throws Exception { @Test public void otherError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Completable.error(new TestException())) @@ -76,7 +76,7 @@ public void otherError() { @Test public void takeMain() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Completable.fromAction(new Action() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybeTest.java index ea9e8f2330..a855c09c26 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybeTest.java @@ -27,7 +27,7 @@ public class FlowableConcatWithMaybeTest extends RxJavaTest { @Test public void normalEmpty() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Maybe.fromAction(new Action() { @@ -43,7 +43,7 @@ public void run() throws Exception { @Test public void normalNonEmpty() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Maybe.just(100)) @@ -68,7 +68,7 @@ public void backpressure() { @Test public void mainError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.error(new TestException()) .concatWith(Maybe.fromAction(new Action() { @@ -84,7 +84,7 @@ public void run() throws Exception { @Test public void otherError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Maybe.error(new TestException())) @@ -95,7 +95,7 @@ public void otherError() { @Test public void takeMain() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Maybe.fromAction(new Action() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingleTest.java index c846a2f5d5..4301746a86 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingleTest.java @@ -26,7 +26,7 @@ public class FlowableConcatWithSingleTest extends RxJavaTest { @Test public void normal() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Single.just(100)) @@ -51,7 +51,7 @@ public void backpressure() { @Test public void mainError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.error(new TestException()) .concatWith(Single.just(100)) @@ -62,7 +62,7 @@ public void mainError() { @Test public void otherError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Single.error(new TestException())) @@ -73,7 +73,7 @@ public void otherError() { @Test public void takeMain() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5) .concatWith(Single.just(100)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java index e6470fe6c9..d0c0adfb70 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreateTest.java @@ -1063,7 +1063,7 @@ public void subscribe(FlowableEmitter e) throws Exception { @Test public void emittersHasToString() { Map> emitterMap = - new HashMap>(); + new HashMap<>(); emitterMap.put(BackpressureStrategy.MISSING, FlowableCreate.MissingEmitter.class); emitterMap.put(BackpressureStrategy.ERROR, FlowableCreate.ErrorAsyncEmitter.class); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTest.java index 17ae8ef654..bfb33f0e8e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTest.java @@ -289,7 +289,7 @@ public Flowable apply(Integer t1) { @Test public void debounceWithTimeBackpressure() throws InterruptedException { TestScheduler scheduler = new TestScheduler(); - TestSubscriberEx subscriber = new TestSubscriberEx(); + TestSubscriberEx subscriber = new TestSubscriberEx<>(); Flowable.merge( Flowable.just(1), Flowable.just(2).delay(10, TimeUnit.MILLISECONDS, scheduler) @@ -305,7 +305,7 @@ public void debounceWithTimeBackpressure() throws InterruptedException { @Test public void debounceDefaultScheduler() throws Exception { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 1000).debounce(1, TimeUnit.SECONDS).subscribe(ts); @@ -330,7 +330,7 @@ public void dispose() { TestHelper.checkDisposed(PublishProcessor.create().debounce(Functions.justFunction(Flowable.never()))); - Disposable d = new FlowableDebounceTimed.DebounceEmitter(1, 1, null); + Disposable d = new FlowableDebounceTimed.DebounceEmitter<>(1, 1, null); assertFalse(d.isDisposed()); d.dispose(); @@ -426,7 +426,7 @@ public Flowable apply(Flowable f) throws Exception { @Test public void disposeInOnNext() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); BehaviorProcessor.createDefault(1) .debounce(new Function>() { @@ -444,7 +444,7 @@ public Flowable apply(Integer o) throws Exception { @Test public void disposedInOnComplete() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); new Flowable() { @Override @@ -461,7 +461,7 @@ protected void subscribeActual(Subscriber subscriber) { @Test public void emitLate() { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); TestSubscriber ts = Flowable.range(1, 2) .debounce(new Function>() { @@ -505,7 +505,7 @@ public Publisher apply(Flowable f) @Test public void timedDisposedIgnoredBySource() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); new Flowable() { @Override @@ -528,13 +528,13 @@ public void timedBadRequest() { @Test public void timedLateEmit() { - TestSubscriber ts = new TestSubscriber(); - DebounceTimedSubscriber sub = new DebounceTimedSubscriber( + TestSubscriber ts = new TestSubscriber<>(); + DebounceTimedSubscriber sub = new DebounceTimedSubscriber<>( ts, 1, TimeUnit.SECONDS, new TestScheduler().createWorker()); sub.onSubscribe(new BooleanSubscription()); - DebounceEmitter de = new DebounceEmitter(1, 50, sub); + DebounceEmitter de = new DebounceEmitter<>(1, 50, sub); de.emit(); de.emit(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefaultIfEmptyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefaultIfEmptyTest.java index ff99ca80e1..dadec04fbe 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefaultIfEmptyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDefaultIfEmptyTest.java @@ -57,7 +57,7 @@ public void defaultIfEmptyWithEmpty() { @Test public void backpressureEmpty() { - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); Flowable.empty().defaultIfEmpty(1).subscribe(ts); ts.assertNoValues(); ts.assertNotTerminated(); @@ -69,7 +69,7 @@ public void backpressureEmpty() { @Test public void backpressureNonEmpty() { - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); Flowable.just(1, 2, 3).defaultIfEmpty(1).subscribe(ts); ts.assertNoValues(); ts.assertNotTerminated(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOtherTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOtherTest.java index 9ac97e12ab..fde11b5ea1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOtherTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOtherTest.java @@ -31,7 +31,7 @@ public class FlowableDelaySubscriptionOtherTest extends RxJavaTest { public void noPrematureSubscription() { PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -64,7 +64,7 @@ public void accept(Subscription s) { public void noMultipleSubscriptions() { PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -98,7 +98,7 @@ public void accept(Subscription s) { public void completeTriggersSubscription() { PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -131,7 +131,7 @@ public void accept(Subscription s) { public void noPrematureSubscriptionToError() { PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -164,7 +164,7 @@ public void accept(Subscription s) { public void noSubscriptionIfOtherErrors() { PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -198,7 +198,7 @@ public void backpressurePassesThrough() { PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); final AtomicInteger subscribed = new AtomicInteger(); @@ -249,7 +249,7 @@ public void unsubscriptionPropagatesBeforeSubscribe() { PublishProcessor source = PublishProcessor.create(); PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.delaySubscription(other).subscribe(ts); @@ -267,7 +267,7 @@ public void unsubscriptionPropagatesAfterSubscribe() { PublishProcessor source = PublishProcessor.create(); PublishProcessor other = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.delaySubscription(other).subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelayTest.java index c5167f4fcd..79740ee44c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelayTest.java @@ -218,7 +218,7 @@ public void delaySubscriptionCancelBeforeTime() { Flowable result = Flowable.just(1, 2, 3).delaySubscription(100, TimeUnit.MILLISECONDS, scheduler); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); result.subscribe(ts); ts.cancel(); @@ -232,7 +232,7 @@ public void delaySubscriptionCancelBeforeTime() { @Test public void delayWithFlowableNormal1() { PublishProcessor source = PublishProcessor.create(); - final List> delays = new ArrayList>(); + final List> delays = new ArrayList<>(); final int n = 10; for (int i = 0; i < n; i++) { PublishProcessor delay = PublishProcessor.create(); @@ -577,7 +577,7 @@ public void delayWithFlowableReorder() { int n = 3; PublishProcessor source = PublishProcessor.create(); - final List> subjects = new ArrayList>(); + final List> subjects = new ArrayList<>(); for (int i = 0; i < n; i++) { subjects.add(PublishProcessor. create()); } @@ -625,7 +625,7 @@ public void accept(Notification t1) { } }); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); delayed.subscribe(ts); // all will be delivered after 500ms since range does not delay between them scheduler.advanceTimeBy(500L, TimeUnit.MILLISECONDS); @@ -634,7 +634,7 @@ public void accept(Notification t1) { @Test public void backpressureWithTimedDelay() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, Flowable.bufferSize() * 2) .delay(100, TimeUnit.MILLISECONDS) .observeOn(Schedulers.computation()) @@ -662,7 +662,7 @@ public Integer apply(Integer t) { @Test public void backpressureWithSubscriptionTimedDelay() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, Flowable.bufferSize() * 2) .delaySubscription(100, TimeUnit.MILLISECONDS) .delay(100, TimeUnit.MILLISECONDS) @@ -691,7 +691,7 @@ public Integer apply(Integer t) { @Test public void backpressureWithSelectorDelay() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, Flowable.bufferSize() * 2) .delay(new Function>() { @@ -726,7 +726,7 @@ public Integer apply(Integer t) { @Test public void backpressureWithSelectorDelayAndSubscriptionDelay() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, Flowable.bufferSize() * 2) .delay(Flowable.defer(new Supplier>() { @@ -771,7 +771,7 @@ public void errorRunsBeforeOnNext() { PublishProcessor pp = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); pp.delay(1, TimeUnit.SECONDS, test).subscribe(ts); @@ -794,7 +794,7 @@ public void delaySupplierSimple() { Flowable source = Flowable.range(1, 5); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.delaySubscription(Flowable.defer(new Supplier>() { @Override @@ -820,7 +820,7 @@ public void delaySupplierCompletes() { Flowable source = Flowable.range(1, 5); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.delaySubscription(Flowable.defer(new Supplier>() { @Override @@ -847,7 +847,7 @@ public void delaySupplierErrors() { Flowable source = Flowable.range(1, 5); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.delaySubscription(Flowable.defer(new Supplier>() { @Override @@ -904,7 +904,7 @@ public void delaySubscriptionDisposeBeforeTime() { Flowable result = Flowable.just(1, 2, 3).delaySubscription(100, TimeUnit.MILLISECONDS, scheduler); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); result.subscribe(ts); ts.cancel(); @@ -918,7 +918,7 @@ public void delaySubscriptionDisposeBeforeTime() { @Test public void onErrorCalledOnScheduler() throws Exception { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference thread = new AtomicReference(); + final AtomicReference thread = new AtomicReference<>(); Flowable.error(new Exception()) .delay(0, TimeUnit.MILLISECONDS, Schedulers.newThread()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerializeTest.java index 75a05cbd23..d4578db529 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerializeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerializeTest.java @@ -141,7 +141,7 @@ public void completePassThru() { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriberEx ts = new TestSubscriberEx(subscriber); + TestSubscriberEx ts = new TestSubscriberEx<>(subscriber); dematerialize.subscribe(ts); System.out.println(ts.errors()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetachTest.java index c328565218..3c87bfeb9a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetachTest.java @@ -35,9 +35,9 @@ public class FlowableDetachTest extends RxJavaTest { public void just() throws Exception { o = new Object(); - WeakReference wr = new WeakReference(o); + WeakReference wr = new WeakReference<>(o); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(o).count().toFlowable().onTerminateDetach().subscribe(ts); @@ -56,7 +56,7 @@ public void just() throws Exception { @Test public void error() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.error(new TestException()).onTerminateDetach().subscribe(ts); @@ -67,7 +67,7 @@ public void error() { @Test public void empty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.empty().onTerminateDetach().subscribe(ts); @@ -78,7 +78,7 @@ public void empty() { @Test public void range() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 1000).onTerminateDetach().subscribe(ts); @@ -91,9 +91,9 @@ public void range() { public void backpressured() throws Exception { o = new Object(); - WeakReference wr = new WeakReference(o); + WeakReference wr = new WeakReference<>(o); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.just(o).count().toFlowable().onTerminateDetach().subscribe(ts); @@ -117,9 +117,9 @@ public void backpressured() throws Exception { public void justUnsubscribed() throws Exception { o = new Object(); - WeakReference wr = new WeakReference(o); + WeakReference wr = new WeakReference<>(o); - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.just(o).count().toFlowable().onTerminateDetach().subscribe(ts); @@ -135,9 +135,9 @@ public void justUnsubscribed() throws Exception { @Test public void deferredUpstreamProducer() { - final AtomicReference> subscriber = new AtomicReference>(); + final AtomicReference> subscriber = new AtomicReference<>(); - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.unsafeCreate(new Publisher() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNextTest.java index 51b27abad1..47f0d74e0b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNextTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNextTest.java @@ -30,7 +30,7 @@ public class FlowableDoAfterNextTest extends RxJavaTest { - final List values = new ArrayList(); + final List values = new ArrayList<>(); final Consumer afterNext = new Consumer() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterTerminateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterTerminateTest.java index 210ed3879b..cee677cf41 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterTerminateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterTerminateTest.java @@ -91,7 +91,7 @@ public void ifFinallyActionThrowsExceptionShouldNotBeSwallowedAndActionShouldBeC Action finallyAction = Mockito.mock(Action.class); doThrow(new IllegalStateException()).when(finallyAction).run(); - TestSubscriber testSubscriber = new TestSubscriber(); + TestSubscriber testSubscriber = new TestSubscriber<>(); Flowable .just("value") diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinallyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinallyTest.java index 7662162d22..8249383dca 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinallyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinallyTest.java @@ -440,7 +440,7 @@ public void onComplete() { @Test public void eventOrdering() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.error(new TestException()) .doOnCancel(new Action() { @@ -480,7 +480,7 @@ public void run() throws Exception { @Test public void eventOrdering2() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.just(1) .doOnCancel(new Action() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEachTest.java index deea77ea70..fc346086b6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEachTest.java @@ -173,7 +173,7 @@ public void accept(List booleans) { @Test public void onErrorThrows() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.error(new TestException()) .doOnError(new Consumer() { @@ -691,7 +691,7 @@ public void run() throws Exception { @Test public void dispose() { - TestHelper.checkDisposed(Flowable.just(1).doOnEach(new TestSubscriber())); + TestHelper.checkDisposed(Flowable.just(1).doOnEach(new TestSubscriber<>())); } @Test @@ -699,7 +699,7 @@ public void doubleOnSubscribe() { TestHelper.checkDoubleOnSubscribeFlowable(new Function, Flowable>() { @Override public Flowable apply(Flowable f) throws Exception { - return f.doOnEach(new TestSubscriber()); + return f.doOnEach(new TestSubscriber<>()); } }); } @@ -736,7 +736,7 @@ public void doOnNextDoOnErrorCombinedFused() { .compose(new FlowableTransformer() { @Override public Publisher apply(Flowable v) { - return new FlowableDoOnEach(v, + return new FlowableDoOnEach<>(v, new Consumer() { @Override public void accept(Integer v) throws Exception { @@ -752,7 +752,7 @@ public void accept(Throwable e) throws Exception { Functions.EMPTY_ACTION , Functions.EMPTY_ACTION - ); + ); } }) .publish(); @@ -866,7 +866,7 @@ public void doOnNextDoOnErrorCombinedFusedConditional() { .compose(new FlowableTransformer() { @Override public Publisher apply(Flowable v) { - return new FlowableDoOnEach(v, + return new FlowableDoOnEach<>(v, new Consumer() { @Override public void accept(Integer v) throws Exception { @@ -882,7 +882,7 @@ public void accept(Throwable e) throws Exception { Functions.EMPTY_ACTION , Functions.EMPTY_ACTION - ); + ); } }) .filter(Functions.alwaysTrue()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnRequestTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnRequestTest.java index e4eb8543f6..b07470a398 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnRequestTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnRequestTest.java @@ -51,7 +51,7 @@ public void accept(long n) { @Test public void doRequest() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.range(1, 5) // .doOnRequest(new LongConsumer() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnSubscribeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnSubscribeTest.java index f300d0427f..c9ea4359d1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnSubscribeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnSubscribeTest.java @@ -66,7 +66,7 @@ public void doOnUnSubscribeWorksWithRefCount() throws Exception { final AtomicInteger onSubscribed = new AtomicInteger(); final AtomicInteger countBefore = new AtomicInteger(); final AtomicInteger countAfter = new AtomicInteger(); - final AtomicReference> sref = new AtomicReference>(); + final AtomicReference> sref = new AtomicReference<>(); Flowable f = Flowable.unsafeCreate(new Publisher() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java index 0a37faae1e..eab0324ba8 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnUnsubscribeTest.java @@ -67,11 +67,11 @@ public void run() { } }); - List subscriptions = new ArrayList(); - List> subscribers = new ArrayList>(); + List subscriptions = new ArrayList<>(); + List> subscribers = new ArrayList<>(); for (int i = 0; i < subCount; ++i) { - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); subscriptions.add(Disposable.fromSubscription(subscriber)); longs.subscribe(subscriber); subscribers.add(subscriber); @@ -128,11 +128,11 @@ public void run() { .publish() .refCount(); - List subscriptions = new ArrayList(); - List> subscribers = new ArrayList>(); + List subscriptions = new ArrayList<>(); + List> subscribers = new ArrayList<>(); for (int i = 0; i < subCount; ++i) { - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); longs.subscribe(subscriber); subscriptions.add(Disposable.fromSubscription(subscriber)); subscribers.add(subscriber); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java index 596e7b3867..56297cbe05 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtTest.java @@ -72,7 +72,7 @@ public void elementAt() { @Test public void elementAtConstrainsUpstreamRequests() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.fromArray(1, 2, 3, 4) .doOnRequest(new LongConsumer() { @Override @@ -88,7 +88,7 @@ public void accept(long n) throws Throwable { @Test public void elementAtWithDefaultConstrainsUpstreamRequests() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.fromArray(1, 2, 3, 4) .doOnRequest(new LongConsumer() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java index 38e94571c2..86ad5aa874 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java @@ -158,7 +158,7 @@ public void functionCrashUnsubscribes() { PublishProcessor pp = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); pp.filter(new Predicate() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java index d425ce8372..8faad86fef 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybeTest.java @@ -523,7 +523,7 @@ public MaybeSource apply(PublishProcessor v) throws Exception @Test public void disposeInner() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).flatMapMaybe(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java index 2d3699e8e2..1a0cd3854f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingleTest.java @@ -408,7 +408,7 @@ public SingleSource apply(PublishProcessor v) throws Exception @Test public void disposeInner() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).flatMapSingle(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapTest.java index 483d92062a..068f652db0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapTest.java @@ -343,13 +343,13 @@ public Flowable apply(Integer t1) { } }, m); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); - Set expected = new HashSet(Arrays.asList( + Set expected = new HashSet<>(Arrays.asList( 10, 11, 20, 21, 30, 31, 40, 41, 50, 51, 60, 61, 70, 71, 80, 81, 90, 91, 100, 101 )); Assert.assertEquals(expected.size(), ts.values().size()); @@ -374,13 +374,13 @@ public Integer apply(Integer t1, Integer t2) { } }, m); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); - Set expected = new HashSet(Arrays.asList( + Set expected = new HashSet<>(Arrays.asList( 1010, 1011, 2020, 2021, 3030, 3031, 4040, 4041, 5050, 5051, 6060, 6061, 7070, 7071, 8080, 8081, 9090, 9091, 10100, 10101 )); @@ -420,7 +420,7 @@ public void flatMapTransformsMaxConcurrentNormal() { Flowable source = Flowable.fromIterable(Arrays.asList(10, 20, 30)); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriberEx ts = new TestSubscriberEx(subscriber); + TestSubscriberEx ts = new TestSubscriberEx<>(subscriber); Function> just = just(onNext); Function> just2 = just(onError); @@ -447,7 +447,7 @@ public void flatMapRangeMixedAsyncLoop() { if (i % 10 == 0) { System.out.println("flatMapRangeAsyncLoop > " + i); } - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(0, 1000) .flatMap(new Function>() { final Random rnd = new Random(); @@ -471,7 +471,7 @@ public Flowable apply(Integer t) { ts.assertNoErrors(); List list = ts.values(); if (list.size() < 1000) { - Set set = new HashSet(list); + Set set = new HashSet<>(list); for (int j = 0; j < 1000; j++) { if (!set.contains(j)) { System.out.println(j + " missing"); @@ -485,7 +485,7 @@ public Flowable apply(Integer t) { @Test public void flatMapIntPassthruAsync() { for (int i = 0; i < 1000; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 1000).flatMap(new Function>() { @Override @@ -504,7 +504,7 @@ public Flowable apply(Integer t) { @Test public void flatMapTwoNestedSync() { for (final int n : new int[] { 1, 1000, 1000000 }) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2).flatMap(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterableTest.java index e9e25c8af2..c65cf4b5b4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterableTest.java @@ -40,7 +40,7 @@ public class FlowableFlattenIterableTest extends RxJavaTest { @Test public void normal0() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 2) .reduce(new BiFunction() { @@ -72,7 +72,7 @@ public Iterable apply(Integer v) { @Test public void normal() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5).concatMapIterable(mapper) .subscribe(ts); @@ -84,7 +84,7 @@ public void normal() { @Test public void normalViaFlatMap() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5).flatMapIterable(mapper) .subscribe(ts); @@ -96,7 +96,7 @@ public void normalViaFlatMap() { @Test public void normalBackpressured() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.range(1, 5).concatMapIterable(mapper) .subscribe(ts); @@ -126,7 +126,7 @@ public void normalBackpressured() { @Test public void longRunning() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); int n = 1000 * 1000; @@ -140,7 +140,7 @@ public void longRunning() { @Test public void asIntermediate() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); int n = 1000 * 1000; @@ -159,7 +159,7 @@ public Flowable apply(Integer v) { @Test public void just() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).concatMapIterable(mapper) .subscribe(ts); @@ -171,7 +171,7 @@ public void just() { @Test public void justHidden() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).hide().concatMapIterable(mapper) .subscribe(ts); @@ -183,7 +183,7 @@ public void justHidden() { @Test public void empty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.empty().concatMapIterable(mapper) .subscribe(ts); @@ -195,7 +195,7 @@ public void empty() { @Test public void error() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).concatWith(Flowable.error(new TestException())) .concatMapIterable(mapper) @@ -208,7 +208,7 @@ public void error() { @Test public void iteratorHasNextThrowsImmediately() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Iterable it = new Iterable() { @Override @@ -248,7 +248,7 @@ public Iterable apply(Integer v) { @Test public void iteratorHasNextThrowsImmediatelyJust() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Iterable it = new Iterable() { @Override @@ -288,7 +288,7 @@ public Iterable apply(Integer v) { @Test public void iteratorHasNextThrowsSecondCall() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Iterable it = new Iterable() { @Override @@ -332,7 +332,7 @@ public Iterable apply(Integer v) { @Test public void iteratorNextThrows() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Iterable it = new Iterable() { @Override @@ -372,7 +372,7 @@ public Iterable apply(Integer v) { @Test public void iteratorNextThrowsAndUnsubscribes() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Iterable it = new Iterable() { @Override @@ -418,7 +418,7 @@ public Iterable apply(Integer v) { @Test public void mixture() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, 1000) .concatMapIterable(new Function>() { @@ -436,7 +436,7 @@ public Iterable apply(Integer v) { @Test public void emptyInnerThenSingleBackpressured() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Flowable.range(1, 2) .concatMapIterable(new Function>() { @@ -454,7 +454,7 @@ public Iterable apply(Integer v) { @Test public void manyEmptyInnerThenSingleBackpressured() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Flowable.range(1, 1000) .concatMapIterable(new Function>() { @@ -472,7 +472,7 @@ public Iterable apply(Integer v) { @Test public void hasNextIsNotCalledAfterChildUnsubscribedOnNext() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger counter = new AtomicInteger(); @@ -523,7 +523,7 @@ public Iterable apply(Integer v) { @Test public void normalPrefetchViaFlatMap() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5).flatMapIterable(mapper, 2) .subscribe(ts); @@ -827,7 +827,7 @@ public void oneByOne() { @Test public void cancelAfterHasNext() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 3).hide() .flatMapIterable(new Function>() { @@ -974,8 +974,8 @@ public Publisher apply(Flowable f) @Test public void upstreamFusionRejected() { - TestSubscriber ts = new TestSubscriber(); - FlattenIterableSubscriber f = new FlattenIterableSubscriber(ts, + TestSubscriber ts = new TestSubscriber<>(); + FlattenIterableSubscriber f = new FlattenIterableSubscriber<>(ts, Functions.justFunction(Collections.emptyList()), 128); final AtomicLong requested = new AtomicLong(); @@ -1031,8 +1031,8 @@ public void cancel() { public void onErrorLate() { List errors = TestHelper.trackPluginErrors(); try { - TestSubscriberEx ts = new TestSubscriberEx(); - FlattenIterableSubscriber f = new FlattenIterableSubscriber(ts, + TestSubscriberEx ts = new TestSubscriberEx<>(); + FlattenIterableSubscriber f = new FlattenIterableSubscriber<>(ts, Functions.justFunction(Collections.emptyList()), 128); f.onSubscribe(new BooleanSubscription()); @@ -1059,8 +1059,8 @@ public void badRequest() { @Test public void fusedCurrentIteratorEmpty() throws Throwable { - TestSubscriber ts = new TestSubscriber(0); - FlattenIterableSubscriber f = new FlattenIterableSubscriber(ts, + TestSubscriber ts = new TestSubscriber<>(0); + FlattenIterableSubscriber f = new FlattenIterableSubscriber<>(ts, Functions.justFunction(Arrays.asList(1, 2)), 128); f.onSubscribe(new BooleanSubscription()); @@ -1080,8 +1080,8 @@ public void fusedCurrentIteratorEmpty() throws Throwable { @Test public void fusionRequestedState() throws Exception { - TestSubscriber ts = new TestSubscriber(0); - FlattenIterableSubscriber f = new FlattenIterableSubscriber(ts, + TestSubscriber ts = new TestSubscriber<>(0); + FlattenIterableSubscriber f = new FlattenIterableSubscriber<>(ts, Functions.justFunction(Arrays.asList(1, 2)), 128); f.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableForEachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableForEachTest.java index c643c0652b..d9ff63257c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableForEachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableForEachTest.java @@ -27,7 +27,7 @@ public class FlowableForEachTest extends RxJavaTest { @Test public void forEachWile() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5) .doOnNext(new Consumer() { @@ -48,7 +48,7 @@ public boolean test(Integer v) throws Exception { @Test public void forEachWileWithError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 5).concatWith(Flowable.error(new TestException())) .doOnNext(new Consumer() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArrayTest.java index 62607299d6..d4dda81754 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArrayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArrayTest.java @@ -37,7 +37,7 @@ Flowable create(int n) { @Test public void simple() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); create(1000).subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallableTest.java index cc4b1b64bb..f8f3ed670c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallableTest.java @@ -121,7 +121,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber outer = new TestSubscriber(subscriber); + TestSubscriber outer = new TestSubscriber<>(subscriber); fromCallableFlowable .subscribeOn(Schedulers.computation()) @@ -248,7 +248,7 @@ public Object call() throws Exception { public void undeliverableUponCancellation() throws Exception { List errors = TestHelper.trackPluginErrors(); try { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.fromCallable(new Callable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterableTest.java index 1b21baf610..6385d58d95 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromIterableTest.java @@ -119,13 +119,13 @@ public void observableFromIterable() { @Test public void backpressureViaRequest() { - ArrayList list = new ArrayList(Flowable.bufferSize()); + ArrayList list = new ArrayList<>(Flowable.bufferSize()); for (int i = 1; i <= Flowable.bufferSize() + 1; i++) { list.add(i); } Flowable f = Flowable.fromIterable(list); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.assertNoValues(); ts.request(1); @@ -145,7 +145,7 @@ public void backpressureViaRequest() { public void noBackpressure() { Flowable f = Flowable.fromIterable(Arrays.asList(1, 2, 3, 4, 5)); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.assertNoValues(); ts.request(Long.MAX_VALUE); // infinite @@ -161,7 +161,7 @@ public void subscribeMultipleTimes() { Flowable f = Flowable.fromIterable(Arrays.asList(1, 2, 3)); for (int i = 0; i < 10; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); f.subscribe(ts); @@ -333,7 +333,7 @@ public Iterator iterator() { } }; - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.fromIterable(it).subscribe(ts); @@ -366,7 +366,7 @@ public void remove() { } }; - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.fromIterable(it).subscribe(ts); @@ -403,7 +403,7 @@ public void remove() { } }; - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.fromIterable(it).subscribe(ts); @@ -440,7 +440,7 @@ public void remove() { } }; - TestSubscriber ts = new TestSubscriber(5); + TestSubscriber ts = new TestSubscriber<>(5); Flowable.fromIterable(it).subscribe(ts); @@ -473,7 +473,7 @@ public void remove() { } }; - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.fromIterable(it).subscribe(ts); @@ -506,7 +506,7 @@ public void remove() { } }; - TestSubscriber ts = new TestSubscriber(5); + TestSubscriber ts = new TestSubscriber<>(5); Flowable.fromIterable(it).subscribe(ts); @@ -539,7 +539,7 @@ public void remove() { } }; - TestSubscriber ts = new TestSubscriber(5); + TestSubscriber ts = new TestSubscriber<>(5); ts.cancel(); Flowable.fromIterable(it).subscribe(ts); @@ -552,7 +552,7 @@ public void remove() { @Test public void fusionWithConcatMap() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.fromIterable(Arrays.asList(1, 2, 3, 4)).concatMap( new Function>() { @@ -724,7 +724,7 @@ public void normalConditionalLong2() { @Test public void requestRaceConditional() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Runnable r = new Runnable() { @Override @@ -744,7 +744,7 @@ public void run() { @Test public void requestRaceConditional2() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Runnable r = new Runnable() { @Override @@ -764,7 +764,7 @@ public void run() { @Test public void requestCancelConditionalRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Runnable r1 = new Runnable() { @Override @@ -791,7 +791,7 @@ public void run() { @Test public void requestCancelConditionalRace2() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Runnable r1 = new Runnable() { @Override @@ -818,7 +818,7 @@ public void run() { @Test public void requestCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Runnable r1 = new Runnable() { @Override @@ -844,7 +844,7 @@ public void run() { @Test public void requestCancelRace2() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Runnable r1 = new Runnable() { @Override @@ -933,7 +933,7 @@ public void hasNext2Throws() { @Test public void hasNextCancels() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.fromIterable(new Iterable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSourceTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSourceTest.java index 76fff6cb82..fefbf1533e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSourceTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSourceTest.java @@ -38,7 +38,7 @@ public class FlowableFromSourceTest extends RxJavaTest { public void before() { source = new PublishAsyncEmitter(); sourceNoCancel = new PublishAsyncEmitterNoCancel(); - ts = new TestSubscriberEx(0L); + ts = new TestSubscriberEx<>(0L); } @Test diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplierTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplierTest.java index c44954bce3..5753e8ff66 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplierTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplierTest.java @@ -121,7 +121,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber outer = new TestSubscriber(subscriber); + TestSubscriber outer = new TestSubscriber<>(subscriber); fromSupplierFlowable .subscribeOn(Schedulers.computation()) @@ -248,7 +248,7 @@ public Object get() throws Exception { public void undeliverableUponCancellation() throws Exception { List errors = TestHelper.trackPluginErrors(); try { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.fromSupplier(new Supplier() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java index 5dae57500f..a14372b959 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java @@ -111,7 +111,7 @@ public void error() { final AtomicInteger groupCounter = new AtomicInteger(); final AtomicInteger eventCounter = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); grouped.flatMap(new Function, Flowable>() { @@ -156,13 +156,13 @@ public void onNext(String v) { private static Map> toMap(Flowable> flowable) { - final ConcurrentHashMap> result = new ConcurrentHashMap>(); + final ConcurrentHashMap> result = new ConcurrentHashMap<>(); flowable.doOnNext(new Consumer>() { @Override public void accept(final GroupedFlowable f) { - result.put(f.getKey(), new ConcurrentLinkedQueue()); + result.put(f.getKey(), new ConcurrentLinkedQueue<>()); f.subscribe(new Consumer() { @Override @@ -605,7 +605,7 @@ public void accept(String s) { @Test public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsAndThenComplete() throws InterruptedException { final CountDownLatch first = new CountDownLatch(2); // there are two groups to first complete - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Flowable.unsafeCreate(new Publisher() { @Override @@ -684,7 +684,7 @@ public void accept(String s) { public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsWhichThenSubscribesOnAndDelaysAndThenCompletes() throws InterruptedException { System.err.println("----------------------------------------------------------------------------------------------"); final CountDownLatch first = new CountDownLatch(2); // there are two groups to first complete - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Flowable.unsafeCreate(new Publisher() { @Override @@ -776,7 +776,7 @@ public void accept(String s) { @Test public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsWhichThenObservesOnAndDelaysAndThenCompletes() throws InterruptedException { final CountDownLatch first = new CountDownLatch(2); // there are two groups to first complete - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Flowable.unsafeCreate(new Publisher() { @Override @@ -853,7 +853,7 @@ public void accept(String s) { @Test public void groupsWithNestedSubscribeOn() throws InterruptedException { - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Flowable.unsafeCreate(new Publisher() { @Override @@ -910,7 +910,7 @@ public void accept(String s) { @Test public void groupsWithNestedObserveOn() throws InterruptedException { - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Flowable.unsafeCreate(new Publisher() { @Override @@ -1035,7 +1035,7 @@ public Boolean apply(Integer n) { @Test public void groupByBackpressure() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 4000) .groupBy(IS_EVEN2) @@ -1162,7 +1162,7 @@ public String apply(String v) { } }); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); m.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); System.out.println("ts .get " + ts.values()); @@ -1178,7 +1178,7 @@ public void keySelectorThrows() { Flowable m = source.groupBy(fail(0), dbl).flatMap(FLATTEN_INTEGER); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); m.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); assertEquals(1, ts.errors().size()); @@ -1190,7 +1190,7 @@ public void valueSelectorThrows() { Flowable source = Flowable.just(0, 1, 2, 3, 4, 5, 6); Flowable m = source.groupBy(identity, fail(0)).flatMap(FLATTEN_INTEGER); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); m.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); assertEquals(1, ts.errors().size()); @@ -1204,7 +1204,7 @@ public void innerEscapeCompleted() { Flowable m = source.groupBy(identity, dbl).flatMap(FLATTEN_INTEGER); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); m.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -1218,7 +1218,7 @@ public void innerEscapeCompleted() { public void exceptionIfSubscribeToChildMoreThanOnce() { Flowable source = Flowable.just(0); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); Flowable> m = source.groupBy(identity, dbl); @@ -1247,7 +1247,7 @@ public void error2() { Flowable m = source.groupBy(identity, dbl).flatMap(FLATTEN_INTEGER); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); m.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); assertEquals(1, ts.errors().size()); @@ -1256,7 +1256,7 @@ public void error2() { @Test public void groupByBackpressure3() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 4000).groupBy(IS_EVEN2).flatMap(new Function, Flowable>() { @@ -1313,7 +1313,7 @@ public void accept(Notification t1) { @Test public void groupByBackpressure2() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 4000) .doOnNext(new Consumer() { @@ -1362,7 +1362,7 @@ public Flowable apply(GroupedFlowable t) { @Test public void groupByWithNullKey() { final String[] key = new String[]{"uninitialized"}; - final List values = new ArrayList(); + final List values = new ArrayList<>(); Flowable.just("a", "b", "c").groupBy(new Function() { @Override @@ -1398,7 +1398,7 @@ public void subscribe(Subscriber subscriber) { } } ); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); f.groupBy(new Function() { @@ -1416,11 +1416,11 @@ public Integer apply(Integer integer) { @Test public void groupByShouldPropagateError() { final Throwable e = new RuntimeException("Oops"); - final TestSubscriberEx inner1 = new TestSubscriberEx(); - final TestSubscriberEx inner2 = new TestSubscriberEx(); + final TestSubscriberEx inner1 = new TestSubscriberEx<>(); + final TestSubscriberEx inner2 = new TestSubscriberEx<>(); final TestSubscriberEx> outer - = new TestSubscriberEx>(new DefaultSubscriber>() { + = new TestSubscriberEx<>(new DefaultSubscriber>() { @Override public void onComplete() { @@ -1557,7 +1557,7 @@ public Object apply(Integer i) { */ @Test public void backpressureInnerDoesntOverflowOuter() { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); PublishProcessor pp = PublishProcessor.create(); @@ -1586,7 +1586,7 @@ public void accept(GroupedFlowable g) { @Test public void backpressureInnerDoesntOverflowOuterMissingBackpressure() { - TestSubscriber> ts = new TestSubscriber>(1); + TestSubscriber> ts = new TestSubscriber<>(1); Flowable.fromArray(1, 2) .groupBy(new Function() { @@ -1611,9 +1611,9 @@ public void accept(GroupedFlowable g) { @Test public void oneGroupInnerRequestsTwiceBuffer() { // FIXME: delayed requesting in groupBy results in group abandonment - TestSubscriber ts1 = new TestSubscriber(1L); + TestSubscriber ts1 = new TestSubscriber<>(1L); - final TestSubscriber ts2 = new TestSubscriber(0L); + final TestSubscriber ts2 = new TestSubscriber<>(0L); Flowable.range(1, Flowable.bufferSize() * 2) .groupBy(new Function() { @@ -1886,7 +1886,7 @@ public Map apply(final Consumer notify) throws Exceptio @Test public void mapFactoryExpiryCompletesGroupedFlowable() { - final List completed = new CopyOnWriteArrayList(); + final List completed = new CopyOnWriteArrayList<>(); Function, Map> evictingMapFactory = createEvictingMapFactorySynchronousOnly(1); PublishSubject subject = PublishSubject.create(); TestSubscriberEx ts = subject.toFlowable(BackpressureStrategy.BUFFER) @@ -1916,7 +1916,7 @@ public Integer apply(Integer n) throws Exception { @Test public void mapFactoryWithExpiringGuavaCacheDemonstrationCodeForUseInJavadoc() { //javadoc will be a version of this using lambdas and without assertions - final List completed = new CopyOnWriteArrayList(); + final List completed = new CopyOnWriteArrayList<>(); //size should be less than 5 to notice the effect Function, Map> evictingMapFactory = createEvictingMapFactoryGuava(3); int numValues = 1000; @@ -1999,7 +1999,7 @@ public void onRemoval(RemovalNotification notification) { } }; - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); Flowable stream = source // .doOnCancel(new Action() { @Override @@ -2091,8 +2091,8 @@ public Publisher apply(GroupedFlowable g) th //not thread safe private static final class SingleThreadEvictingHashMap implements Map { - private final List list = new ArrayList(); - private final Map map = new HashMap(); + private final List list = new ArrayList<>(); + private final Map map = new HashMap<>(); private final int maxSize; private final Consumer evictedListener; @@ -2214,15 +2214,16 @@ private static Function, Map> createEvictingMa @Override public Map apply(final Consumer notify) throws Exception { - return new SingleThreadEvictingHashMap(maxSize, new Consumer() { - @Override - public void accept(Object object) { - try { - notify.accept(object); - } catch (Throwable e) { - throw new RuntimeException(e); - } - }}); + return new SingleThreadEvictingHashMap<>(maxSize, new Consumer() { + @Override + public void accept(Object object) { + try { + notify.accept(object); + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + }); }}; return evictingMapFactory; } @@ -2231,7 +2232,7 @@ public void accept(Object object) { public void cancelOverFlatmapRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final PublishProcessor pp = PublishProcessor.create(); @@ -2274,7 +2275,7 @@ public void run() { @Test public void abandonedGroupsNoDataloss() { - final List> groups = new ArrayList>(); + final List> groups = new ArrayList<>(); Flowable.range(1, 1000) .groupBy(new Function() { @@ -2303,8 +2304,8 @@ public void accept(GroupedFlowable v) throws Throwable { @Test public void newGroupValueSelectorFails() { - TestSubscriber ts1 = new TestSubscriber(); - final TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); + final TestSubscriber ts2 = new TestSubscriber<>(); Flowable.just(1) .groupBy(Functions.identity(), new Function() { @@ -2330,8 +2331,8 @@ public void accept(GroupedFlowable g) throws Throwable { @Test public void existingGroupValueSelectorFails() { - TestSubscriber ts1 = new TestSubscriber(); - final TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); + final TestSubscriber ts2 = new TestSubscriber<>(); Flowable.just(1, 2) .groupBy(Functions.justFunction(1), new Function() { @@ -2399,7 +2400,7 @@ public Integer apply(Integer t) throws Throwable { throw new TestException(); } }) - .subscribeWith(new TestSubscriberEx>(0L)); + .subscribeWith(new TestSubscriberEx<>(0L)); assertTrue(pp.offer(1)); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsTest.java index e88319a1dd..2558b4a433 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsTest.java @@ -61,7 +61,7 @@ public void accept(Integer t) { @Test public void completedOkFlowable() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(1, 10).ignoreElements().toFlowable().subscribe(ts); ts.assertNoErrors(); ts.assertNoValues(); @@ -70,7 +70,7 @@ public void completedOkFlowable() { @Test public void errorReceivedFlowable() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); TestException ex = new TestException("boo"); Flowable.error(ex).ignoreElements().toFlowable().subscribe(ts); ts.assertNoValues(); @@ -173,7 +173,7 @@ public void accept(Integer t) { @Test public void completedOk() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Flowable.range(1, 10).ignoreElements().subscribe(to); to.assertNoErrors(); to.assertNoValues(); @@ -182,7 +182,7 @@ public void completedOk() { @Test public void errorReceived() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); TestException ex = new TestException("boo"); Flowable.error(ex).ignoreElements().subscribe(to); to.assertNoValues(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalTest.java index a5ec88c154..68b8bd59fd 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalTest.java @@ -40,7 +40,7 @@ public void badRequest() { @Test public void cancelledOnRun() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); IntervalSubscriber is = new IntervalSubscriber(ts); ts.onSubscribe(is); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotificationTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotificationTest.java index 3ec6b8f771..0deec9376b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotificationTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotificationTest.java @@ -29,7 +29,7 @@ public class FlowableMapNotificationTest extends RxJavaTest { @Test public void just() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1) .flatMap( new Function>() { @@ -61,7 +61,7 @@ public Flowable get() { public void backpressure() { TestSubscriber ts = TestSubscriber.create(0L); - new FlowableMapNotification(Flowable.range(1, 3), + new FlowableMapNotification<>(Flowable.range(1, 3), new Function() { @Override public Integer apply(Integer item) { @@ -105,7 +105,7 @@ public void noBackpressure() { PublishProcessor pp = PublishProcessor.create(); - new FlowableMapNotification(pp, + new FlowableMapNotification<>(pp, new Function() { @Override public Integer apply(Integer item) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java index 35dba45e2b..c12aafa21d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java @@ -155,7 +155,7 @@ public String apply(Map map) { @Test public void mapWithError() { - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); Flowable w = Flowable.just("one", "fail", "two", "three", "fail"); Flowable m = w.map(new Function() { @@ -261,7 +261,7 @@ public Integer apply(Integer i) { } private static Map getMap(String prefix) { - Map m = new HashMap(); + Map m = new HashMap<>(); m.put("firstName", prefix + "First"); m.put("lastName", prefix + "Last"); return m; @@ -272,7 +272,7 @@ public void functionCrashUnsubscribes() { PublishProcessor pp = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); pp.map(new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterializeTest.java index a6f63f3cf2..1d798e6272 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterializeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterializeTest.java @@ -102,7 +102,7 @@ public void multipleSubscribes() throws InterruptedException, ExecutionException @Test public void backpressureOnEmptyStream() { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); Flowable. empty().materialize().subscribe(ts); ts.assertNoValues(); ts.request(1); @@ -113,7 +113,7 @@ public void backpressureOnEmptyStream() { @Test public void backpressureNoError() { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); Flowable.just(1, 2, 3).materialize().subscribe(ts); ts.assertNoValues(); ts.request(1); @@ -127,7 +127,7 @@ public void backpressureNoError() { @Test public void backpressureNoErrorAsync() throws InterruptedException { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); Flowable.just(1, 2, 3) .materialize() .subscribeOn(Schedulers.computation()) @@ -148,7 +148,7 @@ public void backpressureNoErrorAsync() throws InterruptedException { @Test public void backpressureWithError() { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); Flowable. error(new IllegalArgumentException()).materialize().subscribe(ts); ts.assertNoValues(); ts.request(1); @@ -158,7 +158,7 @@ public void backpressureWithError() { @Test public void backpressureWithEmissionThenError() { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); IllegalArgumentException ex = new IllegalArgumentException(); Flowable.fromIterable(Arrays.asList(1)).concatWith(Flowable. error(ex)).materialize() .subscribe(ts); @@ -175,7 +175,7 @@ public void backpressureWithEmissionThenError() { @Test public void withCompletionCausingError() { - TestSubscriberEx> ts = new TestSubscriberEx>(); + TestSubscriberEx> ts = new TestSubscriberEx<>(); final RuntimeException ex = new RuntimeException("boo"); Flowable.empty().materialize().doOnNext(new Consumer() { @Override @@ -190,7 +190,7 @@ public void accept(Object t) { @Test public void unsubscribeJustBeforeCompletionNotificationShouldPreventThatNotificationArriving() { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); Flowable.empty().materialize() .subscribe(ts); @@ -204,7 +204,7 @@ private static class TestNotificationSubscriber extends DefaultSubscriber> notifications = new Vector>(); + List> notifications = new Vector<>(); @Override public void onComplete() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java index f80323abdd..7a528e8cb2 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java @@ -257,7 +257,7 @@ public void mergeArray() { public void mergeList() { final Flowable f1 = Flowable.unsafeCreate(new TestSynchronousFlowable()); final Flowable f2 = Flowable.unsafeCreate(new TestSynchronousFlowable()); - List> listOfFlowables = new ArrayList>(); + List> listOfFlowables = new ArrayList<>(); listOfFlowables.add(f1); listOfFlowables.add(f2); @@ -438,7 +438,7 @@ public void onNext(String args) { @Test public void errorInParentFlowable() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.mergeDelayError( Flowable.just(Flowable.just(1), Flowable.just(2)) .startWithItem(Flowable. error(new RuntimeException())) @@ -467,7 +467,7 @@ public void subscribe(Subscriber> op) { stringSubscriber = TestHelper.mockSubscriber(); - TestSubscriberEx ts = new TestSubscriberEx(stringSubscriber); + TestSubscriberEx ts = new TestSubscriberEx<>(stringSubscriber); Flowable m = Flowable.mergeDelayError(parentFlowable); m.subscribe(ts); System.out.println("testErrorInParentFlowableDelayed | " + i); @@ -506,7 +506,7 @@ public void run() { @Test public void delayErrorMaxConcurrent() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable source = Flowable.mergeDelayError(Flowable.just( Flowable.just(1).hide(), Flowable.error(new TestException())) @@ -517,7 +517,7 @@ public void accept(long t1) { } }), 1); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.subscribe(ts); @@ -532,7 +532,7 @@ public void accept(long t1) { public void mergeIterable() { final Flowable f1 = Flowable.unsafeCreate(new TestSynchronousFlowable()); final Flowable f2 = Flowable.unsafeCreate(new TestSynchronousFlowable()); - List> listOfFlowables = new ArrayList>(); + List> listOfFlowables = new ArrayList<>(); listOfFlowables.add(f1); listOfFlowables.add(f2); @@ -574,7 +574,7 @@ public void iterableMaxConcurrent() { @SuppressWarnings("unchecked") @Test public void iterableMaxConcurrentError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); PublishProcessor pp1 = PublishProcessor.create(); PublishProcessor pp2 = PublishProcessor.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeMaxConcurrentTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeMaxConcurrentTest.java index 979c29a9aa..b2582430bb 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeMaxConcurrentTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeMaxConcurrentTest.java @@ -34,14 +34,14 @@ public class FlowableMergeMaxConcurrentTest extends RxJavaTest { @Test public void whenMaxConcurrentIsOne() { for (int i = 0; i < 100; i++) { - List> os = new ArrayList>(); + List> os = new ArrayList<>(); os.add(Flowable.just("one", "two", "three", "four", "five").subscribeOn(Schedulers.newThread())); os.add(Flowable.just("one", "two", "three", "four", "five").subscribeOn(Schedulers.newThread())); os.add(Flowable.just("one", "two", "three", "four", "five").subscribeOn(Schedulers.newThread())); List expected = Arrays.asList("one", "two", "three", "four", "five", "one", "two", "three", "four", "five", "one", "two", "three", "four", "five"); Iterator iter = Flowable.merge(os, 1).blockingIterable().iterator(); - List actual = new ArrayList(); + List actual = new ArrayList<>(); while (iter.hasNext()) { actual.add(iter.next()); } @@ -57,8 +57,8 @@ public void maxConcurrent() { int maxConcurrent = 2 + (times % 10); AtomicInteger subscriptionCount = new AtomicInteger(0); - List> os = new ArrayList>(); - List scos = new ArrayList(); + List> os = new ArrayList<>(); + List scos = new ArrayList<>(); for (int i = 0; i < observableCount; i++) { SubscriptionCheckObservable sco = new SubscriptionCheckObservable(subscriptionCount, maxConcurrent); scos.add(sco); @@ -66,7 +66,7 @@ public void maxConcurrent() { } Iterator iter = Flowable.merge(os, maxConcurrent).blockingIterable().iterator(); - List actual = new ArrayList(); + List actual = new ArrayList<>(); while (iter.hasNext()) { actual.add(iter.next()); } @@ -118,7 +118,7 @@ public void run() { @Test public void mergeALotOfSourcesOneByOneSynchronously() { int n = 10000; - List> sourceList = new ArrayList>(n); + List> sourceList = new ArrayList<>(n); for (int i = 0; i < n; i++) { sourceList.add(Flowable.just(i)); } @@ -134,7 +134,7 @@ public void mergeALotOfSourcesOneByOneSynchronously() { @Test public void mergeALotOfSourcesOneByOneSynchronouslyTakeHalf() { int n = 10000; - List> sourceList = new ArrayList>(n); + List> sourceList = new ArrayList<>(n); for (int i = 0; i < n; i++) { sourceList.add(Flowable.just(i)); } @@ -150,9 +150,9 @@ public void mergeALotOfSourcesOneByOneSynchronouslyTakeHalf() { @Test public void simple() { for (int i = 1; i < 100; i++) { - TestSubscriberEx ts = new TestSubscriberEx(); - List> sourceList = new ArrayList>(i); - List result = new ArrayList(i); + TestSubscriberEx ts = new TestSubscriberEx<>(); + List> sourceList = new ArrayList<>(i); + List result = new ArrayList<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Flowable.just(j)); result.add(j); @@ -169,9 +169,9 @@ public void simple() { @Test public void simpleOneLess() { for (int i = 2; i < 100; i++) { - TestSubscriberEx ts = new TestSubscriberEx(); - List> sourceList = new ArrayList>(i); - List result = new ArrayList(i); + TestSubscriberEx ts = new TestSubscriberEx<>(); + List> sourceList = new ArrayList<>(i); + List result = new ArrayList<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Flowable.just(j)); result.add(j); @@ -201,9 +201,9 @@ public void simpleAsyncLoop() { @Test public void simpleAsync() { for (int i = 1; i < 50; i++) { - TestSubscriber ts = new TestSubscriber(); - List> sourceList = new ArrayList>(i); - Set expected = new HashSet(i); + TestSubscriber ts = new TestSubscriber<>(); + List> sourceList = new ArrayList<>(i); + Set expected = new HashSet<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Flowable.just(j).subscribeOn(Schedulers.io())); expected.add(j); @@ -213,7 +213,7 @@ public void simpleAsync() { ts.awaitDone(1, TimeUnit.SECONDS); ts.assertNoErrors(); - Set actual = new HashSet(ts.values()); + Set actual = new HashSet<>(ts.values()); assertEquals(expected, actual); } @@ -233,9 +233,9 @@ public void simpleOneLessAsync() { if (System.currentTimeMillis() - t > TimeUnit.SECONDS.toMillis(9)) { break; } - TestSubscriber ts = new TestSubscriber(); - List> sourceList = new ArrayList>(i); - Set expected = new HashSet(i); + TestSubscriber ts = new TestSubscriber<>(); + List> sourceList = new ArrayList<>(i); + Set expected = new HashSet<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Flowable.just(j).subscribeOn(Schedulers.io())); expected.add(j); @@ -245,7 +245,7 @@ public void simpleOneLessAsync() { ts.awaitDone(1, TimeUnit.SECONDS); ts.assertNoErrors(); - Set actual = new HashSet(ts.values()); + Set actual = new HashSet<>(ts.values()); assertEquals(expected, actual); } @@ -253,7 +253,7 @@ public void simpleOneLessAsync() { @Test public void backpressureHonored() throws Exception { - List> sourceList = new ArrayList>(3); + List> sourceList = new ArrayList<>(3); sourceList.add(Flowable.range(0, 100000).subscribeOn(Schedulers.io())); sourceList.add(Flowable.range(0, 100000).subscribeOn(Schedulers.io())); @@ -284,13 +284,13 @@ public void onNext(Integer t) { @Test public void take() throws Exception { - List> sourceList = new ArrayList>(3); + List> sourceList = new ArrayList<>(3); sourceList.add(Flowable.range(0, 100000).subscribeOn(Schedulers.io())); sourceList.add(Flowable.range(0, 100000).subscribeOn(Schedulers.io())); sourceList.add(Flowable.range(0, 100000).subscribeOn(Schedulers.io())); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.merge(sourceList, 2).take(5).subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java index 22526d1922..6b3f72e2f9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java @@ -115,7 +115,7 @@ public void mergeArray() { public void mergeList() { final Flowable f1 = Flowable.unsafeCreate(new TestSynchronousFlowable()); final Flowable f2 = Flowable.unsafeCreate(new TestSynchronousFlowable()); - List> listOfFlowables = new ArrayList>(); + List> listOfFlowables = new ArrayList<>(); listOfFlowables.add(f1); listOfFlowables.add(f2); @@ -201,7 +201,7 @@ public void mergeArrayWithThreading() { final TestASynchronousFlowable f2 = new TestASynchronousFlowable(); Flowable m = Flowable.merge(Flowable.unsafeCreate(f1), Flowable.unsafeCreate(f2)); - TestSubscriber ts = new TestSubscriber(stringSubscriber); + TestSubscriber ts = new TestSubscriber<>(stringSubscriber); m.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -231,7 +231,7 @@ public void synchronizationOfMultipleSequences() throws Throwable { final AtomicInteger concurrentCounter = new AtomicInteger(); final AtomicInteger totalCounter = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); Flowable m = Flowable.merge(Flowable.unsafeCreate(f1), Flowable.unsafeCreate(f2)); m.subscribe(new DefaultSubscriber() { @@ -424,7 +424,7 @@ public void unsubscribeAsFlowablesComplete() { AtomicBoolean os2 = new AtomicBoolean(false); Flowable f2 = createFlowableOf5IntervalsOf1SecondIncrementsWithSubscriptionHook(scheduler2, os2); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.merge(f1, f2).subscribe(ts); // we haven't incremented time so nothing should be received yet @@ -466,7 +466,7 @@ public void earlyUnsubscribe() { AtomicBoolean os2 = new AtomicBoolean(false); Flowable f2 = createFlowableOf5IntervalsOf1SecondIncrementsWithSubscriptionHook(scheduler2, os2); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.merge(f1, f2).subscribe(ts); // we haven't incremented time so nothing should be received yet @@ -542,7 +542,7 @@ public void concurrency() { for (int i = 0; i < 10; i++) { Flowable merge = Flowable.merge(f.onBackpressureBuffer(), f.onBackpressureBuffer(), f.onBackpressureBuffer()); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); merge.subscribe(ts); ts.awaitDone(3, TimeUnit.SECONDS); @@ -595,7 +595,7 @@ public void run() { for (int i = 0; i < 10; i++) { Flowable merge = Flowable.merge(f, f, f); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); merge.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -642,7 +642,7 @@ public void run() { for (int i = 0; i < 10; i++) { Flowable merge = Flowable.merge(f.onBackpressureBuffer(), f.onBackpressureBuffer(), f.onBackpressureBuffer()); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); merge.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -866,7 +866,7 @@ public void onNext(Integer t) { @Test public void merge1AsyncStreamOf1() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNAsyncStreamsOfN(1, 1).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -875,7 +875,7 @@ public void merge1AsyncStreamOf1() { @Test public void merge1AsyncStreamOf1000() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNAsyncStreamsOfN(1, 1000).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -884,7 +884,7 @@ public void merge1AsyncStreamOf1000() { @Test public void merge10AsyncStreamOf1000() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNAsyncStreamsOfN(10, 1000).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -893,7 +893,7 @@ public void merge10AsyncStreamOf1000() { @Test public void merge1000AsyncStreamOf1000() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNAsyncStreamsOfN(1000, 1000).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -902,7 +902,7 @@ public void merge1000AsyncStreamOf1000() { @Test public void merge2000AsyncStreamOf100() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNAsyncStreamsOfN(2000, 100).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -911,7 +911,7 @@ public void merge2000AsyncStreamOf100() { @Test public void merge100AsyncStreamOf1() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNAsyncStreamsOfN(100, 1).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -933,7 +933,7 @@ public Flowable apply(Integer i) { @Test public void merge1SyncStreamOf1() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNSyncStreamsOfN(1, 1).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -942,7 +942,7 @@ public void merge1SyncStreamOf1() { @Test public void merge1SyncStreamOf1000000() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNSyncStreamsOfN(1, 1000000).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -951,7 +951,7 @@ public void merge1SyncStreamOf1000000() { @Test public void merge1000SyncStreamOf1000() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNSyncStreamsOfN(1000, 1000).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -960,7 +960,7 @@ public void merge1000SyncStreamOf1000() { @Test public void merge10000SyncStreamOf10() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNSyncStreamsOfN(10000, 10).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -969,7 +969,7 @@ public void merge10000SyncStreamOf10() { @Test public void merge1000000SyncStreamOf1() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); mergeNSyncStreamsOfN(1000000, 1).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -1016,7 +1016,7 @@ public boolean hasNext() { @Test public void mergeManyAsyncSingle() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable> os = Flowable.range(1, 10000) .map(new Function>() { @@ -1051,7 +1051,7 @@ public void subscribe(Subscriber s) { @Test public void shouldCompleteAfterApplyingBackpressure_NormalPath() { Flowable source = Flowable.mergeDelayError(Flowable.just(Flowable.range(1, 2))); - TestSubscriberEx subscriber = new TestSubscriberEx(0L); + TestSubscriberEx subscriber = new TestSubscriberEx<>(0L); source.subscribe(subscriber); subscriber.request(3); // 1, 2, - with request(2) we get the 1 and 2 but not the subscriber.assertValues(1, 2); @@ -1061,7 +1061,7 @@ public void shouldCompleteAfterApplyingBackpressure_NormalPath() { @Test public void shouldCompleteAfterApplyingBackpressure_FastPath() { Flowable source = Flowable.mergeDelayError(Flowable.just(Flowable.just(1))); - TestSubscriberEx subscriber = new TestSubscriberEx(0L); + TestSubscriberEx subscriber = new TestSubscriberEx<>(0L); source.subscribe(subscriber); subscriber.request(2); // 1, - should work as per .._NormalPath above subscriber.assertValue(1); @@ -1072,7 +1072,7 @@ public void shouldCompleteAfterApplyingBackpressure_FastPath() { public void shouldNotCompleteIfThereArePendingScalarSynchronousEmissionsWhenTheLastInnerSubscriberCompletes() { TestScheduler scheduler = new TestScheduler(); Flowable source = Flowable.mergeDelayError(Flowable.just(1L), Flowable.timer(1, TimeUnit.SECONDS, scheduler).skip(1)); - TestSubscriberEx subscriber = new TestSubscriberEx(0L); + TestSubscriberEx subscriber = new TestSubscriberEx<>(0L); source.subscribe(subscriber); scheduler.advanceTimeBy(1, TimeUnit.SECONDS); subscriber.assertNoValues(); @@ -1089,7 +1089,7 @@ public void shouldNotCompleteIfThereArePendingScalarSynchronousEmissionsWhenTheL public void delayedErrorsShouldBeEmittedWhenCompleteAfterApplyingBackpressure_NormalPath() { Throwable exception = new Throwable(); Flowable source = Flowable.mergeDelayError(Flowable.range(1, 2), Flowable.error(exception)); - TestSubscriberEx subscriber = new TestSubscriberEx(0L); + TestSubscriberEx subscriber = new TestSubscriberEx<>(0L); source.subscribe(subscriber); subscriber.request(3); // 1, 2, subscriber.assertValues(1, 2); @@ -1101,7 +1101,7 @@ public void delayedErrorsShouldBeEmittedWhenCompleteAfterApplyingBackpressure_No public void delayedErrorsShouldBeEmittedWhenCompleteAfterApplyingBackpressure_FastPath() { Throwable exception = new Throwable(); Flowable source = Flowable.mergeDelayError(Flowable.just(1), Flowable.error(exception)); - TestSubscriberEx subscriber = new TestSubscriberEx(0L); + TestSubscriberEx subscriber = new TestSubscriberEx<>(0L); source.subscribe(subscriber); subscriber.request(2); // 1, subscriber.assertValue(1); @@ -1112,7 +1112,7 @@ public void delayedErrorsShouldBeEmittedWhenCompleteAfterApplyingBackpressure_Fa @Test public void shouldNotCompleteWhileThereAreStillScalarSynchronousEmissionsInTheQueue() { Flowable source = Flowable.merge(Flowable.just(1), Flowable.just(2)); - TestSubscriber subscriber = new TestSubscriber(1L); + TestSubscriber subscriber = new TestSubscriber<>(1L); source.subscribe(subscriber); subscriber.assertValue(1); subscriber.request(1); @@ -1123,7 +1123,7 @@ public void shouldNotCompleteWhileThereAreStillScalarSynchronousEmissionsInTheQu public void shouldNotReceivedDelayedErrorWhileThereAreStillScalarSynchronousEmissionsInTheQueue() { Throwable exception = new Throwable(); Flowable source = Flowable.mergeDelayError(Flowable.just(1), Flowable.just(2), Flowable.error(exception)); - TestSubscriberEx subscriber = new TestSubscriberEx(0L); + TestSubscriberEx subscriber = new TestSubscriberEx<>(0L); subscriber.request(1); source.subscribe(subscriber); subscriber.assertValue(1); @@ -1137,7 +1137,7 @@ public void shouldNotReceivedDelayedErrorWhileThereAreStillScalarSynchronousEmis public void shouldNotReceivedDelayedErrorWhileThereAreStillNormalEmissionsInTheQueue() { Throwable exception = new Throwable(); Flowable source = Flowable.mergeDelayError(Flowable.range(1, 2), Flowable.range(3, 2), Flowable.error(exception)); - TestSubscriberEx subscriber = new TestSubscriberEx(0L); + TestSubscriberEx subscriber = new TestSubscriberEx<>(0L); subscriber.request(3); source.subscribe(subscriber); subscriber.assertValues(1, 2, 3); @@ -1152,7 +1152,7 @@ public void mergeKeepsRequesting() throws InterruptedException { //for (int i = 0; i < 5000; i++) { //System.out.println(i + "......................................................................."); final CountDownLatch latch = new CountDownLatch(1); - final ConcurrentLinkedQueue messages = new ConcurrentLinkedQueue(); + final ConcurrentLinkedQueue messages = new ConcurrentLinkedQueue<>(); Flowable.range(1, 2) // produce many integers per second @@ -1280,7 +1280,7 @@ public Flowable apply(Integer t) { ; void runMerge(Function> func, TestSubscriberEx ts) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (int i = 0; i < 1000; i++) { list.add(i); } @@ -1298,12 +1298,12 @@ void runMerge(Function> func, TestSubscriberEx()); + runMerge(toScalar, new TestSubscriberEx<>()); } @Test public void fastMergeHiddenScalar() { - runMerge(toHiddenScalar, new TestSubscriberEx()); + runMerge(toHiddenScalar, new TestSubscriberEx<>()); } @Test @@ -1466,7 +1466,7 @@ public void flatMapMaxConcurrentJustRange() { public void noInnerReordering() { TestSubscriber ts = TestSubscriber.create(0); FlowableFlatMap.MergeSubscriber, Integer> ms = - new FlowableFlatMap.MergeSubscriber, Integer>(ts, Functions.>identity(), false, 128, 128); + new FlowableFlatMap.MergeSubscriber<>(ts, Functions.>identity(), false, 128, 128); ms.onSubscribe(new BooleanSubscription()); PublishProcessor pp = PublishProcessor.create(); @@ -1488,7 +1488,7 @@ public void noInnerReordering() { public void noOuterScalarReordering() { TestSubscriber ts = TestSubscriber.create(0); FlowableFlatMap.MergeSubscriber, Integer> ms = - new FlowableFlatMap.MergeSubscriber, Integer>(ts, Functions.>identity(), false, 128, 128); + new FlowableFlatMap.MergeSubscriber<>(ts, Functions.>identity(), false, 128, 128); ms.onSubscribe(new BooleanSubscription()); ms.onNext(Flowable.just(1)); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletableTest.java index 9008168d48..c074796be6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletableTest.java @@ -29,7 +29,7 @@ public class FlowableMergeWithCompletableTest extends RxJavaTest { @Test public void normal() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 5).mergeWith( Completable.fromAction(new Action() { @@ -71,7 +71,7 @@ public void cancel() { @Test public void normalBackpressured() { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Flowable.range(1, 5).mergeWith( Completable.fromAction(new Action() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybeTest.java index 604daa4950..e5ef48a426 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybeTest.java @@ -230,7 +230,7 @@ public void onSuccessFastPathBackpressuredRace() { final PublishProcessor pp = PublishProcessor.create(); final MaybeSubject cs = MaybeSubject.create(); - final TestSubscriber ts = pp.mergeWith(cs).subscribeWith(new TestSubscriber(0)); + final TestSubscriber ts = pp.mergeWith(cs).subscribeWith(new TestSubscriber<>(0)); Runnable r1 = new Runnable() { @Override @@ -259,7 +259,7 @@ public void run() { public void onErrorMainOverflow() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> subscriber = new AtomicReference>(); + final AtomicReference> subscriber = new AtomicReference<>(); TestSubscriber ts = new Flowable() { @Override protected void subscribeActual(Subscriber s) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingleTest.java index 4c5cd09d5b..a48ee16fef 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingleTest.java @@ -87,7 +87,7 @@ public void cancel() { @Test public void normalBackpressured() { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Flowable.range(1, 5).mergeWith( Single.just(100) @@ -226,7 +226,7 @@ public void onSuccessFastPathBackpressuredRace() { final PublishProcessor pp = PublishProcessor.create(); final SingleSubject cs = SingleSubject.create(); - final TestSubscriber ts = pp.mergeWith(cs).subscribeWith(new TestSubscriber(0)); + final TestSubscriber ts = pp.mergeWith(cs).subscribeWith(new TestSubscriber<>(0)); Runnable r1 = new Runnable() { @Override @@ -255,7 +255,7 @@ public void run() { public void onErrorMainOverflow() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> subscriber = new AtomicReference>(); + final AtomicReference> subscriber = new AtomicReference<>(); TestSubscriber ts = new Flowable() { @Override protected void subscribeActual(Subscriber s) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java index 994926b237..73efdfc426 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java @@ -65,7 +65,7 @@ public void ordering() throws InterruptedException { Subscriber subscriber = TestHelper.mockSubscriber(); InOrder inOrder = inOrder(subscriber); - TestSubscriberEx ts = new TestSubscriberEx(subscriber); + TestSubscriberEx ts = new TestSubscriberEx<>(subscriber); obs.observeOn(Schedulers.computation()).subscribe(ts); @@ -384,7 +384,7 @@ public void afterUnsubscribeCalledThenObserverOnNextNeverCalled() { final TestScheduler testScheduler = new TestScheduler(); final Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); Flowable.just(1, 2, 3) .observeOn(testScheduler) @@ -519,7 +519,7 @@ public boolean hasNext() { } }); - TestSubscriber testSubscriber = new TestSubscriber(); + TestSubscriber testSubscriber = new TestSubscriber<>(); flowable .take(7) .observeOn(Schedulers.newThread()) @@ -547,7 +547,7 @@ public void subscribe(Subscriber subscriber) { }); - TestSubscriberEx testSubscriber = new TestSubscriberEx(new DefaultSubscriber() { + TestSubscriberEx testSubscriber = new TestSubscriberEx<>(new DefaultSubscriber() { @Override public void onComplete() { @@ -590,7 +590,7 @@ public void onNext(Integer t) { @Test public void asyncChild() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, 100000).observeOn(Schedulers.newThread()).observeOn(Schedulers.newThread()).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -602,7 +602,7 @@ public void onErrorCutsAheadOfOnNext() { final PublishProcessor processor = PublishProcessor.create(); final AtomicLong counter = new AtomicLong(); - TestSubscriberEx ts = new TestSubscriberEx(new DefaultSubscriber() { + TestSubscriberEx ts = new TestSubscriberEx<>(new DefaultSubscriber() { @Override public void onComplete() { @@ -649,7 +649,7 @@ public void onNext(Long t) { */ @Test public void hotOperatorBackpressure() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.interval(0, 1, TimeUnit.MICROSECONDS) .observeOn(Schedulers.computation()) .map(new Function() { @@ -697,7 +697,7 @@ public void accept(Notification n) { }); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.combineLatest(timer, Flowable. never(), new BiFunction() { @@ -757,7 +757,7 @@ public void onNext(Integer t) { @Test public void noMoreRequestsAfterUnsubscribe() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); - final List requests = Collections.synchronizedList(new ArrayList()); + final List requests = Collections.synchronizedList(new ArrayList<>()); Flowable.range(1, 1000000) .doOnRequest(new LongConsumer() { @@ -873,7 +873,7 @@ public void fixedReplenishPattern() { TestScheduler test = new TestScheduler(); - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.range(1, 100) .doOnRequest(new LongConsumer() { @@ -918,9 +918,9 @@ public void bufferSizesWork() { @Test public void synchronousRebatching() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 50) .doOnRequest(new LongConsumer() { @@ -1594,7 +1594,7 @@ public void onNext(Integer t) { @Test public void syncFusedCancelAfterRequest2() { - final TestSubscriber ts = new TestSubscriber(2L); + final TestSubscriber ts = new TestSubscriber<>(2L); Flowable.range(1, 2) .observeOn(Schedulers.single()) @@ -1630,7 +1630,7 @@ public void onNext(Integer t) { @Test public void syncFusedCancelAfterRequestConditional2() { - final TestSubscriber ts = new TestSubscriber(2L); + final TestSubscriber ts = new TestSubscriber<>(2L); Flowable.range(1, 2) .observeOn(Schedulers.single()) @@ -1644,7 +1644,7 @@ public void syncFusedCancelAfterRequestConditional2() { @Test public void nonFusedCancelAfterRequestConditional2() { - final TestSubscriber ts = new TestSubscriber(2L); + final TestSubscriber ts = new TestSubscriber<>(2L); Flowable.range(1, 2).hide() .observeOn(Schedulers.single()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategyTest.java index 91a2b3b7c5..327078ad60 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategyTest.java @@ -58,7 +58,7 @@ public void run() throws Exception { } private TestSubscriber createTestSubscriber() { - return new TestSubscriber(new DefaultSubscriber() { + return new TestSubscriber<>(new DefaultSubscriber() { @Override protected void onStart() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferTest.java index 3d32d2c808..a877b1a390 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferTest.java @@ -39,7 +39,7 @@ public class FlowableOnBackpressureBufferTest extends RxJavaTest { @Test public void noBackpressureSupport() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); // this will be ignored ts.request(100); // we take 500 so it unsubscribes @@ -53,7 +53,7 @@ public void noBackpressureSupport() { public void fixBackpressureWithBuffer() throws InterruptedException { final CountDownLatch l1 = new CountDownLatch(100); final CountDownLatch l2 = new CountDownLatch(150); - TestSubscriber ts = new TestSubscriber(new DefaultSubscriber() { + TestSubscriber ts = new TestSubscriber<>(new DefaultSubscriber() { @Override protected void onStart() { @@ -110,17 +110,19 @@ public void fixBackpressureBufferZeroCapacity() throws InterruptedException { public void fixBackpressureBoundedBuffer() throws InterruptedException { final CountDownLatch l1 = new CountDownLatch(100); final CountDownLatch backpressureCallback = new CountDownLatch(1); - TestSubscriber ts = new TestSubscriber(new DefaultSubscriber() { + TestSubscriber ts = new TestSubscriber<>(new DefaultSubscriber() { @Override protected void onStart() { } @Override - public void onComplete() { } + public void onComplete() { + } @Override - public void onError(Throwable e) { } + public void onError(Throwable e) { + } @Override public void onNext(Long t) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDropTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDropTest.java index dabf934ce3..811e27f766 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDropTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDropTest.java @@ -32,7 +32,7 @@ public class FlowableOnBackpressureDropTest extends RxJavaTest { @Test public void noBackpressureSupport() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); // this will be ignored ts.request(100); // we take 500 so it unsubscribes @@ -44,7 +44,7 @@ public void noBackpressureSupport() { @Test public void withObserveOn() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, Flowable.bufferSize() * 10).onBackpressureDrop().observeOn(Schedulers.io()).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); } @@ -53,7 +53,7 @@ public void withObserveOn() throws InterruptedException { public void fixBackpressureWithBuffer() throws InterruptedException { final CountDownLatch l1 = new CountDownLatch(100); final CountDownLatch l2 = new CountDownLatch(150); - TestSubscriber ts = new TestSubscriber(new DefaultSubscriber() { + TestSubscriber ts = new TestSubscriber<>(new DefaultSubscriber() { @Override protected void onStart() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureErrorTest.java index a2288abcb1..76fad7bf49 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureErrorTest.java @@ -37,7 +37,7 @@ public void doubleOnSubscribe() { TestHelper.checkDoubleOnSubscribeFlowable(new Function, Publisher>() { @Override public Publisher apply(Flowable f) throws Exception { - return new FlowableOnBackpressureError(f); + return new FlowableOnBackpressureError<>(f); } }); } @@ -47,7 +47,7 @@ public void badSource() { TestHelper.checkBadSourceFlowable(new Function, Object>() { @Override public Object apply(Flowable f) throws Exception { - return new FlowableOnBackpressureError(f); + return new FlowableOnBackpressureError<>(f); } }, false, 1, 1, 1); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatestTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatestTest.java index bb52136b0e..438ebb1415 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatestTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatestTest.java @@ -30,7 +30,7 @@ public class FlowableOnBackpressureLatestTest extends RxJavaTest { @Test public void simple() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(1, 5).onBackpressureLatest().subscribe(ts); @@ -41,7 +41,7 @@ public void simple() { @Test public void simpleError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(1, 5).concatWith(Flowable.error(new TestException())) .onBackpressureLatest().subscribe(ts); @@ -53,7 +53,7 @@ public void simpleError() { @Test public void simpleBackpressure() { - TestSubscriber ts = new TestSubscriber(2L); + TestSubscriber ts = new TestSubscriber<>(2L); Flowable.range(1, 5).onBackpressureLatest().subscribe(ts); @@ -65,7 +65,7 @@ public void simpleBackpressure() { @Test public void synchronousDrop() { PublishProcessor source = PublishProcessor.create(); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); source.onBackpressureLatest().subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFlowableTest.java index 742b1e5853..b8829893fa 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFlowableTest.java @@ -148,7 +148,7 @@ public void run() { @Test public void backpressure() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, 100000) .onErrorResumeWith(Flowable.just(1)) .observeOn(Schedulers.computation()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFunctionTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFunctionTest.java index 72a0bf0161..51de11ae7b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFunctionTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorResumeNextViaFunctionTest.java @@ -39,7 +39,7 @@ public class FlowableOnErrorResumeNextViaFunctionTest extends RxJavaTest { @Test public void resumeNextWithSynchronousExecution() { - final AtomicReference receivedException = new AtomicReference(); + final AtomicReference receivedException = new AtomicReference<>(); Flowable w = Flowable.unsafeCreate(new Publisher() { @Override @@ -79,7 +79,7 @@ public Flowable apply(Throwable t1) { @Test public void resumeNextWithAsyncExecution() { - final AtomicReference receivedException = new AtomicReference(); + final AtomicReference receivedException = new AtomicReference<>(); Subscription s = mock(Subscription.class); TestFlowable w = new TestFlowable(s, "one"); Function> resume = new Function>() { @@ -176,7 +176,7 @@ public Flowable apply(Throwable t1) { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber, Long.MAX_VALUE); + TestSubscriber ts = new TestSubscriber<>(subscriber, Long.MAX_VALUE); flowable.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -228,7 +228,7 @@ public void run() { @Test public void backpressure() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, 100000) .onErrorResumeNext(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturnTest.java index 324cfe0eec..ecde026f95 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturnTest.java @@ -39,7 +39,7 @@ public class FlowableOnErrorReturnTest extends RxJavaTest { public void resumeNext() { TestFlowable f = new TestFlowable("one"); Flowable w = Flowable.unsafeCreate(f); - final AtomicReference capturedException = new AtomicReference(); + final AtomicReference capturedException = new AtomicReference<>(); Flowable flowable = w.onErrorReturn(new Function() { @@ -74,7 +74,7 @@ public String apply(Throwable e) { public void functionThrowsError() { TestFlowable f = new TestFlowable("one"); Flowable w = Flowable.unsafeCreate(f); - final AtomicReference capturedException = new AtomicReference(); + final AtomicReference capturedException = new AtomicReference<>(); Flowable flowable = w.onErrorReturn(new Function() { @@ -132,7 +132,7 @@ public String apply(Throwable t1) { }); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber, Long.MAX_VALUE); + TestSubscriber ts = new TestSubscriber<>(subscriber, Long.MAX_VALUE); flowable.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -146,7 +146,7 @@ public String apply(Throwable t1) { @Test public void backpressure() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(0, 100000) .onErrorReturn(new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java index f62616097e..a660eb7429 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java @@ -37,7 +37,7 @@ public class FlowablePublishFunctionTest extends RxJavaTest { @Test public void concatTakeFirstLastCompletes() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 3).publish(new Function, Flowable>() { @Override @@ -210,7 +210,7 @@ public Flowable apply(Flowable f) { @Test public void overflowMissingBackpressureException() { - TestSubscriberEx ts = new TestSubscriberEx(0); + TestSubscriberEx ts = new TestSubscriberEx<>(0); PublishProcessor pp = PublishProcessor.create(); @@ -236,11 +236,11 @@ public Flowable apply(Flowable f) { @Test public void overflowMissingBackpressureExceptionDelayed() { - TestSubscriberEx ts = new TestSubscriberEx(0); + TestSubscriberEx ts = new TestSubscriberEx<>(0); PublishProcessor pp = PublishProcessor.create(); - new FlowablePublishMulticast(pp, new Function, Flowable>() { + new FlowablePublishMulticast<>(pp, new Function, Flowable>() { @Override public Flowable apply(Flowable f) { return f; @@ -349,7 +349,7 @@ public Integer apply(Integer v) throws Exception { @Test public void error() { - new FlowablePublishMulticast(Flowable.just(1).concatWith(Flowable.error(new TestException())), + new FlowablePublishMulticast<>(Flowable.just(1).concatWith(Flowable.error(new TestException())), Functions.>identity(), 16, true) .test() .assertFailure(TestException.class, 1); @@ -428,7 +428,7 @@ public void inputOutputSubscribeRace2() { @Test public void sourceSubscriptionDelayed() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts1 = new TestSubscriber(0L); + final TestSubscriber ts1 = new TestSubscriber<>(0L); Flowable.just(1) .publish(new Function, Publisher>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticastTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticastTest.java index 186285e1f2..0fb61a84c5 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticastTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticastTest.java @@ -32,7 +32,7 @@ public class FlowablePublishMulticastTest extends RxJavaTest { @Test public void asyncFusedInput() { - MulticastProcessor mp = new MulticastProcessor(128, true); + MulticastProcessor mp = new MulticastProcessor<>(128, true); UnicastProcessor up = UnicastProcessor.create(); @@ -51,7 +51,7 @@ public void asyncFusedInput() { @Test public void fusionRejectedInput() { - MulticastProcessor mp = new MulticastProcessor(128, true); + MulticastProcessor mp = new MulticastProcessor<>(128, true); mp.onSubscribe(new QueueSubscription() { @@ -106,10 +106,10 @@ public void cancel() { public void addRemoveRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final MulticastProcessor mp = new MulticastProcessor(128, true); + final MulticastProcessor mp = new MulticastProcessor<>(128, true); - final MulticastSubscription ms1 = new MulticastSubscription(null, mp); - final MulticastSubscription ms2 = new MulticastSubscription(null, mp); + final MulticastSubscription ms1 = new MulticastSubscription<>(null, mp); + final MulticastSubscription ms2 = new MulticastSubscription<>(null, mp); assertTrue(mp.add(ms1)); @@ -133,9 +133,9 @@ public void run() { @Test public void removeNotFound() { - MulticastProcessor mp = new MulticastProcessor(128, true); + MulticastProcessor mp = new MulticastProcessor<>(128, true); - MulticastSubscription ms1 = new MulticastSubscription(null, mp); + MulticastSubscription ms1 = new MulticastSubscription<>(null, mp); assertTrue(mp.add(ms1)); mp.remove(null); @@ -143,9 +143,9 @@ public void removeNotFound() { @Test public void errorAllCancelled() { - MulticastProcessor mp = new MulticastProcessor(128, true); + MulticastProcessor mp = new MulticastProcessor<>(128, true); - MulticastSubscription ms1 = new MulticastSubscription(null, mp); + MulticastSubscription ms1 = new MulticastSubscription<>(null, mp); assertTrue(mp.add(ms1)); ms1.set(Long.MIN_VALUE); @@ -155,9 +155,9 @@ public void errorAllCancelled() { @Test public void completeAllCancelled() { - MulticastProcessor mp = new MulticastProcessor(128, true); + MulticastProcessor mp = new MulticastProcessor<>(128, true); - MulticastSubscription ms1 = new MulticastSubscription(null, mp); + MulticastSubscription ms1 = new MulticastSubscription<>(null, mp); assertTrue(mp.add(ms1)); ms1.set(Long.MIN_VALUE); @@ -167,9 +167,9 @@ public void completeAllCancelled() { @Test public void cancelledWhileFindingRequests() { - final MulticastProcessor mp = new MulticastProcessor(128, true); + final MulticastProcessor mp = new MulticastProcessor<>(128, true); - final MulticastSubscription ms1 = new MulticastSubscription(null, mp); + final MulticastSubscription ms1 = new MulticastSubscription<>(null, mp); assertTrue(mp.add(ms1)); @@ -182,9 +182,9 @@ public void cancelledWhileFindingRequests() { @Test public void negativeRequest() { - final MulticastProcessor mp = new MulticastProcessor(128, true); + final MulticastProcessor mp = new MulticastProcessor<>(128, true); - final MulticastSubscription ms1 = new MulticastSubscription(null, mp); + final MulticastSubscription ms1 = new MulticastSubscription<>(null, mp); List errors = TestHelper.trackPluginErrors(); try { @@ -198,19 +198,19 @@ public void negativeRequest() { @Test public void outputCancellerDoubleOnSubscribe() { - TestHelper.doubleOnSubscribe(new OutputCanceller(new TestSubscriber(), null)); + TestHelper.doubleOnSubscribe(new OutputCanceller<>(new TestSubscriber<>(), null)); } @Test public void dontDropItemsWhenNoReadyConsumers() { - final MulticastProcessor mp = new MulticastProcessor(128, true); + final MulticastProcessor mp = new MulticastProcessor<>(128, true); mp.onSubscribe(new BooleanSubscription()); mp.onNext(1); - TestSubscriber ts = new TestSubscriber(); - final MulticastSubscription ms1 = new MulticastSubscription(ts, mp); + TestSubscriber ts = new TestSubscriber<>(); + final MulticastSubscription ms1 = new MulticastSubscription<>(ts, mp); ts.onSubscribe(ms1); assertTrue(mp.add(ms1)); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java index 0ecba3926f..ec7c88da09 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java @@ -128,7 +128,7 @@ public void run() { }); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.merge(fast, slow).subscribe(ts); is.connect(); ts.awaitDone(5, TimeUnit.SECONDS); @@ -148,7 +148,7 @@ public void accept(Integer t1) { } }); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); xs.publish(new Function, Flowable>() { @Override @@ -175,7 +175,7 @@ public boolean test(Integer i) { @Test public void takeUntilWithPublishedStream() { Flowable xs = Flowable.range(0, Flowable.bufferSize() * 2); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ConnectableFlowable xsp = xs.publish(); xsp.takeUntil(xsp.skipWhile(new Predicate() { @@ -211,7 +211,7 @@ public void run() { final AtomicBoolean child1Unsubscribed = new AtomicBoolean(); final AtomicBoolean child2Unsubscribed = new AtomicBoolean(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); final TestSubscriber ts1 = new TestSubscriber() { @Override @@ -259,7 +259,7 @@ public void connectWithNoSubscriber() { cf.connect(); // Emit 0 scheduler.advanceTimeBy(15, TimeUnit.MILLISECONDS); - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); cf.subscribe(subscriber); // Emit 1 and 2 scheduler.advanceTimeBy(50, TimeUnit.MILLISECONDS); @@ -271,7 +271,7 @@ public void connectWithNoSubscriber() { public void subscribeAfterDisconnectThenConnect() { ConnectableFlowable source = Flowable.just(1).publish(); - TestSubscriberEx ts1 = new TestSubscriberEx(); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); source.subscribe(ts1); @@ -283,7 +283,7 @@ public void subscribeAfterDisconnectThenConnect() { source.reset(); - TestSubscriberEx ts2 = new TestSubscriberEx(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(); source.subscribe(ts2); @@ -301,7 +301,7 @@ public void subscribeAfterDisconnectThenConnect() { public void noSubscriberRetentionOnCompleted() { FlowablePublish source = (FlowablePublish)Flowable.just(1).publish(); - TestSubscriberEx ts1 = new TestSubscriberEx(); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); source.subscribe(ts1); @@ -353,7 +353,7 @@ static boolean checkPublishDisposed(Disposable d) { public void zeroRequested() { ConnectableFlowable source = Flowable.just(1).publish(); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); source.subscribe(ts); @@ -408,9 +408,9 @@ public void syncFusedObserveOn() { Flowable obs = cf.observeOn(Schedulers.computation()); for (int i = 0; i < 1000; i++) { for (int j = 1; j < 6; j++) { - List> tss = new ArrayList>(); + List> tss = new ArrayList<>(); for (int k = 1; k < j; k++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); tss.add(ts); obs.subscribe(ts); } @@ -435,9 +435,9 @@ public void syncFusedObserveOn2() { Flowable obs = cf.observeOn(ImmediateThinScheduler.INSTANCE); for (int i = 0; i < 1000; i++) { for (int j = 1; j < 6; j++) { - List> tss = new ArrayList>(); + List> tss = new ArrayList<>(); for (int k = 1; k < j; k++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); tss.add(ts); obs.subscribe(ts); } @@ -461,9 +461,9 @@ public void asyncFusedObserveOn() { ConnectableFlowable cf = Flowable.range(0, 1000).observeOn(ImmediateThinScheduler.INSTANCE).publish(); for (int i = 0; i < 1000; i++) { for (int j = 1; j < 6; j++) { - List> tss = new ArrayList>(); + List> tss = new ArrayList<>(); for (int k = 1; k < j; k++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); tss.add(ts); cf.subscribe(ts); } @@ -488,9 +488,9 @@ public void observeOn() { Flowable obs = cf.observeOn(Schedulers.computation()); for (int i = 0; i < 1000; i++) { for (int j = 1; j < 6; j++) { - List> tss = new ArrayList>(); + List> tss = new ArrayList<>(); for (int k = 1; k < j; k++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); tss.add(ts); obs.subscribe(ts); } @@ -539,7 +539,7 @@ public void addRemoveRace() { final TestSubscriber ts = cf.test(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -694,7 +694,7 @@ public void subscribeDisconnectRace() { final ConnectableFlowable cf = pp.publish(); final Disposable d = cf.connect(); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -938,7 +938,7 @@ protected void subscribeActual(Subscriber s) { public void disposeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference ref = new AtomicReference(); + final AtomicReference ref = new AtomicReference<>(); final ConnectableFlowable cf = new Flowable() { @Override @@ -963,7 +963,7 @@ public void run() { @Test public void removeNotPresent() { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); final ConnectableFlowable cf = new Flowable() { @Override @@ -976,7 +976,7 @@ protected void subscribeActual(Subscriber s) { cf.connect(); - ref.get().add(new InnerSubscription(new TestSubscriber(), ref.get())); + ref.get().add(new InnerSubscription<>(new TestSubscriber<>(), ref.get())); ref.get().remove(null); } @@ -999,7 +999,7 @@ public void onNext(Integer t) { ts1.assertResult(1); - TestSubscriber ts2 = new TestSubscriber(0); + TestSubscriber ts2 = new TestSubscriber<>(0); cf.subscribe(ts2); ts2 @@ -1012,7 +1012,7 @@ public void onNext(Integer t) { public void subscriberLiveSwap() { final ConnectableFlowable cf = Flowable.range(1, 5).publish(); - final TestSubscriber ts2 = new TestSubscriber(0); + final TestSubscriber ts2 = new TestSubscriber<>(0); TestSubscriber ts1 = new TestSubscriber() { @Override @@ -1038,7 +1038,7 @@ public void onNext(Integer t) { @Test public void selectorSubscriberSwap() { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); Flowable.range(1, 5).publish(new Function, Publisher>() { @Override @@ -1061,7 +1061,7 @@ public Publisher apply(Flowable f) throws Exception { @Test public void leavingSubscriberOverrequests() { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); PublishProcessor pp = PublishProcessor.create(); @@ -1205,7 +1205,7 @@ public Publisher apply(Integer first) @Test public void publishFunctionCancelOuterAfterOneInner() { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); PublishProcessor pp = PublishProcessor.create(); @@ -1231,7 +1231,7 @@ public void onNext(Integer t) { @Test public void publishFunctionCancelOuterAfterOneInnerBackpressured() { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); PublishProcessor pp = PublishProcessor.create(); @@ -1261,7 +1261,7 @@ public void publishCancelOneAsync() { final PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); pp.publish(new Function, Publisher>() { @Override @@ -1300,9 +1300,9 @@ public void publishCancelOneAsync2() { ConnectableFlowable cf = pp.publish(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); cf.subscribe(new FlowableSubscriber() { @SuppressWarnings("unchecked") @@ -1500,7 +1500,7 @@ public boolean test(List v) throws Exception { @Test public void altConnectCrash() { try { - new FlowablePublish(Flowable.empty(), 128) + new FlowablePublish<>(Flowable.empty(), 128) .connect(new Consumer() { @Override public void accept(Disposable t) throws Exception { @@ -1517,7 +1517,7 @@ public void accept(Disposable t) throws Exception { public void altConnectRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { final ConnectableFlowable cf = - new FlowablePublish(Flowable.never(), 128); + new FlowablePublish<>(Flowable.never(), 128); Runnable r = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeLongTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeLongTest.java index afc677aefe..96401918ef 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeLongTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeLongTest.java @@ -98,7 +98,7 @@ public void rangeWithOverflow5() { public void backpressureViaRequest() { Flowable f = Flowable.rangeLong(1, Flowable.bufferSize()); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.assertNoValues(); ts.request(1); @@ -119,14 +119,14 @@ public void backpressureViaRequest() { @Test public void noBackpressure() { - ArrayList list = new ArrayList(Flowable.bufferSize() * 2); + ArrayList list = new ArrayList<>(Flowable.bufferSize() * 2); for (long i = 1; i <= Flowable.bufferSize() * 2 + 1; i++) { list.add(i); } Flowable f = Flowable.rangeLong(1, list.size()); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.assertNoValues(); ts.request(Long.MAX_VALUE); // infinite @@ -139,11 +139,11 @@ public void noBackpressure() { void withBackpressureOneByOne(long start) { Flowable source = Flowable.rangeLong(start, 100); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.request(1); source.subscribe(ts); - List list = new ArrayList(100); + List list = new ArrayList<>(100); for (long i = 0; i < 100; i++) { list.add(i + start); ts.request(1); @@ -154,11 +154,11 @@ void withBackpressureOneByOne(long start) { void withBackpressureAllAtOnce(long start) { Flowable source = Flowable.rangeLong(start, 100); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.request(100); source.subscribe(ts); - List list = new ArrayList(100); + List list = new ArrayList<>(100); for (long i = 0; i < 100; i++) { list.add(i + start); } @@ -184,11 +184,11 @@ public void withBackpressureAllAtOnce() { public void withBackpressureRequestWayMore() { Flowable source = Flowable.rangeLong(50, 100); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.request(150); source.subscribe(ts); - List list = new ArrayList(100); + List list = new ArrayList<>(100); for (long i = 0; i < 100; i++) { list.add(i + 50); } @@ -257,7 +257,7 @@ public void onNext(Long t) { @Test public void nearMaxValueWithoutBackpressure() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.rangeLong(Long.MAX_VALUE - 1L, 2L).subscribe(ts); ts.assertComplete(); @@ -267,7 +267,7 @@ public void nearMaxValueWithoutBackpressure() { @Test public void nearMaxValueWithBackpressure() { - TestSubscriber ts = new TestSubscriber(3L); + TestSubscriber ts = new TestSubscriber<>(3L); Flowable.rangeLong(Long.MAX_VALUE - 1L, 2L).subscribe(ts); ts.assertComplete(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeTest.java index a8d42213c9..39ab84ad63 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRangeTest.java @@ -98,7 +98,7 @@ public void rangeWithOverflow5() { public void backpressureViaRequest() { Flowable f = Flowable.range(1, Flowable.bufferSize()); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.assertNoValues(); ts.request(1); @@ -119,14 +119,14 @@ public void backpressureViaRequest() { @Test public void noBackpressure() { - ArrayList list = new ArrayList(Flowable.bufferSize() * 2); + ArrayList list = new ArrayList<>(Flowable.bufferSize() * 2); for (int i = 1; i <= Flowable.bufferSize() * 2 + 1; i++) { list.add(i); } Flowable f = Flowable.range(1, list.size()); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.assertNoValues(); ts.request(Long.MAX_VALUE); // infinite @@ -139,11 +139,11 @@ public void noBackpressure() { void withBackpressureOneByOne(int start) { Flowable source = Flowable.range(start, 100); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.request(1); source.subscribe(ts); - List list = new ArrayList(100); + List list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { list.add(i + start); ts.request(1); @@ -154,11 +154,11 @@ void withBackpressureOneByOne(int start) { void withBackpressureAllAtOnce(int start) { Flowable source = Flowable.range(start, 100); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.request(100); source.subscribe(ts); - List list = new ArrayList(100); + List list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { list.add(i + start); } @@ -184,11 +184,11 @@ public void withBackpressureAllAtOnce() { public void withBackpressureRequestWayMore() { Flowable source = Flowable.range(50, 100); - TestSubscriberEx ts = new TestSubscriberEx(0L); + TestSubscriberEx ts = new TestSubscriberEx<>(0L); ts.request(150); source.subscribe(ts); - List list = new ArrayList(100); + List list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { list.add(i + 50); } @@ -257,7 +257,7 @@ public void onNext(Integer t) { @Test public void nearMaxValueWithoutBackpressure() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(Integer.MAX_VALUE - 1, 2).subscribe(ts); ts.assertComplete(); @@ -267,7 +267,7 @@ public void nearMaxValueWithoutBackpressure() { @Test public void nearMaxValueWithBackpressure() { - TestSubscriber ts = new TestSubscriber(3L); + TestSubscriber ts = new TestSubscriber<>(3L); Flowable.range(Integer.MAX_VALUE - 1, 2).subscribe(ts); ts.assertComplete(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java index 975740cfe3..169409ed1b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCountTest.java @@ -205,8 +205,8 @@ public void run() { .publish().refCount(); for (int i = 0; i < 10; i++) { - TestSubscriber ts1 = new TestSubscriber(); - TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); + TestSubscriber ts2 = new TestSubscriber<>(); r.subscribe(ts1); r.subscribe(ts2); try { @@ -248,7 +248,7 @@ public void run() { } }); - TestSubscriberEx s = new TestSubscriberEx(); + TestSubscriberEx s = new TestSubscriberEx<>(); f.publish().refCount().subscribeOn(Schedulers.newThread()).subscribe(s); System.out.println("send unsubscribe"); // wait until connected @@ -293,7 +293,7 @@ public void accept(Subscription s) { } }); - TestSubscriberEx s = new TestSubscriberEx(); + TestSubscriberEx s = new TestSubscriberEx<>(); f.publish().refCount().subscribeOn(Schedulers.computation()).subscribe(s); System.out.println("send unsubscribe"); @@ -386,7 +386,7 @@ public void refCount() { Flowable interval = Flowable.interval(100, TimeUnit.MILLISECONDS, s).publish().refCount(); // subscribe list1 - final List list1 = new ArrayList(); + final List list1 = new ArrayList<>(); Disposable d1 = interval.subscribe(new Consumer() { @Override public void accept(Long t1) { @@ -401,7 +401,7 @@ public void accept(Long t1) { assertEquals(1L, list1.get(1).longValue()); // subscribe list2 - final List list2 = new ArrayList(); + final List list2 = new ArrayList<>(); Disposable d2 = interval.subscribe(new Consumer() { @Override public void accept(Long t1) { @@ -446,7 +446,7 @@ public void accept(Long t1) { // subscribing a new one should start over because the source should have been unsubscribed // subscribe list3 - final List list3 = new ArrayList(); + final List list3 = new ArrayList<>(); interval.subscribe(new Consumer() { @Override public void accept(Long t1) { @@ -517,8 +517,8 @@ public Integer apply(Integer t1, Integer t2) { }) .publish().refCount(); - TestSubscriberEx ts1 = new TestSubscriberEx(); - TestSubscriberEx ts2 = new TestSubscriberEx(); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(); combined.subscribe(ts1); combined.subscribe(ts2); @@ -1184,7 +1184,7 @@ public void unsubscribeSubscribeRace() { final TestSubscriber ts1 = source.test(0); - final TestSubscriber ts2 = new TestSubscriber(0); + final TestSubscriber ts2 = new TestSubscriber<>(0); Runnable r1 = new Runnable() { @Override @@ -1388,7 +1388,7 @@ protected void subscribeActual(Subscriber subscriber) { @Test public void timeoutResetsSource() { - TestConnectableFlowable tcf = new TestConnectableFlowable(); + TestConnectableFlowable tcf = new TestConnectableFlowable<>(); FlowableRefCount o = (FlowableRefCount)tcf.refCount(); RefConnection rc = new RefConnection(o); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatTest.java index c99e09a45a..29959688ad 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatTest.java @@ -164,7 +164,7 @@ public void repeatAndDistinctUnbounded() { .repeat(3) .distinct(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); src.subscribe(ts); @@ -176,8 +176,8 @@ public void repeatAndDistinctUnbounded() { /** Issue #2844: wrong target of request. */ @Test public void repeatRetarget() { - final List concatBase = new ArrayList(); - TestSubscriber ts = new TestSubscriber(); + final List concatBase = new ArrayList<>(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2) .repeat(5) .concatMap(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayEagerTruncateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayEagerTruncateTest.java index d17526e47b..f263b98f66 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayEagerTruncateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayEagerTruncateTest.java @@ -732,14 +732,14 @@ public boolean isDisposed() { @Test public void boundedReplayBuffer() { - BoundedReplayBuffer buf = new BoundedReplayBuffer(true); + BoundedReplayBuffer buf = new BoundedReplayBuffer<>(true); buf.addLast(new Node(1, 0)); buf.addLast(new Node(2, 1)); buf.addLast(new Node(3, 2)); buf.addLast(new Node(4, 3)); buf.addLast(new Node(5, 4)); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.collect(values); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), values); @@ -762,8 +762,8 @@ public void boundedReplayBuffer() { @Test public void timedAndSizedTruncation() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, true); - List values = new ArrayList(); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, true); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -808,8 +808,8 @@ public void accept(long t) { }); ConnectableFlowable cf = source.replay(); - TestSubscriberEx ts1 = new TestSubscriberEx(10L); - TestSubscriberEx ts2 = new TestSubscriberEx(90L); + TestSubscriberEx ts1 = new TestSubscriberEx<>(10L); + TestSubscriberEx ts2 = new TestSubscriberEx<>(90L); cf.subscribe(ts1); cf.subscribe(ts2); @@ -839,8 +839,8 @@ public void accept(long t) { }); ConnectableFlowable cf = source.replay(50, true); - TestSubscriberEx ts1 = new TestSubscriberEx(10L); - TestSubscriberEx ts2 = new TestSubscriberEx(90L); + TestSubscriberEx ts1 = new TestSubscriberEx<>(10L); + TestSubscriberEx ts2 = new TestSubscriberEx<>(90L); cf.subscribe(ts1); cf.subscribe(ts2); @@ -862,7 +862,7 @@ public void accept(long t) { public void coldReplayNoBackpressure() { Flowable source = Flowable.range(0, 1000).replay().autoConnect(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.subscribe(ts); @@ -880,7 +880,7 @@ public void coldReplayNoBackpressure() { public void coldReplayBackpressure() { Flowable source = Flowable.range(0, 1000).replay().autoConnect(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); ts.request(10); source.subscribe(ts); @@ -961,7 +961,7 @@ public void unsubscribeSource() throws Throwable { @Test public void take() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable cached = Flowable.range(1, 100).replay().autoConnect(); cached.take(10).subscribe(ts); @@ -975,7 +975,7 @@ public void take() { public void async() { Flowable source = Flowable.range(1, 10000); for (int i = 0; i < 100; i++) { - TestSubscriberEx ts1 = new TestSubscriberEx(); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); Flowable cached = source.replay().autoConnect(); @@ -986,7 +986,7 @@ public void async() { ts1.assertTerminated(); assertEquals(10000, ts1.values().size()); - TestSubscriberEx ts2 = new TestSubscriberEx(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(); cached.observeOn(Schedulers.computation()).subscribe(ts2); ts2.awaitDone(2, TimeUnit.SECONDS); @@ -1005,14 +1005,14 @@ public void asyncComeAndGo() { Flowable output = cached.observeOn(Schedulers.computation(), false, 1024); - List> list = new ArrayList>(100); + List> list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); list.add(ts); output.skip(i * 10).take(10).subscribe(ts); } - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 10; i++) { expected.add((long)(i - 10)); } @@ -1046,7 +1046,7 @@ public void subscribe(Subscriber t) { } }); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); firehose.replay().autoConnect().observeOn(Schedulers.computation()).takeLast(100).subscribe(ts); ts.awaitDone(3, TimeUnit.SECONDS); @@ -1062,14 +1062,14 @@ public void valuesAndThenError() { .concatWith(Flowable.error(new TestException())) .replay().autoConnect(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.subscribe(ts); ts.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); ts.assertNotComplete(); Assert.assertEquals(1, ts.errors().size()); - TestSubscriberEx ts2 = new TestSubscriberEx(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(); source.subscribe(ts2); ts2.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); @@ -1110,7 +1110,7 @@ public void onNext(Integer t) { public void unboundedLeavesEarly() { PublishProcessor source = PublishProcessor.create(); - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable out = source .doOnRequest(new LongConsumer() { @@ -1120,8 +1120,8 @@ public void accept(long t) { } }).replay().autoConnect(); - TestSubscriber ts1 = new TestSubscriber(5L); - TestSubscriber ts2 = new TestSubscriber(10L); + TestSubscriber ts1 = new TestSubscriber<>(5L); + TestSubscriber ts2 = new TestSubscriber<>(10L); out.subscribe(ts1); out.subscribe(ts2); @@ -1135,7 +1135,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ConnectableFlowable source = Flowable.range(1, 10).replay(1, true); source.connect(); - TestSubscriber ts1 = new TestSubscriber(2L); + TestSubscriber ts1 = new TestSubscriber<>(2L); source.subscribe(ts1); @@ -1143,7 +1143,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts1.assertNoErrors(); ts1.cancel(); - TestSubscriber ts2 = new TestSubscriber(2L); + TestSubscriber ts2 = new TestSubscriber<>(2L); source.subscribe(ts2); @@ -1151,7 +1151,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts2.assertNoErrors(); ts2.cancel(); - TestSubscriber ts21 = new TestSubscriber(1L); + TestSubscriber ts21 = new TestSubscriber<>(1L); source.subscribe(ts21); @@ -1159,7 +1159,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts21.assertNoErrors(); ts21.cancel(); - TestSubscriber ts22 = new TestSubscriber(1L); + TestSubscriber ts22 = new TestSubscriber<>(1L); source.subscribe(ts22); @@ -1167,7 +1167,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts22.assertNoErrors(); ts22.cancel(); - TestSubscriber ts3 = new TestSubscriber(); + TestSubscriber ts3 = new TestSubscriber<>(); source.subscribe(ts3); @@ -1182,7 +1182,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ConnectableFlowable source = Flowable.range(1, 10).replay(2, true); source.connect(); - TestSubscriber ts1 = new TestSubscriber(2L); + TestSubscriber ts1 = new TestSubscriber<>(2L); source.subscribe(ts1); @@ -1190,7 +1190,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts1.assertNoErrors(); ts1.cancel(); - TestSubscriber ts11 = new TestSubscriber(2L); + TestSubscriber ts11 = new TestSubscriber<>(2L); source.subscribe(ts11); @@ -1198,7 +1198,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts11.assertNoErrors(); ts11.cancel(); - TestSubscriber ts2 = new TestSubscriber(3L); + TestSubscriber ts2 = new TestSubscriber<>(3L); source.subscribe(ts2); @@ -1206,7 +1206,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts2.assertNoErrors(); ts2.cancel(); - TestSubscriber ts21 = new TestSubscriber(1L); + TestSubscriber ts21 = new TestSubscriber<>(1L); source.subscribe(ts21); @@ -1214,7 +1214,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts21.assertNoErrors(); ts21.cancel(); - TestSubscriber ts22 = new TestSubscriber(1L); + TestSubscriber ts22 = new TestSubscriber<>(1L); source.subscribe(ts22); @@ -1222,7 +1222,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts22.assertNoErrors(); ts22.cancel(); - TestSubscriber ts3 = new TestSubscriber(); + TestSubscriber ts3 = new TestSubscriber<>(); source.subscribe(ts3); @@ -1286,8 +1286,8 @@ public void subscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableFlowable cf = Flowable.range(1, 3).replay(); - final TestSubscriber ts1 = new TestSubscriber(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); + final TestSubscriber ts2 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -1312,8 +1312,8 @@ public void addRemoveRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableFlowable cf = Flowable.range(1, 3).replay(); - final TestSubscriber ts1 = new TestSubscriber(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); + final TestSubscriber ts2 = new TestSubscriber<>(); cf.subscribe(ts1); @@ -1412,7 +1412,7 @@ public void subscribeOnNextRace() { final ConnectableFlowable cf = pp.replay(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -1441,7 +1441,7 @@ public void unsubscribeOnNextRace() { final ConnectableFlowable cf = pp.replay(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); cf.subscribe(ts1); @@ -1470,7 +1470,7 @@ public void unsubscribeReplayRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableFlowable cf = Flowable.range(1, 1000).replay(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); cf.connect(); @@ -1592,12 +1592,12 @@ public void replayMaxInt() { @Test public void timedAndSizedTruncationError() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, true); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, true); Assert.assertFalse(buf.hasCompleted()); Assert.assertFalse(buf.hasError()); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -1635,8 +1635,8 @@ public void timedAndSizedTruncationError() { @Test public void sizedTruncation() { - SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer(2, true); - List values = new ArrayList(); + SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer<>(2, true); + List values = new ArrayList<>(); buf.next(1); buf.next(2); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayTest.java index b8a7ba3e9d..67763fce0b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplayTest.java @@ -732,14 +732,14 @@ public boolean isDisposed() { @Test public void boundedReplayBuffer() { - BoundedReplayBuffer buf = new BoundedReplayBuffer(false); + BoundedReplayBuffer buf = new BoundedReplayBuffer<>(false); buf.addLast(new Node(1, 0)); buf.addLast(new Node(2, 1)); buf.addLast(new Node(3, 2)); buf.addLast(new Node(4, 3)); buf.addLast(new Node(5, 4)); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.collect(values); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), values); @@ -763,8 +763,8 @@ public void boundedReplayBuffer() { @Test public void timedAndSizedTruncation() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, false); - List values = new ArrayList(); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, false); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -809,8 +809,8 @@ public void accept(long t) { }); ConnectableFlowable cf = source.replay(); - TestSubscriberEx ts1 = new TestSubscriberEx(10L); - TestSubscriberEx ts2 = new TestSubscriberEx(90L); + TestSubscriberEx ts1 = new TestSubscriberEx<>(10L); + TestSubscriberEx ts2 = new TestSubscriberEx<>(90L); cf.subscribe(ts1); cf.subscribe(ts2); @@ -840,8 +840,8 @@ public void accept(long t) { }); ConnectableFlowable cf = source.replay(50); - TestSubscriberEx ts1 = new TestSubscriberEx(10L); - TestSubscriberEx ts2 = new TestSubscriberEx(90L); + TestSubscriberEx ts1 = new TestSubscriberEx<>(10L); + TestSubscriberEx ts2 = new TestSubscriberEx<>(90L); cf.subscribe(ts1); cf.subscribe(ts2); @@ -863,7 +863,7 @@ public void accept(long t) { public void coldReplayNoBackpressure() { Flowable source = Flowable.range(0, 1000).replay().autoConnect(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.subscribe(ts); @@ -881,7 +881,7 @@ public void coldReplayNoBackpressure() { public void coldReplayBackpressure() { Flowable source = Flowable.range(0, 1000).replay().autoConnect(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); ts.request(10); source.subscribe(ts); @@ -962,7 +962,7 @@ public void unsubscribeSource() throws Throwable { @Test public void take() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable cached = Flowable.range(1, 100).replay().autoConnect(); cached.take(10).subscribe(ts); @@ -976,7 +976,7 @@ public void take() { public void async() { Flowable source = Flowable.range(1, 10000); for (int i = 0; i < 100; i++) { - TestSubscriberEx ts1 = new TestSubscriberEx(); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); Flowable cached = source.replay().autoConnect(); @@ -987,7 +987,7 @@ public void async() { ts1.assertTerminated(); assertEquals(10000, ts1.values().size()); - TestSubscriberEx ts2 = new TestSubscriberEx(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(); cached.observeOn(Schedulers.computation()).subscribe(ts2); ts2.awaitDone(2, TimeUnit.SECONDS); @@ -1006,14 +1006,14 @@ public void asyncComeAndGo() { Flowable output = cached.observeOn(Schedulers.computation(), false, 1024); - List> list = new ArrayList>(100); + List> list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); list.add(ts); output.skip(i * 10).take(10).subscribe(ts); } - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 10; i++) { expected.add((long)(i - 10)); } @@ -1047,7 +1047,7 @@ public void subscribe(Subscriber t) { } }); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); firehose.replay().autoConnect().observeOn(Schedulers.computation()).takeLast(100).subscribe(ts); ts.awaitDone(3, TimeUnit.SECONDS); @@ -1063,14 +1063,14 @@ public void valuesAndThenError() { .concatWith(Flowable.error(new TestException())) .replay().autoConnect(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.subscribe(ts); ts.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); ts.assertNotComplete(); Assert.assertEquals(1, ts.errors().size()); - TestSubscriberEx ts2 = new TestSubscriberEx(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(); source.subscribe(ts2); ts2.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); @@ -1111,7 +1111,7 @@ public void onNext(Integer t) { public void unboundedLeavesEarly() { PublishProcessor source = PublishProcessor.create(); - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable out = source .doOnRequest(new LongConsumer() { @@ -1121,8 +1121,8 @@ public void accept(long t) { } }).replay().autoConnect(); - TestSubscriber ts1 = new TestSubscriber(5L); - TestSubscriber ts2 = new TestSubscriber(10L); + TestSubscriber ts1 = new TestSubscriber<>(5L); + TestSubscriber ts2 = new TestSubscriber<>(10L); out.subscribe(ts1); out.subscribe(ts2); @@ -1136,7 +1136,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ConnectableFlowable source = Flowable.range(1, 10).replay(1); source.connect(); - TestSubscriber ts1 = new TestSubscriber(2L); + TestSubscriber ts1 = new TestSubscriber<>(2L); source.subscribe(ts1); @@ -1144,7 +1144,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts1.assertNoErrors(); ts1.cancel(); - TestSubscriber ts2 = new TestSubscriber(2L); + TestSubscriber ts2 = new TestSubscriber<>(2L); source.subscribe(ts2); @@ -1152,7 +1152,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts2.assertNoErrors(); ts2.cancel(); - TestSubscriber ts21 = new TestSubscriber(1L); + TestSubscriber ts21 = new TestSubscriber<>(1L); source.subscribe(ts21); @@ -1160,7 +1160,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts21.assertNoErrors(); ts21.cancel(); - TestSubscriber ts22 = new TestSubscriber(1L); + TestSubscriber ts22 = new TestSubscriber<>(1L); source.subscribe(ts22); @@ -1168,7 +1168,7 @@ public void subscribersComeAndGoAtRequestBoundaries() { ts22.assertNoErrors(); ts22.cancel(); - TestSubscriber ts3 = new TestSubscriber(); + TestSubscriber ts3 = new TestSubscriber<>(); source.subscribe(ts3); @@ -1183,7 +1183,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ConnectableFlowable source = Flowable.range(1, 10).replay(2); source.connect(); - TestSubscriber ts1 = new TestSubscriber(2L); + TestSubscriber ts1 = new TestSubscriber<>(2L); source.subscribe(ts1); @@ -1191,7 +1191,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts1.assertNoErrors(); ts1.cancel(); - TestSubscriber ts11 = new TestSubscriber(2L); + TestSubscriber ts11 = new TestSubscriber<>(2L); source.subscribe(ts11); @@ -1199,7 +1199,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts11.assertNoErrors(); ts11.cancel(); - TestSubscriber ts2 = new TestSubscriber(3L); + TestSubscriber ts2 = new TestSubscriber<>(3L); source.subscribe(ts2); @@ -1207,7 +1207,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts2.assertNoErrors(); ts2.cancel(); - TestSubscriber ts21 = new TestSubscriber(1L); + TestSubscriber ts21 = new TestSubscriber<>(1L); source.subscribe(ts21); @@ -1215,7 +1215,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts21.assertNoErrors(); ts21.cancel(); - TestSubscriber ts22 = new TestSubscriber(1L); + TestSubscriber ts22 = new TestSubscriber<>(1L); source.subscribe(ts22); @@ -1223,7 +1223,7 @@ public void subscribersComeAndGoAtRequestBoundaries2() { ts22.assertNoErrors(); ts22.cancel(); - TestSubscriber ts3 = new TestSubscriber(); + TestSubscriber ts3 = new TestSubscriber<>(); source.subscribe(ts3); @@ -1287,8 +1287,8 @@ public void subscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableFlowable cf = Flowable.range(1, 3).replay(); - final TestSubscriber ts1 = new TestSubscriber(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); + final TestSubscriber ts2 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -1313,8 +1313,8 @@ public void addRemoveRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableFlowable cf = Flowable.range(1, 3).replay(); - final TestSubscriber ts1 = new TestSubscriber(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); + final TestSubscriber ts2 = new TestSubscriber<>(); cf.subscribe(ts1); @@ -1413,7 +1413,7 @@ public void subscribeOnNextRace() { final ConnectableFlowable cf = pp.replay(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -1442,7 +1442,7 @@ public void unsubscribeOnNextRace() { final ConnectableFlowable cf = pp.replay(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); cf.subscribe(ts1); @@ -1471,7 +1471,7 @@ public void unsubscribeReplayRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableFlowable cf = Flowable.range(1, 1000).replay(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); cf.connect(); @@ -1593,12 +1593,12 @@ public void replayMaxInt() { @Test public void timedAndSizedTruncationError() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, false); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, false); Assert.assertFalse(buf.hasCompleted()); Assert.assertFalse(buf.hasError()); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -1636,8 +1636,8 @@ public void timedAndSizedTruncationError() { @Test public void sizedTruncation() { - SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer(2, false); - List values = new ArrayList(); + SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer<>(2, false); + List values = new ArrayList<>(); buf.next(1); buf.next(2); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java index e2db70633a..4de1cfb8b0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java @@ -63,7 +63,7 @@ public void subscribe(Subscriber t1) { } }); - TestSubscriber ts = new TestSubscriber(consumer); + TestSubscriber ts = new TestSubscriber<>(consumer); producer.retryWhen(new Function, Flowable>() { @Override @@ -117,7 +117,7 @@ public void retryIndefinitely() { Subscriber subscriber = TestHelper.mockSubscriber(); int numRetries = 20; Flowable origin = Flowable.unsafeCreate(new FuncWithErrors(numRetries)); - origin.retry().subscribe(new TestSubscriber(subscriber)); + origin.retry().subscribe(new TestSubscriber<>(subscriber)); InOrder inOrder = inOrder(subscriber); // should show 3 attempts @@ -136,7 +136,7 @@ public void schedulingNotificationHandler() { Subscriber subscriber = TestHelper.mockSubscriber(); int numRetries = 2; Flowable origin = Flowable.unsafeCreate(new FuncWithErrors(numRetries)); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); origin.retryWhen(new Function, Flowable>() { @Override public Flowable apply(Flowable t1) { @@ -203,7 +203,7 @@ public Integer apply(Throwable t1) { public void onCompletedFromNotificationHandler() { Subscriber subscriber = TestHelper.mockSubscriber(); Flowable origin = Flowable.unsafeCreate(new FuncWithErrors(1)); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); origin.retryWhen(new Function, Flowable>() { @Override public Flowable apply(Flowable t1) { @@ -498,7 +498,7 @@ public void cancel() { public void sourceFlowableCallsUnsubscribe() throws InterruptedException { final AtomicInteger subsCount = new AtomicInteger(0); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Publisher onSubscribe = new Publisher() { @Override @@ -529,7 +529,7 @@ public void subscribe(Subscriber s) { public void sourceFlowableRetry1() throws InterruptedException { final AtomicInteger subsCount = new AtomicInteger(0); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Publisher onSubscribe = new Publisher() { @Override @@ -548,7 +548,7 @@ public void subscribe(Subscriber s) { public void sourceFlowableRetry0() throws InterruptedException { final AtomicInteger subsCount = new AtomicInteger(0); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Publisher onSubscribe = new Publisher() { @Override @@ -674,7 +674,7 @@ public void unsubscribeAfterError() { SlowFlowable so = new SlowFlowable(100, 0, "testUnsubscribeAfterError"); Flowable f = Flowable.unsafeCreate(so).retry(5); - AsyncSubscriber async = new AsyncSubscriber(subscriber); + AsyncSubscriber async = new AsyncSubscriber<>(subscriber); f.subscribe(async); @@ -698,7 +698,7 @@ public void timeoutWithRetry() { SlowFlowable sf = new SlowFlowable(100, 10, "testTimeoutWithRetry"); Flowable f = Flowable.unsafeCreate(sf).timeout(80, TimeUnit.MILLISECONDS).retry(5); - AsyncSubscriber async = new AsyncSubscriber(subscriber); + AsyncSubscriber async = new AsyncSubscriber<>(subscriber); f.subscribe(async); @@ -720,7 +720,7 @@ public void retryWithBackpressure() throws InterruptedException { for (int i = 0; i < 400; i++) { Subscriber subscriber = TestHelper.mockSubscriber(); Flowable origin = Flowable.unsafeCreate(new FuncWithErrors(numRetries)); - TestSubscriberEx ts = new TestSubscriberEx(subscriber); + TestSubscriberEx ts = new TestSubscriberEx<>(subscriber); origin.retry().observeOn(Schedulers.computation()).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -751,7 +751,7 @@ public void retryWithBackpressureParallel() throws InterruptedException { } final AtomicInteger timeouts = new AtomicInteger(); - final Map> data = new ConcurrentHashMap>(); + final Map> data = new ConcurrentHashMap<>(); int m = 5000; final CountDownLatch cdl = new CountDownLatch(m); @@ -763,11 +763,11 @@ public void run() { final AtomicInteger nexts = new AtomicInteger(); try { Flowable origin = Flowable.unsafeCreate(new FuncWithErrors(numRetries)); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); origin.retry() .observeOn(Schedulers.computation()).subscribe(ts); ts.awaitDone(2500, TimeUnit.MILLISECONDS); - List onNextEvents = new ArrayList(ts.values()); + List onNextEvents = new ArrayList<>(ts.values()); if (onNextEvents.size() != numRetries + 2) { for (Throwable t : ts.errors()) { onNextEvents.add(t.toString()); @@ -865,7 +865,7 @@ public Flowable apply(GroupedFlowable t1) { return t1.take(1); } }, NUM_MSG) // Must request as many groups as groupBy produces to avoid MBE - .subscribe(new TestSubscriber(subscriber)); + .subscribe(new TestSubscriber<>(subscriber)); InOrder inOrder = inOrder(subscriber); // should show 3 attempts @@ -910,7 +910,7 @@ public Flowable apply(GroupedFlowable t1) { return t1.take(1); } }) - .subscribe(new TestSubscriber(subscriber)); + .subscribe(new TestSubscriber<>(subscriber)); InOrder inOrder = inOrder(subscriber); // should show 3 attempts diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWithPredicateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWithPredicateTest.java index 9a69dd3ede..b70b742201 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWithPredicateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWithPredicateTest.java @@ -236,7 +236,7 @@ public void unsubscribeAfterError() { .unsafeCreate(so) .retry(retry5); - FlowableRetryTest.AsyncSubscriber async = new FlowableRetryTest.AsyncSubscriber(subscriber); + FlowableRetryTest.AsyncSubscriber async = new FlowableRetryTest.AsyncSubscriber<>(subscriber); f.subscribe(async); @@ -263,7 +263,7 @@ public void timeoutWithRetry() { .timeout(80, TimeUnit.MILLISECONDS) .retry(retry5); - FlowableRetryTest.AsyncSubscriber async = new FlowableRetryTest.AsyncSubscriber(subscriber); + FlowableRetryTest.AsyncSubscriber async = new FlowableRetryTest.AsyncSubscriber<>(subscriber); f.subscribe(async); @@ -279,7 +279,7 @@ public void timeoutWithRetry() { @Test public void issue2826() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final RuntimeException e = new RuntimeException("You shall not pass"); final AtomicInteger c = new AtomicInteger(); Flowable.just(1).map(new Function() { @@ -313,7 +313,7 @@ public Integer apply(Integer t1) { @Test public void issue3008RetryWithPredicate() { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); final AtomicBoolean isFirst = new AtomicBoolean(true); Flowable. just(1L, 2L, 3L).map(new Function() { @Override @@ -341,7 +341,7 @@ public void accept(Long t) { @Test public void issue3008RetryInfinite() { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); final AtomicBoolean isFirst = new AtomicBoolean(true); Flowable. just(1L, 2L, 3L).map(new Function() { @Override @@ -365,7 +365,7 @@ public void accept(Long t) { @Test public void backpressure() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable source = Flowable .just(1) @@ -377,7 +377,7 @@ public void accept(long t) { } }); - TestSubscriber ts = new TestSubscriber(3L); + TestSubscriber ts = new TestSubscriber<>(3L); source .retry(new BiPredicate() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMapTest.java index 60ddbbff52..a4abfce00c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMapTest.java @@ -59,7 +59,7 @@ public Integer get() throws Exception { static final class OneCallablePublisher implements Publisher, Supplier { @Override public void subscribe(Subscriber s) { - s.onSubscribe(new ScalarSubscription(s, 1)); + s.onSubscribe(new ScalarSubscription<>(s, 1)); } @Override @@ -70,7 +70,7 @@ public Integer get() throws Exception { @Test public void tryScalarXMap() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); assertTrue(FlowableScalarXMap.tryScalarXMapSubscribe(new CallablePublisher(), ts, new Function>() { @Override public Publisher apply(Integer f) throws Exception { @@ -83,7 +83,7 @@ public Publisher apply(Integer f) throws Exception { @Test public void emptyXMap() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); assertTrue(FlowableScalarXMap.tryScalarXMapSubscribe(new EmptyCallablePublisher(), ts, new Function>() { @Override @@ -97,7 +97,7 @@ public Publisher apply(Integer f) throws Exception { @Test public void mapperCrashes() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); assertTrue(FlowableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), ts, new Function>() { @Override @@ -111,7 +111,7 @@ public Publisher apply(Integer f) throws Exception { @Test public void mapperToJust() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); assertTrue(FlowableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), ts, new Function>() { @Override @@ -125,7 +125,7 @@ public Publisher apply(Integer f) throws Exception { @Test public void mapperToEmpty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); assertTrue(FlowableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), ts, new Function>() { @Override @@ -139,7 +139,7 @@ public Publisher apply(Integer f) throws Exception { @Test public void mapperToCrashingCallable() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); assertTrue(FlowableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), ts, new Function>() { @Override @@ -177,8 +177,8 @@ public Publisher apply(Integer v) throws Exception { @Test public void scalarDisposableStateCheck() { - TestSubscriber ts = new TestSubscriber(); - ScalarSubscription sd = new ScalarSubscription(ts, 1); + TestSubscriber ts = new TestSubscriber<>(); + ScalarSubscription sd = new ScalarSubscription<>(ts, 1); ts.onSubscribe(sd); assertFalse(sd.isCancelled()); @@ -211,8 +211,8 @@ public void scalarDisposableStateCheck() { @Test public void scalarDisposableRunDisposeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(); - final ScalarSubscription sd = new ScalarSubscription(ts, 1); + TestSubscriber ts = new TestSubscriber<>(); + final ScalarSubscription sd = new ScalarSubscription<>(ts, 1); ts.onSubscribe(sd); Runnable r1 = new Runnable() { @@ -235,7 +235,7 @@ public void run() { @Test public void cancelled() { - ScalarSubscription scalar = new ScalarSubscription(new TestSubscriber(), 1); + ScalarSubscription scalar = new ScalarSubscription<>(new TestSubscriber<>(), 1); assertFalse(scalar.isCancelled()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanTest.java index f918d152be..2988da7570 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanTest.java @@ -114,7 +114,7 @@ public Integer apply(Integer t1, Integer t2) { @Test public void shouldNotEmitUntilAfterSubscription() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 100).scan(0, new BiFunction() { @Override @@ -263,7 +263,7 @@ public void seedFactory() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @@ -289,7 +289,7 @@ public void seedFactoryFlowable() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @@ -315,7 +315,7 @@ public Integer apply(Integer t1, Integer t2) { } }).take(1); - TestSubscriberEx subscriber = new TestSubscriberEx(); + TestSubscriberEx subscriber = new TestSubscriberEx<>(); f.subscribe(subscriber); subscriber.assertValue(0); subscriber.assertTerminated(); @@ -324,7 +324,7 @@ public Integer apply(Integer t1, Integer t2) { @Test public void scanShouldNotRequestZero() { - final AtomicReference producer = new AtomicReference(); + final AtomicReference producer = new AtomicReference<>(); Flowable f = Flowable.unsafeCreate(new Publisher() { @Override public void subscribe(final Subscriber subscriber) { @@ -445,7 +445,7 @@ public Integer apply(Integer a, Integer b) throws Exception { public void unsubscribeScan() { FlowableEventStream.getEventStream("HTTP-ClusterB", 20) - .scan(new HashMap(), new BiFunction, Event, HashMap>() { + .scan(new HashMap<>(), new BiFunction, Event, HashMap>() { @Override public HashMap apply(HashMap accum, Event perInstanceEvent) { accum.put("instance", perInstanceEvent.instanceId); @@ -463,7 +463,7 @@ public void accept(HashMap v) { @Test public void scanWithSeedDoesNotEmitErrorTwiceIfScanFunctionThrows() { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); Consumer errorConsumer = new Consumer() { @Override public void accept(Throwable t) throws Exception { @@ -543,7 +543,7 @@ public void scanNoSeed() { @Test public void scanNoSeedDoesNotEmitErrorTwiceIfScanFunctionThrows() { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); Consumer errorConsumer = new Consumer() { @Override public void accept(Throwable t) throws Exception { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualTest.java index e94bf06167..e5e74e8227 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualTest.java @@ -384,7 +384,7 @@ protected void subscribeActual(Subscriber s) { }; for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final PublishProcessor pp = PublishProcessor.create(); @@ -487,7 +487,7 @@ protected void subscribeActual(Subscriber s) { }; for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); final PublishProcessor pp = PublishProcessor.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleTest.java index 3e2fdae2a0..099f63f049 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleTest.java @@ -74,7 +74,7 @@ public void singleWithEmptyFlowable() { @Test public void singleDoesNotRequestMoreThanItNeedsIf1Then2RequestedFlowable() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.just(1) // .doOnRequest(new LongConsumer() { @@ -115,7 +115,7 @@ public void onNext(Integer t) { @Test public void singleDoesNotRequestMoreThanItNeedsIf3RequestedFlowable() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.just(1) // .doOnRequest(new LongConsumer() { @@ -155,7 +155,7 @@ public void onNext(Integer t) { @Test public void singleRequestsExactlyWhatItNeedsIf1RequestedFlowable() { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); Flowable.just(1) // .doOnRequest(new LongConsumer() { @@ -705,7 +705,7 @@ public Integer apply(Integer i1, Integer i2) { @Test public void singleElementOperatorDoNotSwallowExceptionWhenDone() { final Throwable exception = new RuntimeException("some error"); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); try { RxJavaPlugins.setErrorHandler(new Consumer() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTest.java index be4230486a..53e4e24a93 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTest.java @@ -89,7 +89,7 @@ public void skipLastWithZeroCount() { @Test public void skipLastWithBackpressure() { Flowable f = Flowable.range(0, Flowable.bufferSize() * 2).skipLast(Flowable.bufferSize() + 10); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); f.observeOn(Schedulers.computation()).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipTest.java index 26128e5889..deb672124a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipTest.java @@ -143,7 +143,7 @@ public void skipError() { @Test public void backpressureMultipleSmallAsyncRequests() throws InterruptedException { final AtomicLong requests = new AtomicLong(0); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.interval(100, TimeUnit.MILLISECONDS) .doOnRequest(new LongConsumer() { @Override @@ -162,7 +162,7 @@ public void accept(long n) { @Test public void requestOverflowDoesNotOccur() { - TestSubscriberEx ts = new TestSubscriberEx(Long.MAX_VALUE - 1); + TestSubscriberEx ts = new TestSubscriberEx<>(Long.MAX_VALUE - 1); Flowable.range(1, 10).skip(5).subscribe(ts); ts.assertTerminated(); ts.assertComplete(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOnTest.java index a10283358e..a9f66d733e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOnTest.java @@ -41,7 +41,7 @@ public void issue813() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch doneLatch = new CountDownLatch(1); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable .unsafeCreate(new Publisher() { @@ -79,7 +79,7 @@ public void subscribe( @Test public void onError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.unsafeCreate(new Publisher() { @Override @@ -152,7 +152,7 @@ public Disposable schedule(@NonNull final Runnable action, final long delayTime, @Test public void unsubscribeInfiniteStream() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger count = new AtomicInteger(); Flowable.unsafeCreate(new Publisher() { @@ -178,7 +178,7 @@ public void subscribe(Subscriber sub) { @Test public void backpressureReschedulesCorrectly() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(10); - TestSubscriberEx ts = new TestSubscriberEx(new DefaultSubscriber() { + TestSubscriberEx ts = new TestSubscriberEx<>(new DefaultSubscriber() { @Override public void onComplete() { @@ -208,7 +208,7 @@ public void onNext(Integer t) { @Test public void setProducerSynchronousRequest() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2, 3).lift(new FlowableOperator() { @Override @@ -283,11 +283,11 @@ public void dispose() { public void deferredRequestRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Worker w = Schedulers.computation().createWorker(); - final SubscribeOnSubscriber so = new SubscribeOnSubscriber(ts, w, Flowable.never(), true); + final SubscribeOnSubscriber so = new SubscribeOnSubscriber<>(ts, w, Flowable.never(), true); ts.onSubscribe(so); final BooleanSubscription bs = new BooleanSubscription(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmptyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmptyTest.java index 9ee55f4dfa..de348ee6a0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmptyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmptyTest.java @@ -142,7 +142,7 @@ public void subscribe(final Subscriber subscriber) { @Test public void switchRequestAlternativeObservableWithBackpressure() { - TestSubscriber ts = new TestSubscriber(1L); + TestSubscriber ts = new TestSubscriber<>(1L); Flowable.empty().switchIfEmpty(Flowable.just(1, 2, 3)).subscribe(ts); @@ -156,7 +156,7 @@ public void switchRequestAlternativeObservableWithBackpressure() { @Test public void backpressureNoRequest() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.empty().switchIfEmpty(Flowable.just(1, 2, 3)).subscribe(ts); ts.assertNoValues(); ts.assertNoErrors(); @@ -164,7 +164,7 @@ public void backpressureNoRequest() { @Test public void backpressureOnFirstObservable() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.just(1, 2, 3).switchIfEmpty(Flowable.just(4, 5, 6)).subscribe(ts); ts.assertNotComplete(); ts.assertNoErrors(); @@ -173,7 +173,7 @@ public void backpressureOnFirstObservable() { @Test public void requestsNotLost() throws InterruptedException { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Flowable.unsafeCreate(new Publisher() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchTest.java index fa83213599..fb571c1109 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchTest.java @@ -450,7 +450,7 @@ public void backpressure() { publishCompleted(o2, 50); publishCompleted(o3, 55); - final TestSubscriberEx testSubscriber = new TestSubscriberEx(); + final TestSubscriberEx testSubscriber = new TestSubscriberEx<>(); Flowable.switchOnNext(o).subscribe(new DefaultSubscriber() { private int requested; @@ -549,7 +549,7 @@ public void onNext(String t) { @Test public void initialRequestsAreAdditive() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.switchOnNext( Flowable.interval(100, TimeUnit.MILLISECONDS) .map( @@ -568,7 +568,7 @@ public Flowable apply(Long t) { @Test public void initialRequestsDontOverflow() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.switchOnNext( Flowable.interval(100, TimeUnit.MILLISECONDS) .map(new Function>() { @@ -585,7 +585,7 @@ public Flowable apply(Long t) { @Test public void secondaryRequestsDontOverflow() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.switchOnNext( Flowable.interval(100, TimeUnit.MILLISECONDS) .map(new Function>() { @@ -639,7 +639,7 @@ public void delayErrors() { @Test public void switchOnNextPrefetch() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable source = Flowable.range(1, 10).hide().doOnNext(new Consumer() { @Override @@ -656,7 +656,7 @@ public void accept(Integer v) throws Exception { @Test public void switchOnNextDelayError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable source = Flowable.range(1, 10).hide().doOnNext(new Consumer() { @Override @@ -673,7 +673,7 @@ public void accept(Integer v) throws Exception { @Test public void switchOnNextDelayErrorPrefetch() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable source = Flowable.range(1, 10).hide().doOnNext(new Consumer() { @Override @@ -1082,7 +1082,7 @@ protected void subscribeActual(Subscriber s) { @Test public void drainCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final PublishProcessor pp = PublishProcessor.create(); @@ -1176,7 +1176,7 @@ public Object apply(Integer w) throws Exception { public void undeliverableUponCancel() { List errors = TestHelper.trackPluginErrors(); try { - final TestSubscriberEx ts = new TestSubscriberEx(); + final TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1) .map(new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOneTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOneTest.java index 8acf9c3026..373fe4fbe4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOneTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOneTest.java @@ -30,7 +30,7 @@ public class FlowableTakeLastOneTest extends RxJavaTest { @Test public void lastOfManyReturnsLast() { - TestSubscriberEx s = new TestSubscriberEx(); + TestSubscriberEx s = new TestSubscriberEx<>(); Flowable.range(1, 10).takeLast(1).subscribe(s); s.assertValue(10); s.assertNoErrors(); @@ -41,7 +41,7 @@ public void lastOfManyReturnsLast() { @Test public void lastOfEmptyReturnsEmpty() { - TestSubscriberEx s = new TestSubscriberEx(); + TestSubscriberEx s = new TestSubscriberEx<>(); Flowable.empty().takeLast(1).subscribe(s); s.assertNoValues(); s.assertNoErrors(); @@ -52,7 +52,7 @@ public void lastOfEmptyReturnsEmpty() { @Test public void lastOfOneReturnsLast() { - TestSubscriberEx s = new TestSubscriberEx(); + TestSubscriberEx s = new TestSubscriberEx<>(); Flowable.just(1).takeLast(1).subscribe(s); s.assertValue(1); s.assertNoErrors(); @@ -81,7 +81,7 @@ public void run() { @Test public void lastWithBackpressure() { - MySubscriber s = new MySubscriber(0); + MySubscriber s = new MySubscriber<>(0); Flowable.just(1).takeLast(1).subscribe(s); assertEquals(0, s.list.size()); s.requestMore(1); @@ -111,7 +111,7 @@ private static class MySubscriber extends DefaultSubscriber { this.initialRequest = initialRequest; } - final List list = new ArrayList(); + final List list = new ArrayList<>(); public void requestMore(long n) { request(n); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTest.java index 764e932405..c62ff85db0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTest.java @@ -96,7 +96,7 @@ public void takeLastWithNegativeCount() { @Test public void backpressure1() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 100000).takeLast(1) .observeOn(Schedulers.newThread()) .map(newSlowProcessor()).subscribe(ts); @@ -107,7 +107,7 @@ public void backpressure1() { @Test public void backpressure2() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 100000).takeLast(Flowable.bufferSize() * 4) .observeOn(Schedulers.newThread()).map(newSlowProcessor()).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -283,7 +283,7 @@ public void onNext(Integer integer) { @Test public void requestOverflow() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.range(1, 100).takeLast(50).subscribe(new DefaultSubscriber() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimedTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimedTest.java index 745cdf01dd..3f8c22a47f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimedTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimedTest.java @@ -208,7 +208,7 @@ public void takeLastTimedWithZeroCapacity() { public void continuousDelivery() { TestScheduler scheduler = new TestScheduler(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); PublishProcessor pp = PublishProcessor.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest.java index 3174c62721..fa9ea0725d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest.java @@ -290,7 +290,7 @@ public void subscribe(Subscriber op) { @Test public void takeObserveOn() { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); INFINITE_OBSERVABLE.onBackpressureDrop() .observeOn(Schedulers.newThread()).take(1).subscribe(ts); @@ -305,7 +305,7 @@ public void takeObserveOn() { @Test public void producerRequestThroughTake() { - TestSubscriber ts = new TestSubscriber(3); + TestSubscriber ts = new TestSubscriber<>(3); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -331,7 +331,7 @@ public void cancel() { @Test public void producerRequestThroughTakeIsModified() { - TestSubscriber ts = new TestSubscriber(3); + TestSubscriber ts = new TestSubscriber<>(3); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -358,7 +358,7 @@ public void cancel() { @Test public void interrupt() throws InterruptedException { - final AtomicReference exception = new AtomicReference(); + final AtomicReference exception = new AtomicReference<>(); final CountDownLatch latch = new CountDownLatch(1); Flowable.just(1).subscribeOn(Schedulers.computation()).take(1) .subscribe(new Consumer() { @@ -384,7 +384,7 @@ public void accept(Integer t1) { @Test public void doesntRequestMoreThanNeededFromUpstream() throws InterruptedException { final AtomicLong requests = new AtomicLong(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.interval(100, TimeUnit.MILLISECONDS) // .doOnRequest(new LongConsumer() { @@ -429,7 +429,7 @@ public void onNext(Integer t) { public void reentrantTake() { final PublishProcessor source = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.take(1).doOnNext(new Consumer() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest2.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest2.java index 14987ce16a..089c148099 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest2.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeTest2.java @@ -32,7 +32,7 @@ // moved tests from FlowableLimitTest to here (limit removed as operator) public class FlowableTakeTest2 extends RxJavaTest implements LongConsumer, Action { - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); static final Long CANCELLED = -100L; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicateTest.java index 4bdda03eb1..4019ec4fd9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicateTest.java @@ -142,7 +142,7 @@ public boolean test(Integer v) { @Test public void backpressure() { - TestSubscriber ts = new TestSubscriber(5L); + TestSubscriber ts = new TestSubscriber<>(5L); Flowable.range(1, 1000).takeUntil(new Predicate() { @Override @@ -158,7 +158,7 @@ public boolean test(Integer v) { @Test public void errorIncludesLastValueAsCause() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final TestException e = new TestException("Forced failure"); Predicate predicate = new Predicate() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilTest.java index 7db84c13d8..150143ba6f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilTest.java @@ -190,7 +190,7 @@ public void untilFires() { PublishProcessor source = PublishProcessor.create(); PublishProcessor until = PublishProcessor.create(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.takeUntil(until).subscribe(ts); @@ -216,7 +216,7 @@ public void mainCompletes() { PublishProcessor source = PublishProcessor.create(); PublishProcessor until = PublishProcessor.create(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.takeUntil(until).subscribe(ts); @@ -240,7 +240,7 @@ public void downstreamUnsubscribes() { PublishProcessor source = PublishProcessor.create(); PublishProcessor until = PublishProcessor.create(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); source.takeUntil(until).take(1).subscribe(ts); @@ -262,7 +262,7 @@ public void downstreamUnsubscribes() { public void backpressure() { PublishProcessor until = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.range(1, 10).takeUntil(until).subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhileTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhileTest.java index 90f9cc4a59..f2c964cf28 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhileTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhileTest.java @@ -236,7 +236,7 @@ public boolean test(Integer t1) { return t1 < 100; } }); - TestSubscriber ts = new TestSubscriber(5L); + TestSubscriber ts = new TestSubscriber<>(5L); source.subscribe(ts); @@ -257,7 +257,7 @@ public boolean test(Integer t1) { return t1 < 2; } }); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); source.subscribe(ts); @@ -269,7 +269,7 @@ public boolean test(Integer t1) { @Test public void errorCauseIncludesLastValue() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just("abc").takeWhile(new Predicate() { @Override public boolean test(String t1) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java index ac517661cb..496bce4a38 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java @@ -61,11 +61,11 @@ public void timeInterval() { processor.onComplete(); inOrder.verify(subscriber, times(1)).onNext( - new Timed(1, 1000, TIME_UNIT)); + new Timed<>(1, 1000, TIME_UNIT)); inOrder.verify(subscriber, times(1)).onNext( - new Timed(2, 2000, TIME_UNIT)); + new Timed<>(2, 2000, TIME_UNIT)); inOrder.verify(subscriber, times(1)).onNext( - new Timed(3, 3000, TIME_UNIT)); + new Timed<>(3, 3000, TIME_UNIT)); inOrder.verify(subscriber, times(1)).onComplete(); inOrder.verifyNoMoreInteractions(); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTests.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTests.java index 144d0da6a8..88a3bc2253 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTests.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTests.java @@ -52,7 +52,7 @@ public void setUp() { @Test public void shouldNotTimeoutIfOnNextWithinTimeout() { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); withTimeout.subscribe(ts); @@ -67,7 +67,7 @@ public void shouldNotTimeoutIfOnNextWithinTimeout() { @Test public void shouldNotTimeoutIfSecondOnNextWithinTimeout() { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); withTimeout.subscribe(ts); @@ -83,7 +83,7 @@ public void shouldNotTimeoutIfSecondOnNextWithinTimeout() { @Test public void shouldTimeoutIfOnNextNotWithinTimeout() { - TestSubscriberEx subscriber = new TestSubscriberEx(); + TestSubscriberEx subscriber = new TestSubscriberEx<>(); withTimeout.subscribe(subscriber); @@ -93,8 +93,8 @@ public void shouldTimeoutIfOnNextNotWithinTimeout() { @Test public void shouldTimeoutIfSecondOnNextNotWithinTimeout() { - TestSubscriberEx subscriber = new TestSubscriberEx(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriberEx subscriber = new TestSubscriberEx<>(); + TestSubscriber ts = new TestSubscriber<>(subscriber); withTimeout.subscribe(subscriber); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); underlyingSubject.onNext("One"); @@ -107,7 +107,7 @@ public void shouldTimeoutIfSecondOnNextNotWithinTimeout() { @Test public void shouldCompleteIfUnderlyingComletes() { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); withTimeout.subscribe(subscriber); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); underlyingSubject.onComplete(); @@ -120,7 +120,7 @@ public void shouldCompleteIfUnderlyingComletes() { @Test public void shouldErrorIfUnderlyingErrors() { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); withTimeout.subscribe(subscriber); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); underlyingSubject.onError(new UnsupportedOperationException()); @@ -135,7 +135,7 @@ public void shouldSwitchToOtherIfOnNextNotWithinTimeout() { Flowable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); source.subscribe(ts); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -158,7 +158,7 @@ public void shouldSwitchToOtherIfOnErrorNotWithinTimeout() { Flowable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); source.subscribe(ts); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -181,7 +181,7 @@ public void shouldSwitchToOtherIfOnCompletedNotWithinTimeout() { Flowable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); source.subscribe(ts); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -204,7 +204,7 @@ public void shouldSwitchToOtherAndCanBeUnsubscribedIfOnNextNotWithinTimeout() { Flowable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); source.subscribe(ts); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -234,7 +234,7 @@ public void shouldTimeoutIfSynchronizedFlowableEmitFirstOnNextNotWithinTimeout() final CountDownLatch exit = new CountDownLatch(1); final CountDownLatch timeoutSetuped = new CountDownLatch(1); - final TestSubscriberEx subscriber = new TestSubscriberEx(); + final TestSubscriberEx subscriber = new TestSubscriberEx<>(); new Thread(new Runnable() { @@ -283,7 +283,7 @@ public void subscribe(Subscriber subscriber) { TestScheduler testScheduler = new TestScheduler(); Flowable observableWithTimeout = never.timeout(1000, TimeUnit.MILLISECONDS, testScheduler); - TestSubscriberEx subscriber = new TestSubscriberEx(); + TestSubscriberEx subscriber = new TestSubscriberEx<>(); observableWithTimeout.subscribe(subscriber); testScheduler.advanceTimeBy(2000, TimeUnit.MILLISECONDS); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutWithSelectorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutWithSelectorTest.java index ff0b91113a..7e55fa0d77 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutWithSelectorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutWithSelectorTest.java @@ -329,7 +329,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(subscriber).onComplete(); - final TestSubscriber ts = new TestSubscriber(subscriber); + final TestSubscriber ts = new TestSubscriber<>(subscriber); new Thread(new Runnable() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimerTest.java index bc8b67d737..e23228f943 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimerTest.java @@ -63,7 +63,7 @@ public void timerOnce() { @Test public void timerPeriodically() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.interval(100, 100, TimeUnit.MILLISECONDS, scheduler).subscribe(ts); @@ -91,7 +91,7 @@ public void timerPeriodically() { @Test public void interval() { Flowable w = Flowable.interval(1, TimeUnit.SECONDS, scheduler); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); w.subscribe(ts); ts.assertNoValues(); @@ -116,8 +116,8 @@ public void interval() { public void withMultipleSubscribersStartingAtSameTime() { Flowable w = Flowable.interval(1, TimeUnit.SECONDS, scheduler); - TestSubscriber ts1 = new TestSubscriber(); - TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); + TestSubscriber ts2 = new TestSubscriber<>(); w.subscribe(ts1); w.subscribe(ts2); @@ -153,7 +153,7 @@ public void withMultipleSubscribersStartingAtSameTime() { public void withMultipleStaggeredSubscribers() { Flowable w = Flowable.interval(1, TimeUnit.SECONDS, scheduler); - TestSubscriber ts1 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); w.subscribe(ts1); @@ -161,7 +161,7 @@ public void withMultipleStaggeredSubscribers() { scheduler.advanceTimeTo(2, TimeUnit.SECONDS); - TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts2 = new TestSubscriber<>(); w.subscribe(ts2); @@ -193,7 +193,7 @@ public void withMultipleStaggeredSubscribers() { public void withMultipleStaggeredSubscribersAndPublish() { ConnectableFlowable w = Flowable.interval(1, TimeUnit.SECONDS, scheduler).publish(); - TestSubscriber ts1 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); w.subscribe(ts1); w.connect(); @@ -202,7 +202,7 @@ public void withMultipleStaggeredSubscribersAndPublish() { scheduler.advanceTimeTo(2, TimeUnit.SECONDS); - TestSubscriber ts2 = new TestSubscriber(); + TestSubscriber ts2 = new TestSubscriber<>(); w.subscribe(ts2); ts1.assertValues(0L, 1L); @@ -309,7 +309,7 @@ public void backpressureNotReady() { @Test public void timerCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final TestScheduler scheduler = new TestScheduler(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimestampTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimestampTest.java index 08bbcdc50f..05274da619 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimestampTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimestampTest.java @@ -53,9 +53,9 @@ public void timestampWithScheduler() { InOrder inOrder = inOrder(subscriber); - inOrder.verify(subscriber, times(1)).onNext(new Timed(1, 0, TimeUnit.MILLISECONDS)); - inOrder.verify(subscriber, times(1)).onNext(new Timed(2, 100, TimeUnit.MILLISECONDS)); - inOrder.verify(subscriber, times(1)).onNext(new Timed(3, 200, TimeUnit.MILLISECONDS)); + inOrder.verify(subscriber, times(1)).onNext(new Timed<>(1, 0, TimeUnit.MILLISECONDS)); + inOrder.verify(subscriber, times(1)).onNext(new Timed<>(2, 100, TimeUnit.MILLISECONDS)); + inOrder.verify(subscriber, times(1)).onNext(new Timed<>(3, 200, TimeUnit.MILLISECONDS)); verify(subscriber, never()).onError(any(Throwable.class)); verify(subscriber, never()).onComplete(); @@ -77,9 +77,9 @@ public void timestampWithScheduler2() { InOrder inOrder = inOrder(subscriber); - inOrder.verify(subscriber, times(1)).onNext(new Timed(1, 0, TimeUnit.MILLISECONDS)); - inOrder.verify(subscriber, times(1)).onNext(new Timed(2, 0, TimeUnit.MILLISECONDS)); - inOrder.verify(subscriber, times(1)).onNext(new Timed(3, 200, TimeUnit.MILLISECONDS)); + inOrder.verify(subscriber, times(1)).onNext(new Timed<>(1, 0, TimeUnit.MILLISECONDS)); + inOrder.verify(subscriber, times(1)).onNext(new Timed<>(2, 0, TimeUnit.MILLISECONDS)); + inOrder.verify(subscriber, times(1)).onNext(new Timed<>(3, 200, TimeUnit.MILLISECONDS)); verify(subscriber, never()).onError(any(Throwable.class)); verify(subscriber, never()).onComplete(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToCompletableTest.java index 84c52ab2c3..cac1fa61b7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToCompletableTest.java @@ -73,7 +73,7 @@ public void emptyObservable() { @Test public void neverObservable() { - TestSubscriberEx subscriber = new TestSubscriberEx(); + TestSubscriberEx subscriber = new TestSubscriberEx<>(); Completable cmp = Flowable.never().ignoreElements(); cmp.toFlowable().subscribe(subscriber); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToFutureTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToFutureTest.java index 28a61aa223..524a7216e1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToFutureTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToFutureTest.java @@ -38,7 +38,7 @@ public void success() throws Exception { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); Flowable.fromFuture(future).subscribe(ts); @@ -60,7 +60,7 @@ public void successOperatesOnSuppliedScheduler() throws Exception { Subscriber subscriber = TestHelper.mockSubscriber(); TestScheduler scheduler = new TestScheduler(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); Flowable.fromFuture(future, scheduler).subscribe(ts); @@ -80,7 +80,7 @@ public void failure() throws Exception { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); Flowable.fromFuture(future).subscribe(ts); @@ -101,7 +101,7 @@ public void cancelledBeforeSubscribe() throws Exception { Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); ts.cancel(); Flowable.fromFuture(future).subscribe(ts); @@ -147,7 +147,7 @@ public Object get(long timeout, TimeUnit unit) throws InterruptedException, Exec Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); Flowable futureObservable = Flowable.fromFuture(future); futureObservable.subscribeOn(Schedulers.computation()).subscribe(ts); @@ -163,9 +163,9 @@ public Object get(long timeout, TimeUnit unit) throws InterruptedException, Exec @Test public void backpressure() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); - FutureTask f = new FutureTask(new Runnable() { + FutureTask f = new FutureTask<>(new Runnable() { @Override public void run() { @@ -187,7 +187,7 @@ public void run() { @Test public void withTimeoutNoTimeout() { - FutureTask task = new FutureTask(new Runnable() { + FutureTask task = new FutureTask<>(new Runnable() { @Override public void run() { @@ -207,7 +207,7 @@ public void run() { @Test public void withTimeoutTimeout() { - FutureTask task = new FutureTask(new Runnable() { + FutureTask task = new FutureTask<>(new Runnable() { @Override public void run() { @@ -225,7 +225,7 @@ public void run() { @Test public void withTimeoutNoTimeoutScheduler() { - FutureTask task = new FutureTask(new Runnable() { + FutureTask task = new FutureTask<>(new Runnable() { @Override public void run() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java index 66d2164b70..73d1455d5b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java @@ -92,7 +92,7 @@ public void listWithBlockingFirstFlowable() { @Test public void backpressureHonoredFlowable() { Flowable> w = Flowable.just(1, 2, 3, 4, 5).toList().toFlowable(); - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); w.subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMapTest.java index 8635158cc3..149a3f893c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMapTest.java @@ -54,7 +54,7 @@ public void toMapFlowable() { Flowable> mapped = source.toMap(lengthFunc).toFlowable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -73,7 +73,7 @@ public void toMapWithValueSelectorFlowable() { Flowable> mapped = source.toMap(lengthFunc, duplicate).toFlowable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -101,7 +101,7 @@ public Integer apply(String t1) { }; Flowable> mapped = source.toMap(lengthFuncErr).toFlowable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -131,7 +131,7 @@ public String apply(String t1) { Flowable> mapped = source.toMap(lengthFunc, duplicateErr).toFlowable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -177,7 +177,7 @@ public String apply(String v) { } }, mapFactory).toFlowable(); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); @@ -213,7 +213,7 @@ public String apply(String v) { } }, mapFactory).toFlowable(); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); @@ -231,7 +231,7 @@ public void toMap() { Single> mapped = source.toMap(lengthFunc); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -249,7 +249,7 @@ public void toMapWithValueSelector() { Single> mapped = source.toMap(lengthFunc, duplicate); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -276,7 +276,7 @@ public Integer apply(String t1) { }; Single> mapped = source.toMap(lengthFuncErr); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -305,7 +305,7 @@ public String apply(String t1) { Single> mapped = source.toMap(lengthFunc, duplicateErr); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -350,7 +350,7 @@ public String apply(String v) { } }, mapFactory); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); @@ -385,7 +385,7 @@ public String apply(String v) { } }, mapFactory); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMultimapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMultimapTest.java index 2413ed6afe..06e71d6beb 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMultimapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToMultimapTest.java @@ -55,7 +55,7 @@ public void toMultimapFlowable() { Flowable>> mapped = source.toMultimap(lengthFunc).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -72,7 +72,7 @@ public void toMultimapWithValueSelectorFlowable() { Flowable>> mapped = source.toMultimap(lengthFunc, duplicate).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -114,11 +114,11 @@ public String apply(String v) { mapFactory, new Function>() { @Override public Collection apply(Integer e) { - return new ArrayList(); + return new ArrayList<>(); } }).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -137,9 +137,9 @@ public void toMultimapWithCollectionFactoryFlowable() { @Override public Collection apply(Integer t1) { if (t1 == 2) { - return new ArrayList(); + return new ArrayList<>(); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -153,16 +153,16 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Flowable>> mapped = source .toMultimap(lengthFunc, identity, mapSupplier, collectionFactory).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, new HashSet(Arrays.asList("eee"))); + expected.put(3, new HashSet<>(Arrays.asList("eee"))); mapped.subscribe(objectSubscriber); @@ -187,7 +187,7 @@ public Integer apply(String t1) { Flowable>> mapped = source.toMultimap(lengthFuncErr).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -214,7 +214,7 @@ public String apply(String t1) { Flowable>> mapped = source.toMultimap(lengthFunc, duplicateErr).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -244,7 +244,7 @@ public String apply(String v) { } }, mapFactory).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -265,7 +265,7 @@ public Collection apply(Integer t1) { if (t1 == 2) { throw new RuntimeException("Forced failure"); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -279,14 +279,14 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Flowable>> mapped = source.toMultimap(lengthFunc, identity, mapSupplier, collectionFactory).toFlowable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Collections.singleton("eee")); @@ -303,7 +303,7 @@ public void toMultimap() { Single>> mapped = source.toMultimap(lengthFunc); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -319,7 +319,7 @@ public void toMultimapWithValueSelector() { Single>> mapped = source.toMultimap(lengthFunc, duplicate); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -360,11 +360,11 @@ public String apply(String v) { mapFactory, new Function>() { @Override public Collection apply(Integer e) { - return new ArrayList(); + return new ArrayList<>(); } }); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -382,9 +382,9 @@ public void toMultimapWithCollectionFactory() { @Override public Collection apply(Integer t1) { if (t1 == 2) { - return new ArrayList(); + return new ArrayList<>(); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -398,16 +398,16 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Single>> mapped = source .toMultimap(lengthFunc, identity, mapSupplier, collectionFactory); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, new HashSet(Arrays.asList("eee"))); + expected.put(3, new HashSet<>(Arrays.asList("eee"))); mapped.subscribe(singleObserver); @@ -431,7 +431,7 @@ public Integer apply(String t1) { Single>> mapped = source.toMultimap(lengthFuncErr); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -457,7 +457,7 @@ public String apply(String t1) { Single>> mapped = source.toMultimap(lengthFunc, duplicateErr); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -486,7 +486,7 @@ public String apply(String v) { } }, mapFactory); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -506,7 +506,7 @@ public Collection apply(Integer t1) { if (t1 == 2) { throw new RuntimeException("Forced failure"); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -520,14 +520,14 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Single>> mapped = source.toMultimap(lengthFunc, identity, mapSupplier, collectionFactory); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Collections.singleton("eee")); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java index 96ec70ef69..6012db0add 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java @@ -72,7 +72,7 @@ public void withFollowingFirstFlowable() { @Test public void backpressureHonoredFlowable() { Flowable> w = Flowable.just(1, 3, 2, 5, 4).toSortedList().toFlowable(); - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); w.subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOnTest.java index 6457f5ae26..1f37dbddce 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOnTest.java @@ -38,7 +38,7 @@ public void unsubscribeWhenSubscribeOnAndUnsubscribeOnAreOnSameThread() throws I UIEventLoopScheduler uiEventLoop = new UIEventLoopScheduler(); try { final ThreadSubscription subscription = new ThreadSubscription(); - final AtomicReference subscribeThread = new AtomicReference(); + final AtomicReference subscribeThread = new AtomicReference<>(); Flowable w = Flowable.unsafeCreate(new Publisher() { @Override @@ -54,7 +54,7 @@ public void subscribe(Subscriber t1) { } }); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); w.subscribeOn(uiEventLoop).observeOn(Schedulers.computation()) .unsubscribeOn(uiEventLoop) .take(2) @@ -87,7 +87,7 @@ public void unsubscribeWhenSubscribeOnAndUnsubscribeOnAreOnDifferentThreads() th UIEventLoopScheduler uiEventLoop = new UIEventLoopScheduler(); try { final ThreadSubscription subscription = new ThreadSubscription(); - final AtomicReference subscribeThread = new AtomicReference(); + final AtomicReference subscribeThread = new AtomicReference<>(); Flowable w = Flowable.unsafeCreate(new Publisher() { @Override @@ -103,7 +103,7 @@ public void subscribe(Subscriber t1) { } }); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); w.subscribeOn(Schedulers.newThread()).observeOn(Schedulers.computation()) .unsubscribeOn(uiEventLoop) .take(2) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java index 37d9195175..f32b599df4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsingTest.java @@ -233,7 +233,7 @@ public Flowable apply(Disposable subscription) { @Test public void usingDisposesEagerlyBeforeCompletion() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); Supplier resourceFactory = createResourceFactory(events); final Action completion = createOnCompletedAction(events); final Action unsub = createUnsubAction(events); @@ -260,7 +260,7 @@ public Flowable apply(Resource resource) { @Test public void usingDoesNotDisposesEagerlyBeforeCompletion() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); Supplier resourceFactory = createResourceFactory(events); final Action completion = createOnCompletedAction(events); final Action unsub = createUnsubAction(events); @@ -287,7 +287,7 @@ public Flowable apply(Resource resource) { @Test public void usingDisposesEagerlyBeforeError() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); Supplier resourceFactory = createResourceFactory(events); final Consumer onError = createOnErrorAction(events); final Action unsub = createUnsubAction(events); @@ -315,7 +315,7 @@ public Flowable apply(Resource resource) { @Test public void usingDoesNotDisposesEagerlyBeforeError() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); final Supplier resourceFactory = createResourceFactory(events); final Consumer onError = createOnErrorAction(events); final Action unsub = createUnsubAction(events); @@ -604,7 +604,7 @@ public Flowable apply(Flowable f) @Test public void eagerDisposedOnComplete() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.using(Functions.justSupplier(1), Functions.justFunction(new Flowable() { @Override @@ -619,7 +619,7 @@ protected void subscribeActual(Subscriber subscriber) { @Test public void eagerDisposedOnError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.using(Functions.justSupplier(1), Functions.justFunction(new Flowable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java index 8b001803c0..c17e2ccfae 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java @@ -43,7 +43,7 @@ public void windowViaFlowableNormal1() { final Subscriber subscriber = TestHelper.mockSubscriber(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Subscriber> wo = new DefaultSubscriber>() { @Override @@ -100,7 +100,7 @@ public void windowViaFlowableBoundaryCompletes() { final Subscriber subscriber = TestHelper.mockSubscriber(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Subscriber> wo = new DefaultSubscriber>() { @Override @@ -156,7 +156,7 @@ public void windowViaFlowableBoundaryThrows() { final Subscriber subscriber = TestHelper.mockSubscriber(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Subscriber> wo = new DefaultSubscriber>() { @Override @@ -206,7 +206,7 @@ public void windowViaFlowableThrows() { final Subscriber subscriber = TestHelper.mockSubscriber(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Subscriber> wo = new DefaultSubscriber>() { @Override @@ -408,7 +408,7 @@ public Flowable apply( public void mainAndBoundaryBothError() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); TestSubscriberEx> ts = Flowable.error(new TestException("main")) .window(new Flowable() { @@ -447,8 +447,8 @@ public void mainCompleteBoundaryErrorRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); TestSubscriberEx> ts = new Flowable() { @Override @@ -497,8 +497,8 @@ public void run() { @Test public void mainNextBoundaryNextRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); TestSubscriber> ts = new Flowable() { @Override @@ -540,8 +540,8 @@ public void run() { @Test public void takeOneAnotherBoundary() { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); TestSubscriberEx> ts = new Flowable() { @Override @@ -572,8 +572,8 @@ protected void subscribeActual(Subscriber subscriber) { @Test public void disposeMainBoundaryCompleteRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); final TestSubscriber> ts = new Flowable() { @Override @@ -629,8 +629,8 @@ public void disposeMainBoundaryErrorRace() { final TestException ex = new TestException(); for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); final TestSubscriber> ts = new Flowable() { @Override @@ -709,7 +709,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstream() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(Flowable.never()) .doOnNext(new Consumer>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java index 4536d7a8e4..e9a7c77518 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java @@ -99,7 +99,7 @@ public void skipAndCountWindowsWithGaps() { @Test public void windowUnsubscribeNonOverlapping() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final AtomicInteger count = new AtomicInteger(); Flowable.merge(Flowable.range(1, 10000).doOnNext(new Consumer() { @@ -121,7 +121,7 @@ public void accept(Integer t1) { @Test public void windowUnsubscribeNonOverlappingAsyncSource() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final AtomicInteger count = new AtomicInteger(); Flowable.merge(Flowable.range(1, 100000) .doOnNext(new Consumer() { @@ -145,7 +145,7 @@ public void accept(Integer t1) { @Test public void windowUnsubscribeOverlapping() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final AtomicInteger count = new AtomicInteger(); Flowable.merge(Flowable.range(1, 10000).doOnNext(new Consumer() { @@ -164,7 +164,7 @@ public void accept(Integer t1) { @Test public void windowUnsubscribeOverlappingAsyncSource() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final AtomicInteger count = new AtomicInteger(); Flowable.merge(Flowable.range(1, 100000) .doOnNext(new Consumer() { @@ -187,7 +187,7 @@ public void accept(Integer t1) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -198,7 +198,7 @@ private List list(String... args) { public void backpressureOuter() { Flowable> source = Flowable.range(1, 10).window(3); - final List list = new ArrayList(); + final List list = new ArrayList<>(); final Subscriber subscriber = TestHelper.mockSubscriber(); @@ -271,7 +271,7 @@ public void subscribe(Subscriber s) { @Test public void takeFlatMapCompletes() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final int indicator = 999999999; @@ -293,7 +293,7 @@ public Flowable apply(Flowable w) { @SuppressWarnings("unchecked") @Test public void backpressureOuterInexact() { - TestSubscriber> ts = new TestSubscriber>(0L); + TestSubscriber> ts = new TestSubscriber<>(0L); Flowable.range(1, 5) .window(2, 1) @@ -476,7 +476,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstreamSize() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(10) .take(1) @@ -530,7 +530,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstreamSkip() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(5, 10) .take(1) @@ -584,7 +584,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstreamOverlap() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(5, 3) .take(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java index 9420794dc5..55db25f654 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java @@ -46,8 +46,8 @@ public void before() { @Test public void flowableBasedOpenerAndCloser() { - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); Flowable source = Flowable.unsafeCreate(new Publisher() { @Override @@ -96,7 +96,7 @@ public void subscribe(Subscriber subscriber) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -128,7 +128,7 @@ public void accept(Flowable stringFlowable) { stringFlowable.subscribe(new DefaultSubscriber() { @Override public void onComplete() { - lists.add(new ArrayList(list)); + lists.add(new ArrayList<>(list)); list.clear(); } @@ -153,7 +153,7 @@ public void noUnsubscribeAndNoLeak() { PublishProcessor open = PublishProcessor.create(); final PublishProcessor close = PublishProcessor.create(); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); source.window(open, new Function>() { @Override @@ -197,7 +197,7 @@ public void unsubscribeAll() { PublishProcessor open = PublishProcessor.create(); final PublishProcessor close = PublishProcessor.create(); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); source.window(open, new Function>() { @Override @@ -481,7 +481,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstream() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(Flowable.just(1).concatWith(Flowable.never()), Functions.justFunction(Flowable.never())) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java index eb72dab5f7..86da1bb09c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java @@ -46,8 +46,8 @@ public void before() { @Test public void timedAndCount() { - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); Flowable source = Flowable.unsafeCreate(new Publisher() { @Override @@ -82,8 +82,8 @@ public void subscribe(Subscriber subscriber) { @Test public void timed() { - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); Flowable source = Flowable.unsafeCreate(new Publisher() { @Override @@ -111,7 +111,7 @@ public void subscribe(Subscriber subscriber) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -143,7 +143,7 @@ public void accept(Flowable stringFlowable) { stringFlowable.subscribe(new DefaultSubscriber() { @Override public void onComplete() { - lists.add(new ArrayList(list)); + lists.add(new ArrayList<>(list)); list.clear(); } @@ -166,8 +166,8 @@ public void exactWindowSize() { Flowable> source = Flowable.range(1, 10) .window(1, TimeUnit.MINUTES, scheduler, 3); - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); source.subscribe(observeWindow(list, lists)); @@ -184,7 +184,7 @@ public void exactWindowSize() { @Test public void takeFlatMapCompletes() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicInteger wip = new AtomicInteger(); @@ -514,7 +514,7 @@ public void overlapBackpressure2() { PublishProcessor pp = PublishProcessor.create(); - final TestSubscriber tsInner = new TestSubscriber(); + final TestSubscriber tsInner = new TestSubscriber<>(); TestSubscriber> ts = pp.window(2, 1, TimeUnit.SECONDS, scheduler) .doOnNext(new Consumer>() { @@ -1204,7 +1204,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstreamExactTime() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(10, TimeUnit.MINUTES) .take(1) @@ -1254,7 +1254,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstreamExactTimeAndSize() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(10, TimeUnit.MINUTES, 100) .take(1) @@ -1304,7 +1304,7 @@ public Publisher apply(Flowable w) throws Throwable { public void windowAbandonmentCancelsUpstreamExactTimeSkip() { PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestSubscriber> ts = pp.window(10, 15, TimeUnit.MINUTES) .take(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java index 20814684da..544ca11bc6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java @@ -90,7 +90,7 @@ public void emptySource() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -116,7 +116,7 @@ public void emptyOther() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -142,7 +142,7 @@ public void unsubscription() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); result.subscribe(ts); @@ -169,7 +169,7 @@ public void sourceThrows() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -197,7 +197,7 @@ public void otherThrows() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -225,7 +225,7 @@ public void functionThrows() { Flowable result = source.withLatestFrom(other, COMBINER_ERROR); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -251,7 +251,7 @@ public void noDownstreamUnsubscribe() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); result.subscribe(ts); @@ -267,7 +267,7 @@ public void backpressure() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); result.subscribe(ts); @@ -319,7 +319,7 @@ public void manySources() { PublishProcessor pp3 = PublishProcessor.create(); PublishProcessor main = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); main.withLatestFrom(new Flowable[] { pp1, pp2, pp3 }, toArray) .subscribe(ts); @@ -366,7 +366,7 @@ public void manySourcesIterable() { PublishProcessor pp3 = PublishProcessor.create(); PublishProcessor main = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); main.withLatestFrom(Arrays.>asList(pp1, pp2, pp3), toArray) .subscribe(ts); @@ -411,8 +411,8 @@ public void manySourcesIterableSweep() { for (String val : new String[] { "1" /*, null*/ }) { int n = 35; for (int i = 0; i < n; i++) { - List> sources = new ArrayList>(); - List expected = new ArrayList(); + List> sources = new ArrayList<>(); + List expected = new ArrayList<>(); expected.add(val); for (int j = 0; j < i; j++) { @@ -420,7 +420,7 @@ public void manySourcesIterableSweep() { expected.add(String.valueOf(val)); } - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); PublishProcessor main = PublishProcessor.create(); @@ -443,7 +443,7 @@ public void backpressureNoSignal() { PublishProcessor pp1 = PublishProcessor.create(); PublishProcessor pp2 = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.range(1, 10).withLatestFrom(new Flowable[] { pp1, pp2 }, toArray) .subscribe(ts); @@ -465,7 +465,7 @@ public void backpressureWithSignal() { PublishProcessor pp1 = PublishProcessor.create(); PublishProcessor pp2 = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.range(1, 3).withLatestFrom(new Flowable[] { pp1, pp2 }, toArray) .subscribe(ts); @@ -495,7 +495,7 @@ public void backpressureWithSignal() { @Test public void withEmpty() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.range(1, 3).withLatestFrom( new Flowable[] { Flowable.just(1), Flowable.empty() }, toArray) @@ -508,7 +508,7 @@ public void withEmpty() { @Test public void withError() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.range(1, 3).withLatestFrom( new Flowable[] { Flowable.just(1), Flowable.error(new TestException()) }, toArray) @@ -521,7 +521,7 @@ public void withError() { @Test public void withMainError() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.error(new TestException()).withLatestFrom( new Flowable[] { Flowable.just(1), Flowable.just(1) }, toArray) @@ -536,7 +536,7 @@ public void withMainError() { public void with2Others() { Flowable just = Flowable.just(1); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); just.withLatestFrom(just, just, new Function3>() { @Override @@ -555,7 +555,7 @@ public List apply(Integer a, Integer b, Integer c) { public void with3Others() { Flowable just = Flowable.just(1); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); just.withLatestFrom(just, just, just, new Function4>() { @Override @@ -574,7 +574,7 @@ public List apply(Integer a, Integer b, Integer c, Integer d) { public void with4Others() { Flowable just = Flowable.just(1); - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); just.withLatestFrom(just, just, just, just, new Function5>() { @Override @@ -609,7 +609,7 @@ public Object apply(Integer a, Integer b, Integer c) throws Exception { @Test public void manyIteratorThrows() { Flowable.just(1) - .withLatestFrom(new CrashingMappedIterable>(1, 100, 100, new Function>() { + .withLatestFrom(new CrashingMappedIterable<>(1, 100, 100, new Function>() { @Override public Flowable apply(Integer v) throws Exception { return Flowable.just(2); @@ -733,7 +733,7 @@ public void singleRequestNotForgottenWhenNoData() { Flowable result = source.withLatestFrom(other, COMBINER); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); result.subscribe(ts); @@ -794,7 +794,7 @@ public Object apply(Object a, Integer b, Integer c, Integer d) } }); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -837,7 +837,7 @@ public Object apply(Object a, Integer b, Integer c, Integer d) } }); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java index d6921bfd91..d40b3908c9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java @@ -352,7 +352,7 @@ public void aggregatorUnsubscribe() { PublishProcessor r2 = PublishProcessor.create(); /* define a Subscriber to receive aggregated events */ Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); Flowable.zip(r1, r2, zipr2).subscribe(ts); @@ -770,7 +770,7 @@ public String apply(Integer a, Integer b) { } }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); os.subscribe(new Consumer() { @Override @@ -797,7 +797,7 @@ public String apply(Integer a, Integer b) { } }).take(5); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); os.subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -822,7 +822,7 @@ public String apply(Integer a, Integer b) { } }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); os.subscribe(new DefaultSubscriber() { @Override @@ -886,7 +886,7 @@ public String apply(Notification t1, Notification t2) { }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); f.subscribe(new Consumer() { @Override @@ -915,7 +915,7 @@ public String apply(Integer t1, String t2) { }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); f.subscribe(new Consumer() { @Override @@ -942,7 +942,7 @@ public Object apply(final Object[] args) { } }); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); f.subscribe(ts); ts.awaitDone(200, TimeUnit.MILLISECONDS); ts.assertNoValues(); @@ -976,7 +976,7 @@ public void backpressureSync() { Flowable f1 = createInfiniteFlowable(generatedA); Flowable f2 = createInfiniteFlowable(generatedB); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.zip(f1, f2, new BiFunction() { @Override @@ -1000,7 +1000,7 @@ public void backpressureAsync() { Flowable f1 = createInfiniteFlowable(generatedA).subscribeOn(Schedulers.computation()); Flowable f2 = createInfiniteFlowable(generatedB).subscribeOn(Schedulers.computation()); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.zip(f1, f2, new BiFunction() { @Override @@ -1024,7 +1024,7 @@ public void downstreamBackpressureRequestsWithFiniteSyncFlowables() { Flowable f1 = createInfiniteFlowable(generatedA).take(Flowable.bufferSize() * 2); Flowable f2 = createInfiniteFlowable(generatedB).take(Flowable.bufferSize() * 2); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.zip(f1, f2, new BiFunction() { @Override @@ -1049,7 +1049,7 @@ public void downstreamBackpressureRequestsWithInfiniteAsyncFlowables() { Flowable f1 = createInfiniteFlowable(generatedA).subscribeOn(Schedulers.computation()); Flowable f2 = createInfiniteFlowable(generatedB).subscribeOn(Schedulers.computation()); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.zip(f1, f2, new BiFunction() { @Override @@ -1074,7 +1074,7 @@ public void downstreamBackpressureRequestsWithInfiniteSyncFlowables() { Flowable f1 = createInfiniteFlowable(generatedA); Flowable f2 = createInfiniteFlowable(generatedB); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.zip(f1, f2, new BiFunction() { @Override @@ -1189,7 +1189,7 @@ public Integer apply(Integer i1, Integer i2) { return i1 + i2; } }); - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 1026; i++) { expected.add(i * 3); } @@ -1247,7 +1247,7 @@ public Integer apply(Integer t1, Integer t2) { @Test public void zipRequest1() { Flowable src = Flowable.just(1).subscribeOn(Schedulers.computation()); - TestSubscriber ts = new TestSubscriber(1L); + TestSubscriber ts = new TestSubscriber<>(1L); Flowable.zip(src, src, new BiFunction() { @Override @@ -1871,7 +1871,7 @@ public Integer apply(Integer a, Integer b) throws Exception { public void firstErrorPreventsSecondSubscription() { final AtomicInteger counter = new AtomicInteger(); - List> flowableList = new ArrayList>(); + List> flowableList = new ArrayList<>(); flowableList.add(Flowable.create(new FlowableOnSubscribe() { @Override public void subscribe(FlowableEmitter e) From d6d698213b6c4091f4d5a0fb47f0922d0d0618e3 Mon Sep 17 00:00:00 2001 From: Dmitry Volkov <42734936+dvolkovv@users.noreply.github.com> Date: Sun, 22 Dec 2019 13:59:44 +0300 Subject: [PATCH 017/665] diamond operators in unit tests, observable package (#6786) --- .../BlockingObservableNextTest.java | 8 +- .../BlockingObservableToIteratorTest.java | 8 +- .../internal/operators/observable/Burst.java | 4 +- .../observable/ObservableAllTest.java | 4 +- .../observable/ObservableAmbTest.java | 4 +- .../observable/ObservableAnyTest.java | 4 +- .../observable/ObservableBlockingTest.java | 22 ++--- .../observable/ObservableBufferTest.java | 81 ++++++++++--------- .../observable/ObservableCacheTest.java | 30 +++---- .../observable/ObservableCollectTest.java | 26 +++--- .../ObservableCombineLatestTest.java | 12 +-- .../ObservableConcatMapEagerTest.java | 6 +- .../ObservableConcatMapSchedulerTest.java | 2 +- .../observable/ObservableConcatTest.java | 48 +++++------ .../ObservableConcatWithCompletableTest.java | 8 +- .../ObservableConcatWithMaybeTest.java | 10 +-- .../ObservableConcatWithSingleTest.java | 8 +- .../observable/ObservableDebounceTest.java | 18 ++--- .../ObservableDelaySubscriptionOtherTest.java | 10 +-- .../observable/ObservableDelayTest.java | 26 +++--- .../ObservableDematerializeTest.java | 2 +- .../observable/ObservableDetachTest.java | 12 +-- .../observable/ObservableDistinctTest.java | 4 +- .../ObservableDistinctUntilChangedTest.java | 4 +- .../observable/ObservableDoAfterNextTest.java | 14 ++-- .../observable/ObservableDoFinallyTest.java | 24 +++--- .../observable/ObservableDoOnEachTest.java | 20 ++--- .../ObservableDoOnSubscribeTest.java | 2 +- .../ObservableDoOnUnsubscribeTest.java | 12 +-- .../observable/ObservableFilterTest.java | 6 +- .../ObservableFlatMapCompletableTest.java | 4 +- .../ObservableFlatMapMaybeTest.java | 2 +- .../ObservableFlatMapSingleTest.java | 2 +- .../observable/ObservableFlatMapTest.java | 26 +++--- .../observable/ObservableForEachTest.java | 4 +- .../ObservableFromCallableTest.java | 6 +- .../ObservableFromIterableTest.java | 10 +-- .../ObservableFromSupplierTest.java | 6 +- .../observable/ObservableFromTest.java | 2 +- .../observable/ObservableGroupByTest.java | 56 ++++++------- .../ObservableIgnoreElementsTest.java | 8 +- .../observable/ObservableIntervalTest.java | 2 +- .../ObservableMapNotificationTest.java | 2 +- .../observable/ObservableMapTest.java | 8 +- .../observable/ObservableMaterializeTest.java | 4 +- .../ObservableMergeDelayErrorTest.java | 6 +- .../ObservableMergeMaxConcurrentTest.java | 46 +++++------ .../observable/ObservableMergeTest.java | 44 +++++----- .../ObservableMergeWithCompletableTest.java | 4 +- .../ObservableMergeWithMaybeTest.java | 2 +- .../ObservableMergeWithSingleTest.java | 2 +- .../observable/ObservableObserveOnTest.java | 16 ++-- .../ObservableOnErrorResumeNextTest.java | 8 +- .../ObservableOnErrorResumeWithTest.java | 2 +- .../ObservableOnErrorReturnTest.java | 8 +- .../observable/ObservablePublishTest.java | 28 +++---- .../observable/ObservableRangeLongTest.java | 10 +-- .../observable/ObservableRangeTest.java | 8 +- .../observable/ObservableRedoTest.java | 2 +- .../observable/ObservableRefCountTest.java | 22 ++--- .../observable/ObservableRepeatTest.java | 6 +- .../ObservableReplayEagerTruncateTest.java | 48 +++++------ .../observable/ObservableReplayTest.java | 48 +++++------ .../ObservableResourceWrapperTest.java | 12 +-- .../observable/ObservableRetryTest.java | 30 +++---- .../ObservableRetryWithPredicateTest.java | 10 +-- .../observable/ObservableScalarXMapTest.java | 22 ++--- .../observable/ObservableScanTest.java | 10 +-- .../observable/ObservableSingleTest.java | 2 +- .../observable/ObservableSkipLastTest.java | 2 +- .../observable/ObservableSkipTest.java | 2 +- .../observable/ObservableSubscribeOnTest.java | 8 +- .../observable/ObservableSwitchTest.java | 6 +- .../observable/ObservableTakeLastOneTest.java | 6 +- .../observable/ObservableTakeLastTest.java | 4 +- .../observable/ObservableTakeTest.java | 6 +- .../ObservableTakeUntilPredicateTest.java | 2 +- .../observable/ObservableTakeUntilTest.java | 6 +- .../observable/ObservableTakeWhileTest.java | 4 +- .../ObservableTimeIntervalTest.java | 6 +- .../observable/ObservableTimeoutTests.java | 24 +++--- .../ObservableTimeoutWithSelectorTest.java | 2 +- .../observable/ObservableTimerTest.java | 18 ++--- .../observable/ObservableTimestampTest.java | 12 +-- .../observable/ObservableToFutureTest.java | 10 +-- .../observable/ObservableToMapTest.java | 24 +++--- .../observable/ObservableToMultimapTest.java | 60 +++++++------- .../ObservableUnsubscribeOnTest.java | 8 +- .../observable/ObservableUsingTest.java | 12 +-- .../ObservableWindowWithObservableTest.java | 32 ++++---- .../ObservableWindowWithSizeTest.java | 20 ++--- ...vableWindowWithStartEndObservableTest.java | 14 ++-- .../ObservableWindowWithTimeTest.java | 24 +++--- .../ObservableWithLatestFromTest.java | 38 ++++----- .../observable/ObservableZipTest.java | 20 ++--- 95 files changed, 675 insertions(+), 672 deletions(-) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java index 0fcec3a172..357d512e9b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNextTest.java @@ -63,7 +63,7 @@ public void run() { } static Iterable next(ObservableSource source) { - return new BlockingObservableNext(source); + return new BlockingObservableNext<>(source); } @Test @@ -356,7 +356,7 @@ public void remove() { @Test public void nextObserverError() { - NextObserver no = new NextObserver(); + NextObserver no = new NextObserver<>(); List errors = TestHelper.trackPluginErrors(); try { @@ -370,7 +370,7 @@ public void nextObserverError() { @Test public void nextObserverOnNext() throws Exception { - NextObserver no = new NextObserver(); + NextObserver no = new NextObserver<>(); no.setWaiting(); no.onNext(Notification.createOnNext(1)); @@ -383,7 +383,7 @@ public void nextObserverOnNext() throws Exception { @Test public void nextObserverOnCompleteOnNext() throws Exception { - NextObserver no = new NextObserver(); + NextObserver no = new NextObserver<>(); no.setWaiting(); no.onNext(Notification.createOnComplete()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java index 3658c24b97..4b6e2bc5cd 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableToIteratorTest.java @@ -73,7 +73,7 @@ public void subscribe(Observer observer) { @Test public void dispose() { - BlockingObservableIterator it = new BlockingObservableIterator(128); + BlockingObservableIterator it = new BlockingObservableIterator<>(128); assertFalse(it.isDisposed()); @@ -84,7 +84,7 @@ public void dispose() { @Test public void interruptWait() { - BlockingObservableIterator it = new BlockingObservableIterator(128); + BlockingObservableIterator it = new BlockingObservableIterator<>(128); try { Thread.currentThread().interrupt(); @@ -97,14 +97,14 @@ public void interruptWait() { @Test(expected = NoSuchElementException.class) public void emptyThrowsNoSuch() { - BlockingObservableIterator it = new BlockingObservableIterator(128); + BlockingObservableIterator it = new BlockingObservableIterator<>(128); it.onComplete(); it.next(); } @Test(expected = UnsupportedOperationException.class) public void remove() { - BlockingObservableIterator it = new BlockingObservableIterator(128); + BlockingObservableIterator it = new BlockingObservableIterator<>(128); it.remove(); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java index 7e3dfbf8bf..2bed435879 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/Burst.java @@ -53,7 +53,7 @@ public static Builder item(T item) { @SafeVarargs public static Builder items(T... items) { - return new Builder(Arrays.asList(items)); + return new Builder<>(Arrays.asList(items)); } public static final class Builder { @@ -71,7 +71,7 @@ public Observable error(Throwable e) { } public Observable create() { - return new Burst(error, items); + return new Burst<>(error, items); } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java index 38be122378..293607d66b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllTest.java @@ -146,7 +146,7 @@ public Observable apply(Boolean t1) { @Test public void predicateThrowsExceptionAndValueInCauseMessageObservable() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); @@ -277,7 +277,7 @@ public Observable apply(Boolean t1) { @Test public void predicateThrowsExceptionAndValueInCauseMessage() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java index 14c720257a..239d8339ea 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java @@ -178,7 +178,7 @@ public void accept(Disposable d) { //this stream emits second Observable o2 = Observable.just(1).doOnSubscribe(incrementer) .delay(100, TimeUnit.MILLISECONDS).subscribeOn(Schedulers.computation()); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.ambArray(o1, o2).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -212,7 +212,7 @@ public void ambCancelsOthers() { PublishSubject source2 = PublishSubject.create(); PublishSubject source3 = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.ambArray(source1, source2, source3).subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java index 6ce9410340..35faf30c19 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnyTest.java @@ -247,7 +247,7 @@ public Observable apply(Boolean t1) { @Test public void predicateThrowsExceptionAndValueInCauseMessageObservable() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); Observable.just("Boo!").any(new Predicate() { @@ -469,7 +469,7 @@ public Observable apply(Boolean t1) { @Test public void predicateThrowsExceptionAndValueInCauseMessage() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final IllegalArgumentException ex = new IllegalArgumentException(); Observable.just("Boo!").any(new Predicate() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java index c8454452b4..a2d9854569 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingTest.java @@ -48,7 +48,7 @@ public void blockingFirstDefault() { @Test public void blockingSubscribeConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -64,7 +64,7 @@ public void accept(Integer v) throws Exception { @Test public void blockingSubscribeConsumerConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -80,7 +80,7 @@ public void accept(Integer v) throws Exception { @Test public void blockingSubscribeConsumerConsumerError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); TestException ex = new TestException(); @@ -100,7 +100,7 @@ public void accept(Object v) throws Exception { @Test public void blockingSubscribeConsumerConsumerAction() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Consumer cons = new Consumer() { @Override @@ -123,7 +123,7 @@ public void run() throws Exception { @Test public void blockingSubscribeObserver() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.range(1, 5) .subscribeOn(Schedulers.computation()) @@ -156,7 +156,7 @@ public void onComplete() { @Test public void blockingSubscribeObserverError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); final TestException ex = new TestException(); @@ -232,7 +232,7 @@ public void utilityClass() { @Test public void disposeUpFront() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.dispose(); Observable.just(1).blockingSubscribe(to); @@ -242,7 +242,7 @@ public void disposeUpFront() { @SuppressWarnings("rawtypes") @Test public void delayed() throws Exception { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); final Observer[] s = { null }; Schedulers.single().scheduleDirect(new Runnable() { @@ -271,14 +271,14 @@ protected void subscribeActual(Observer observer) { @Test public void interrupt() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Thread.currentThread().interrupt(); Observable.never().blockingSubscribe(to); } @Test public void onCompleteDelayed() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.empty().delay(100, TimeUnit.MILLISECONDS) .blockingSubscribe(to); @@ -288,7 +288,7 @@ public void onCompleteDelayed() { @Test public void blockingCancelUpfront() { - BlockingFirstObserver o = new BlockingFirstObserver(); + BlockingFirstObserver o = new BlockingFirstObserver<>(); assertFalse(o.isDisposed()); o.dispose(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java index 34ae5ed780..eeee099b0d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java @@ -273,7 +273,7 @@ public void accept(List t1) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -303,7 +303,7 @@ public void bufferStopsWhenUnsubscribed1() { Observable source = Observable.never(); Observer> o = TestHelper.mockObserver(); - TestObserver> to = new TestObserver>(o); + TestObserver> to = new TestObserver<>(o); source.buffer(100, 200, TimeUnit.MILLISECONDS, scheduler) .doOnNext(new Consumer>() { @@ -781,7 +781,7 @@ public void bufferBoundaryHint() { } static HashSet set(Integer... values) { - return new HashSet(Arrays.asList(values)); + return new HashSet<>(Arrays.asList(values)); } @SuppressWarnings("unchecked") @@ -791,7 +791,7 @@ public void bufferIntoCustomCollection() { .buffer(3, new Supplier>() { @Override public Collection get() throws Exception { - return new HashSet(); + return new HashSet<>(); } }) .test() @@ -805,7 +805,7 @@ public void bufferSkipIntoCustomCollection() { .buffer(3, 3, new Supplier>() { @Override public Collection get() throws Exception { - return new HashSet(); + return new HashSet<>(); } }) .test() @@ -865,7 +865,7 @@ public Collection get() throws Exception { if (count++ == 1) { throw new TestException(); } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -885,7 +885,7 @@ public Collection get() throws Exception { if (count++ == 1) { throw new TestException(); } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -905,7 +905,7 @@ public Collection get() throws Exception { if (count++ == 1) { throw new TestException(); } else { - return new ArrayList(); + return new ArrayList<>(); } } }) @@ -925,7 +925,7 @@ public Collection get() throws Exception { if (count++ == 1) { return null; } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -945,7 +945,7 @@ public Collection get() throws Exception { if (count++ == 1) { return null; } else { - return new ArrayList(); + return new ArrayList<>(); } } }, false) @@ -965,7 +965,7 @@ public Collection get() throws Exception { if (count++ == 1) { return null; } else { - return new ArrayList(); + return new ArrayList<>(); } } }) @@ -1016,7 +1016,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }) .test() @@ -1034,7 +1034,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }) .test() @@ -1134,7 +1134,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }, true) .test(); @@ -1587,7 +1587,7 @@ public List get() throws Exception { if (++calls == 2) { throw new TestException(); } - return new ArrayList(); + return new ArrayList<>(); } }).test(); @@ -1609,7 +1609,7 @@ protected void subscribeActual(Observer observer) { } }; - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); Observable b = new Observable() { @Override protected void subscribeActual(Observer observer) { @@ -1653,10 +1653,10 @@ public void timedCancelledUpfront() { public void timedInternalState() { TestScheduler sch = new TestScheduler(); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); - BufferExactUnboundedObserver> sub = new BufferExactUnboundedObserver>( - to, Functions.justSupplier((List)new ArrayList()), 1, TimeUnit.SECONDS, sch); + BufferExactUnboundedObserver> sub = new BufferExactUnboundedObserver<>( + to, Functions.justSupplier((List) new ArrayList()), 1, TimeUnit.SECONDS, sch); sub.onSubscribe(Disposable.empty()); @@ -1672,7 +1672,7 @@ public void timedInternalState() { assertTrue(sub.isDisposed()); - sub.buffer = new ArrayList(); + sub.buffer = new ArrayList<>(); sub.enter(); sub.onComplete(); } @@ -1703,10 +1703,10 @@ public Observable> apply(Observable f) public void timedSkipInternalState() { TestScheduler sch = new TestScheduler(); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); - BufferSkipBoundedObserver> sub = new BufferSkipBoundedObserver>( - to, Functions.justSupplier((List)new ArrayList()), 1, 1, TimeUnit.SECONDS, sch.createWorker()); + BufferSkipBoundedObserver> sub = new BufferSkipBoundedObserver<>( + to, Functions.justSupplier((List) new ArrayList()), 1, 1, TimeUnit.SECONDS, sch.createWorker()); sub.onSubscribe(Disposable.empty()); @@ -1722,19 +1722,20 @@ public void timedSkipInternalState() { public void timedSkipCancelWhenSecondBuffer() { TestScheduler sch = new TestScheduler(); - final TestObserver> to = new TestObserver>(); + final TestObserver> to = new TestObserver<>(); - BufferSkipBoundedObserver> sub = new BufferSkipBoundedObserver>( + BufferSkipBoundedObserver> sub = new BufferSkipBoundedObserver<>( to, new Supplier>() { - int calls; - @Override - public List get() throws Exception { - if (++calls == 2) { - to.dispose(); - } - return new ArrayList(); - } - }, 1, 1, TimeUnit.SECONDS, sch.createWorker()); + int calls; + + @Override + public List get() throws Exception { + if (++calls == 2) { + to.dispose(); + } + return new ArrayList<>(); + } + }, 1, 1, TimeUnit.SECONDS, sch.createWorker()); sub.onSubscribe(Disposable.empty()); @@ -1747,11 +1748,11 @@ public List get() throws Exception { public void timedSizeBufferAlreadyCleared() { TestScheduler sch = new TestScheduler(); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); BufferExactBoundedObserver> sub = - new BufferExactBoundedObserver>( - to, Functions.justSupplier((List)new ArrayList()), + new BufferExactBoundedObserver<>( + to, Functions.justSupplier((List) new ArrayList()), 1, TimeUnit.SECONDS, 1, false, sch.createWorker()) ; @@ -1790,10 +1791,10 @@ public ObservableSource> apply(Observable o) @Test public void bufferExactState() { - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); - BufferExactObserver> sub = new BufferExactObserver>( - to, 1, Functions.justSupplier((List)new ArrayList()) + BufferExactObserver> sub = new BufferExactObserver<>( + to, 1, Functions.justSupplier((List) new ArrayList()) ); sub.onComplete(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java index 20c4fde308..8f4440e41b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCacheTest.java @@ -36,11 +36,11 @@ public class ObservableCacheTest extends RxJavaTest { @Test public void coldReplayNoBackpressure() { - ObservableCache source = new ObservableCache(Observable.range(0, 1000), 16); + ObservableCache source = new ObservableCache<>(Observable.range(0, 1000), 16); assertFalse("Source is connected!", source.isConnected()); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.subscribe(to); @@ -119,9 +119,9 @@ public void unsubscribeSource() throws Throwable { @Test public void take() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - ObservableCache cached = new ObservableCache(Observable.range(1, 1000), 16); + ObservableCache cached = new ObservableCache<>(Observable.range(1, 1000), 16); cached.take(10).subscribe(to); to.assertNoErrors(); @@ -135,9 +135,9 @@ public void take() { public void async() { Observable source = Observable.range(1, 10000); for (int i = 0; i < 100; i++) { - TestObserver to1 = new TestObserver(); + TestObserver to1 = new TestObserver<>(); - ObservableCache cached = new ObservableCache(source, 16); + ObservableCache cached = new ObservableCache<>(source, 16); cached.observeOn(Schedulers.computation()).subscribe(to1); @@ -146,7 +146,7 @@ public void async() { to1.assertComplete(); assertEquals(10000, to1.values().size()); - TestObserver to2 = new TestObserver(); + TestObserver to2 = new TestObserver<>(); cached.observeOn(Schedulers.computation()).subscribe(to2); to2.awaitDone(2, TimeUnit.SECONDS); @@ -161,18 +161,18 @@ public void asyncComeAndGo() { Observable source = Observable.interval(1, 1, TimeUnit.MILLISECONDS) .take(1000) .subscribeOn(Schedulers.io()); - ObservableCache cached = new ObservableCache(source, 16); + ObservableCache cached = new ObservableCache<>(source, 16); Observable output = cached.observeOn(Schedulers.computation()); - List> list = new ArrayList>(100); + List> list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); list.add(to); output.skip(i * 10).take(10).subscribe(to); } - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 10; i++) { expected.add((long)(i - 10)); } @@ -206,7 +206,7 @@ public void subscribe(Observer t) { } }); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); firehose.cache().observeOn(Schedulers.computation()).takeLast(100).subscribe(to); to.awaitDone(3, TimeUnit.SECONDS); @@ -222,14 +222,14 @@ public void valuesAndThenError() { .concatWith(Observable.error(new TestException())) .cache(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.subscribe(to); to.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); to.assertNotComplete(); to.assertError(TestException.class); - TestObserver to2 = new TestObserver(); + TestObserver to2 = new TestObserver<>(); source.subscribe(to2); to2.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); @@ -296,7 +296,7 @@ public void subscribeEmitRace() { cache.test(); - final TestObserverEx to = new TestObserverEx(); + final TestObserverEx to = new TestObserverEx<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java index e31a2299be..d28bd20e97 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java @@ -36,7 +36,7 @@ public void collectToListObservable() { .collect(new Supplier>() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -85,7 +85,7 @@ public void accept(StringBuilder sb, Integer v) { @Test public void collectorFailureDoesNotResultInTwoErrorEmissionsObservable() { try { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); RxJavaPlugins.setErrorHandler(addToList(list)); final RuntimeException e1 = new RuntimeException(); final RuntimeException e2 = new RuntimeException(); @@ -146,14 +146,14 @@ public void accept(Object o, Integer t) { @Test public void collectIntoObservable() { Observable.just(1, 1, 1, 1, 2) - .collectInto(new HashSet(), new BiConsumer, Integer>() { + .collectInto(new HashSet<>(), new BiConsumer, Integer>() { @Override public void accept(HashSet s, Integer v) throws Exception { s.add(v); } }).toObservable() .test() - .assertResult(new HashSet(Arrays.asList(1, 2))); + .assertResult(new HashSet<>(Arrays.asList(1, 2))); } @Test @@ -162,7 +162,7 @@ public void collectToList() { .collect(new Supplier>() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -211,7 +211,7 @@ public void accept(StringBuilder sb, Integer v) { @Test public void collectorFailureDoesNotResultInTwoErrorEmissions() { try { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); RxJavaPlugins.setErrorHandler(addToList(list)); final RuntimeException e1 = new RuntimeException(); final RuntimeException e2 = new RuntimeException(); @@ -270,14 +270,14 @@ public void accept(Object o, Integer t) { @Test public void collectInto() { Observable.just(1, 1, 1, 1, 2) - .collectInto(new HashSet(), new BiConsumer, Integer>() { + .collectInto(new HashSet<>(), new BiConsumer, Integer>() { @Override public void accept(HashSet s, Integer v) throws Exception { s.add(v); } }) .test() - .assertResult(new HashSet(Arrays.asList(1, 2))); + .assertResult(new HashSet<>(Arrays.asList(1, 2))); } @Test @@ -285,7 +285,7 @@ public void dispose() { TestHelper.checkDisposed(Observable.range(1, 3).collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -297,7 +297,7 @@ public void accept(List a, Integer b) throws Exception { TestHelper.checkDisposed(Observable.range(1, 3).collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -315,7 +315,7 @@ public SingleSource> apply(Observable o) throws Exception return o.collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -332,7 +332,7 @@ public ObservableSource> apply(Observable o) throws Excep return o.collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -352,7 +352,7 @@ public Object apply(Observable o) throws Exception { return o.collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java index 4dca384928..c5f93d4222 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java @@ -436,8 +436,8 @@ public List apply(Object[] args) { }; for (int i = 1; i <= n; i++) { System.out.println("test1ToNSources: " + i + " sources"); - List> sources = new ArrayList>(); - List values = new ArrayList(); + List> sources = new ArrayList<>(); + List values = new ArrayList<>(); for (int j = 0; j < i; j++) { sources.add(Observable.just(j)); values.add(j); @@ -467,8 +467,8 @@ public List apply(Object[] args) { }; for (int i = 1; i <= n; i++) { System.out.println("test1ToNSourcesScheduled: " + i + " sources"); - List> sources = new ArrayList>(); - List values = new ArrayList(); + List> sources = new ArrayList<>(); + List values = new ArrayList<>(); for (int j = 0; j < i; j++) { sources.add(Observable.just(j).subscribeOn(Schedulers.io())); values.add(j); @@ -753,7 +753,7 @@ public void accept(Notification n) { } }).take(SIZE); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.combineLatest(timer, Observable. never(), new BiFunction() { @Override @@ -873,7 +873,7 @@ public Object apply(Object a, Object b) throws Exception { @Test public void cancelWhileSubscribing() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.combineLatest( Observable.just(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java index 4e9a2b0d1f..fe1cafe56b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java @@ -197,7 +197,7 @@ public Observable apply(Integer t) { @Before public void before() { - to = new TestObserver(); + to = new TestObserver<>(); } @Test @@ -723,7 +723,7 @@ public void run() { @Test public void mapperCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1).hide() .concatMapEager(new Function>() { @@ -837,7 +837,7 @@ public ObservableSource apply(Integer i) throws Exception { public void maxConcurrencyOf2() { List[] list = new ArrayList[100]; for (int i = 0; i < 100; i++) { - List lst = new ArrayList(); + List lst = new ArrayList<>(); list[i] = lst; for (int k = 1; k <= 10; k++) { lst.add((i) * 10 + k); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java index fa72be7ac6..a82553b9a4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapSchedulerTest.java @@ -367,7 +367,7 @@ public void concatMapRangeAsyncLoopIssue2876() { if (i % 1000 == 0) { System.out.println("concatMapRangeAsyncLoop > " + i); } - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.range(0, 1000) .concatMap(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java index 354c2afc27..c5b00fdf7c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java @@ -63,7 +63,7 @@ public void concatWithList() { final Observable odds = Observable.fromArray(o); final Observable even = Observable.fromArray(e); - final List> list = new ArrayList>(); + final List> list = new ArrayList<>(); list.add(odds); list.add(even); Observable concat = Observable.concat(Observable.fromIterable(list)); @@ -108,8 +108,8 @@ public void subscribe(Observer> observer) { public void simpleAsyncConcat() { Observer observer = TestHelper.mockObserver(); - TestObservable o1 = new TestObservable("one", "two", "three"); - TestObservable o2 = new TestObservable("four", "five", "six"); + TestObservable o1 = new TestObservable<>("one", "two", "three"); + TestObservable o2 = new TestObservable<>("four", "five", "six"); Observable.concat(Observable.unsafeCreate(o1), Observable.unsafeCreate(o2)).subscribe(observer); @@ -148,12 +148,12 @@ public void nestedAsyncConcatLoop() throws Throwable { public void nestedAsyncConcat() throws InterruptedException { Observer observer = TestHelper.mockObserver(); - final TestObservable o1 = new TestObservable("one", "two", "three"); - final TestObservable o2 = new TestObservable("four", "five", "six"); - final TestObservable o3 = new TestObservable("seven", "eight", "nine"); + final TestObservable o1 = new TestObservable<>("one", "two", "three"); + final TestObservable o2 = new TestObservable<>("four", "five", "six"); + final TestObservable o3 = new TestObservable<>("seven", "eight", "nine"); final CountDownLatch allowThird = new CountDownLatch(1); - final AtomicReference parent = new AtomicReference(); + final AtomicReference parent = new AtomicReference<>(); final CountDownLatch parentHasStarted = new CountDownLatch(1); final CountDownLatch parentHasFinished = new CountDownLatch(1); @@ -272,7 +272,7 @@ public void blockedObservableOfObservables() { final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); @SuppressWarnings("unchecked") - TestObservable> observableOfObservables = new TestObservable>(callOnce, okToContinue, odds, even); + TestObservable> observableOfObservables = new TestObservable<>(callOnce, okToContinue, odds, even); Observable concatF = Observable.concat(Observable.unsafeCreate(observableOfObservables)); concatF.subscribe(observer); try { @@ -304,14 +304,14 @@ public void blockedObservableOfObservables() { @Test public void concatConcurrentWithInfinity() { - final TestObservable w1 = new TestObservable("one", "two", "three"); + final TestObservable w1 = new TestObservable<>("one", "two", "three"); //This Observable will send "hello" MAX_VALUE time. - final TestObservable w2 = new TestObservable("hello", Integer.MAX_VALUE); + final TestObservable w2 = new TestObservable<>("hello", Integer.MAX_VALUE); Observer observer = TestHelper.mockObserver(); @SuppressWarnings("unchecked") - TestObservable> observableOfObservables = new TestObservable>(Observable.unsafeCreate(w1), Observable.unsafeCreate(w2)); + TestObservable> observableOfObservables = new TestObservable<>(Observable.unsafeCreate(w1), Observable.unsafeCreate(w2)); Observable concatF = Observable.concat(Observable.unsafeCreate(observableOfObservables)); concatF.take(50).subscribe(observer); @@ -339,8 +339,8 @@ public void concatNonBlockingObservables() { final CountDownLatch okToContinueW1 = new CountDownLatch(1); final CountDownLatch okToContinueW2 = new CountDownLatch(1); - final TestObservable w1 = new TestObservable(null, okToContinueW1, "one", "two", "three"); - final TestObservable w2 = new TestObservable(null, okToContinueW2, "four", "five", "six"); + final TestObservable w1 = new TestObservable<>(null, okToContinueW1, "one", "two", "three"); + final TestObservable w2 = new TestObservable<>(null, okToContinueW2, "four", "five", "six"); Observer observer = TestHelper.mockObserver(); @@ -390,11 +390,11 @@ public void subscribe(Observer> observer) { public void concatUnsubscribe() { final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); - final TestObservable w1 = new TestObservable("one", "two", "three"); - final TestObservable w2 = new TestObservable(callOnce, okToContinue, "four", "five", "six"); + final TestObservable w1 = new TestObservable<>("one", "two", "three"); + final TestObservable w2 = new TestObservable<>(callOnce, okToContinue, "four", "five", "six"); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); final Observable concat = Observable.concat(Observable.unsafeCreate(w1), Observable.unsafeCreate(w2)); @@ -432,14 +432,14 @@ public void concatUnsubscribe() { public void concatUnsubscribeConcurrent() { final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); - final TestObservable w1 = new TestObservable("one", "two", "three"); - final TestObservable w2 = new TestObservable(callOnce, okToContinue, "four", "five", "six"); + final TestObservable w1 = new TestObservable<>("one", "two", "three"); + final TestObservable w2 = new TestObservable<>(callOnce, okToContinue, "four", "five", "six"); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); @SuppressWarnings("unchecked") - TestObservable> observableOfObservables = new TestObservable>(Observable.unsafeCreate(w1), Observable.unsafeCreate(w2)); + TestObservable> observableOfObservables = new TestObservable<>(Observable.unsafeCreate(w1), Observable.unsafeCreate(w2)); Observable concatF = Observable.concat(Observable.unsafeCreate(observableOfObservables)); concatF.subscribe(to); @@ -617,7 +617,7 @@ public Observable apply(Integer v) { result.subscribe(o); - List list = new ArrayList(n); + List list = new ArrayList<>(n); for (int i = 0; i < n; i++) { list.add(i); } @@ -642,7 +642,7 @@ public Observable apply(Integer v) { result.subscribe(o); - List list = new ArrayList(n); + List list = new ArrayList<>(n); for (int i = 0; i < n / 2; i++) { list.add(i); } @@ -674,7 +674,7 @@ public void subscribe(Observer observer) { }); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.concat(o, o).subscribe(to); to.awaitDone(500, TimeUnit.MILLISECONDS); to.assertTerminated(); @@ -745,7 +745,7 @@ public void concatMapRangeAsyncLoopIssue2876() { if (i % 1000 == 0) { System.out.println("concatMapRangeAsyncLoop > " + i); } - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.range(0, 1000) .concatMap(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java index 4ca6d1bb7b..fc2a8b8531 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletableTest.java @@ -28,7 +28,7 @@ public class ObservableConcatWithCompletableTest extends RxJavaTest { @Test public void normal() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Completable.fromAction(new Action() { @@ -44,7 +44,7 @@ public void run() throws Exception { @Test public void mainError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.error(new TestException()) .concatWith(Completable.fromAction(new Action() { @@ -60,7 +60,7 @@ public void run() throws Exception { @Test public void otherError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Completable.error(new TestException())) @@ -71,7 +71,7 @@ public void otherError() { @Test public void takeMain() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Completable.fromAction(new Action() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java index cd35a91ad1..16ba059b8b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybeTest.java @@ -28,7 +28,7 @@ public class ObservableConcatWithMaybeTest extends RxJavaTest { @Test public void normalEmpty() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Maybe.fromAction(new Action() { @@ -44,7 +44,7 @@ public void run() throws Exception { @Test public void normalNonEmpty() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Maybe.just(100)) @@ -55,7 +55,7 @@ public void normalNonEmpty() { @Test public void mainError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.error(new TestException()) .concatWith(Maybe.fromAction(new Action() { @@ -71,7 +71,7 @@ public void run() throws Exception { @Test public void otherError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Maybe.error(new TestException())) @@ -82,7 +82,7 @@ public void otherError() { @Test public void takeMain() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Maybe.fromAction(new Action() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java index 45c0b6f269..2d05b58c2d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingleTest.java @@ -27,7 +27,7 @@ public class ObservableConcatWithSingleTest extends RxJavaTest { @Test public void normal() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Single.just(100)) @@ -38,7 +38,7 @@ public void normal() { @Test public void mainError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.error(new TestException()) .concatWith(Single.just(100)) @@ -49,7 +49,7 @@ public void mainError() { @Test public void otherError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Single.error(new TestException())) @@ -60,7 +60,7 @@ public void otherError() { @Test public void takeMain() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5) .concatWith(Single.just(100)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java index af60b93018..d59431a677 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTest.java @@ -288,7 +288,7 @@ public Observable apply(Integer t1) { @Test public void debounceWithTimeBackpressure() throws InterruptedException { TestScheduler scheduler = new TestScheduler(); - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); Observable.merge( Observable.just(1), @@ -317,7 +317,7 @@ public void dispose() { TestHelper.checkDisposed(PublishSubject.create().debounce(Functions.justFunction(Observable.never()))); - Disposable d = new ObservableDebounceTimed.DebounceEmitter(1, 1, null); + Disposable d = new ObservableDebounceTimed.DebounceEmitter<>(1, 1, null); assertFalse(d.isDisposed()); d.dispose(); @@ -395,7 +395,7 @@ public Observable apply(Observable o) throws Exception { @Test public void disposeInOnNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); BehaviorSubject.createDefault(1) .debounce(new Function>() { @@ -413,7 +413,7 @@ public ObservableSource apply(Integer o) throws Exception { @Test public void disposedInOnComplete() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); new Observable() { @Override @@ -430,7 +430,7 @@ protected void subscribeActual(Observer observer) { @Test public void emitLate() { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); TestObserver to = Observable.range(1, 2) .debounce(new Function>() { @@ -469,7 +469,7 @@ public Publisher apply(Flowable f) @Test public void timedDisposedIgnoredBySource() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); new Observable() { @Override @@ -487,13 +487,13 @@ protected void subscribeActual( @Test public void timedLateEmit() { - TestObserver to = new TestObserver(); - DebounceTimedObserver sub = new DebounceTimedObserver( + TestObserver to = new TestObserver<>(); + DebounceTimedObserver sub = new DebounceTimedObserver<>( to, 1, TimeUnit.SECONDS, new TestScheduler().createWorker()); sub.onSubscribe(Disposable.empty()); - DebounceEmitter de = new DebounceEmitter(1, 50, sub); + DebounceEmitter de = new DebounceEmitter<>(1, 50, sub); de.run(); de.run(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelaySubscriptionOtherTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelaySubscriptionOtherTest.java index 604321a714..93b09456dd 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelaySubscriptionOtherTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelaySubscriptionOtherTest.java @@ -32,7 +32,7 @@ public class ObservableDelaySubscriptionOtherTest extends RxJavaTest { public void noPrematureSubscription() { PublishSubject other = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -65,7 +65,7 @@ public void accept(Disposable d) { public void noMultipleSubscriptions() { PublishSubject other = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -99,7 +99,7 @@ public void accept(Disposable d) { public void completeTriggersSubscription() { PublishSubject other = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -132,7 +132,7 @@ public void accept(Disposable d) { public void noPrematureSubscriptionToError() { PublishSubject other = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final AtomicInteger subscribed = new AtomicInteger(); @@ -165,7 +165,7 @@ public void accept(Disposable d) { public void noSubscriptionIfOtherErrors() { PublishSubject other = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final AtomicInteger subscribed = new AtomicInteger(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelayTest.java index cc93594bb8..82b60d4701 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelayTest.java @@ -219,7 +219,7 @@ public void delaySubscriptionDisposeBeforeTime() { Observable result = Observable.just(1, 2, 3).delaySubscription(100, TimeUnit.MILLISECONDS, scheduler); Observer o = TestHelper.mockObserver(); - TestObserver to = new TestObserver(o); + TestObserver to = new TestObserver<>(o); result.subscribe(to); to.dispose(); @@ -233,7 +233,7 @@ public void delaySubscriptionDisposeBeforeTime() { @Test public void delayWithObservableNormal1() { PublishSubject source = PublishSubject.create(); - final List> delays = new ArrayList>(); + final List> delays = new ArrayList<>(); final int n = 10; for (int i = 0; i < n; i++) { PublishSubject delay = PublishSubject.create(); @@ -584,7 +584,7 @@ public void delayWithObservableReorder() { int n = 3; PublishSubject source = PublishSubject.create(); - final List> subjects = new ArrayList>(); + final List> subjects = new ArrayList<>(); for (int i = 0; i < n; i++) { subjects.add(PublishSubject. create()); } @@ -632,7 +632,7 @@ public void accept(Notification t1) { } }); - TestObserver observer = new TestObserver(); + TestObserver observer = new TestObserver<>(); delayed.subscribe(observer); // all will be delivered after 500ms since range does not delay between them scheduler.advanceTimeBy(500L, TimeUnit.MILLISECONDS); @@ -641,7 +641,7 @@ public void accept(Notification t1) { @Test public void backpressureWithTimedDelay() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, Flowable.bufferSize() * 2) .delay(100, TimeUnit.MILLISECONDS) .observeOn(Schedulers.computation()) @@ -669,7 +669,7 @@ public Integer apply(Integer t) { @Test public void backpressureWithSubscriptionTimedDelay() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, Flowable.bufferSize() * 2) .delaySubscription(100, TimeUnit.MILLISECONDS) .delay(100, TimeUnit.MILLISECONDS) @@ -698,7 +698,7 @@ public Integer apply(Integer t) { @Test public void backpressureWithSelectorDelay() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, Flowable.bufferSize() * 2) .delay(new Function>() { @@ -733,7 +733,7 @@ public Integer apply(Integer t) { @Test public void backpressureWithSelectorDelayAndSubscriptionDelay() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, Flowable.bufferSize() * 2) .delay(Observable.timer(500, TimeUnit.MILLISECONDS) , new Function>() { @@ -773,7 +773,7 @@ public void errorRunsBeforeOnNext() { PublishSubject ps = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); ps.delay(1, TimeUnit.SECONDS, test).subscribe(to); @@ -796,7 +796,7 @@ public void delaySupplierSimple() { Observable source = Observable.range(1, 5); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.delaySubscription(ps).subscribe(to); @@ -817,7 +817,7 @@ public void delaySupplierCompletes() { Observable source = Observable.range(1, 5); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.delaySubscription(ps).subscribe(to); @@ -839,7 +839,7 @@ public void delaySupplierErrors() { Observable source = Observable.range(1, 5); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.delaySubscription(ps).subscribe(to); @@ -866,7 +866,7 @@ public void delayWithTimeDelayError() throws Exception { @Test public void onErrorCalledOnScheduler() throws Exception { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference thread = new AtomicReference(); + final AtomicReference thread = new AtomicReference<>(); Observable.error(new Exception()) .delay(0, TimeUnit.MILLISECONDS, Schedulers.newThread()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java index 5d3e2b6be1..8c7b6017ab 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerializeTest.java @@ -140,7 +140,7 @@ public void completePassThru() { Observer observer = TestHelper.mockObserver(); - TestObserverEx to = new TestObserverEx(observer); + TestObserverEx to = new TestObserverEx<>(observer); dematerialize.subscribe(to); System.out.println(to.errors()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetachTest.java index ef67be319e..95cbf96f76 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetachTest.java @@ -33,9 +33,9 @@ public class ObservableDetachTest extends RxJavaTest { public void just() throws Exception { o = new Object(); - WeakReference wr = new WeakReference(o); + WeakReference wr = new WeakReference<>(o); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(o).count().toObservable().onTerminateDetach().subscribe(to); @@ -54,7 +54,7 @@ public void just() throws Exception { @Test public void error() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.error(new TestException()).onTerminateDetach().subscribe(to); @@ -65,7 +65,7 @@ public void error() { @Test public void empty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.empty().onTerminateDetach().subscribe(to); @@ -76,7 +76,7 @@ public void empty() { @Test public void range() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 1000).onTerminateDetach().subscribe(to); @@ -89,7 +89,7 @@ public void range() { public void justUnsubscribed() throws Exception { o = new Object(); - WeakReference wr = new WeakReference(o); + WeakReference wr = new WeakReference<>(o); TestObserver to = Observable.just(o).count().toObservable().onTerminateDetach().test(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java index 888a39cb89..1f2614f284 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctTest.java @@ -115,7 +115,7 @@ public void error() { @Test public void fusedSync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.just(1, 1, 2, 1, 3, 2, 4, 5, 4) .distinct() @@ -127,7 +127,7 @@ public void fusedSync() { @Test public void fusedAsync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); UnicastSubject us = UnicastSubject.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java index 13122988cf..da1d67e819 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java @@ -148,7 +148,7 @@ public boolean test(String a, String b) { @Test public void fused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.just(1, 2, 2, 3, 3, 4, 5) .distinctUntilChanged(new BiPredicate() { @@ -167,7 +167,7 @@ public boolean test(Integer a, Integer b) throws Exception { @Test public void fusedAsync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); UnicastSubject up = UnicastSubject.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java index f5f2e8feb6..5b6cd18987 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java @@ -31,7 +31,7 @@ public class ObservableDoAfterNextTest extends RxJavaTest { - final List values = new ArrayList(); + final List values = new ArrayList<>(); final Consumer afterNext = new Consumer() { @Override @@ -101,7 +101,7 @@ public void empty() { @Test public void syncFused() { - TestObserverEx to0 = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.SYNC); Observable.range(1, 5) .doAfterNext(afterNext) @@ -115,7 +115,7 @@ public void syncFused() { @Test public void asyncFusedRejected() { - TestObserverEx to0 = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.ASYNC); Observable.range(1, 5) .doAfterNext(afterNext) @@ -129,7 +129,7 @@ public void asyncFusedRejected() { @Test public void asyncFused() { - TestObserverEx to0 = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.ASYNC); UnicastSubject up = UnicastSubject.create(); @@ -196,7 +196,7 @@ public void emptyConditional() { @Test public void syncFusedConditional() { - TestObserverEx to0 = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.SYNC); Observable.range(1, 5) .doAfterNext(afterNext) @@ -211,7 +211,7 @@ public void syncFusedConditional() { @Test public void asyncFusedRejectedConditional() { - TestObserverEx to0 = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.ASYNC); Observable.range(1, 5) .doAfterNext(afterNext) @@ -226,7 +226,7 @@ public void asyncFusedRejectedConditional() { @Test public void asyncFusedConditional() { - TestObserverEx to0 = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.ASYNC); UnicastSubject up = UnicastSubject.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java index 9fd860816a..a0eeccb405 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java @@ -99,7 +99,7 @@ public Observable apply(Observable f) throws Exception { @Test public void syncFused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); Observable.range(1, 5) .doFinally(this) @@ -113,7 +113,7 @@ public void syncFused() { @Test public void syncFusedBoundary() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC | QueueFuseable.BOUNDARY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC | QueueFuseable.BOUNDARY); Observable.range(1, 5) .doFinally(this) @@ -127,7 +127,7 @@ public void syncFusedBoundary() { @Test public void asyncFused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); UnicastSubject up = UnicastSubject.create(); TestHelper.emit(up, 1, 2, 3, 4, 5); @@ -144,7 +144,7 @@ public void asyncFused() { @Test public void asyncFusedBoundary() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC | QueueFuseable.BOUNDARY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC | QueueFuseable.BOUNDARY); UnicastSubject up = UnicastSubject.create(); TestHelper.emit(up, 1, 2, 3, 4, 5); @@ -206,7 +206,7 @@ public void normalTakeConditional() { @Test public void syncFusedConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); Observable.range(1, 5) .doFinally(this) @@ -221,7 +221,7 @@ public void syncFusedConditional() { @Test public void nonFused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); Observable.range(1, 5).hide() .doFinally(this) @@ -235,7 +235,7 @@ public void nonFused() { @Test public void nonFusedConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); Observable.range(1, 5).hide() .doFinally(this) @@ -250,7 +250,7 @@ public void nonFusedConditional() { @Test public void syncFusedBoundaryConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC | QueueFuseable.BOUNDARY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC | QueueFuseable.BOUNDARY); Observable.range(1, 5) .doFinally(this) @@ -265,7 +265,7 @@ public void syncFusedBoundaryConditional() { @Test public void asyncFusedConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); UnicastSubject up = UnicastSubject.create(); TestHelper.emit(up, 1, 2, 3, 4, 5); @@ -283,7 +283,7 @@ public void asyncFusedConditional() { @Test public void asyncFusedBoundaryConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC | QueueFuseable.BOUNDARY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC | QueueFuseable.BOUNDARY); UnicastSubject up = UnicastSubject.create(); TestHelper.emit(up, 1, 2, 3, 4, 5); @@ -446,7 +446,7 @@ public void onComplete() { @Test public void eventOrdering() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.error(new TestException()) .doOnDispose(new Action() { @@ -486,7 +486,7 @@ public void run() throws Exception { @Test public void eventOrdering2() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.just(1) .doOnDispose(new Action() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java index 30f0cb2d6f..0482d23906 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java @@ -202,7 +202,7 @@ public void accept(List booleans) { @Test public void onErrorThrows() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.error(new TestException()) .doOnError(new Consumer() { @@ -472,7 +472,7 @@ public void accept(Throwable e) throws Exception { @Test @Ignore("Fusion not supported yet") // TODO decide/implement fusion public void fused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); final int[] call = { 0, 0 }; @@ -502,7 +502,7 @@ public void run() throws Exception { @Test @Ignore("Fusion not supported yet") // TODO decide/implement fusion public void fusedOnErrorCrash() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); final int[] call = { 0 }; @@ -531,7 +531,7 @@ public void run() throws Exception { @Test @Ignore("Fusion not supported yet") // TODO decide/implement fusion public void fusedConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); final int[] call = { 0, 0 }; @@ -562,7 +562,7 @@ public void run() throws Exception { @Test @Ignore("Fusion not supported yet") // TODO decide/implement fusion public void fusedOnErrorCrashConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); final int[] call = { 0 }; @@ -592,7 +592,7 @@ public void run() throws Exception { @Test @Ignore("Fusion not supported yet") // TODO decide/implement fusion public void fusedAsync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); final int[] call = { 0, 0 }; @@ -626,7 +626,7 @@ public void run() throws Exception { @Test @Ignore("Fusion not supported yet") // TODO decide/implement fusion public void fusedAsyncConditional() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); final int[] call = { 0, 0 }; @@ -661,7 +661,7 @@ public void run() throws Exception { @Test @Ignore("Fusion not supported yet") // TODO decide/implement fusion public void fusedAsyncConditional2() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); final int[] call = { 0, 0 }; @@ -695,7 +695,7 @@ public void run() throws Exception { @Test public void dispose() { - TestHelper.checkDisposed(Observable.just(1).doOnEach(new TestObserver())); + TestHelper.checkDisposed(Observable.just(1).doOnEach(new TestObserver<>())); } @Test @@ -703,7 +703,7 @@ public void doubleOnSubscribe() { TestHelper.checkDoubleOnSubscribeObservable(new Function, ObservableSource>() { @Override public ObservableSource apply(Observable o) throws Exception { - return o.doOnEach(new TestObserver()); + return o.doOnEach(new TestObserver<>()); } }); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java index 26cd2b340b..5a16beda68 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnSubscribeTest.java @@ -69,7 +69,7 @@ public void doOnUnSubscribeWorksWithRefCount() throws Exception { final AtomicInteger onSubscribed = new AtomicInteger(); final AtomicInteger countBefore = new AtomicInteger(); final AtomicInteger countAfter = new AtomicInteger(); - final AtomicReference> sref = new AtomicReference>(); + final AtomicReference> sref = new AtomicReference<>(); Observable o = Observable.unsafeCreate(new ObservableSource() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnUnsubscribeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnUnsubscribeTest.java index c90105fb04..a95f372d18 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnUnsubscribeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnUnsubscribeTest.java @@ -68,11 +68,11 @@ public void run() { } }); - List subscriptions = new ArrayList(); - List> subscribers = new ArrayList>(); + List subscriptions = new ArrayList<>(); + List> subscribers = new ArrayList<>(); for (int i = 0; i < subCount; ++i) { - TestObserver observer = new TestObserver(); + TestObserver observer = new TestObserver<>(); subscriptions.add(observer); longs.subscribe(observer); subscribers.add(observer); @@ -131,11 +131,11 @@ public void run() { .publish() .refCount(); - List subscriptions = new ArrayList(); - List> subscribers = new ArrayList>(); + List subscriptions = new ArrayList<>(); + List> subscribers = new ArrayList<>(); for (int i = 0; i < subCount; ++i) { - TestObserver observer = new TestObserver(); + TestObserver observer = new TestObserver<>(); longs.subscribe(observer); subscriptions.add(observer); subscribers.add(observer); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilterTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilterTest.java index 22cd6c060a..6bbb838da5 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilterTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilterTest.java @@ -68,7 +68,7 @@ public ObservableSource apply(Observable o) throws Exception { @Test public void fusedSync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.range(1, 5) .filter(new Predicate() { @@ -85,7 +85,7 @@ public boolean test(Integer v) throws Exception { @Test public void fusedAsync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); UnicastSubject us = UnicastSubject.create(); @@ -106,7 +106,7 @@ public boolean test(Integer v) throws Exception { @Test public void fusedReject() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY | QueueFuseable.BOUNDARY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY | QueueFuseable.BOUNDARY); Observable.range(1, 5) .filter(new Predicate() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java index f5bf13b231..95baba1448 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableTest.java @@ -169,7 +169,7 @@ public CompletableSource apply(Integer v) throws Exception { @Test public void fusedObservable() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.range(1, 10) .flatMapCompletable(new Function() { @@ -334,7 +334,7 @@ public CompletableSource apply(Integer v) throws Exception { @Test public void fused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.range(1, 10) .flatMapCompletable(new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java index e1133ee311..a1540dcfca 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybeTest.java @@ -429,7 +429,7 @@ public MaybeSource apply(PublishSubject v) throws Exception { @Test public void disposeInner() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1).flatMapMaybe(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java index b8c7b351be..b990bd407c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingleTest.java @@ -344,7 +344,7 @@ public SingleSource apply(PublishSubject v) throws Exception { @Test public void disposeInner() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1).flatMapSingle(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java index 38defaf049..00f2523a33 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapTest.java @@ -338,13 +338,13 @@ public Observable apply(Integer t1) { } }, m); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); - Set expected = new HashSet(Arrays.asList( + Set expected = new HashSet<>(Arrays.asList( 10, 11, 20, 21, 30, 31, 40, 41, 50, 51, 60, 61, 70, 71, 80, 81, 90, 91, 100, 101 )); Assert.assertEquals(expected.size(), to.values().size()); @@ -369,13 +369,13 @@ public Integer apply(Integer t1, Integer t2) { } }, m); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); - Set expected = new HashSet(Arrays.asList( + Set expected = new HashSet<>(Arrays.asList( 1010, 1011, 2020, 2021, 3030, 3031, 4040, 4041, 5050, 5051, 6060, 6061, 7070, 7071, 8080, 8081, 9090, 9091, 10100, 10101 )); @@ -415,7 +415,7 @@ public void flatMapTransformsMaxConcurrentNormal() { Observable source = Observable.fromIterable(Arrays.asList(10, 20, 30)); Observer o = TestHelper.mockObserver(); - TestObserverEx to = new TestObserverEx(o); + TestObserverEx to = new TestObserverEx<>(o); Function> just = just(onError); source.flatMap(just(onNext), just, just0(onComplete), m).subscribe(to); @@ -440,7 +440,7 @@ public void flatMapRangeMixedAsyncLoop() { if (i % 10 == 0) { System.out.println("flatMapRangeAsyncLoop > " + i); } - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.range(0, 1000) .flatMap(new Function>() { final Random rnd = new Random(); @@ -464,7 +464,7 @@ public Observable apply(Integer t) { to.assertNoErrors(); List list = to.values(); if (list.size() < 1000) { - Set set = new HashSet(list); + Set set = new HashSet<>(list); for (int j = 0; j < 1000; j++) { if (!set.contains(j)) { System.out.println(j + " missing"); @@ -478,7 +478,7 @@ public Observable apply(Integer t) { @Test public void flatMapIntPassthruAsync() { for (int i = 0; i < 1000; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 1000).flatMap(new Function>() { @Override @@ -497,7 +497,7 @@ public Observable apply(Integer t) { @Test public void flatMapTwoNestedSync() { for (final int n : new int[] { 1, 1000, 1000000 }) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1, 2).flatMap(new Function>() { @Override @@ -972,10 +972,10 @@ public void onNext(Integer t) { @Test public void fusedSourceCrashResumeWithNextSource() { final UnicastSubject fusedSource = UnicastSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); ObservableFlatMap.MergeObserver merger = - new ObservableFlatMap.MergeObserver(to, new Function>() { + new ObservableFlatMap.MergeObserver<>(to, new Function>() { @Override public Observable apply(Integer t) throws Exception { @@ -984,7 +984,9 @@ public Observable apply(Integer t) .map(new Function() { @Override public Integer apply(Integer v) - throws Exception { throw new TestException(); } + throws Exception { + throw new TestException(); + } }) .compose(TestHelper.observableStripBoundary()); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableForEachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableForEachTest.java index cd7273a282..677b78414f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableForEachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableForEachTest.java @@ -33,7 +33,7 @@ public class ObservableForEachTest extends RxJavaTest { @Test public void forEachWile() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.range(1, 5) .doOnNext(new Consumer() { @@ -54,7 +54,7 @@ public boolean test(Integer v) throws Exception { @Test public void forEachWileWithError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.range(1, 5).concatWith(Observable.error(new TestException())) .doOnNext(new Consumer() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallableTest.java index 060a871855..43b1d96fef 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallableTest.java @@ -121,7 +121,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromCallableObservable .subscribeOn(Schedulers.computation()) @@ -262,7 +262,7 @@ public Object call() throws Exception { @Test public void disposedOnCall() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.fromCallable(new Callable() { @Override @@ -280,7 +280,7 @@ public Integer call() throws Exception { public void disposedOnCallThrows() { List errors = TestHelper.trackPluginErrors(); try { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.fromCallable(new Callable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterableTest.java index 4cd33b78e9..d744472b66 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterableTest.java @@ -119,7 +119,7 @@ public void observableFromIterable() { public void noBackpressure() { Observable o = Observable.fromIterable(Arrays.asList(1, 2, 3, 4, 5)); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); o.subscribe(to); @@ -132,7 +132,7 @@ public void subscribeMultipleTimes() { Observable o = Observable.fromIterable(Arrays.asList(1, 2, 3)); for (int i = 0; i < 10; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); o.subscribe(to); @@ -235,7 +235,7 @@ public void onNext(Integer t) { @Test public void fusionWithConcatMap() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.fromIterable(Arrays.asList(1, 2, 3, 4)).concatMap( new Function>() { @@ -266,7 +266,7 @@ public void hasNext2Throws() { @Test public void hasNextCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.fromIterable(new Iterable() { @Override @@ -303,7 +303,7 @@ public void remove() { @Test public void fusionRejected() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); Observable.fromIterable(Arrays.asList(1, 2, 3)) .subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplierTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplierTest.java index bf6e9d4e3b..2b181a56fe 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplierTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplierTest.java @@ -121,7 +121,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromSupplierObservable .subscribeOn(Schedulers.computation()) @@ -262,7 +262,7 @@ public Object get() throws Exception { @Test public void disposedOnCall() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.fromSupplier(new Supplier() { @Override @@ -280,7 +280,7 @@ public Integer get() throws Exception { public void disposedOnCallThrows() { List errors = TestHelper.trackPluginErrors(); try { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.fromSupplier(new Supplier() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromTest.java index 778520fa4f..0bc7a09997 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromTest.java @@ -77,7 +77,7 @@ public ObservableSource apply(Flowable f) throws Exception { @Test public void fusionRejected() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); Observable.fromArray(1, 2, 3) .subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java index 1eaf76cda8..28d3f2d643 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupByTest.java @@ -105,7 +105,7 @@ public void error() { final AtomicInteger groupCounter = new AtomicInteger(); final AtomicInteger eventCounter = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); grouped.flatMap(new Function, Observable>() { @@ -148,13 +148,13 @@ public void onNext(String v) { private static Map> toMap(Observable> observable) { - final ConcurrentHashMap> result = new ConcurrentHashMap>(); + final ConcurrentHashMap> result = new ConcurrentHashMap<>(); observable.doOnNext(new Consumer>() { @Override public void accept(final GroupedObservable o) { - result.put(o.getKey(), new ConcurrentLinkedQueue()); + result.put(o.getKey(), new ConcurrentLinkedQueue<>()); o.subscribe(new Consumer() { @Override @@ -597,7 +597,7 @@ public void accept(String s) { @Test public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsAndThenComplete() throws InterruptedException { final CountDownLatch first = new CountDownLatch(2); // there are two groups to first complete - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Observable.unsafeCreate(new ObservableSource() { @Override @@ -676,7 +676,7 @@ public void accept(String s) { public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsWhichThenSubscribesOnAndDelaysAndThenCompletes() throws InterruptedException { System.err.println("----------------------------------------------------------------------------------------------"); final CountDownLatch first = new CountDownLatch(2); // there are two groups to first complete - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Observable.unsafeCreate(new ObservableSource() { @Override @@ -768,7 +768,7 @@ public void accept(String s) { @Test public void firstGroupsCompleteAndParentSlowToThenEmitFinalGroupsWhichThenObservesOnAndDelaysAndThenCompletes() throws InterruptedException { final CountDownLatch first = new CountDownLatch(2); // there are two groups to first complete - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Observable.unsafeCreate(new ObservableSource() { @Override @@ -845,7 +845,7 @@ public void accept(String s) { @Test public void groupsWithNestedSubscribeOn() throws InterruptedException { - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Observable.unsafeCreate(new ObservableSource() { @Override @@ -902,7 +902,7 @@ public void accept(String s) { @Test public void groupsWithNestedObserveOn() throws InterruptedException { - final ArrayList results = new ArrayList(); + final ArrayList results = new ArrayList<>(); Observable.unsafeCreate(new ObservableSource() { @Override @@ -1027,7 +1027,7 @@ public Boolean apply(Integer n) { @Test public void groupByBackpressure() throws InterruptedException { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 4000) .groupBy(IS_EVEN2) @@ -1154,7 +1154,7 @@ public String apply(String v) { } }); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); m.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); System.out.println("ts .get " + to.values()); @@ -1170,7 +1170,7 @@ public void keySelectorThrows() { Observable m = source.groupBy(fail(0), dbl).flatMap(FLATTEN_INTEGER); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); m.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); assertEquals(1, to.errors().size()); @@ -1182,7 +1182,7 @@ public void valueSelectorThrows() { Observable source = Observable.just(0, 1, 2, 3, 4, 5, 6); Observable m = source.groupBy(identity, fail(0)).flatMap(FLATTEN_INTEGER); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); m.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); assertEquals(1, to.errors().size()); @@ -1196,7 +1196,7 @@ public void innerEscapeCompleted() { Observable m = source.groupBy(identity, dbl).flatMap(FLATTEN_INTEGER); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); m.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -1210,7 +1210,7 @@ public void innerEscapeCompleted() { public void exceptionIfSubscribeToChildMoreThanOnce() { Observable source = Observable.just(0); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); Observable> m = source.groupBy(identity, dbl); @@ -1239,7 +1239,7 @@ public void error2() { Observable m = source.groupBy(identity, dbl).flatMap(FLATTEN_INTEGER); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); m.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); assertEquals(1, to.errors().size()); @@ -1248,7 +1248,7 @@ public void error2() { @Test public void groupByBackpressure3() throws InterruptedException { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 4000).groupBy(IS_EVEN2).flatMap(new Function, Observable>() { @@ -1305,7 +1305,7 @@ public void accept(Notification t1) { @Test public void groupByBackpressure2() throws InterruptedException { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 4000).groupBy(IS_EVEN2).flatMap(new Function, Observable>() { @@ -1346,7 +1346,7 @@ public Observable apply(GroupedObservable t) { @Test public void groupByWithNullKey() { final String[] key = new String[]{"uninitialized"}; - final List values = new ArrayList(); + final List values = new ArrayList<>(); Observable.just("a", "b", "c").groupBy(new Function() { @Override @@ -1382,7 +1382,7 @@ public void subscribe(Observer observer) { } } ); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); o.groupBy(new Function() { @@ -1400,11 +1400,11 @@ public Integer apply(Integer integer) { @Test public void groupByShouldPropagateError() { final Throwable e = new RuntimeException("Oops"); - final TestObserverEx inner1 = new TestObserverEx(); - final TestObserverEx inner2 = new TestObserverEx(); + final TestObserverEx inner1 = new TestObserverEx<>(); + final TestObserverEx inner2 = new TestObserverEx<>(); final TestObserverEx> outer - = new TestObserverEx>(new DefaultObserver>() { + = new TestObserverEx<>(new DefaultObserver>() { @Override public void onComplete() { @@ -1546,7 +1546,7 @@ public void delayErrorSimpleComplete() { public void cancelOverFlatmapRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); final PublishSubject ps = PublishSubject.create(); @@ -1589,7 +1589,7 @@ public void run() { @Test public void abandonedGroupsNoDataloss() { - final List> groups = new ArrayList>(); + final List> groups = new ArrayList<>(); Observable.range(1, 1000) .groupBy(new Function() { @@ -1618,8 +1618,8 @@ public void accept(GroupedObservable v) throws Throwable { @Test public void newGroupValueSelectorFails() { - TestObserver to1 = new TestObserver(); - final TestObserver to2 = new TestObserver(); + TestObserver to1 = new TestObserver<>(); + final TestObserver to2 = new TestObserver<>(); Observable.just(1) .groupBy(Functions.identity(), new Function() { @@ -1645,8 +1645,8 @@ public void accept(GroupedObservable g) throws Throwable { @Test public void existingGroupValueSelectorFails() { - TestObserver to1 = new TestObserver(); - final TestObserver to2 = new TestObserver(); + TestObserver to1 = new TestObserver<>(); + final TestObserver to2 = new TestObserver<>(); Observable.just(1, 2) .groupBy(Functions.justFunction(1), new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsTest.java index e5704dfd46..43af8631e9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsTest.java @@ -58,7 +58,7 @@ public void accept(Integer t) { @Test public void completedOkObservable() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.range(1, 10).ignoreElements().toObservable().subscribe(to); to.assertNoErrors(); to.assertNoValues(); @@ -67,7 +67,7 @@ public void completedOkObservable() { @Test public void errorReceivedObservable() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); TestException ex = new TestException("boo"); Observable.error(ex).ignoreElements().toObservable().subscribe(to); to.assertNoValues(); @@ -120,7 +120,7 @@ public void accept(Integer t) { @Test public void completedOk() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.range(1, 10).ignoreElements().subscribe(to); to.assertNoErrors(); to.assertNoValues(); @@ -129,7 +129,7 @@ public void completedOk() { @Test public void errorReceived() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); TestException ex = new TestException("boo"); Observable.error(ex).ignoreElements().subscribe(to); to.assertNoValues(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIntervalTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIntervalTest.java index 11fe98a669..4f188093c5 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIntervalTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIntervalTest.java @@ -40,7 +40,7 @@ public void cancel() { @Test public void cancelledOnRun() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); IntervalObserver is = new IntervalObserver(to); to.onSubscribe(is); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java index 7a84d0c981..81b470ff99 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotificationTest.java @@ -27,7 +27,7 @@ public class ObservableMapNotificationTest extends RxJavaTest { @Test public void just() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1) .flatMap( new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapTest.java index da5651214f..538feaaa7f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapTest.java @@ -306,7 +306,7 @@ public Integer apply(Integer i) { // } private static Map getMap(String prefix) { - Map m = new HashMap(); + Map m = new HashMap<>(); m.put("firstName", prefix + "First"); m.put("lastName", prefix + "Last"); return m; @@ -350,7 +350,7 @@ public ObservableSource apply(Observable o) throws Exception { @Test public void fusedSync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.range(1, 5) .map(Functions.identity()) @@ -362,7 +362,7 @@ public void fusedSync() { @Test public void fusedAsync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); UnicastSubject us = UnicastSubject.create(); @@ -378,7 +378,7 @@ public void fusedAsync() { @Test public void fusedReject() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY | QueueFuseable.BOUNDARY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY | QueueFuseable.BOUNDARY); Observable.range(1, 5) .map(Functions.identity()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java index daf82d84ee..0e1df287bc 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterializeTest.java @@ -102,7 +102,7 @@ public void multipleSubscribes() throws InterruptedException, ExecutionException @Test public void withCompletionCausingError() { - TestObserverEx> to = new TestObserverEx>(); + TestObserverEx> to = new TestObserverEx<>(); final RuntimeException ex = new RuntimeException("boo"); Observable.empty().materialize().doOnNext(new Consumer() { @Override @@ -119,7 +119,7 @@ private static class TestLocalObserver extends DefaultObserver> notifications = new Vector>(); + List> notifications = new Vector<>(); @Override public void onComplete() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java index 877b75b70a..db6450d097 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java @@ -252,7 +252,7 @@ public void mergeArray() { public void mergeList() { final Observable o1 = Observable.unsafeCreate(new TestSynchronousObservable()); final Observable o2 = Observable.unsafeCreate(new TestSynchronousObservable()); - List> listOfObservables = new ArrayList>(); + List> listOfObservables = new ArrayList<>(); listOfObservables.add(o1); listOfObservables.add(o2); @@ -433,7 +433,7 @@ public void onNext(String args) { @Test public void errorInParentObservable() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.mergeDelayError( Observable.just(Observable.just(1), Observable.just(2)) .startWithItem(Observable. error(new RuntimeException())) @@ -462,7 +462,7 @@ public void subscribe(Observer> op) { Observer stringObserver = TestHelper.mockObserver(); - TestObserverEx to = new TestObserverEx(stringObserver); + TestObserverEx to = new TestObserverEx<>(stringObserver); Observable m = Observable.mergeDelayError(parentObservable); m.subscribe(to); System.out.println("testErrorInParentObservableDelayed | " + i); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java index ac69ffbaa7..9ab92edc12 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeMaxConcurrentTest.java @@ -42,14 +42,14 @@ public void before() { @Test public void whenMaxConcurrentIsOne() { for (int i = 0; i < 100; i++) { - List> os = new ArrayList>(); + List> os = new ArrayList<>(); os.add(Observable.just("one", "two", "three", "four", "five").subscribeOn(Schedulers.newThread())); os.add(Observable.just("one", "two", "three", "four", "five").subscribeOn(Schedulers.newThread())); os.add(Observable.just("one", "two", "three", "four", "five").subscribeOn(Schedulers.newThread())); List expected = Arrays.asList("one", "two", "three", "four", "five", "one", "two", "three", "four", "five", "one", "two", "three", "four", "five"); Iterator iter = Observable.merge(os, 1).blockingIterable().iterator(); - List actual = new ArrayList(); + List actual = new ArrayList<>(); while (iter.hasNext()) { actual.add(iter.next()); } @@ -65,8 +65,8 @@ public void maxConcurrent() { int maxConcurrent = 2 + (times % 10); AtomicInteger subscriptionCount = new AtomicInteger(0); - List> os = new ArrayList>(); - List scos = new ArrayList(); + List> os = new ArrayList<>(); + List scos = new ArrayList<>(); for (int i = 0; i < observableCount; i++) { SubscriptionCheckObservable sco = new SubscriptionCheckObservable(subscriptionCount, maxConcurrent); scos.add(sco); @@ -74,7 +74,7 @@ public void maxConcurrent() { } Iterator iter = Observable.merge(os, maxConcurrent).blockingIterable().iterator(); - List actual = new ArrayList(); + List actual = new ArrayList<>(); while (iter.hasNext()) { actual.add(iter.next()); } @@ -126,7 +126,7 @@ public void run() { @Test public void mergeALotOfSourcesOneByOneSynchronously() { int n = 10000; - List> sourceList = new ArrayList>(n); + List> sourceList = new ArrayList<>(n); for (int i = 0; i < n; i++) { sourceList.add(Observable.just(i)); } @@ -142,7 +142,7 @@ public void mergeALotOfSourcesOneByOneSynchronously() { @Test public void mergeALotOfSourcesOneByOneSynchronouslyTakeHalf() { int n = 10000; - List> sourceList = new ArrayList>(n); + List> sourceList = new ArrayList<>(n); for (int i = 0; i < n; i++) { sourceList.add(Observable.just(i)); } @@ -158,9 +158,9 @@ public void mergeALotOfSourcesOneByOneSynchronouslyTakeHalf() { @Test public void simple() { for (int i = 1; i < 100; i++) { - TestObserverEx to = new TestObserverEx(); - List> sourceList = new ArrayList>(i); - List result = new ArrayList(i); + TestObserverEx to = new TestObserverEx<>(); + List> sourceList = new ArrayList<>(i); + List result = new ArrayList<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Observable.just(j)); result.add(j); @@ -177,9 +177,9 @@ public void simple() { @Test public void simpleOneLess() { for (int i = 2; i < 100; i++) { - TestObserverEx to = new TestObserverEx(); - List> sourceList = new ArrayList>(i); - List result = new ArrayList(i); + TestObserverEx to = new TestObserverEx<>(); + List> sourceList = new ArrayList<>(i); + List result = new ArrayList<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Observable.just(j)); result.add(j); @@ -209,9 +209,9 @@ public void simpleAsyncLoop() { @Test public void simpleAsync() { for (int i = 1; i < 50; i++) { - TestObserver to = new TestObserver(); - List> sourceList = new ArrayList>(i); - Set expected = new HashSet(i); + TestObserver to = new TestObserver<>(); + List> sourceList = new ArrayList<>(i); + Set expected = new HashSet<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Observable.just(j).subscribeOn(Schedulers.io())); expected.add(j); @@ -221,7 +221,7 @@ public void simpleAsync() { to.awaitDone(1, TimeUnit.SECONDS); to.assertNoErrors(); - Set actual = new HashSet(to.values()); + Set actual = new HashSet<>(to.values()); assertEquals(expected, actual); } @@ -241,9 +241,9 @@ public void simpleOneLessAsync() { if (System.currentTimeMillis() - t > TimeUnit.SECONDS.toMillis(9)) { break; } - TestObserver to = new TestObserver(); - List> sourceList = new ArrayList>(i); - Set expected = new HashSet(i); + TestObserver to = new TestObserver<>(); + List> sourceList = new ArrayList<>(i); + Set expected = new HashSet<>(i); for (int j = 1; j <= i; j++) { sourceList.add(Observable.just(j).subscribeOn(Schedulers.io())); expected.add(j); @@ -253,7 +253,7 @@ public void simpleOneLessAsync() { to.awaitDone(1, TimeUnit.SECONDS); to.assertNoErrors(); - Set actual = new HashSet(to.values()); + Set actual = new HashSet<>(to.values()); assertEquals(expected, actual); } @@ -261,13 +261,13 @@ public void simpleOneLessAsync() { @Test public void take() throws Exception { - List> sourceList = new ArrayList>(3); + List> sourceList = new ArrayList<>(3); sourceList.add(Observable.range(0, 100000).subscribeOn(Schedulers.io())); sourceList.add(Observable.range(0, 100000).subscribeOn(Schedulers.io())); sourceList.add(Observable.range(0, 100000).subscribeOn(Schedulers.io())); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.merge(sourceList, 2).take(5).subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java index c198ff83f0..92d30f8a85 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java @@ -112,7 +112,7 @@ public void mergeArray() { public void mergeList() { final Observable o1 = Observable.unsafeCreate(new TestSynchronousObservable()); final Observable o2 = Observable.unsafeCreate(new TestSynchronousObservable()); - List> listOfObservables = new ArrayList>(); + List> listOfObservables = new ArrayList<>(); listOfObservables.add(o1); listOfObservables.add(o2); @@ -191,7 +191,7 @@ public void mergeArrayWithThreading() { final TestASynchronousObservable o2 = new TestASynchronousObservable(); Observable m = Observable.merge(Observable.unsafeCreate(o1), Observable.unsafeCreate(o2)); - TestObserver to = new TestObserver(stringObserver); + TestObserver to = new TestObserver<>(stringObserver); m.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); @@ -408,7 +408,7 @@ public void unsubscribeAsObservablesComplete() { AtomicBoolean os2 = new AtomicBoolean(false); Observable o2 = createObservableOf5IntervalsOf1SecondIncrementsWithSubscriptionHook(scheduler2, os2); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.merge(o1, o2).subscribe(to); // we haven't incremented time so nothing should be received yet @@ -450,7 +450,7 @@ public void earlyUnsubscribe() { AtomicBoolean os2 = new AtomicBoolean(false); Observable o2 = createObservableOf5IntervalsOf1SecondIncrementsWithSubscriptionHook(scheduler2, os2); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.merge(o1, o2).subscribe(to); // we haven't incremented time so nothing should be received yet @@ -523,7 +523,7 @@ public void concurrency() { for (int i = 0; i < 10; i++) { Observable merge = Observable.merge(o, o, o); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); merge.subscribe(to); to.awaitDone(3, TimeUnit.SECONDS); @@ -576,7 +576,7 @@ public void run() { for (int i = 0; i < 10; i++) { Observable merge = Observable.merge(o, o, o); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); merge.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); @@ -623,7 +623,7 @@ public void run() { for (int i = 0; i < 10; i++) { Observable merge = Observable.merge(o, o, o); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); merge.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); @@ -806,7 +806,7 @@ public void onNext(Integer t) { @Test public void merge1AsyncStreamOf1() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNAsyncStreamsOfN(1, 1).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -815,7 +815,7 @@ public void merge1AsyncStreamOf1() { @Test public void merge1AsyncStreamOf1000() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNAsyncStreamsOfN(1, 1000).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -824,7 +824,7 @@ public void merge1AsyncStreamOf1000() { @Test public void merge10AsyncStreamOf1000() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNAsyncStreamsOfN(10, 1000).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -833,7 +833,7 @@ public void merge10AsyncStreamOf1000() { @Test public void merge1000AsyncStreamOf1000() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNAsyncStreamsOfN(1000, 1000).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -842,7 +842,7 @@ public void merge1000AsyncStreamOf1000() { @Test public void merge2000AsyncStreamOf100() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNAsyncStreamsOfN(2000, 100).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -851,7 +851,7 @@ public void merge2000AsyncStreamOf100() { @Test public void merge100AsyncStreamOf1() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNAsyncStreamsOfN(100, 1).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -873,7 +873,7 @@ public Observable apply(Integer i) { @Test public void merge1SyncStreamOf1() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNSyncStreamsOfN(1, 1).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -882,7 +882,7 @@ public void merge1SyncStreamOf1() { @Test public void merge1SyncStreamOf1000000() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNSyncStreamsOfN(1, 1000000).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -891,7 +891,7 @@ public void merge1SyncStreamOf1000000() { @Test public void merge1000SyncStreamOf1000() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNSyncStreamsOfN(1000, 1000).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -900,7 +900,7 @@ public void merge1000SyncStreamOf1000() { @Test public void merge10000SyncStreamOf10() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNSyncStreamsOfN(10000, 10).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -909,7 +909,7 @@ public void merge10000SyncStreamOf10() { @Test public void merge1000000SyncStreamOf1() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); mergeNSyncStreamsOfN(1000000, 1).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -956,7 +956,7 @@ public boolean hasNext() { @Test public void mergeManyAsyncSingle() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable> os = Observable.range(1, 10000) .map(new Function>() { @@ -1004,7 +1004,7 @@ public Observable apply(Integer t) { ; void runMerge(Function> func, TestObserverEx to) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (int i = 0; i < 1000; i++) { list.add(i); } @@ -1022,12 +1022,12 @@ void runMerge(Function> func, TestObserverEx()); + runMerge(toScalar, new TestObserverEx<>()); } @Test public void fastMergeHiddenScalar() { - runMerge(toHiddenScalar, new TestObserverEx()); + runMerge(toHiddenScalar, new TestObserverEx<>()); } @Test diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java index e5414dfe7f..e561eaac1e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletableTest.java @@ -29,7 +29,7 @@ public class ObservableMergeWithCompletableTest extends RxJavaTest { @Test public void normal() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5).mergeWith( Completable.fromAction(new Action() { @@ -46,7 +46,7 @@ public void run() throws Exception { @Test public void take() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5).mergeWith( Completable.complete() diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java index 4949ad2081..43f0a7fff7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybeTest.java @@ -178,7 +178,7 @@ public void onNext(Integer t) { public void onErrorMainOverflow() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> observerRef = new AtomicReference>(); + final AtomicReference> observerRef = new AtomicReference<>(); TestObserver to = new Observable() { @Override protected void subscribeActual(Observer observer) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java index 28c6099d4c..c6b083f41d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingleTest.java @@ -170,7 +170,7 @@ public void onNext(Integer t) { public void onErrorMainOverflow() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> observerRef = new AtomicReference>(); + final AtomicReference> observerRef = new AtomicReference<>(); TestObserver to = new Observable() { @Override protected void subscribeActual(Observer observer) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java index 286d115edf..579b713313 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java @@ -66,7 +66,7 @@ public void ordering() throws InterruptedException { Observer observer = TestHelper.mockObserver(); InOrder inOrder = inOrder(observer); - TestObserverEx to = new TestObserverEx(observer); + TestObserverEx to = new TestObserverEx<>(observer); obs.observeOn(Schedulers.computation()).subscribe(to); @@ -387,7 +387,7 @@ public void afterUnsubscribeCalledThenObserverOnNextNeverCalled() { final TestScheduler testScheduler = new TestScheduler(); final Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); Observable.just(1, 2, 3) .observeOn(testScheduler) @@ -428,7 +428,7 @@ public boolean hasNext() { } }); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); o .take(7) .observeOn(Schedulers.newThread()) @@ -441,7 +441,7 @@ public boolean hasNext() { @Test public void asyncChild() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(0, 100000).observeOn(Schedulers.newThread()).observeOn(Schedulers.newThread()).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); @@ -565,7 +565,7 @@ public void inputAsyncFusedErrorDelayed() { @Test public void outputFused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.range(1, 5).hide() .observeOn(Schedulers.single()) @@ -578,7 +578,7 @@ public void outputFused() { @Test public void outputFusedReject() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); Observable.range(1, 5).hide() .observeOn(Schedulers.single()) @@ -591,7 +591,7 @@ public void outputFusedReject() { @Test public void inputOutputAsyncFusedError() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); UnicastSubject us = UnicastSubject.create(); @@ -611,7 +611,7 @@ public void inputOutputAsyncFusedError() { @Test public void inputOutputAsyncFusedErrorDelayed() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); UnicastSubject us = UnicastSubject.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java index 69a9b0f7cd..386b6c96a1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeNextTest.java @@ -37,7 +37,7 @@ public class ObservableOnErrorResumeNextTest extends RxJavaTest { @Test public void resumeNextWithSynchronousExecution() { - final AtomicReference receivedException = new AtomicReference(); + final AtomicReference receivedException = new AtomicReference<>(); Observable w = Observable.unsafeCreate(new ObservableSource() { @Override @@ -77,7 +77,7 @@ public Observable apply(Throwable t1) { @Test public void resumeNextWithAsyncExecution() { - final AtomicReference receivedException = new AtomicReference(); + final AtomicReference receivedException = new AtomicReference<>(); Subscription s = mock(Subscription.class); TestObservable w = new TestObservable(s, "one"); Function> resume = new Function>() { @@ -174,7 +174,7 @@ public Observable apply(Throwable t1) { Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); o.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); @@ -226,7 +226,7 @@ public void run() { @Test public void backpressure() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(0, 100000) .onErrorResumeNext(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeWithTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeWithTest.java index 217335ec08..f60ea43a25 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeWithTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorResumeWithTest.java @@ -146,7 +146,7 @@ public void run() { @Test public void backpressure() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(0, 100000) .onErrorResumeWith(Observable.just(1)) .observeOn(Schedulers.computation()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java index d9520fe6d2..69c6c6889c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturnTest.java @@ -37,7 +37,7 @@ public class ObservableOnErrorReturnTest extends RxJavaTest { public void resumeNext() { TestObservable f = new TestObservable("one"); Observable w = Observable.unsafeCreate(f); - final AtomicReference capturedException = new AtomicReference(); + final AtomicReference capturedException = new AtomicReference<>(); Observable observable = w.onErrorReturn(new Function() { @@ -72,7 +72,7 @@ public String apply(Throwable e) { public void functionThrowsError() { TestObservable f = new TestObservable("one"); Observable w = Observable.unsafeCreate(f); - final AtomicReference capturedException = new AtomicReference(); + final AtomicReference capturedException = new AtomicReference<>(); Observable observable = w.onErrorReturn(new Function() { @@ -130,7 +130,7 @@ public String apply(Throwable t1) { }); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); observable.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); @@ -144,7 +144,7 @@ public String apply(Throwable t1) { @Test public void backpressure() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(0, 100000) .onErrorReturn(new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java index ee80dc2cab..16cb03d00e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishTest.java @@ -126,7 +126,7 @@ public void run() { }); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.merge(fast, slow).subscribe(to); is.connect(); to.awaitDone(5, TimeUnit.SECONDS); @@ -146,7 +146,7 @@ public void accept(Integer t1) { } }); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); xs.publish(new Function, Observable>() { @Override @@ -173,7 +173,7 @@ public boolean test(Integer i) { @Test public void takeUntilWithPublishedStream() { Observable xs = Observable.range(0, Flowable.bufferSize() * 2); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); ConnectableObservable xsp = xs.publish(); xsp.takeUntil(xsp.skipWhile(new Predicate() { @@ -209,7 +209,7 @@ public void run() { final AtomicBoolean child1Unsubscribed = new AtomicBoolean(); final AtomicBoolean child2Unsubscribed = new AtomicBoolean(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); final TestObserver to1 = new TestObserver() { @Override @@ -257,7 +257,7 @@ public void connectWithNoSubscriber() { co.connect(); // Emit 0 scheduler.advanceTimeBy(15, TimeUnit.MILLISECONDS); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); co.subscribe(to); // Emit 1 and 2 scheduler.advanceTimeBy(50, TimeUnit.MILLISECONDS); @@ -270,7 +270,7 @@ public void connectWithNoSubscriber() { public void subscribeAfterDisconnectThenConnect() { ConnectableObservable source = Observable.just(1).publish(); - TestObserverEx to1 = new TestObserverEx(); + TestObserverEx to1 = new TestObserverEx<>(); source.subscribe(to1); @@ -282,7 +282,7 @@ public void subscribeAfterDisconnectThenConnect() { source.reset(); - TestObserverEx to2 = new TestObserverEx(); + TestObserverEx to2 = new TestObserverEx<>(); source.subscribe(to2); @@ -300,7 +300,7 @@ public void subscribeAfterDisconnectThenConnect() { public void noSubscriberRetentionOnCompleted() { ObservablePublish source = (ObservablePublish)Observable.just(1).publish(); - TestObserverEx to1 = new TestObserverEx(); + TestObserverEx to1 = new TestObserverEx<>(); source.subscribe(to1); @@ -382,9 +382,9 @@ public void observeOn() { Observable obs = co.observeOn(Schedulers.computation()); for (int i = 0; i < 1000; i++) { for (int j = 1; j < 6; j++) { - List> tos = new ArrayList>(); + List> tos = new ArrayList<>(); for (int k = 1; k < j; k++) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); tos.add(to); obs.subscribe(to); } @@ -480,7 +480,7 @@ public void addRemoveRace() { final TestObserver to = co.test(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -628,7 +628,7 @@ public void subscribeDisconnectRace() { final ConnectableObservable co = ps.publish(); final Disposable d = co.connect(); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -762,7 +762,7 @@ public void disposedUpfront() { @Test public void altConnectCrash() { try { - new ObservablePublish(Observable.empty()) + new ObservablePublish<>(Observable.empty()) .connect(new Consumer() { @Override public void accept(Disposable t) throws Exception { @@ -779,7 +779,7 @@ public void accept(Disposable t) throws Exception { public void altConnectRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { final ConnectableObservable co = - new ObservablePublish(Observable.never()); + new ObservablePublish<>(Observable.never()); Runnable r = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLongTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLongTest.java index e25cb17b8a..7a58572d2f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLongTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLongTest.java @@ -93,14 +93,14 @@ public void rangeWithOverflow5() { @Test public void noBackpressure() { - ArrayList list = new ArrayList(Flowable.bufferSize() * 2); + ArrayList list = new ArrayList<>(Flowable.bufferSize() * 2); for (long i = 1; i <= Flowable.bufferSize() * 2 + 1; i++) { list.add(i); } Observable o = Observable.rangeLong(1, list.size()); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); o.subscribe(to); @@ -137,7 +137,7 @@ public void onNext(Long t) { @Test public void nearMaxValueWithoutBackpressure() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.rangeLong(Long.MAX_VALUE - 1L, 2L).subscribe(to); to.assertComplete(); @@ -171,7 +171,7 @@ public void noOverflow() { @Test public void fused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Observable.rangeLong(1, 2).subscribe(to); @@ -181,7 +181,7 @@ public void fused() { @Test public void fusedReject() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); Observable.rangeLong(1, 2).subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeTest.java index 2f9374ad5a..e789b55093 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeTest.java @@ -94,14 +94,14 @@ public void rangeWithOverflow5() { @Test public void noBackpressure() { - ArrayList list = new ArrayList(Flowable.bufferSize() * 2); + ArrayList list = new ArrayList<>(Flowable.bufferSize() * 2); for (int i = 1; i <= Flowable.bufferSize() * 2 + 1; i++) { list.add(i); } Observable o = Observable.range(1, list.size()); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); o.subscribe(to); @@ -138,7 +138,7 @@ public void onNext(Integer t) { @Test public void nearMaxValueWithoutBackpressure() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(Integer.MAX_VALUE - 1, 2).subscribe(to); to.assertComplete(); @@ -158,7 +158,7 @@ public void negativeCount() { @Test public void requestWrongFusion() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ASYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); Observable.range(1, 5) .subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRedoTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRedoTest.java index cac0b75d1a..2dc15f230e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRedoTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRedoTest.java @@ -24,7 +24,7 @@ public class ObservableRedoTest extends RxJavaTest { @Test public void redoCancel() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1) .repeatWhen(new Function, ObservableSource>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java index 7e862e1ad2..d9b98de8f9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCountTest.java @@ -190,8 +190,8 @@ public void run() { .publish().refCount(); for (int i = 0; i < 10; i++) { - TestObserver to1 = new TestObserver(); - TestObserver to2 = new TestObserver(); + TestObserver to1 = new TestObserver<>(); + TestObserver to2 = new TestObserver<>(); r.subscribe(to1); r.subscribe(to2); try { @@ -233,7 +233,7 @@ public void run() { } }); - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); o.publish().refCount().subscribeOn(Schedulers.newThread()).subscribe(observer); System.out.println("send unsubscribe"); // wait until connected @@ -278,7 +278,7 @@ public void accept(Disposable d) { } }); - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); o.publish().refCount().subscribeOn(Schedulers.computation()).subscribe(observer); System.out.println("send unsubscribe"); @@ -367,7 +367,7 @@ public void refCount() { Observable interval = Observable.interval(100, TimeUnit.MILLISECONDS, s).publish().refCount(); // subscribe list1 - final List list1 = new ArrayList(); + final List list1 = new ArrayList<>(); Disposable d1 = interval.subscribe(new Consumer() { @Override public void accept(Long t1) { @@ -382,7 +382,7 @@ public void accept(Long t1) { assertEquals(1L, list1.get(1).longValue()); // subscribe list2 - final List list2 = new ArrayList(); + final List list2 = new ArrayList<>(); Disposable d2 = interval.subscribe(new Consumer() { @Override public void accept(Long t1) { @@ -427,7 +427,7 @@ public void accept(Long t1) { // subscribing a new one should start over because the source should have been unsubscribed // subscribe list3 - final List list3 = new ArrayList(); + final List list3 = new ArrayList<>(); interval.subscribe(new Consumer() { @Override public void accept(Long t1) { @@ -498,8 +498,8 @@ public Integer apply(Integer t1, Integer t2) { }) .publish().refCount(); - TestObserverEx to1 = new TestObserverEx(); - TestObserverEx to2 = new TestObserverEx(); + TestObserverEx to1 = new TestObserverEx<>(); + TestObserverEx to2 = new TestObserverEx<>(); combined.subscribe(to1); combined.subscribe(to2); @@ -1133,7 +1133,7 @@ public void unsubscribeSubscribeRace() { final TestObserver to1 = source.test(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -1337,7 +1337,7 @@ protected void subscribeActual(Observer observer) { @Test public void timeoutResetsSource() { - TestConnectableObservable tco = new TestConnectableObservable(); + TestConnectableObservable tco = new TestConnectableObservable<>(); ObservableRefCount o = (ObservableRefCount)tco.refCount(); RefConnection rc = new RefConnection(o); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java index f7b1d93b0b..78c793f1b4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatTest.java @@ -164,7 +164,7 @@ public void repeatAndDistinctUnbounded() { .repeat(3) .distinct(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); src.subscribe(to); @@ -176,8 +176,8 @@ public void repeatAndDistinctUnbounded() { /** Issue #2844: wrong target of request. */ @Test public void repeatRetarget() { - final List concatBase = new ArrayList(); - TestObserver to = new TestObserver(); + final List concatBase = new ArrayList<>(); + TestObserver to = new TestObserver<>(); Observable.just(1, 2) .repeat(5) .concatMap(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java index ba3a7d6974..fe2c59d088 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayEagerTruncateTest.java @@ -729,7 +729,7 @@ void truncate() { buf.addLast(new Node(4)); buf.addLast(new Node(5)); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.collect(values); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), values); @@ -753,8 +753,8 @@ void truncate() { @Test public void timedAndSizedTruncation() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, false); - List values = new ArrayList(); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, false); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -792,12 +792,12 @@ public void timedAndSizedTruncation() { @Test public void timedAndSizedTruncationError() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, false); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, false); Assert.assertFalse(buf.hasCompleted()); Assert.assertFalse(buf.hasError()); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -835,8 +835,8 @@ public void timedAndSizedTruncationError() { @Test public void sizedTruncation() { - SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer(2, false); - List values = new ArrayList(); + SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer<>(2, false); + List values = new ArrayList<>(); buf.next(1); buf.next(2); @@ -871,7 +871,7 @@ public void sizedTruncation() { public void coldReplayNoBackpressure() { Observable source = Observable.range(0, 1000).replay().autoConnect(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.subscribe(to); @@ -949,7 +949,7 @@ public void unsubscribeSource() throws Throwable { @Test public void take() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable cached = Observable.range(1, 100).replay().autoConnect(); cached.take(10).subscribe(to); @@ -965,7 +965,7 @@ public void take() { public void async() { Observable source = Observable.range(1, 10000); for (int i = 0; i < 100; i++) { - TestObserverEx to1 = new TestObserverEx(); + TestObserverEx to1 = new TestObserverEx<>(); Observable cached = source.replay().autoConnect(); @@ -976,7 +976,7 @@ public void async() { to1.assertTerminated(); assertEquals(10000, to1.values().size()); - TestObserverEx to2 = new TestObserverEx(); + TestObserverEx to2 = new TestObserverEx<>(); cached.observeOn(Schedulers.computation()).subscribe(to2); to2.awaitDone(2, TimeUnit.SECONDS); @@ -995,14 +995,14 @@ public void asyncComeAndGo() { Observable output = cached.observeOn(Schedulers.computation()); - List> list = new ArrayList>(100); + List> list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); list.add(to); output.skip(i * 10).take(10).subscribe(to); } - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 10; i++) { expected.add((long)(i - 10)); } @@ -1036,7 +1036,7 @@ public void subscribe(Observer t) { } }); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); firehose.replay().autoConnect().observeOn(Schedulers.computation()).takeLast(100).subscribe(to); to.awaitDone(3, TimeUnit.SECONDS); @@ -1052,14 +1052,14 @@ public void valuesAndThenError() { .concatWith(Observable.error(new TestException())) .replay().autoConnect(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.subscribe(to); to.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); to.assertNotComplete(); Assert.assertEquals(1, to.errors().size()); - TestObserverEx to2 = new TestObserverEx(); + TestObserverEx to2 = new TestObserverEx<>(); source.subscribe(to2); to2.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); @@ -1130,8 +1130,8 @@ public void subscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableObservable co = Observable.range(1, 3).replay(); - final TestObserver to1 = new TestObserver(); - final TestObserver to2 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); + final TestObserver to2 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -1156,8 +1156,8 @@ public void addRemoveRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableObservable co = Observable.range(1, 3).replay(); - final TestObserver to1 = new TestObserver(); - final TestObserver to2 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); + final TestObserver to2 = new TestObserver<>(); co.subscribe(to1); @@ -1256,7 +1256,7 @@ public void subscribeOnNextRace() { final ConnectableObservable co = ps.replay(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -1285,7 +1285,7 @@ public void unsubscribeOnNextRace() { final ConnectableObservable co = ps.replay(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); co.subscribe(to1); @@ -1314,7 +1314,7 @@ public void unsubscribeReplayRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableObservable co = Observable.range(1, 1000).replay(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); co.connect(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java index e55d5ff716..9b31f51e8d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplayTest.java @@ -729,7 +729,7 @@ void truncate() { buf.addLast(new Node(4)); buf.addLast(new Node(5)); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.collect(values); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), values); @@ -753,8 +753,8 @@ void truncate() { @Test public void timedAndSizedTruncation() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, false); - List values = new ArrayList(); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, false); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -792,12 +792,12 @@ public void timedAndSizedTruncation() { @Test public void timedAndSizedTruncationError() { TestScheduler test = new TestScheduler(); - SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer(2, 2000, TimeUnit.MILLISECONDS, test, false); + SizeAndTimeBoundReplayBuffer buf = new SizeAndTimeBoundReplayBuffer<>(2, 2000, TimeUnit.MILLISECONDS, test, false); Assert.assertFalse(buf.hasCompleted()); Assert.assertFalse(buf.hasError()); - List values = new ArrayList(); + List values = new ArrayList<>(); buf.next(1); test.advanceTimeBy(1, TimeUnit.SECONDS); @@ -835,8 +835,8 @@ public void timedAndSizedTruncationError() { @Test public void sizedTruncation() { - SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer(2, false); - List values = new ArrayList(); + SizeBoundReplayBuffer buf = new SizeBoundReplayBuffer<>(2, false); + List values = new ArrayList<>(); buf.next(1); buf.next(2); @@ -871,7 +871,7 @@ public void sizedTruncation() { public void coldReplayNoBackpressure() { Observable source = Observable.range(0, 1000).replay().autoConnect(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.subscribe(to); @@ -949,7 +949,7 @@ public void unsubscribeSource() throws Throwable { @Test public void take() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable cached = Observable.range(1, 100).replay().autoConnect(); cached.take(10).subscribe(to); @@ -965,7 +965,7 @@ public void take() { public void async() { Observable source = Observable.range(1, 10000); for (int i = 0; i < 100; i++) { - TestObserverEx to1 = new TestObserverEx(); + TestObserverEx to1 = new TestObserverEx<>(); Observable cached = source.replay().autoConnect(); @@ -976,7 +976,7 @@ public void async() { to1.assertTerminated(); assertEquals(10000, to1.values().size()); - TestObserverEx to2 = new TestObserverEx(); + TestObserverEx to2 = new TestObserverEx<>(); cached.observeOn(Schedulers.computation()).subscribe(to2); to2.awaitDone(2, TimeUnit.SECONDS); @@ -995,14 +995,14 @@ public void asyncComeAndGo() { Observable output = cached.observeOn(Schedulers.computation()); - List> list = new ArrayList>(100); + List> list = new ArrayList<>(100); for (int i = 0; i < 100; i++) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); list.add(to); output.skip(i * 10).take(10).subscribe(to); } - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 10; i++) { expected.add((long)(i - 10)); } @@ -1036,7 +1036,7 @@ public void subscribe(Observer t) { } }); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); firehose.replay().autoConnect().observeOn(Schedulers.computation()).takeLast(100).subscribe(to); to.awaitDone(3, TimeUnit.SECONDS); @@ -1052,14 +1052,14 @@ public void valuesAndThenError() { .concatWith(Observable.error(new TestException())) .replay().autoConnect(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.subscribe(to); to.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); to.assertNotComplete(); Assert.assertEquals(1, to.errors().size()); - TestObserverEx to2 = new TestObserverEx(); + TestObserverEx to2 = new TestObserverEx<>(); source.subscribe(to2); to2.assertValues(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); @@ -1130,8 +1130,8 @@ public void subscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableObservable co = Observable.range(1, 3).replay(); - final TestObserver to1 = new TestObserver(); - final TestObserver to2 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); + final TestObserver to2 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -1156,8 +1156,8 @@ public void addRemoveRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableObservable co = Observable.range(1, 3).replay(); - final TestObserver to1 = new TestObserver(); - final TestObserver to2 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); + final TestObserver to2 = new TestObserver<>(); co.subscribe(to1); @@ -1256,7 +1256,7 @@ public void subscribeOnNextRace() { final ConnectableObservable co = ps.replay(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -1285,7 +1285,7 @@ public void unsubscribeOnNextRace() { final ConnectableObservable co = ps.replay(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); co.subscribe(to1); @@ -1314,7 +1314,7 @@ public void unsubscribeReplayRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final ConnectableObservable co = Observable.range(1, 1000).replay(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); co.connect(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java index d5a70e0e47..228725c0a0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableResourceWrapperTest.java @@ -27,8 +27,8 @@ public class ObservableResourceWrapperTest extends RxJavaTest { @Test public void disposed() { - TestObserver to = new TestObserver(); - ObserverResourceWrapper orw = new ObserverResourceWrapper(to); + TestObserver to = new TestObserver<>(); + ObserverResourceWrapper orw = new ObserverResourceWrapper<>(to); Disposable d = Disposable.empty(); @@ -43,16 +43,16 @@ public void disposed() { @Test public void doubleOnSubscribe() { - TestObserver to = new TestObserver(); - ObserverResourceWrapper orw = new ObserverResourceWrapper(to); + TestObserver to = new TestObserver<>(); + ObserverResourceWrapper orw = new ObserverResourceWrapper<>(to); TestHelper.doubleOnSubscribe(orw); } @Test public void onErrorDisposes() { - TestObserver to = new TestObserver(); - ObserverResourceWrapper orw = new ObserverResourceWrapper(to); + TestObserver to = new TestObserver<>(); + ObserverResourceWrapper orw = new ObserverResourceWrapper<>(to); Disposable d = Disposable.empty(); Disposable d1 = Disposable.empty(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java index 82d96dc5d7..381ee0a790 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java @@ -64,7 +64,7 @@ public void subscribe(Observer t1) { } }); - TestObserver to = new TestObserver(consumer); + TestObserver to = new TestObserver<>(consumer); producer.retryWhen(new Function, Observable>() { @Override @@ -117,7 +117,7 @@ public void retryIndefinitely() { Observer observer = TestHelper.mockObserver(); int numRetries = 20; Observable origin = Observable.unsafeCreate(new FuncWithErrors(numRetries)); - origin.retry().subscribe(new TestObserver(observer)); + origin.retry().subscribe(new TestObserver<>(observer)); InOrder inOrder = inOrder(observer); // should show 3 attempts @@ -136,7 +136,7 @@ public void schedulingNotificationHandler() { Observer observer = TestHelper.mockObserver(); int numRetries = 2; Observable origin = Observable.unsafeCreate(new FuncWithErrors(numRetries)); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); origin.retryWhen(new Function, Observable>() { @Override public Observable apply(Observable t1) { @@ -205,7 +205,7 @@ public Integer apply(Throwable t1) { public void onCompletedFromNotificationHandler() { Observer observer = TestHelper.mockObserver(); Observable origin = Observable.unsafeCreate(new FuncWithErrors(1)); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); origin.retryWhen(new Function, Observable>() { @Override public Observable apply(Observable t1) { @@ -455,7 +455,7 @@ public void run() { public void sourceObservableCallsUnsubscribe() throws InterruptedException { final AtomicInteger subsCount = new AtomicInteger(0); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); ObservableSource onSubscribe = new ObservableSource() { @Override @@ -486,7 +486,7 @@ public void subscribe(Observer observer) { public void sourceObservableRetry1() throws InterruptedException { final AtomicInteger subsCount = new AtomicInteger(0); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); ObservableSource onSubscribe = new ObservableSource() { @Override @@ -505,7 +505,7 @@ public void subscribe(Observer observer) { public void sourceObservableRetry0() throws InterruptedException { final AtomicInteger subsCount = new AtomicInteger(0); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); ObservableSource onSubscribe = new ObservableSource() { @Override @@ -625,7 +625,7 @@ public void unsubscribeAfterError() { SlowObservable so = new SlowObservable(100, 0, "testUnsubscribeAfterError"); Observable o = Observable.unsafeCreate(so).retry(5); - AsyncObserver async = new AsyncObserver(observer); + AsyncObserver async = new AsyncObserver<>(observer); o.subscribe(async); @@ -649,7 +649,7 @@ public void timeoutWithRetry() { SlowObservable so = new SlowObservable(100, 10, "testTimeoutWithRetry"); Observable o = Observable.unsafeCreate(so).timeout(80, TimeUnit.MILLISECONDS).retry(5); - AsyncObserver async = new AsyncObserver(observer); + AsyncObserver async = new AsyncObserver<>(observer); o.subscribe(async); @@ -671,7 +671,7 @@ public void retryWithBackpressure() throws InterruptedException { for (int i = 0; i < 400; i++) { Observer observer = TestHelper.mockObserver(); Observable origin = Observable.unsafeCreate(new FuncWithErrors(NUM_RETRIES)); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); origin.retry().observeOn(Schedulers.computation()).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); @@ -702,7 +702,7 @@ public void retryWithBackpressureParallel() throws InterruptedException { } final AtomicInteger timeouts = new AtomicInteger(); - final Map> data = new ConcurrentHashMap>(); + final Map> data = new ConcurrentHashMap<>(); int m = 5000; final CountDownLatch cdl = new CountDownLatch(m); @@ -714,11 +714,11 @@ public void run() { final AtomicInteger nexts = new AtomicInteger(); try { Observable origin = Observable.unsafeCreate(new FuncWithErrors(NUM_RETRIES)); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); origin.retry() .observeOn(Schedulers.computation()).subscribe(to); to.awaitDone(2500, TimeUnit.MILLISECONDS); - List onNextEvents = new ArrayList(to.values()); + List onNextEvents = new ArrayList<>(to.values()); if (onNextEvents.size() != NUM_RETRIES + 2) { for (Throwable t : to.errors()) { onNextEvents.add(t.toString()); @@ -816,7 +816,7 @@ public Observable apply(GroupedObservable t1) { return t1.take(1); } }) - .subscribe(new TestObserver(observer)); + .subscribe(new TestObserver<>(observer)); InOrder inOrder = inOrder(observer); // should show 3 attempts @@ -861,7 +861,7 @@ public Observable apply(GroupedObservable t1) { return t1.take(1); } }) - .subscribe(new TestObserver(observer)); + .subscribe(new TestObserver<>(observer)); InOrder inOrder = inOrder(observer); // should show 3 attempts diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java index d7d20c1347..3b57f6c2e1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWithPredicateTest.java @@ -235,7 +235,7 @@ public void unsubscribeAfterError() { .unsafeCreate(so) .retry(retry5); - ObservableRetryTest.AsyncObserver async = new ObservableRetryTest.AsyncObserver(observer); + ObservableRetryTest.AsyncObserver async = new ObservableRetryTest.AsyncObserver<>(observer); o.subscribe(async); @@ -262,7 +262,7 @@ public void timeoutWithRetry() { .timeout(80, TimeUnit.MILLISECONDS) .retry(retry5); - ObservableRetryTest.AsyncObserver async = new ObservableRetryTest.AsyncObserver(observer); + ObservableRetryTest.AsyncObserver async = new ObservableRetryTest.AsyncObserver<>(observer); o.subscribe(async); @@ -278,7 +278,7 @@ public void timeoutWithRetry() { @Test public void issue2826() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final RuntimeException e = new RuntimeException("You shall not pass"); final AtomicInteger c = new AtomicInteger(); Observable.just(1).map(new Function() { @@ -312,7 +312,7 @@ public Integer apply(Integer t1) { @Test public void issue3008RetryWithPredicate() { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); final AtomicBoolean isFirst = new AtomicBoolean(true); Observable. just(1L, 2L, 3L).map(new Function() { @Override @@ -340,7 +340,7 @@ public void accept(Long t) { @Test public void issue3008RetryInfinite() { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); final AtomicBoolean isFirst = new AtomicBoolean(true); Observable. just(1L, 2L, 3L).map(new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMapTest.java index 13b01e6888..bffd568b70 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMapTest.java @@ -59,7 +59,7 @@ public Integer get() throws Exception { static final class OneCallablePublisher implements ObservableSource, Supplier { @Override public void subscribe(Observer observer) { - ScalarDisposable sd = new ScalarDisposable(observer, 1); + ScalarDisposable sd = new ScalarDisposable<>(observer, 1); observer.onSubscribe(sd); sd.run(); } @@ -72,7 +72,7 @@ public Integer get() throws Exception { @Test public void tryScalarXMap() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); assertTrue(ObservableScalarXMap.tryScalarXMapSubscribe(new CallablePublisher(), to, new Function>() { @Override public ObservableSource apply(Integer f) throws Exception { @@ -85,7 +85,7 @@ public ObservableSource apply(Integer f) throws Exception { @Test public void emptyXMap() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); assertTrue(ObservableScalarXMap.tryScalarXMapSubscribe(new EmptyCallablePublisher(), to, new Function>() { @Override @@ -99,7 +99,7 @@ public ObservableSource apply(Integer f) throws Exception { @Test public void mapperCrashes() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); assertTrue(ObservableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), to, new Function>() { @Override @@ -113,7 +113,7 @@ public ObservableSource apply(Integer f) throws Exception { @Test public void mapperToJust() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); assertTrue(ObservableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), to, new Function>() { @Override @@ -127,7 +127,7 @@ public ObservableSource apply(Integer f) throws Exception { @Test public void mapperToEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); assertTrue(ObservableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), to, new Function>() { @Override @@ -141,7 +141,7 @@ public ObservableSource apply(Integer f) throws Exception { @Test public void mapperToCrashingCallable() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); assertTrue(ObservableScalarXMap.tryScalarXMapSubscribe(new OneCallablePublisher(), to, new Function>() { @Override @@ -179,8 +179,8 @@ public ObservableSource apply(Integer v) throws Exception { @Test public void scalarDisposableStateCheck() { - TestObserver to = new TestObserver(); - ScalarDisposable sd = new ScalarDisposable(to, 1); + TestObserver to = new TestObserver<>(); + ScalarDisposable sd = new ScalarDisposable<>(to, 1); to.onSubscribe(sd); assertFalse(sd.isDisposed()); @@ -213,8 +213,8 @@ public void scalarDisposableStateCheck() { @Test public void scalarDisposableRunDisposeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestObserver to = new TestObserver(); - final ScalarDisposable sd = new ScalarDisposable(to, 1); + TestObserver to = new TestObserver<>(); + final ScalarDisposable sd = new ScalarDisposable<>(to, 1); to.onSubscribe(sd); Runnable r1 = new Runnable() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java index 590273ce60..c482d121be 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanTest.java @@ -114,7 +114,7 @@ public Integer apply(Integer t1, Integer t2) { @Test public void shouldNotEmitUntilAfterSubscription() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 100).scan(0, new BiFunction() { @Override @@ -181,7 +181,7 @@ public void seedFactory() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @@ -208,7 +208,7 @@ public Integer apply(Integer t1, Integer t2) { }).take(1); - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); o.subscribe(observer); observer.assertValue(0); @@ -220,7 +220,7 @@ public Integer apply(Integer t1, Integer t2) { public void initialValueEmittedNoProducer() { PublishSubject source = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.scan(0, new BiFunction() { @Override @@ -310,7 +310,7 @@ public Object apply(Object a, Object b) throws Exception { public void scanFunctionThrowsAndUpstreamErrorsDoesNotResultInTwoTerminalEvents() { final RuntimeException err = new RuntimeException(); final RuntimeException err2 = new RuntimeException(); - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); final Consumer errorConsumer = new Consumer() { @Override public void accept(Throwable t) throws Exception { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleTest.java index 4c7faeaf3e..2290c645ca 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleTest.java @@ -465,7 +465,7 @@ public Integer apply(Integer i1, Integer i2) { @Test public void singleElementOperatorDoNotSwallowExceptionWhenDone() { final Throwable exception = new RuntimeException("some error"); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); try { RxJavaPlugins.setErrorHandler(new Consumer() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTest.java index 7b87b3e329..8edf026fe1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTest.java @@ -84,7 +84,7 @@ public void skipLastWithZeroCount() { @Test public void skipLastWithBackpressure() { Observable o = Observable.range(0, Flowable.bufferSize() * 2).skipLast(Flowable.bufferSize() + 10); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); o.observeOn(Schedulers.computation()).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertNoErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipTest.java index d122e23399..fbd5572246 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipTest.java @@ -138,7 +138,7 @@ public void skipError() { @Test public void requestOverflowDoesNotOccur() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.range(1, 10).skip(5).subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java index 26ffe95a71..53ccf2b3a6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOnTest.java @@ -36,7 +36,7 @@ public void issue813() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch doneLatch = new CountDownLatch(1); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable .unsafeCreate(new ObservableSource() { @@ -74,7 +74,7 @@ public void subscribe( @Test public void onError() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.unsafeCreate(new ObservableSource() { @Override @@ -147,7 +147,7 @@ public Disposable schedule(@NonNull final Runnable action, final long delayTime, @Test public void unsubscribeInfiniteStream() throws InterruptedException { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final AtomicInteger count = new AtomicInteger(); Observable.unsafeCreate(new ObservableSource() { @@ -174,7 +174,7 @@ public void subscribe(Observer sub) { public void cancelBeforeActualSubscribe() { TestScheduler test = new TestScheduler(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1).hide() .subscribeOn(test).subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java index 5eb40104fc..88f43e7452 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchTest.java @@ -995,7 +995,7 @@ public void outerInnerErrorRaceIgnoreDispose() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> obs1 = new AtomicReference>(); + final AtomicReference> obs1 = new AtomicReference<>(); final Observable ps1 = new Observable() { @Override protected void subscribeActual( @@ -1003,7 +1003,7 @@ protected void subscribeActual( obs1.set(observer); } }; - final AtomicReference> obs2 = new AtomicReference>(); + final AtomicReference> obs2 = new AtomicReference<>(); final Observable ps2 = new Observable() { @Override protected void subscribeActual( @@ -1203,7 +1203,7 @@ public Object apply(Integer w) throws Exception { public void undeliverableUponCancel() { List errors = TestHelper.trackPluginErrors(); try { - final TestObserverEx to = new TestObserverEx(); + final TestObserverEx to = new TestObserverEx<>(); Observable.just(1) .map(new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOneTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOneTest.java index 63161e7fb8..92536eb95e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOneTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOneTest.java @@ -28,7 +28,7 @@ public class ObservableTakeLastOneTest extends RxJavaTest { @Test public void lastOfManyReturnsLast() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.range(1, 10).takeLast(1).subscribe(to); to.assertValue(10); to.assertNoErrors(); @@ -37,7 +37,7 @@ public void lastOfManyReturnsLast() { @Test public void lastOfEmptyReturnsEmpty() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.empty().takeLast(1).subscribe(to); to.assertNoValues(); to.assertNoErrors(); @@ -46,7 +46,7 @@ public void lastOfEmptyReturnsEmpty() { @Test public void lastOfOneReturnsLast() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1).takeLast(1).subscribe(to); to.assertValue(1); to.assertNoErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTest.java index 90429038e6..2aa9e37985 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTest.java @@ -90,7 +90,7 @@ public void takeLastWithNegativeCount() { @Test public void backpressure1() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 100000).takeLast(1) .observeOn(Schedulers.newThread()) .map(newSlowProcessor()).subscribe(to); @@ -101,7 +101,7 @@ public void backpressure1() { @Test public void backpressure2() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 100000).takeLast(Flowable.bufferSize() * 4) .observeOn(Schedulers.newThread()).map(newSlowProcessor()).subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java index 2225f3022b..f83fb8595d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeTest.java @@ -257,7 +257,7 @@ public void subscribe(Observer op) { @Test public void takeObserveOn() { Observer o = TestHelper.mockObserver(); - TestObserver to = new TestObserver(o); + TestObserver to = new TestObserver<>(o); INFINITE_OBSERVABLE .observeOn(Schedulers.newThread()).take(1).subscribe(to); @@ -272,7 +272,7 @@ public void takeObserveOn() { @Test public void interrupt() throws InterruptedException { - final AtomicReference exception = new AtomicReference(); + final AtomicReference exception = new AtomicReference<>(); final CountDownLatch latch = new CountDownLatch(1); Observable.just(1).subscribeOn(Schedulers.computation()).take(1) .subscribe(new Consumer() { @@ -317,7 +317,7 @@ public void onNext(Integer t) { public void reentrantTake() { final PublishSubject source = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.take(1).doOnNext(new Consumer() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java index 9566d64c9a..143fe9d3ca 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicateTest.java @@ -141,7 +141,7 @@ public boolean test(Integer v) { @Test public void errorIncludesLastValueAsCause() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final TestException e = new TestException("Forced failure"); Predicate predicate = (new Predicate() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilTest.java index 790af0c882..5e831b03b3 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilTest.java @@ -190,7 +190,7 @@ public void untilFires() { PublishSubject source = PublishSubject.create(); PublishSubject until = PublishSubject.create(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.takeUntil(until).subscribe(to); @@ -217,7 +217,7 @@ public void mainCompletes() { PublishSubject source = PublishSubject.create(); PublishSubject until = PublishSubject.create(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.takeUntil(until).subscribe(to); @@ -242,7 +242,7 @@ public void downstreamUnsubscribes() { PublishSubject source = PublishSubject.create(); PublishSubject until = PublishSubject.create(); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); source.takeUntil(until).take(1).subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java index b348cda2ce..134e1f93f6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhileTest.java @@ -223,7 +223,7 @@ public boolean test(Integer t1) { return t1 < 2; } }); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); source.subscribe(to); @@ -236,7 +236,7 @@ public boolean test(Integer t1) { @Test public void errorCauseIncludesLastValue() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just("abc").takeWhile(new Predicate() { @Override public boolean test(String t1) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java index 7eecc75fb5..3b849ab978 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java @@ -60,11 +60,11 @@ public void timeInterval() { subject.onComplete(); inOrder.verify(observer, times(1)).onNext( - new Timed(1, 1000, TIME_UNIT)); + new Timed<>(1, 1000, TIME_UNIT)); inOrder.verify(observer, times(1)).onNext( - new Timed(2, 2000, TIME_UNIT)); + new Timed<>(2, 2000, TIME_UNIT)); inOrder.verify(observer, times(1)).onNext( - new Timed(3, 3000, TIME_UNIT)); + new Timed<>(3, 3000, TIME_UNIT)); inOrder.verify(observer, times(1)).onComplete(); inOrder.verifyNoMoreInteractions(); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java index f0001ab276..fe7b2cd370 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTests.java @@ -51,7 +51,7 @@ public void setUp() { @Test public void shouldNotTimeoutIfOnNextWithinTimeout() { Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); withTimeout.subscribe(to); @@ -66,7 +66,7 @@ public void shouldNotTimeoutIfOnNextWithinTimeout() { @Test public void shouldNotTimeoutIfSecondOnNextWithinTimeout() { Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); withTimeout.subscribe(to); @@ -82,7 +82,7 @@ public void shouldNotTimeoutIfSecondOnNextWithinTimeout() { @Test public void shouldTimeoutIfOnNextNotWithinTimeout() { - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); withTimeout.subscribe(observer); @@ -92,7 +92,7 @@ public void shouldTimeoutIfOnNextNotWithinTimeout() { @Test public void shouldTimeoutIfSecondOnNextNotWithinTimeout() { - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); withTimeout.subscribe(observer); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); underlyingSubject.onNext("One"); @@ -104,7 +104,7 @@ public void shouldTimeoutIfSecondOnNextNotWithinTimeout() { @Test public void shouldCompleteIfUnderlyingComletes() { Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); withTimeout.subscribe(observer); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); underlyingSubject.onComplete(); @@ -117,7 +117,7 @@ public void shouldCompleteIfUnderlyingComletes() { @Test public void shouldErrorIfUnderlyingErrors() { Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); withTimeout.subscribe(observer); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); underlyingSubject.onError(new UnsupportedOperationException()); @@ -132,7 +132,7 @@ public void shouldSwitchToOtherIfOnNextNotWithinTimeout() { Observable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); source.subscribe(to); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -155,7 +155,7 @@ public void shouldSwitchToOtherIfOnErrorNotWithinTimeout() { Observable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); source.subscribe(to); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -178,7 +178,7 @@ public void shouldSwitchToOtherIfOnCompletedNotWithinTimeout() { Observable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); source.subscribe(to); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -201,7 +201,7 @@ public void shouldSwitchToOtherAndCanBeUnsubscribedIfOnNextNotWithinTimeout() { Observable source = underlyingSubject.timeout(TIMEOUT, TIME_UNIT, testScheduler, other); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); source.subscribe(to); testScheduler.advanceTimeBy(2, TimeUnit.SECONDS); @@ -231,7 +231,7 @@ public void shouldTimeoutIfSynchronizedObservableEmitFirstOnNextNotWithinTimeout final CountDownLatch exit = new CountDownLatch(1); final CountDownLatch timeoutSetuped = new CountDownLatch(1); - final TestObserverEx observer = new TestObserverEx(); + final TestObserverEx observer = new TestObserverEx<>(); new Thread(new Runnable() { @@ -280,7 +280,7 @@ public void subscribe(Observer observer) { TestScheduler testScheduler = new TestScheduler(); Observable observableWithTimeout = never.timeout(1000, TimeUnit.MILLISECONDS, testScheduler); - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); observableWithTimeout.subscribe(observer); testScheduler.advanceTimeBy(2000, TimeUnit.MILLISECONDS); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java index 9e6540e0b3..60cb5a8b92 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutWithSelectorTest.java @@ -329,7 +329,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(o).onComplete(); - final TestObserver to = new TestObserver(o); + final TestObserver to = new TestObserver<>(o); new Thread(new Runnable() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java index d77ee456db..8eddbce491 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimerTest.java @@ -64,7 +64,7 @@ public void timerOnce() { @Test public void timerPeriodically() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.interval(100, 100, TimeUnit.MILLISECONDS, scheduler).subscribe(to); @@ -92,7 +92,7 @@ public void timerPeriodically() { @Test public void interval() { Observable w = Observable.interval(1, TimeUnit.SECONDS, scheduler); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); w.subscribe(to); to.assertNoValues(); @@ -117,8 +117,8 @@ public void interval() { public void withMultipleSubscribersStartingAtSameTime() { Observable w = Observable.interval(1, TimeUnit.SECONDS, scheduler); - TestObserver to1 = new TestObserver(); - TestObserver to2 = new TestObserver(); + TestObserver to1 = new TestObserver<>(); + TestObserver to2 = new TestObserver<>(); w.subscribe(to1); w.subscribe(to2); @@ -154,7 +154,7 @@ public void withMultipleSubscribersStartingAtSameTime() { public void withMultipleStaggeredSubscribers() { Observable w = Observable.interval(1, TimeUnit.SECONDS, scheduler); - TestObserver to1 = new TestObserver(); + TestObserver to1 = new TestObserver<>(); w.subscribe(to1); @@ -162,7 +162,7 @@ public void withMultipleStaggeredSubscribers() { scheduler.advanceTimeTo(2, TimeUnit.SECONDS); - TestObserver to2 = new TestObserver(); + TestObserver to2 = new TestObserver<>(); w.subscribe(to2); @@ -194,7 +194,7 @@ public void withMultipleStaggeredSubscribers() { public void withMultipleStaggeredSubscribersAndPublish() { ConnectableObservable w = Observable.interval(1, TimeUnit.SECONDS, scheduler).publish(); - TestObserver to1 = new TestObserver(); + TestObserver to1 = new TestObserver<>(); w.subscribe(to1); w.connect(); @@ -203,7 +203,7 @@ public void withMultipleStaggeredSubscribersAndPublish() { scheduler.advanceTimeTo(2, TimeUnit.SECONDS); - TestObserver to2 = new TestObserver(); + TestObserver to2 = new TestObserver<>(); w.subscribe(to2); to1.assertValues(0L, 1L); @@ -348,7 +348,7 @@ public Long apply(Long v) throws Exception { @Test public void cancelledAndRun() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); TimerObserver tm = new TimerObserver(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimestampTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimestampTest.java index 7148a44a4a..597333ea66 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimestampTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimestampTest.java @@ -52,9 +52,9 @@ public void timestampWithScheduler() { InOrder inOrder = inOrder(observer); - inOrder.verify(observer, times(1)).onNext(new Timed(1, 0, TimeUnit.MILLISECONDS)); - inOrder.verify(observer, times(1)).onNext(new Timed(2, 100, TimeUnit.MILLISECONDS)); - inOrder.verify(observer, times(1)).onNext(new Timed(3, 200, TimeUnit.MILLISECONDS)); + inOrder.verify(observer, times(1)).onNext(new Timed<>(1, 0, TimeUnit.MILLISECONDS)); + inOrder.verify(observer, times(1)).onNext(new Timed<>(2, 100, TimeUnit.MILLISECONDS)); + inOrder.verify(observer, times(1)).onNext(new Timed<>(3, 200, TimeUnit.MILLISECONDS)); verify(observer, never()).onError(any(Throwable.class)); verify(observer, never()).onComplete(); @@ -76,9 +76,9 @@ public void timestampWithScheduler2() { InOrder inOrder = inOrder(observer); - inOrder.verify(observer, times(1)).onNext(new Timed(1, 0, TimeUnit.MILLISECONDS)); - inOrder.verify(observer, times(1)).onNext(new Timed(2, 0, TimeUnit.MILLISECONDS)); - inOrder.verify(observer, times(1)).onNext(new Timed(3, 200, TimeUnit.MILLISECONDS)); + inOrder.verify(observer, times(1)).onNext(new Timed<>(1, 0, TimeUnit.MILLISECONDS)); + inOrder.verify(observer, times(1)).onNext(new Timed<>(2, 0, TimeUnit.MILLISECONDS)); + inOrder.verify(observer, times(1)).onNext(new Timed<>(3, 200, TimeUnit.MILLISECONDS)); verify(observer, never()).onError(any(Throwable.class)); verify(observer, never()).onComplete(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToFutureTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToFutureTest.java index 481fd8ae65..164d7f6fd6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToFutureTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToFutureTest.java @@ -37,7 +37,7 @@ public void success() throws Exception { Observer o = TestHelper.mockObserver(); - TestObserver to = new TestObserver(o); + TestObserver to = new TestObserver<>(o); Observable.fromFuture(future).subscribe(to); @@ -59,7 +59,7 @@ public void successOperatesOnSuppliedScheduler() throws Exception { Observer o = TestHelper.mockObserver(); TestScheduler scheduler = new TestScheduler(); - TestObserver to = new TestObserver(o); + TestObserver to = new TestObserver<>(o); Observable.fromFuture(future, scheduler).subscribe(to); @@ -79,7 +79,7 @@ public void failure() throws Exception { Observer o = TestHelper.mockObserver(); - TestObserver to = new TestObserver(o); + TestObserver to = new TestObserver<>(o); Observable.fromFuture(future).subscribe(to); @@ -100,7 +100,7 @@ public void cancelledBeforeSubscribe() throws Exception { Observer o = TestHelper.mockObserver(); - TestObserver to = new TestObserver(o); + TestObserver to = new TestObserver<>(o); to.dispose(); Observable.fromFuture(future).subscribe(to); @@ -146,7 +146,7 @@ public Object get(long timeout, TimeUnit unit) throws InterruptedException, Exec Observer o = TestHelper.mockObserver(); - TestObserver to = new TestObserver(o); + TestObserver to = new TestObserver<>(o); Observable futureObservable = Observable.fromFuture(future); futureObservable.subscribeOn(Schedulers.computation()).subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMapTest.java index 25136fcd9d..8838ff4b5a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMapTest.java @@ -55,7 +55,7 @@ public void toMapObservable() { Observable> mapped = source.toMap(lengthFunc).toObservable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -74,7 +74,7 @@ public void toMapWithValueSelectorObservable() { Observable> mapped = source.toMap(lengthFunc, duplicate).toObservable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -102,7 +102,7 @@ public Integer apply(String t1) { }; Observable> mapped = source.toMap(lengthFuncErr).toObservable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -132,7 +132,7 @@ public String apply(String t1) { Observable> mapped = source.toMap(lengthFunc, duplicateErr).toObservable(); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -178,7 +178,7 @@ public String apply(String v) { } }, mapFactory).toObservable(); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); @@ -214,7 +214,7 @@ public String apply(String v) { } }, mapFactory).toObservable(); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); @@ -232,7 +232,7 @@ public void toMap() { Single> mapped = source.toMap(lengthFunc); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -250,7 +250,7 @@ public void toMapWithValueSelector() { Single> mapped = source.toMap(lengthFunc, duplicate); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -277,7 +277,7 @@ public Integer apply(String t1) { }; Single> mapped = source.toMap(lengthFuncErr); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "a"); expected.put(2, "bb"); expected.put(3, "ccc"); @@ -306,7 +306,7 @@ public String apply(String t1) { Single> mapped = source.toMap(lengthFunc, duplicateErr); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put(1, "aa"); expected.put(2, "bbbb"); expected.put(3, "cccccc"); @@ -351,7 +351,7 @@ public String apply(String v) { } }, mapFactory); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); @@ -386,7 +386,7 @@ public String apply(String v) { } }, mapFactory); - Map expected = new LinkedHashMap(); + Map expected = new LinkedHashMap<>(); expected.put(2, "bb"); expected.put(3, "ccc"); expected.put(4, "dddd"); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMultimapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMultimapTest.java index ef7f0feec4..0853b0acad 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMultimapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToMultimapTest.java @@ -55,7 +55,7 @@ public void toMultimapObservable() { Observable>> mapped = source.toMultimap(lengthFunc).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -72,7 +72,7 @@ public void toMultimapWithValueSelectorObservable() { Observable>> mapped = source.toMultimap(lengthFunc, duplicate).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -114,11 +114,11 @@ public String apply(String v) { mapFactory, new Function>() { @Override public Collection apply(Integer v) { - return new ArrayList(); + return new ArrayList<>(); } }).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -137,9 +137,9 @@ public void toMultimapWithCollectionFactoryObservable() { @Override public Collection apply(Integer t1) { if (t1 == 2) { - return new ArrayList(); + return new ArrayList<>(); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -153,16 +153,16 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Observable>> mapped = source .toMultimap(lengthFunc, identity, mapSupplier, collectionFactory).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, new HashSet(Arrays.asList("eee"))); + expected.put(3, new HashSet<>(Arrays.asList("eee"))); mapped.subscribe(objectObserver); @@ -187,7 +187,7 @@ public Integer apply(String t1) { Observable>> mapped = source.toMultimap(lengthFuncErr).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -214,7 +214,7 @@ public String apply(String t1) { Observable>> mapped = source.toMultimap(lengthFunc, duplicateErr).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -244,7 +244,7 @@ public String apply(String v) { } }, mapFactory).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -265,7 +265,7 @@ public Collection apply(Integer t1) { if (t1 == 2) { throw new RuntimeException("Forced failure"); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -279,14 +279,14 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Observable>> mapped = source.toMultimap(lengthFunc, identity, mapSupplier, collectionFactory).toObservable(); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Collections.singleton("eee")); @@ -303,7 +303,7 @@ public void toMultimap() { Single>> mapped = source.toMultimap(lengthFunc); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -319,7 +319,7 @@ public void toMultimapWithValueSelector() { Single>> mapped = source.toMultimap(lengthFunc, duplicate); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -360,11 +360,11 @@ public String apply(String v) { mapFactory, new Function>() { @Override public Collection apply(Integer v) { - return new ArrayList(); + return new ArrayList<>(); } }); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -382,9 +382,9 @@ public void toMultimapWithCollectionFactory() { @Override public Collection apply(Integer t1) { if (t1 == 2) { - return new ArrayList(); + return new ArrayList<>(); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -398,16 +398,16 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Single>> mapped = source .toMultimap(lengthFunc, identity, mapSupplier, collectionFactory); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); - expected.put(3, new HashSet(Arrays.asList("eee"))); + expected.put(3, new HashSet<>(Arrays.asList("eee"))); mapped.subscribe(singleObserver); @@ -431,7 +431,7 @@ public Integer apply(String t1) { Single>> mapped = source.toMultimap(lengthFuncErr); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("a", "b")); expected.put(2, Arrays.asList("cc", "dd")); @@ -457,7 +457,7 @@ public String apply(String t1) { Single>> mapped = source.toMultimap(lengthFunc, duplicateErr); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(1, Arrays.asList("aa", "bb")); expected.put(2, Arrays.asList("cccc", "dddd")); @@ -486,7 +486,7 @@ public String apply(String v) { } }, mapFactory); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Arrays.asList("eee", "fff")); @@ -506,7 +506,7 @@ public Collection apply(Integer t1) { if (t1 == 2) { throw new RuntimeException("Forced failure"); } else { - return new HashSet(); + return new HashSet<>(); } } }; @@ -520,14 +520,14 @@ public String apply(String v) { Supplier>> mapSupplier = new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }; Single>> mapped = source.toMultimap(lengthFunc, identity, mapSupplier, collectionFactory); - Map> expected = new HashMap>(); + Map> expected = new HashMap<>(); expected.put(2, Arrays.asList("cc", "dd")); expected.put(3, Collections.singleton("eee")); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java index 8a4ac039b7..da58ffdb2a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOnTest.java @@ -37,7 +37,7 @@ public void unsubscribeWhenSubscribeOnAndUnsubscribeOnAreOnSameThread() throws I UIEventLoopScheduler uiEventLoop = new UIEventLoopScheduler(); try { final ThreadSubscription subscription = new ThreadSubscription(); - final AtomicReference subscribeThread = new AtomicReference(); + final AtomicReference subscribeThread = new AtomicReference<>(); Observable w = Observable.unsafeCreate(new ObservableSource() { @Override @@ -53,7 +53,7 @@ public void subscribe(Observer t1) { } }); - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); w.subscribeOn(uiEventLoop).observeOn(Schedulers.computation()) .unsubscribeOn(uiEventLoop) @@ -87,7 +87,7 @@ public void unsubscribeWhenSubscribeOnAndUnsubscribeOnAreOnDifferentThreads() th UIEventLoopScheduler uiEventLoop = new UIEventLoopScheduler(); try { final ThreadSubscription subscription = new ThreadSubscription(); - final AtomicReference subscribeThread = new AtomicReference(); + final AtomicReference subscribeThread = new AtomicReference<>(); Observable w = Observable.unsafeCreate(new ObservableSource() { @Override @@ -103,7 +103,7 @@ public void subscribe(Observer t1) { } }); - TestObserverEx observer = new TestObserverEx(); + TestObserverEx observer = new TestObserverEx<>(); w.subscribeOn(Schedulers.newThread()).observeOn(Schedulers.computation()) .unsubscribeOn(uiEventLoop) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java index 6312ed7d42..143524319d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsingTest.java @@ -232,7 +232,7 @@ public Observable apply(Disposable subscription) { @Test public void usingDisposesEagerlyBeforeCompletion() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); Supplier resourceFactory = createResourceFactory(events); final Action completion = createOnCompletedAction(events); final Action unsub = createUnsubAction(events); @@ -259,7 +259,7 @@ public Observable apply(Resource resource) { @Test public void usingDoesNotDisposesEagerlyBeforeCompletion() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); Supplier resourceFactory = createResourceFactory(events); final Action completion = createOnCompletedAction(events); final Action unsub = createUnsubAction(events); @@ -286,7 +286,7 @@ public Observable apply(Resource resource) { @Test public void usingDisposesEagerlyBeforeError() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); Supplier resourceFactory = createResourceFactory(events); final Consumer onError = createOnErrorAction(events); final Action unsub = createUnsubAction(events); @@ -314,7 +314,7 @@ public Observable apply(Resource resource) { @Test public void usingDoesNotDisposesEagerlyBeforeError() { - final List events = new ArrayList(); + final List events = new ArrayList<>(); final Supplier resourceFactory = createResourceFactory(events); final Consumer onError = createOnErrorAction(events); final Action unsub = createUnsubAction(events); @@ -533,7 +533,7 @@ public ObservableSource apply(Observable o) @Test public void eagerDisposedOnComplete() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.using(Functions.justSupplier(1), Functions.justFunction(new Observable() { @Override @@ -548,7 +548,7 @@ protected void subscribeActual(Observer observer) { @Test public void eagerDisposedOnError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.using(Functions.justSupplier(1), Functions.justFunction(new Observable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java index 63bb37eafa..bfb972626b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithObservableTest.java @@ -44,7 +44,7 @@ public void windowViaObservableNormal1() { final Observer o = TestHelper.mockObserver(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Observer> wo = new DefaultObserver>() { @Override @@ -101,7 +101,7 @@ public void windowViaObservableBoundaryCompletes() { final Observer o = TestHelper.mockObserver(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Observer> wo = new DefaultObserver>() { @Override @@ -157,7 +157,7 @@ public void windowViaObservableBoundaryThrows() { final Observer o = TestHelper.mockObserver(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Observer> wo = new DefaultObserver>() { @Override @@ -207,7 +207,7 @@ public void windowViaObservableSourceThrows() { final Observer o = TestHelper.mockObserver(); - final List> values = new ArrayList>(); + final List> values = new ArrayList<>(); Observer> wo = new DefaultObserver>() { @Override @@ -368,7 +368,7 @@ public ObservableSource apply( public void mainAndBoundaryBothError() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> ref = new AtomicReference<>(); TestObserverEx> to = Observable.error(new TestException("main")) .window(new Observable() { @@ -407,8 +407,8 @@ public void mainCompleteBoundaryErrorRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); TestObserverEx> to = new Observable() { @Override @@ -457,8 +457,8 @@ public void run() { @Test public void mainNextBoundaryNextRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); TestObserver> to = new Observable() { @Override @@ -500,8 +500,8 @@ public void run() { @Test public void takeOneAnotherBoundary() { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); TestObserverEx> to = new Observable() { @Override @@ -532,8 +532,8 @@ protected void subscribeActual(Observer observer) { @Test public void disposeMainBoundaryCompleteRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); final TestObserver> to = new Observable() { @Override @@ -590,8 +590,8 @@ public void disposeMainBoundaryErrorRace() { final TestException ex = new TestException(); for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - final AtomicReference> refMain = new AtomicReference>(); - final AtomicReference> ref = new AtomicReference>(); + final AtomicReference> refMain = new AtomicReference<>(); + final AtomicReference> ref = new AtomicReference<>(); final TestObserver> to = new Observable() { @Override @@ -671,7 +671,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstream() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(Observable.never()) .doOnNext(new Consumer>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java index 274c9712a7..a897430ab7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java @@ -36,7 +36,7 @@ public class ObservableWindowWithSizeTest extends RxJavaTest { private static List> toLists(Observable> observables) { - final List> lists = new ArrayList>(); + final List> lists = new ArrayList<>(); Observable.concatEager(observables.map(new Function, Observable>>() { @Override public Observable> apply(Observable xs) { @@ -106,7 +106,7 @@ public void skipAndCountWindowsWithGaps() { @Test public void windowUnsubscribeNonOverlapping() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final AtomicInteger count = new AtomicInteger(); Observable.merge(Observable.range(1, 10000).doOnNext(new Consumer() { @@ -128,7 +128,7 @@ public void accept(Integer t1) { @Test public void windowUnsubscribeNonOverlappingAsyncSource() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final AtomicInteger count = new AtomicInteger(); Observable.merge(Observable.range(1, 100000) @@ -161,7 +161,7 @@ public void accept(Integer t1) { @Test public void windowUnsubscribeOverlapping() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final AtomicInteger count = new AtomicInteger(); Observable.merge(Observable.range(1, 10000).doOnNext(new Consumer() { @@ -183,7 +183,7 @@ public void accept(Integer t1) { @Test public void windowUnsubscribeOverlappingAsyncSource() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); final AtomicInteger count = new AtomicInteger(); Observable.merge(Observable.range(1, 100000) @@ -208,7 +208,7 @@ public void accept(Integer t1) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -241,7 +241,7 @@ public void subscribe(Observer observer) { @Test public void takeFlatMapCompletes() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final int indicator = 999999999; @@ -405,7 +405,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstreamSize() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(10) .take(1) @@ -459,7 +459,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstreamSkip() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(5, 10) .take(1) @@ -513,7 +513,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstreamOverlap() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(5, 3) .take(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java index d3466c7eb4..0c7dc8056b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java @@ -47,8 +47,8 @@ public void before() { @Test public void observableBasedOpenerAndCloser() { - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); Observable source = Observable.unsafeCreate(new ObservableSource() { @Override @@ -97,7 +97,7 @@ public void subscribe(Observer innerObserver) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -129,7 +129,7 @@ public void accept(Observable stringObservable) { stringObservable.subscribe(new DefaultObserver() { @Override public void onComplete() { - lists.add(new ArrayList(list)); + lists.add(new ArrayList<>(list)); list.clear(); } @@ -154,7 +154,7 @@ public void noUnsubscribeAndNoLeak() { PublishSubject open = PublishSubject.create(); final PublishSubject close = PublishSubject.create(); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); source.window(open, new Function>() { @Override @@ -199,7 +199,7 @@ public void unsubscribeAll() { PublishSubject open = PublishSubject.create(); final PublishSubject close = PublishSubject.create(); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); source.window(open, new Function>() { @Override @@ -464,7 +464,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstream() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(Observable.just(1).concatWith(Observable.never()), Functions.justFunction(Observable.never())) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java index 51591516a6..13f662c88e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithTimeTest.java @@ -46,8 +46,8 @@ public void before() { @Test public void timedAndCount() { - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); Observable source = Observable.unsafeCreate(new ObservableSource() { @Override @@ -82,8 +82,8 @@ public void subscribe(Observer observer) { @Test public void timed() { - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); Observable source = Observable.unsafeCreate(new ObservableSource() { @Override @@ -111,7 +111,7 @@ public void subscribe(Observer observer) { } private List list(String... args) { - List list = new ArrayList(); + List list = new ArrayList<>(); for (String arg : args) { list.add(arg); } @@ -143,7 +143,7 @@ public void accept(Observable stringObservable) { stringObservable.subscribe(new DefaultObserver() { @Override public void onComplete() { - lists.add(new ArrayList(list)); + lists.add(new ArrayList<>(list)); list.clear(); } @@ -166,8 +166,8 @@ public void exactWindowSize() { Observable> source = Observable.range(1, 10) .window(1, TimeUnit.MINUTES, scheduler, 3); - final List list = new ArrayList(); - final List> lists = new ArrayList>(); + final List list = new ArrayList<>(); + final List> lists = new ArrayList<>(); source.subscribe(observeWindow(list, lists)); @@ -184,7 +184,7 @@ public void exactWindowSize() { @Test public void takeFlatMapCompletes() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); final AtomicInteger wip = new AtomicInteger(); @@ -978,7 +978,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstreamExactTime() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(10, TimeUnit.MINUTES) .take(1) @@ -1028,7 +1028,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstreamExactTimeAndSize() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(10, TimeUnit.MINUTES, 100) .take(1) @@ -1078,7 +1078,7 @@ public Observable apply(Observable w) throws Throwable { public void windowAbandonmentCancelsUpstreamExactTimeSkip() { PublishSubject ps = PublishSubject.create(); - final AtomicReference> inner = new AtomicReference>(); + final AtomicReference> inner = new AtomicReference<>(); TestObserver> to = ps.window(10, 15, TimeUnit.MINUTES) .take(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java index 00b4e73f18..a754b21b8c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java @@ -91,7 +91,7 @@ public void emptySource() { Observable result = source.withLatestFrom(other, COMBINER); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); result.subscribe(to); @@ -117,7 +117,7 @@ public void emptyOther() { Observable result = source.withLatestFrom(other, COMBINER); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); result.subscribe(to); @@ -143,7 +143,7 @@ public void unsubscription() { Observable result = source.withLatestFrom(other, COMBINER); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); result.subscribe(to); @@ -170,7 +170,7 @@ public void sourceThrows() { Observable result = source.withLatestFrom(other, COMBINER); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); result.subscribe(to); @@ -198,7 +198,7 @@ public void otherThrows() { Observable result = source.withLatestFrom(other, COMBINER); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); result.subscribe(to); @@ -226,7 +226,7 @@ public void functionThrows() { Observable result = source.withLatestFrom(other, COMBINER_ERROR); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); result.subscribe(to); @@ -252,7 +252,7 @@ public void noDownstreamUnsubscribe() { Observable result = source.withLatestFrom(other, COMBINER); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); result.subscribe(to); @@ -276,7 +276,7 @@ public void manySources() { PublishSubject ps3 = PublishSubject.create(); PublishSubject main = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); main.withLatestFrom(new Observable[] { ps1, ps2, ps3 }, toArray) .subscribe(to); @@ -323,7 +323,7 @@ public void manySourcesIterable() { PublishSubject ps3 = PublishSubject.create(); PublishSubject main = PublishSubject.create(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); main.withLatestFrom(Arrays.>asList(ps1, ps2, ps3), toArray) .subscribe(to); @@ -368,8 +368,8 @@ public void manySourcesIterableSweep() { for (String val : new String[] { "1" /*, null*/ }) { int n = 35; for (int i = 0; i < n; i++) { - List> sources = new ArrayList>(); - List expected = new ArrayList(); + List> sources = new ArrayList<>(); + List expected = new ArrayList<>(); expected.add(val); for (int j = 0; j < i; j++) { @@ -377,7 +377,7 @@ public void manySourcesIterableSweep() { expected.add(String.valueOf(val)); } - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); PublishSubject main = PublishSubject.create(); @@ -397,7 +397,7 @@ public void manySourcesIterableSweep() { @Test public void withEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 3).withLatestFrom( new Observable[] { Observable.just(1), Observable.empty() }, toArray) @@ -410,7 +410,7 @@ public void withEmpty() { @Test public void withError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.range(1, 3).withLatestFrom( new Observable[] { Observable.just(1), Observable.error(new TestException()) }, toArray) @@ -423,7 +423,7 @@ public void withError() { @Test public void withMainError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.error(new TestException()).withLatestFrom( new Observable[] { Observable.just(1), Observable.just(1) }, toArray) @@ -438,7 +438,7 @@ public void withMainError() { public void with2Others() { Observable just = Observable.just(1); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); just.withLatestFrom(just, just, new Function3>() { @Override @@ -457,7 +457,7 @@ public List apply(Integer a, Integer b, Integer c) { public void with3Others() { Observable just = Observable.just(1); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); just.withLatestFrom(just, just, just, new Function4>() { @Override @@ -476,7 +476,7 @@ public List apply(Integer a, Integer b, Integer c, Integer d) { public void with4Others() { Observable just = Observable.just(1); - TestObserver> to = new TestObserver>(); + TestObserver> to = new TestObserver<>(); just.withLatestFrom(just, just, just, just, new Function5>() { @Override @@ -511,7 +511,7 @@ public Object apply(Integer a, Integer b, Integer c) throws Exception { @Test public void manyIteratorThrows() { Observable.just(1) - .withLatestFrom(new CrashingMappedIterable>(1, 100, 100, new Function>() { + .withLatestFrom(new CrashingMappedIterable<>(1, 100, 100, new Function>() { @Override public Observable apply(Integer v) throws Exception { return Observable.just(2); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java index b52b90183d..d8c153a285 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java @@ -350,7 +350,7 @@ public void aggregatorUnsubscribe() { PublishSubject r2 = PublishSubject.create(); /* define an Observer to receive aggregated events */ Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); Observable.zip(r1, r2, zipr2).subscribe(to); @@ -768,7 +768,7 @@ public String apply(Integer a, Integer b) { } }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); os.subscribe(new Consumer() { @Override @@ -795,7 +795,7 @@ public String apply(Integer a, Integer b) { } }).take(5); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); os.subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); @@ -820,7 +820,7 @@ public String apply(Integer a, Integer b) { } }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); os.subscribe(new DefaultObserver() { @Override @@ -884,7 +884,7 @@ public String apply(Notification t1, Notification t2) { }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); o.subscribe(new Consumer() { @Override @@ -913,7 +913,7 @@ public String apply(Integer t1, String t2) { }); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); o.subscribe(new Consumer() { @Override @@ -940,7 +940,7 @@ public Object apply(final Object[] args) { } }); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); o.subscribe(to); to.awaitDone(200, TimeUnit.MILLISECONDS); to.assertNoValues(); @@ -974,7 +974,7 @@ public void downstreamBackpressureRequestsWithFiniteSyncObservables() { Observable o1 = createInfiniteObservable(generatedA).take(Observable.bufferSize() * 2); Observable o2 = createInfiniteObservable(generatedB).take(Observable.bufferSize() * 2); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.zip(o1, o2, new BiFunction() { @Override @@ -1089,7 +1089,7 @@ public Integer apply(Integer i1, Integer i2) { return i1 + i2; } }); - List expected = new ArrayList(); + List expected = new ArrayList<>(); for (int i = 0; i < 1026; i++) { expected.add(i * 3); } @@ -1404,7 +1404,7 @@ public Integer apply(Integer t1, Integer t2) throws Exception { public void firstErrorPreventsSecondSubscription() { final AtomicInteger counter = new AtomicInteger(); - List> observableList = new ArrayList>(); + List> observableList = new ArrayList<>(); observableList.add(Observable.create(new ObservableOnSubscribe() { @Override public void subscribe(ObservableEmitter e) From 75da4b504a52466b455a49a342747514fa0dffab Mon Sep 17 00:00:00 2001 From: Dmitry Volkov <42734936+dvolkovv@users.noreply.github.com> Date: Sun, 22 Dec 2019 14:17:23 +0300 Subject: [PATCH 018/665] Diamond operators in tests (#6788) * diamond operators in unit tests, small packages in internal/ * diamond operators in unit tests, small packages in internal/operators --- .../rxjava3/internal/SubscribeWithTest.java | 4 +- .../disposables/DisposableHelperTest.java | 12 +- .../observers/BasicFuseableObserverTest.java | 4 +- .../observers/BlockingFirstObserverTest.java | 2 +- .../observers/BlockingMultiObserverTest.java | 8 +- .../observers/BlockingObserverTest.java | 4 +- .../observers/ConsumerSingleObserverTest.java | 4 +- .../observers/DeferredScalarObserverTest.java | 36 +++--- .../DisposableLambdaObserverTest.java | 8 +- .../observers/FutureObserverTest.java | 8 +- .../observers/LambdaObserverTest.java | 110 +++++++++--------- .../observers/QueueDrainObserverTest.java | 14 +-- .../completable/CompletableAmbTest.java | 4 +- .../CompletableAndThenCompletableabTest.java | 4 +- .../completable/CompletableCacheTest.java | 12 +- .../completable/CompletableConcatTest.java | 8 +- .../completable/CompletableDelayTest.java | 2 +- .../completable/CompletableDetachTest.java | 6 +- .../CompletableFromCallableTest.java | 2 +- .../CompletableFromSupplierTest.java | 2 +- .../CompletableMergeIterableTest.java | 4 +- .../completable/CompletableMergeTest.java | 10 +- .../completable/CompletableTakeUntilTest.java | 4 +- .../completable/CompletableTimeoutTest.java | 2 +- .../CompletableToObservableTest.java | 2 +- .../operators/maybe/MaybeAmbTest.java | 2 +- .../operators/maybe/MaybeCacheTest.java | 6 +- .../maybe/MaybeCallbackObserverTest.java | 12 +- .../maybe/MaybeConcatIterableTest.java | 6 +- .../operators/maybe/MaybeDetachTest.java | 8 +- .../maybe/MaybeDoAfterSuccessTest.java | 2 +- .../maybe/MaybeFlatMapBiSelectorTest.java | 2 +- .../MaybeFlatMapIterableFlowableTest.java | 4 +- .../MaybeFlatMapIterableObservableTest.java | 4 +- .../operators/maybe/MaybeFromActionTest.java | 2 +- .../maybe/MaybeFromCallableTest.java | 2 +- .../operators/maybe/MaybeFromFutureTest.java | 18 +-- .../maybe/MaybeFromRunnableTest.java | 2 +- .../maybe/MaybeFromSupplierTest.java | 2 +- .../operators/maybe/MaybeMergeArrayTest.java | 4 +- .../operators/maybe/MaybeZipIterableTest.java | 6 +- .../mixed/FlowableConcatMapMaybeTest.java | 6 +- .../mixed/FlowableConcatMapSingleTest.java | 6 +- .../FlowableSwitchMapCompletableTest.java | 2 +- .../mixed/FlowableSwitchMapMaybeTest.java | 6 +- .../mixed/FlowableSwitchMapSingleTest.java | 6 +- .../mixed/ObservableConcatMapMaybeTest.java | 6 +- .../mixed/ObservableConcatMapSingleTest.java | 6 +- .../ObservableSwitchMapCompletableTest.java | 2 +- .../mixed/ObservableSwitchMapMaybeTest.java | 6 +- .../mixed/ObservableSwitchMapSingleTest.java | 6 +- .../operators/single/SingleDelayTest.java | 2 +- .../operators/single/SingleDetachTest.java | 6 +- .../single/SingleDoAfterSuccessTest.java | 2 +- .../single/SingleDoAfterTerminateTest.java | 2 +- .../SingleFlatMapIterableFlowableTest.java | 4 +- .../SingleFlatMapIterableObservableTest.java | 4 +- .../single/SingleFromCallableTest.java | 4 +- .../single/SingleFromSupplierTest.java | 4 +- .../single/SingleZipIterableTest.java | 6 +- .../internal/queue/SimpleQueueTest.java | 20 ++-- .../schedulers/InstantPeriodicTaskTest.java | 12 +- .../schedulers/ScheduledRunnableTest.java | 8 +- .../SchedulerMultiWorkerSupportTest.java | 12 +- ...asicFuseableConditionalSubscriberTest.java | 2 +- .../BasicFuseableSubscriberTest.java | 4 +- .../subscribers/BlockingSubscriberTest.java | 12 +- .../subscribers/BoundedSubscriberTest.java | 36 +++--- .../DeferredScalarSubscriberTest.java | 2 +- .../subscribers/FutureSubscriberTest.java | 8 +- .../InnerQueuedSubscriberTest.java | 4 +- .../subscribers/LambdaSubscriberTest.java | 106 ++++++++--------- .../subscribers/QueueDrainSubscriberTest.java | 32 ++--- .../SinglePostCompleteSubscriberTest.java | 2 +- .../subscribers/StrictSubscriberTest.java | 28 ++--- .../SubscriberResourceWrapperTest.java | 8 +- .../DeferredScalarSubscriptionTest.java | 16 +-- .../subscriptions/ScalarSubscriptionTest.java | 8 +- .../subscriptions/SubscriptionHelperTest.java | 18 +-- .../internal/util/AtomicThrowableTest.java | 36 +++--- .../internal/util/CrashingMappedIterable.java | 2 +- .../internal/util/ExceptionHelperTest.java | 2 +- .../util/HalfSerializerObserverTest.java | 4 +- .../util/HalfSerializerSubscriberTest.java | 4 +- .../internal/util/MergerBiFunctionTest.java | 10 +- .../rxjava3/internal/util/MiscUtilTest.java | 36 +++--- .../internal/util/NotificationLiteTest.java | 2 +- .../util/ObservableToFlowabeTestSync.java | 4 +- .../internal/util/OpenHashSetTest.java | 2 +- .../internal/util/QueueDrainHelperTest.java | 74 ++++++------ .../rxjava3/internal/util/TestingHelper.java | 2 +- .../util/VolatileSizeArrayListTest.java | 8 +- 92 files changed, 484 insertions(+), 484 deletions(-) diff --git a/src/test/java/io/reactivex/rxjava3/internal/SubscribeWithTest.java b/src/test/java/io/reactivex/rxjava3/internal/SubscribeWithTest.java index 15a7c1190d..e1b858507a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/SubscribeWithTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/SubscribeWithTest.java @@ -27,14 +27,14 @@ public class SubscribeWithTest extends RxJavaTest { @Test public void withFlowable() { Flowable.range(1, 10) - .subscribeWith(new TestSubscriber()) + .subscribeWith(new TestSubscriber<>()) .assertResult(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); } @Test public void withObservable() { Observable.range(1, 10) - .subscribeWith(new TestObserver()) + .subscribeWith(new TestObserver<>()) .assertResult(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java index 24c8dd784f..0332b02556 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/disposables/DisposableHelperTest.java @@ -54,7 +54,7 @@ public void validationNull() { @Test public void disposeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference d = new AtomicReference(); + final AtomicReference d = new AtomicReference<>(); Runnable r = new Runnable() { @Override @@ -70,7 +70,7 @@ public void run() { @Test public void setReplace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference d = new AtomicReference(); + final AtomicReference d = new AtomicReference<>(); Runnable r = new Runnable() { @Override @@ -86,7 +86,7 @@ public void run() { @Test public void setRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference d = new AtomicReference(); + final AtomicReference d = new AtomicReference<>(); Runnable r = new Runnable() { @Override @@ -101,7 +101,7 @@ public void run() { @Test public void setReplaceNull() { - final AtomicReference d = new AtomicReference(); + final AtomicReference d = new AtomicReference<>(); DisposableHelper.dispose(d); @@ -112,7 +112,7 @@ public void setReplaceNull() { @Test public void dispose() { Disposable u = Disposable.empty(); - final AtomicReference d = new AtomicReference(u); + final AtomicReference d = new AtomicReference<>(u); DisposableHelper.dispose(d); @@ -121,7 +121,7 @@ public void dispose() { @Test public void trySet() { - AtomicReference ref = new AtomicReference(); + AtomicReference ref = new AtomicReference<>(); Disposable d1 = Disposable.empty(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java index a63283b938..1764ea6a47 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/BasicFuseableObserverTest.java @@ -25,7 +25,7 @@ public class BasicFuseableObserverTest extends RxJavaTest { @Test(expected = UnsupportedOperationException.class) public void offer() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); BasicFuseableObserver o = new BasicFuseableObserver(to) { @Nullable @Override @@ -57,7 +57,7 @@ protected boolean beforeDownstream() { @Test(expected = UnsupportedOperationException.class) public void offer2() { - BasicFuseableObserver o = new BasicFuseableObserver(new TestObserver()) { + BasicFuseableObserver o = new BasicFuseableObserver(new TestObserver<>()) { @Nullable @Override public Integer poll() throws Exception { diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java index 845817379a..0b33e6ddff 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingFirstObserverTest.java @@ -25,7 +25,7 @@ public class BlockingFirstObserverTest extends RxJavaTest { @Test public void firstValueOnly() { - BlockingFirstObserver bf = new BlockingFirstObserver(); + BlockingFirstObserver bf = new BlockingFirstObserver<>(); Disposable d = Disposable.empty(); bf.onSubscribe(d); diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java index 399b5506e3..b9ec5d276b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingMultiObserverTest.java @@ -27,7 +27,7 @@ public class BlockingMultiObserverTest extends RxJavaTest { @Test public void dispose() { - BlockingMultiObserver bmo = new BlockingMultiObserver(); + BlockingMultiObserver bmo = new BlockingMultiObserver<>(); bmo.dispose(); Disposable d = Disposable.empty(); @@ -37,7 +37,7 @@ public void dispose() { @Test public void blockingGetDefault() { - final BlockingMultiObserver bmo = new BlockingMultiObserver(); + final BlockingMultiObserver bmo = new BlockingMultiObserver<>(); Schedulers.single().scheduleDirect(new Runnable() { @Override @@ -51,7 +51,7 @@ public void run() { @Test public void blockingAwait() { - final BlockingMultiObserver bmo = new BlockingMultiObserver(); + final BlockingMultiObserver bmo = new BlockingMultiObserver<>(); Schedulers.single().scheduleDirect(new Runnable() { @Override @@ -65,7 +65,7 @@ public void run() { @Test public void blockingGetDefaultInterrupt() { - final BlockingMultiObserver bmo = new BlockingMultiObserver(); + final BlockingMultiObserver bmo = new BlockingMultiObserver<>(); Thread.currentThread().interrupt(); try { diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingObserverTest.java index f5d8fb6117..1a07bdf907 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/BlockingObserverTest.java @@ -25,9 +25,9 @@ public class BlockingObserverTest extends RxJavaTest { @Test public void dispose() { - Queue q = new ArrayDeque(); + Queue q = new ArrayDeque<>(); - BlockingObserver bo = new BlockingObserver(q); + BlockingObserver bo = new BlockingObserver<>(q); bo.dispose(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/ConsumerSingleObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/ConsumerSingleObserverTest.java index aa2e6e22e3..5e264bc4d7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/ConsumerSingleObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/ConsumerSingleObserverTest.java @@ -24,7 +24,7 @@ public final class ConsumerSingleObserverTest extends RxJavaTest { @Test public void onErrorMissingShouldReportNoCustomOnError() { - ConsumerSingleObserver o = new ConsumerSingleObserver(Functions.emptyConsumer(), + ConsumerSingleObserver o = new ConsumerSingleObserver<>(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING); assertFalse(o.hasCustomOnError()); @@ -32,7 +32,7 @@ public void onErrorMissingShouldReportNoCustomOnError() { @Test public void customOnErrorShouldReportCustomOnError() { - ConsumerSingleObserver o = new ConsumerSingleObserver(Functions.emptyConsumer(), + ConsumerSingleObserver o = new ConsumerSingleObserver<>(Functions.emptyConsumer(), Functions.emptyConsumer()); assertTrue(o.hasCustomOnError()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java index f6ebd39d1f..e9668c81ce 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/DeferredScalarObserverTest.java @@ -50,7 +50,7 @@ public void onNext(Integer value) { public void normal() { List errors = TestHelper.trackPluginErrors(); try { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); TakeFirst source = new TakeFirst(to); @@ -73,7 +73,7 @@ public void normal() { @Test public void error() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); TakeFirst source = new TakeFirst(to); @@ -85,7 +85,7 @@ public void error() { @Test public void complete() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); TakeFirst source = new TakeFirst(to); @@ -97,7 +97,7 @@ public void complete() { @Test public void dispose() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); TakeFirst source = new TakeFirst(to); @@ -118,7 +118,7 @@ public void dispose() { public void fused() { List errors = TestHelper.trackPluginErrors(); try { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); TakeFirst source = new TakeFirst(to); @@ -148,7 +148,7 @@ public void fused() { public void fusedReject() { List errors = TestHelper.trackPluginErrors(); try { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); TakeFirst source = new TakeFirst(to); @@ -193,7 +193,7 @@ public void onNext(Integer value) { public void nonfusedTerminateMore() { List errors = TestHelper.trackPluginErrors(); try { - TestObserverEx to = new TestObserverEx(QueueFuseable.NONE); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.NONE); TakeLast source = new TakeLast(to); @@ -218,7 +218,7 @@ public void nonfusedTerminateMore() { public void nonfusedError() { List errors = TestHelper.trackPluginErrors(); try { - TestObserverEx to = new TestObserverEx(QueueFuseable.NONE); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.NONE); TakeLast source = new TakeLast(to); @@ -243,7 +243,7 @@ public void nonfusedError() { public void fusedTerminateMore() { List errors = TestHelper.trackPluginErrors(); try { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); TakeLast source = new TakeLast(to); @@ -268,7 +268,7 @@ public void fusedTerminateMore() { public void fusedError() { List errors = TestHelper.trackPluginErrors(); try { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); TakeLast source = new TakeLast(to); @@ -291,7 +291,7 @@ public void fusedError() { @Test public void disposed() { - TestObserverEx to = new TestObserverEx(QueueFuseable.NONE); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.NONE); TakeLast source = new TakeLast(to); @@ -309,7 +309,7 @@ public void disposed() { @Test public void disposedAfterOnNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); TakeLast source = new TakeLast(new Observer() { Disposable upstream; @@ -346,7 +346,7 @@ public void onComplete() { @Test public void fusedEmpty() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); TakeLast source = new TakeLast(to); @@ -361,7 +361,7 @@ public void fusedEmpty() { @Test public void nonfusedEmpty() { - TestObserverEx to = new TestObserverEx(QueueFuseable.NONE); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.NONE); TakeLast source = new TakeLast(to); @@ -376,7 +376,7 @@ public void nonfusedEmpty() { @Test public void customFusion() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); TakeLast source = new TakeLast(new Observer() { QueueDisposable d; @@ -426,7 +426,7 @@ public void onComplete() { @Test public void customFusionClear() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); TakeLast source = new TakeLast(new Observer() { QueueDisposable d; @@ -465,7 +465,7 @@ public void onComplete() { @Test public void offerThrow() { - TestObserverEx to = new TestObserverEx(QueueFuseable.NONE); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.NONE); TakeLast source = new TakeLast(to); @@ -474,7 +474,7 @@ public void offerThrow() { @Test public void customFusionDontConsume() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); TakeFirst source = new TakeFirst(new Observer() { QueueDisposable d; diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java index cc84102c30..964301f6e4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/DisposableLambdaObserverTest.java @@ -32,8 +32,8 @@ public class DisposableLambdaObserverTest extends RxJavaTest { @Test public void doubleOnSubscribe() { - TestHelper.doubleOnSubscribe(new DisposableLambdaObserver( - new TestObserver(), Functions.emptyConsumer(), Functions.EMPTY_ACTION + TestHelper.doubleOnSubscribe(new DisposableLambdaObserver<>( + new TestObserver<>(), Functions.emptyConsumer(), Functions.EMPTY_ACTION )); } @@ -41,8 +41,8 @@ public void doubleOnSubscribe() { public void disposeCrash() { List errors = TestHelper.trackPluginErrors(); try { - DisposableLambdaObserver o = new DisposableLambdaObserver( - new TestObserver(), Functions.emptyConsumer(), + DisposableLambdaObserver o = new DisposableLambdaObserver<>( + new TestObserver<>(), Functions.emptyConsumer(), new Action() { @Override public void run() throws Exception { diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java index f9caa33808..a4a8f353c2 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/FutureObserverTest.java @@ -36,7 +36,7 @@ public class FutureObserverTest extends RxJavaTest { @Before public void before() { - fo = new FutureObserver(); + fo = new FutureObserver<>(); } @Test @@ -157,7 +157,7 @@ public void onSubscribe() throws Exception { @Test public void cancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final FutureSubscriber fo = new FutureSubscriber(); + final FutureSubscriber fo = new FutureSubscriber<>(); Runnable r = new Runnable() { @Override @@ -188,7 +188,7 @@ public void onErrorCancelRace() { RxJavaPlugins.setErrorHandler(Functions.emptyConsumer()); try { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final FutureSubscriber fo = new FutureSubscriber(); + final FutureSubscriber fo = new FutureSubscriber<>(); final TestException ex = new TestException(); @@ -218,7 +218,7 @@ public void onCompleteCancelRace() { RxJavaPlugins.setErrorHandler(Functions.emptyConsumer()); try { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final FutureSubscriber fo = new FutureSubscriber(); + final FutureSubscriber fo = new FutureSubscriber<>(); if (i % 3 == 0) { fo.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java index 937e11441d..c19b6ff2db 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/LambdaObserverTest.java @@ -35,20 +35,20 @@ public class LambdaObserverTest extends RxJavaTest { @Test public void onSubscribeThrows() { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaObserver o = new LambdaObserver(new Consumer() { + LambdaObserver o = new LambdaObserver<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -72,20 +72,20 @@ public void accept(Disposable d) throws Exception { @Test public void onNextThrows() { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaObserver o = new LambdaObserver(new Consumer() { + LambdaObserver o = new LambdaObserver<>(new Consumer() { @Override public void accept(Object v) throws Exception { throw new TestException(); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -111,20 +111,20 @@ public void onErrorThrows() { List errors = TestHelper.trackPluginErrors(); try { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaObserver o = new LambdaObserver(new Consumer() { + LambdaObserver o = new LambdaObserver<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - throw new TestException("Inner"); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + throw new TestException("Inner"); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -157,20 +157,20 @@ public void onCompleteThrows() { List errors = TestHelper.trackPluginErrors(); try { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaObserver o = new LambdaObserver(new Consumer() { + LambdaObserver o = new LambdaObserver<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { throw new TestException(); @@ -215,20 +215,20 @@ public void subscribeActual(Observer observer) { } }; - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaObserver o = new LambdaObserver(new Consumer() { + LambdaObserver o = new LambdaObserver<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -266,20 +266,20 @@ public void subscribeActual(Observer observer) { } }; - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaObserver o = new LambdaObserver(new Consumer() { + LambdaObserver o = new LambdaObserver<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -304,7 +304,7 @@ public void accept(Disposable d) throws Exception { public void onNextThrowsCancelsUpstream() { PublishSubject ps = PublishSubject.create(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); ps.subscribe(new Consumer() { @Override @@ -333,9 +333,9 @@ public void accept(Throwable e) throws Exception { public void onSubscribeThrowsCancelsUpstream() { PublishSubject ps = PublishSubject.create(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); - ps.subscribe(new LambdaObserver(new Consumer() { + ps.subscribe(new LambdaObserver<>(new Consumer() { @Override public void accept(Integer v) throws Exception { } @@ -363,7 +363,7 @@ public void accept(Disposable d) throws Exception { @Test public void onErrorMissingShouldReportNoCustomOnError() { - LambdaObserver o = new LambdaObserver(Functions.emptyConsumer(), + LambdaObserver o = new LambdaObserver<>(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION, Functions.emptyConsumer()); @@ -373,7 +373,7 @@ public void onErrorMissingShouldReportNoCustomOnError() { @Test public void customOnErrorShouldReportCustomOnError() { - LambdaObserver o = new LambdaObserver(Functions.emptyConsumer(), + LambdaObserver o = new LambdaObserver<>(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, Functions.emptyConsumer()); @@ -385,9 +385,9 @@ public void customOnErrorShouldReportCustomOnError() { public void disposedObserverShouldReportErrorOnGlobalErrorHandler() { List errors = TestHelper.trackPluginErrors(); try { - final List observerErrors = Collections.synchronizedList(new ArrayList()); + final List observerErrors = Collections.synchronizedList(new ArrayList<>()); - LambdaObserver o = new LambdaObserver(Functions.emptyConsumer(), + LambdaObserver o = new LambdaObserver<>(Functions.emptyConsumer(), new Consumer() { @Override public void accept(Throwable t) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java index c8823563e0..9afd31e95f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/observers/QueueDrainObserverTest.java @@ -24,7 +24,7 @@ public class QueueDrainObserverTest extends RxJavaTest { static final QueueDrainObserver createUnordered(TestObserver to, final Disposable d) { - return new QueueDrainObserver(to, new SpscArrayQueue(4)) { + return new QueueDrainObserver(to, new SpscArrayQueue<>(4)) { @Override public void onNext(Integer t) { fastPathEmit(t, false, d); @@ -51,7 +51,7 @@ public void accept(Observer a, Integer v) { } static final QueueDrainObserver createOrdered(TestObserver to, final Disposable d) { - return new QueueDrainObserver(to, new SpscArrayQueue(4)) { + return new QueueDrainObserver(to, new SpscArrayQueue<>(4)) { @Override public void onNext(Integer t) { fastPathOrderedEmit(t, false, d); @@ -79,7 +79,7 @@ public void accept(Observer a, Integer v) { @Test public void unorderedSlowPath() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Disposable d = Disposable.empty(); QueueDrainObserver qd = createUnordered(to, d); to.onSubscribe(Disposable.empty()); @@ -92,7 +92,7 @@ public void unorderedSlowPath() { @Test public void orderedSlowPath() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Disposable d = Disposable.empty(); QueueDrainObserver qd = createOrdered(to, d); to.onSubscribe(Disposable.empty()); @@ -105,7 +105,7 @@ public void orderedSlowPath() { @Test public void orderedSlowPathNonEmptyQueue() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Disposable d = Disposable.empty(); QueueDrainObserver qd = createOrdered(to, d); to.onSubscribe(Disposable.empty()); @@ -120,7 +120,7 @@ public void orderedSlowPathNonEmptyQueue() { public void unorderedOnNextRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Disposable d = Disposable.empty(); final QueueDrainObserver qd = createUnordered(to, d); to.onSubscribe(Disposable.empty()); @@ -142,7 +142,7 @@ public void run() { public void orderedOnNextRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Disposable d = Disposable.empty(); final QueueDrainObserver qd = createOrdered(to, d); to.onSubscribe(Disposable.empty()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java index 8192f338da..60c69b5f20 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAmbTest.java @@ -38,7 +38,7 @@ public class CompletableAmbTest extends RxJavaTest { @Test public void ambLots() { - List ms = new ArrayList(); + List ms = new ArrayList<>(); for (int i = 0; i < 32; i++) { ms.add(Completable.never()); @@ -172,7 +172,7 @@ public void ambArrayOrder() { @Test public void ambRace() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); CompositeDisposable cd = new CompositeDisposable(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAndThenCompletableabTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAndThenCompletableabTest.java index 1137804fe2..c6b201be51 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAndThenCompletableabTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableAndThenCompletableabTest.java @@ -113,7 +113,7 @@ public void run() { @Test public void andThenFirstCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.fromRunnable(new Runnable() { @Override public void run() { @@ -129,7 +129,7 @@ public void run() { @Test public void andThenSecondCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.complete() .andThen(Completable.fromRunnable(new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCacheTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCacheTest.java index 08c93ae7d6..fd286681c8 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCacheTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCacheTest.java @@ -86,7 +86,7 @@ public void error() { public void crossDispose() { PublishSubject ps = PublishSubject.create(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); final TestObserver to2 = new TestObserver() { @Override @@ -111,7 +111,7 @@ public void onComplete() { public void crossDisposeOnError() { PublishSubject ps = PublishSubject.create(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); final TestObserver to2 = new TestObserver() { @Override @@ -174,9 +174,9 @@ public void subscribeRace() { final Completable c = ps.ignoreElements().cache(); - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -209,7 +209,7 @@ public void subscribeDisposeRace() { final TestObserver to1 = c.test(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -237,7 +237,7 @@ public void run() { public void doubleDispose() { PublishSubject ps = PublishSubject.create(); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); ps.ignoreElements().cache() .subscribe(new CompletableObserver() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java index 3180f0b71a..b41086ce3b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcatTest.java @@ -167,7 +167,7 @@ public void arrayCancelled() { @Test public void arrayFirstCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.concatArray(new Completable() { @Override @@ -191,7 +191,7 @@ public void iterableCancelled() { @Test public void iterableFirstCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.concat(Arrays.asList(new Completable() { @Override @@ -215,7 +215,7 @@ public void arrayCancelRace() { final Completable c = Completable.concatArray(a); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -244,7 +244,7 @@ public void iterableCancelRace() { final Completable c = Completable.concat(Arrays.asList(a)); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDelayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDelayTest.java index 117bdb0bcf..94e34fe728 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDelayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDelayTest.java @@ -41,7 +41,7 @@ public void delayCustomScheduler() { @Test public void onErrorCalledOnScheduler() throws Exception { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference thread = new AtomicReference(); + final AtomicReference thread = new AtomicReference<>(); Completable.error(new Exception()) .delay(0, TimeUnit.MILLISECONDS, Schedulers.newThread()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java index b2dfd16c32..b5b52b67c2 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableDetachTest.java @@ -65,7 +65,7 @@ public void onComplete() { @Test public void cancelDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Completable() { @Override @@ -91,7 +91,7 @@ protected void subscribeActual(CompletableObserver observer) { @Test public void completeDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Completable() { @Override @@ -117,7 +117,7 @@ protected void subscribeActual(CompletableObserver observer) { @Test public void errorDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Completable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallableTest.java index da082c514d..6cf88ce667 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromCallableTest.java @@ -142,7 +142,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromCallableObservable .subscribeOn(Schedulers.computation()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplierTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplierTest.java index c7943de244..73dda447c8 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplierTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSupplierTest.java @@ -144,7 +144,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromSupplierObservable .subscribeOn(Schedulers.computation()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterableTest.java index 67253bceb0..bacfc8521a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeIterableTest.java @@ -67,7 +67,7 @@ public void run() { @Test public void cancelAfterHasNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.merge(new Iterable() { @Override @@ -97,7 +97,7 @@ public void remove() { @Test public void cancelAfterNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.merge(new Iterable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java index c1cc06f3c6..3482a53409 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMergeTest.java @@ -43,7 +43,7 @@ public void invalidPrefetch() { @Test public void cancelAfterFirst() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.mergeArray(new Completable() { @Override @@ -60,7 +60,7 @@ protected void subscribeActual(CompletableObserver observer) { @Test public void cancelAfterFirstDelayError() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.mergeArrayDelayError(new Completable() { @Override @@ -428,7 +428,7 @@ protected void subscribeActual(CompletableObserver observer) { @Test public void innerIsDisposed() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.mergeDelayError(Flowable.just(new Completable() { @Override @@ -494,7 +494,7 @@ public void delayErrorIterableCancel() { @Test public void delayErrorIterableCancelAfterHasNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.mergeDelayError(new Iterable() { @Override @@ -525,7 +525,7 @@ public void remove() { @Test public void delayErrorIterableCancelAfterNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Completable.mergeDelayError(new Iterable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java index 1b4bb17263..3458ef9772 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTakeUntilTest.java @@ -186,7 +186,7 @@ public void otherErrorLate() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference ref = new AtomicReference(); + final AtomicReference ref = new AtomicReference<>(); Completable.complete() .takeUntil(new Completable() { @@ -213,7 +213,7 @@ public void otherCompleteLate() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference ref = new AtomicReference(); + final AtomicReference ref = new AtomicReference<>(); Completable.complete() .takeUntil(new Completable() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java index 0b176d808e..a2f0771ab4 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableTimeoutTest.java @@ -155,7 +155,7 @@ public void run() { @Test public void ambRace() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); CompositeDisposable cd = new CompositeDisposable(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java index 4d5b99cb90..f1bbda0353 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservableTest.java @@ -39,7 +39,7 @@ public Observable apply(Completable c) throws Exception { @Test public void fusion() throws Exception { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); ObserverCompletableObserver co = new ObserverCompletableObserver(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java index 50b76aa424..af49d5353b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java @@ -37,7 +37,7 @@ public class MaybeAmbTest extends RxJavaTest { @Test public void ambLots() { - List> ms = new ArrayList>(); + List> ms = new ArrayList<>(); for (int i = 0; i < 32; i++) { ms.add(Maybe.never()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCacheTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCacheTest.java index 41c6d485eb..a8d42ef603 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCacheTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCacheTest.java @@ -154,7 +154,7 @@ public void onlineComplete() { @Test public void crossCancelOnSuccess() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); PublishProcessor pp = PublishProcessor.create(); @@ -178,7 +178,7 @@ public void accept(Integer v) throws Exception { @Test public void crossCancelOnError() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); PublishProcessor pp = PublishProcessor.create(); @@ -201,7 +201,7 @@ public void accept(Object v) throws Exception { @Test public void crossCancelOnComplete() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); PublishProcessor pp = PublishProcessor.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java index 9ebf939b52..e65b01e070 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCallbackObserverTest.java @@ -31,7 +31,7 @@ public class MaybeCallbackObserverTest extends RxJavaTest { @Test public void dispose() { - MaybeCallbackObserver mo = new MaybeCallbackObserver(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION); + MaybeCallbackObserver mo = new MaybeCallbackObserver<>(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION); Disposable d = Disposable.empty(); @@ -50,7 +50,7 @@ public void dispose() { public void onSuccessCrashes() { List errors = TestHelper.trackPluginErrors(); try { - MaybeCallbackObserver mo = new MaybeCallbackObserver( + MaybeCallbackObserver mo = new MaybeCallbackObserver<>( new Consumer() { @Override public void accept(Object v) throws Exception { @@ -74,7 +74,7 @@ public void accept(Object v) throws Exception { public void onErrorCrashes() { List errors = TestHelper.trackPluginErrors(); try { - MaybeCallbackObserver mo = new MaybeCallbackObserver( + MaybeCallbackObserver mo = new MaybeCallbackObserver<>( Functions.emptyConsumer(), new Consumer() { @Override @@ -103,7 +103,7 @@ public void accept(Object v) throws Exception { public void onCompleteCrashes() { List errors = TestHelper.trackPluginErrors(); try { - MaybeCallbackObserver mo = new MaybeCallbackObserver( + MaybeCallbackObserver mo = new MaybeCallbackObserver<>( Functions.emptyConsumer(), Functions.emptyConsumer(), new Action() { @@ -125,7 +125,7 @@ public void run() throws Exception { @Test public void onErrorMissingShouldReportNoCustomOnError() { - MaybeCallbackObserver o = new MaybeCallbackObserver(Functions.emptyConsumer(), + MaybeCallbackObserver o = new MaybeCallbackObserver<>(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); @@ -134,7 +134,7 @@ public void onErrorMissingShouldReportNoCustomOnError() { @Test public void customOnErrorShouldReportCustomOnError() { - MaybeCallbackObserver o = new MaybeCallbackObserver(Functions.emptyConsumer(), + MaybeCallbackObserver o = new MaybeCallbackObserver<>(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java index 4ceb82f1e6..c693e0296a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java @@ -90,7 +90,7 @@ public void run() { @Test public void hasNextThrows() { - Maybe.concat(new CrashingMappedIterable>(100, 1, 100, new Function>() { + Maybe.concat(new CrashingMappedIterable<>(100, 1, 100, new Function>() { @Override public Maybe apply(Integer v) throws Exception { return Maybe.just(1); @@ -102,7 +102,7 @@ public Maybe apply(Integer v) throws Exception { @Test public void nextThrows() { - Maybe.concat(new CrashingMappedIterable>(100, 100, 1, new Function>() { + Maybe.concat(new CrashingMappedIterable<>(100, 100, 1, new Function>() { @Override public Maybe apply(Integer v) throws Exception { return Maybe.just(1); @@ -114,7 +114,7 @@ public Maybe apply(Integer v) throws Exception { @Test public void nextReturnsNull() { - Maybe.concat(new CrashingMappedIterable>(100, 100, 100, new Function>() { + Maybe.concat(new CrashingMappedIterable<>(100, 100, 100, new Function>() { @Override public Maybe apply(Integer v) throws Exception { return null; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java index ba720a02f9..2c96cbfe88 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetachTest.java @@ -65,7 +65,7 @@ public void onComplete() { @Test public void cancelDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Maybe() { @Override @@ -91,7 +91,7 @@ protected void subscribeActual(MaybeObserver observer) { @Test public void completeDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Maybe() { @Override @@ -117,7 +117,7 @@ protected void subscribeActual(MaybeObserver observer) { @Test public void errorDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Maybe() { @Override @@ -143,7 +143,7 @@ protected void subscribeActual(MaybeObserver observer) { @Test public void successDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Maybe() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccessTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccessTest.java index b754c5150f..d772193241 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccessTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccessTest.java @@ -30,7 +30,7 @@ public class MaybeDoAfterSuccessTest extends RxJavaTest { - final List values = new ArrayList(); + final List values = new ArrayList<>(); final Consumer afterSuccess = new Consumer() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelectorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelectorTest.java index 513eb8eee4..f0db482a62 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelectorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelectorTest.java @@ -215,7 +215,7 @@ public Object apply(Integer a, Integer b) throws Exception { @Test public void mapperCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Maybe.just(1) .flatMap(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowableTest.java index a702ac234d..1c840a2295 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowableTest.java @@ -480,7 +480,7 @@ public void slowPathCancelAfterHasNext() { final Integer[] a = new Integer[1000]; Arrays.fill(a, 1); - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Maybe.just(1) .flattenAsFlowable(new Function>() { @@ -524,7 +524,7 @@ public void fastPathCancelAfterHasNext() { final Integer[] a = new Integer[1000]; Arrays.fill(a, 1); - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Maybe.just(1) .flattenAsFlowable(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservableTest.java index e9d955386a..601da820a7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservableTest.java @@ -99,7 +99,7 @@ public Iterable apply(Integer v) throws Exception { @Test public void fused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Maybe.just(1).flattenAsObservable(new Function>() { @Override @@ -117,7 +117,7 @@ public Iterable apply(Integer v) throws Exception { @Test public void fusedNoSync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); Maybe.just(1).flattenAsObservable(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromActionTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromActionTest.java index c797e81d8d..ccd87877cd 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromActionTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromActionTest.java @@ -170,7 +170,7 @@ public void disposedUpfront() throws Throwable { @Test public void cancelWhileRunning() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Maybe.fromAction(new Action() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallableTest.java index eb551defcd..b934f98fdf 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromCallableTest.java @@ -196,7 +196,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromCallableObservable .subscribeOn(Schedulers.computation()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFutureTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFutureTest.java index 82ed2f9b35..98ea62255e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFutureTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFutureTest.java @@ -29,7 +29,7 @@ public class MaybeFromFutureTest extends RxJavaTest { @Test public void cancelImmediately() { - FutureTask ft = new FutureTask(Functions.justCallable(1)); + FutureTask ft = new FutureTask<>(Functions.justCallable(1)); Maybe.fromFuture(ft).test(true) .assertEmpty(); @@ -37,7 +37,7 @@ public void cancelImmediately() { @Test public void timeout() { - FutureTask ft = new FutureTask(Functions.justCallable(1)); + FutureTask ft = new FutureTask<>(Functions.justCallable(1)); Maybe.fromFuture(ft, 1, TimeUnit.MILLISECONDS).test() .awaitDone(5, TimeUnit.SECONDS) @@ -46,7 +46,7 @@ public void timeout() { @Test public void timedWait() { - FutureTask ft = new FutureTask(Functions.justCallable(1)); + FutureTask ft = new FutureTask<>(Functions.justCallable(1)); ft.run(); Maybe.fromFuture(ft, 1, TimeUnit.MILLISECONDS).test() @@ -56,7 +56,7 @@ public void timedWait() { @Test public void interrupt() { - FutureTask ft = new FutureTask(Functions.justCallable(1)); + FutureTask ft = new FutureTask<>(Functions.justCallable(1)); Thread.currentThread().interrupt(); @@ -66,9 +66,9 @@ public void interrupt() { @Test public void cancelWhileRunning() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); - FutureTask ft = new FutureTask(new Runnable() { + FutureTask ft = new FutureTask<>(new Runnable() { @Override public void run() { to.dispose(); @@ -86,9 +86,9 @@ public void run() { @Test public void cancelAndCrashWhileRunning() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); - FutureTask ft = new FutureTask(new Runnable() { + FutureTask ft = new FutureTask<>(new Runnable() { @Override public void run() { to.dispose(); @@ -107,7 +107,7 @@ public void run() { @Test public void futureNull() { - FutureTask ft = new FutureTask(new Runnable() { + FutureTask ft = new FutureTask<>(new Runnable() { @Override public void run() { } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnableTest.java index 6c59ecca27..b5bc8d9727 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnableTest.java @@ -175,7 +175,7 @@ public void disposedUpfront() { @Test public void cancelWhileRunning() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Maybe.fromRunnable(new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplierTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplierTest.java index 9e90177fae..b8196e3917 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplierTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSupplierTest.java @@ -197,7 +197,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromSupplierObservable .subscribeOn(Schedulers.computation()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java index 2212d50eed..86261efc4f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java @@ -95,7 +95,7 @@ public void onComplete() { @SuppressWarnings("unchecked") @Test public void cancel() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Maybe.mergeArray(Maybe.just(1), Maybe.empty(), Maybe.just(2)) .subscribe(ts); @@ -109,7 +109,7 @@ public void cancel() { @SuppressWarnings("unchecked") @Test public void firstErrors() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Maybe.mergeArray(Maybe.error(new TestException()), Maybe.empty(), Maybe.just(2)) .subscribe(ts); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java index 9ce40cee25..f06be6be57 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java @@ -156,7 +156,7 @@ public void run() { @Test public void iteratorThrows() { - Maybe.zip(new CrashingMappedIterable>(1, 100, 100, new Function>() { + Maybe.zip(new CrashingMappedIterable<>(1, 100, 100, new Function>() { @Override public Maybe apply(Integer v) throws Exception { return Maybe.just(v); @@ -168,7 +168,7 @@ public Maybe apply(Integer v) throws Exception { @Test public void hasNextThrows() { - Maybe.zip(new CrashingMappedIterable>(100, 20, 100, new Function>() { + Maybe.zip(new CrashingMappedIterable<>(100, 20, 100, new Function>() { @Override public Maybe apply(Integer v) throws Exception { return Maybe.just(v); @@ -180,7 +180,7 @@ public Maybe apply(Integer v) throws Exception { @Test public void nextThrows() { - Maybe.zip(new CrashingMappedIterable>(100, 100, 5, new Function>() { + Maybe.zip(new CrashingMappedIterable<>(100, 100, 5, new Function>() { @Override public Maybe apply(Integer v) throws Exception { return Maybe.just(v); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java index 47dd3d1284..8a60f6c367 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybeTest.java @@ -291,7 +291,7 @@ public void innerErrorAfterMainError() { try { final PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> obs = new AtomicReference>(); + final AtomicReference> obs = new AtomicReference<>(); TestSubscriberEx ts = pp.concatMapMaybe( new Function>() { @@ -368,9 +368,9 @@ public MaybeSource apply(Integer v) @Test public void cancelNoConcurrentClean() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ConcatMapMaybeSubscriber operator = - new ConcatMapMaybeSubscriber( + new ConcatMapMaybeSubscriber<>( ts, Functions.justFunction(Maybe.never()), 16, ErrorMode.IMMEDIATE); operator.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java index 5da39ff395..a2e5d73c09 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingleTest.java @@ -209,7 +209,7 @@ public void innerErrorAfterMainError() { try { final PublishProcessor pp = PublishProcessor.create(); - final AtomicReference> obs = new AtomicReference>(); + final AtomicReference> obs = new AtomicReference<>(); TestSubscriberEx ts = pp.concatMapSingle( new Function>() { @@ -286,9 +286,9 @@ public SingleSource apply(Integer v) @Test public void cancelNoConcurrentClean() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ConcatMapSingleSubscriber operator = - new ConcatMapSingleSubscriber( + new ConcatMapSingleSubscriber<>( ts, Functions.justFunction(Single.never()), 16, ErrorMode.IMMEDIATE); operator.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletableTest.java index 5cd60792a7..bf18f90b05 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletableTest.java @@ -174,7 +174,7 @@ public CompletableSource apply(Integer f) throws Exception { @Test public void mapperCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Flowable.range(1, 5).switchMapCompletable(new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java index b7b66c3ca4..8fd4b55d6f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybeTest.java @@ -307,7 +307,7 @@ public MaybeSource apply(Integer v) @Test public void disposeBeforeSwitchInOnNext() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1) .switchMapMaybe(new Function>() { @@ -324,7 +324,7 @@ public MaybeSource apply(Integer v) @Test public void disposeOnNextAfterFirst() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2) .switchMapMaybe(new Function>() { @@ -404,7 +404,7 @@ public MaybeSource apply(Integer v) public void innerErrorAfterTermination() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> moRef = new AtomicReference>(); + final AtomicReference> moRef = new AtomicReference<>(); TestSubscriberEx ts = new Flowable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java index ca1cb6d919..a28413ab42 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingleTest.java @@ -255,7 +255,7 @@ public SingleSource apply(Integer v) @Test public void disposeBeforeSwitchInOnNext() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1) .switchMapSingle(new Function>() { @@ -272,7 +272,7 @@ public SingleSource apply(Integer v) @Test public void disposeOnNextAfterFirst() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2) .switchMapSingle(new Function>() { @@ -352,7 +352,7 @@ public SingleSource apply(Integer v) public void innerErrorAfterTermination() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> moRef = new AtomicReference>(); + final AtomicReference> moRef = new AtomicReference<>(); TestSubscriberEx ts = new Flowable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java index 21b9936292..4764b3e50e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java @@ -262,7 +262,7 @@ public void innerErrorAfterMainError() { try { final PublishSubject ps = PublishSubject.create(); - final AtomicReference> obs = new AtomicReference>(); + final AtomicReference> obs = new AtomicReference<>(); TestObserverEx to = ps.concatMapMaybe( new Function>() { @@ -373,9 +373,9 @@ public void scalarEmptySource() { @Test public void cancelNoConcurrentClean() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); ConcatMapMaybeMainObserver operator = - new ConcatMapMaybeMainObserver( + new ConcatMapMaybeMainObserver<>( to, Functions.justFunction(Maybe.never()), 16, ErrorMode.IMMEDIATE); operator.onSubscribe(Disposable.empty()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java index 8f24b87bbb..41659020e5 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java @@ -180,7 +180,7 @@ public void innerErrorAfterMainError() { try { final PublishSubject ps = PublishSubject.create(); - final AtomicReference> obs = new AtomicReference>(); + final AtomicReference> obs = new AtomicReference<>(); TestObserverEx to = ps.concatMapSingle( new Function>() { @@ -313,9 +313,9 @@ public void scalarEmptySource() { @Test public void cancelNoConcurrentClean() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); ConcatMapSingleMainObserver operator = - new ConcatMapSingleMainObserver( + new ConcatMapSingleMainObserver<>( to, Functions.justFunction(Single.never()), 16, ErrorMode.IMMEDIATE); operator.onSubscribe(Disposable.empty()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java index b060b9d03d..33ce7c27da 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletableTest.java @@ -172,7 +172,7 @@ public CompletableSource apply(Integer f) throws Exception { @Test public void mapperCancels() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.range(1, 5).switchMapCompletable(new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java index 7f877089b5..865544b22a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybeTest.java @@ -283,7 +283,7 @@ public MaybeSource apply(Integer v) @Test public void disposeBeforeSwitchInOnNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1) .switchMapMaybe(new Function>() { @@ -300,7 +300,7 @@ public MaybeSource apply(Integer v) @Test public void disposeOnNextAfterFirst() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1, 2) .switchMapMaybe(new Function>() { @@ -380,7 +380,7 @@ public MaybeSource apply(Integer v) public void innerErrorAfterTermination() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> moRef = new AtomicReference>(); + final AtomicReference> moRef = new AtomicReference<>(); TestObserverEx to = new Observable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java index 957c20ba0e..d5de085a1e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingleTest.java @@ -252,7 +252,7 @@ public SingleSource apply(Integer v) @Test public void disposeBeforeSwitchInOnNext() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1).hide() .switchMapSingle(new Function>() { @@ -269,7 +269,7 @@ public SingleSource apply(Integer v) @Test public void disposeOnNextAfterFirst() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Observable.just(1, 2) .switchMapSingle(new Function>() { @@ -349,7 +349,7 @@ public SingleSource apply(Integer v) public void innerErrorAfterTermination() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> moRef = new AtomicReference>(); + final AtomicReference> moRef = new AtomicReference<>(); TestObserverEx to = new Observable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java index 1eaf426511..555b53e30a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayTest.java @@ -138,7 +138,7 @@ public void delaySubscriptionTimeCustomScheduler() throws Exception { @Test public void onErrorCalledOnScheduler() throws Exception { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference thread = new AtomicReference(); + final AtomicReference thread = new AtomicReference<>(); Single.error(new Exception()) .delay(0, TimeUnit.MILLISECONDS, Schedulers.newThread()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java index d582339cd0..ae13a0d9c0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDetachTest.java @@ -65,7 +65,7 @@ public void onSuccess() { @Test public void cancelDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Single() { @Override @@ -91,7 +91,7 @@ protected void subscribeActual(SingleObserver observer) { @Test public void errorDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Single() { @Override @@ -117,7 +117,7 @@ protected void subscribeActual(SingleObserver observer) { @Test public void successDetaches() throws Exception { Disposable d = Disposable.empty(); - final WeakReference wr = new WeakReference(d); + final WeakReference wr = new WeakReference<>(d); TestObserver to = new Single() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccessTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccessTest.java index 679eac1e64..2090015950 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccessTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccessTest.java @@ -30,7 +30,7 @@ public class SingleDoAfterSuccessTest extends RxJavaTest { - final List values = new ArrayList(); + final List values = new ArrayList<>(); final Consumer afterSuccess = new Consumer() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminateTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminateTest.java index 3096263c88..b4c8c1e7ca 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminateTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminateTest.java @@ -39,7 +39,7 @@ public void run() throws Exception { } }; - private final TestObserver to = new TestObserver(); + private final TestObserver to = new TestObserver<>(); @Test public void just() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowableTest.java index 3fa9a2b462..4643d61303 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowableTest.java @@ -467,7 +467,7 @@ public void slowPathCancelAfterHasNext() { final Integer[] a = new Integer[1000]; Arrays.fill(a, 1); - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Single.just(1) .flattenAsFlowable(new Function>() { @@ -511,7 +511,7 @@ public void fastPathCancelAfterHasNext() { final Integer[] a = new Integer[1000]; Arrays.fill(a, 1); - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); Single.just(1) .flattenAsFlowable(new Function>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservableTest.java index 00fc69fdd9..5d2bafc9c9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservableTest.java @@ -86,7 +86,7 @@ public Iterable apply(Integer v) throws Exception { @Test public void fused() { - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); Single.just(1).flattenAsObservable(new Function>() { @Override @@ -104,7 +104,7 @@ public Iterable apply(Integer v) throws Exception { @Test public void fusedNoSync() { - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); Single.just(1).flattenAsObservable(new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallableTest.java index b7c4ebf331..8a97b8a2d6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromCallableTest.java @@ -169,7 +169,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromCallableObservable .subscribeOn(Schedulers.computation()) @@ -230,7 +230,7 @@ public Object call() throws Exception { @Test public void disposedOnCall() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Single.fromCallable(new Callable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplierTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplierTest.java index 479999d8f3..dd259f209b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplierTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleFromSupplierTest.java @@ -170,7 +170,7 @@ public String answer(InvocationOnMock invocation) throws Throwable { Observer observer = TestHelper.mockObserver(); - TestObserver outer = new TestObserver(observer); + TestObserver outer = new TestObserver<>(observer); fromSupplierObservable .subscribeOn(Schedulers.computation()) @@ -231,7 +231,7 @@ public Object get() throws Exception { @Test public void disposedOnCall() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Single.fromSupplier(new Supplier() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java index 1a94e23257..948972dbcb 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java @@ -156,7 +156,7 @@ public void run() { @Test public void iteratorThrows() { - Single.zip(new CrashingMappedIterable>(1, 100, 100, new Function>() { + Single.zip(new CrashingMappedIterable<>(1, 100, 100, new Function>() { @Override public Single apply(Integer v) throws Exception { return Single.just(v); @@ -168,7 +168,7 @@ public Single apply(Integer v) throws Exception { @Test public void hasNextThrows() { - Single.zip(new CrashingMappedIterable>(100, 20, 100, new Function>() { + Single.zip(new CrashingMappedIterable<>(100, 20, 100, new Function>() { @Override public Single apply(Integer v) throws Exception { return Single.just(v); @@ -180,7 +180,7 @@ public Single apply(Integer v) throws Exception { @Test public void nextThrows() { - Single.zip(new CrashingMappedIterable>(100, 100, 5, new Function>() { + Single.zip(new CrashingMappedIterable<>(100, 100, 5, new Function>() { @Override public Single apply(Integer v) throws Exception { return Single.just(v); diff --git a/src/test/java/io/reactivex/rxjava3/internal/queue/SimpleQueueTest.java b/src/test/java/io/reactivex/rxjava3/internal/queue/SimpleQueueTest.java index d58918d04d..e5673b49bc 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/queue/SimpleQueueTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/queue/SimpleQueueTest.java @@ -30,25 +30,25 @@ public class SimpleQueueTest extends RxJavaTest { @Test(expected = NullPointerException.class) public void spscArrayQueueNull() { - SpscArrayQueue q = new SpscArrayQueue(16); + SpscArrayQueue q = new SpscArrayQueue<>(16); q.offer(null); } @Test(expected = NullPointerException.class) public void spscLinkedArrayQueueNull() { - SpscLinkedArrayQueue q = new SpscLinkedArrayQueue(16); + SpscLinkedArrayQueue q = new SpscLinkedArrayQueue<>(16); q.offer(null); } @Test(expected = NullPointerException.class) public void mpscLinkedQueueNull() { - MpscLinkedQueue q = new MpscLinkedQueue(); + MpscLinkedQueue q = new MpscLinkedQueue<>(); q.offer(null); } @Test public void spscArrayQueueBiOffer() { - SpscArrayQueue q = new SpscArrayQueue(16); + SpscArrayQueue q = new SpscArrayQueue<>(16); q.offer(1, 2); assertEquals(1, q.poll()); @@ -58,7 +58,7 @@ public void spscArrayQueueBiOffer() { @Test public void spscLinkedArrayQueueBiOffer() { - SpscLinkedArrayQueue q = new SpscLinkedArrayQueue(16); + SpscLinkedArrayQueue q = new SpscLinkedArrayQueue<>(16); q.offer(1, 2); assertEquals(1, q.poll()); @@ -68,7 +68,7 @@ public void spscLinkedArrayQueueBiOffer() { @Test public void mpscLinkedQueueBiOffer() { - MpscLinkedQueue q = new MpscLinkedQueue(); + MpscLinkedQueue q = new MpscLinkedQueue<>(); q.offer(1, 2); assertEquals(1, q.poll()); @@ -78,7 +78,7 @@ public void mpscLinkedQueueBiOffer() { @Test public void spscBiOfferCapacity() { - SpscArrayQueue q = new SpscArrayQueue(8); + SpscArrayQueue q = new SpscArrayQueue<>(8); assertTrue(q.offer(1, 2)); assertTrue(q.offer(3, 4)); assertTrue(q.offer(5, 6)); @@ -90,7 +90,7 @@ public void spscBiOfferCapacity() { @Test public void spscLinkedNewBufferPeek() { - SpscLinkedArrayQueue q = new SpscLinkedArrayQueue(8); + SpscLinkedArrayQueue q = new SpscLinkedArrayQueue<>(8); assertTrue(q.offer(1, 2)); assertTrue(q.offer(3, 4)); assertTrue(q.offer(5, 6)); @@ -107,7 +107,7 @@ public void spscLinkedNewBufferPeek() { @Test public void mpscOfferPollRace() throws Exception { - final MpscLinkedQueue q = new MpscLinkedQueue(); + final MpscLinkedQueue q = new MpscLinkedQueue<>(); final AtomicInteger c = new AtomicInteger(3); @@ -160,7 +160,7 @@ public void run() { @Test public void spscLinkedArrayQueueNoNepotism() { - SpscLinkedArrayQueue q = new SpscLinkedArrayQueue(16); + SpscLinkedArrayQueue q = new SpscLinkedArrayQueue<>(16); AtomicReferenceArray ara = q.producerBuffer; diff --git a/src/test/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTaskTest.java b/src/test/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTaskTest.java index 577aa468de..7b7299f309 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTaskTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTaskTest.java @@ -156,12 +156,12 @@ public void run() { task.dispose(); - FutureTask f1 = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f1 = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); task.setFirst(f1); assertTrue(f1.isCancelled()); - FutureTask f2 = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f2 = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); task.setRest(f2); assertTrue(f2.isCancelled()); @@ -187,12 +187,12 @@ public void run() { task.dispose(); - FutureTask f1 = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f1 = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); task.setFirst(f1); assertTrue(f1.isCancelled()); - FutureTask f2 = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f2 = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); task.setRest(f2); assertTrue(f2.isCancelled()); @@ -214,7 +214,7 @@ public void run() { } }, exec); - final FutureTask f1 = new FutureTask(Functions.EMPTY_RUNNABLE, null); + final FutureTask f1 = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); Runnable r1 = new Runnable() { @Override public void run() { @@ -251,7 +251,7 @@ public void run() { } }, exec); - final FutureTask f1 = new FutureTask(Functions.EMPTY_RUNNABLE, null); + final FutureTask f1 = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); Runnable r1 = new Runnable() { @Override public void run() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/schedulers/ScheduledRunnableTest.java b/src/test/java/io/reactivex/rxjava3/internal/schedulers/ScheduledRunnableTest.java index 58305b7e9e..a8920fbd93 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/schedulers/ScheduledRunnableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/schedulers/ScheduledRunnableTest.java @@ -65,7 +65,7 @@ public void setFutureCancelRace() { final ScheduledRunnable run = new ScheduledRunnable(Functions.EMPTY_RUNNABLE, set); set.add(run); - final FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, 0); + final FutureTask ft = new FutureTask<>(Functions.EMPTY_RUNNABLE, 0); Runnable r1 = new Runnable() { @Override @@ -94,7 +94,7 @@ public void setFutureRunRace() { final ScheduledRunnable run = new ScheduledRunnable(Functions.EMPTY_RUNNABLE, set); set.add(run); - final FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, 0); + final FutureTask ft = new FutureTask<>(Functions.EMPTY_RUNNABLE, 0); Runnable r1 = new Runnable() { @Override @@ -266,7 +266,7 @@ public void runFuture() { final ScheduledRunnable run = new ScheduledRunnable(Functions.EMPTY_RUNNABLE, set); set.add(run); - final FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, null); + final FutureTask ft = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); Runnable r1 = new Runnable() { @Override @@ -316,7 +316,7 @@ public void run() { final ScheduledRunnable run = new ScheduledRunnable(r0, set); set.add(run); - final FutureTask ft = new FutureTask(run, null); + final FutureTask ft = new FutureTask<>(run, null); Runnable r2 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerMultiWorkerSupportTest.java b/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerMultiWorkerSupportTest.java index 332a115604..21b4514be1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerMultiWorkerSupportTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/schedulers/SchedulerMultiWorkerSupportTest.java @@ -33,7 +33,7 @@ public class SchedulerMultiWorkerSupportTest extends RxJavaTest { @Test public void moreThanMaxWorkers() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); SchedulerMultiWorkerSupport mws = (SchedulerMultiWorkerSupport)Schedulers.computation(); @@ -49,7 +49,7 @@ public void onWorker(int i, Worker w) { @Test public void getShutdownWorkers() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); ComputationScheduler.NONE.createWorkers(max * 2, new WorkerCallback() { @Override @@ -74,14 +74,14 @@ public void distinctThreads() throws Exception { try { final CountDownLatch cdl = new CountDownLatch(max * 2); - final Set threads1 = Collections.synchronizedSet(new HashSet()); + final Set threads1 = Collections.synchronizedSet(new HashSet<>()); - final Set threads2 = Collections.synchronizedSet(new HashSet()); + final Set threads2 = Collections.synchronizedSet(new HashSet<>()); Runnable parallel1 = new Runnable() { @Override public void run() { - final List list1 = new ArrayList(); + final List list1 = new ArrayList<>(); SchedulerMultiWorkerSupport mws = (SchedulerMultiWorkerSupport)Schedulers.computation(); @@ -110,7 +110,7 @@ public void run() { Runnable parallel2 = new Runnable() { @Override public void run() { - final List list2 = new ArrayList(); + final List list2 = new ArrayList<>(); SchedulerMultiWorkerSupport mws = (SchedulerMultiWorkerSupport)Schedulers.computation(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableConditionalSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableConditionalSubscriberTest.java index 0bcfd2d12a..c84e380701 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableConditionalSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableConditionalSubscriberTest.java @@ -75,7 +75,7 @@ public Integer poll() throws Exception { } }; - fcs.onSubscribe(new ScalarSubscription(fcs, 1)); + fcs.onSubscribe(new ScalarSubscription<>(fcs, 1)); TestHelper.assertNoOffer(fcs); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableSubscriberTest.java index b3543ad7d3..87ccd91b92 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BasicFuseableSubscriberTest.java @@ -27,7 +27,7 @@ public class BasicFuseableSubscriberTest extends RxJavaTest { @Test public void offerThrows() { - BasicFuseableSubscriber fcs = new BasicFuseableSubscriber(new TestSubscriber(0L)) { + BasicFuseableSubscriber fcs = new BasicFuseableSubscriber(new TestSubscriber<>(0L)) { @Override public void onNext(Integer t) { @@ -45,7 +45,7 @@ public Integer poll() throws Exception { } }; - fcs.onSubscribe(new ScalarSubscription(fcs, 1)); + fcs.onSubscribe(new ScalarSubscription<>(fcs, 1)); TestHelper.assertNoOffer(fcs); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BlockingSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BlockingSubscriberTest.java index f127e60c44..563a52a984 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BlockingSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BlockingSubscriberTest.java @@ -29,12 +29,12 @@ public class BlockingSubscriberTest extends RxJavaTest { @Test public void doubleOnSubscribe() { - TestHelper.doubleOnSubscribe(new BlockingSubscriber(new ArrayDeque())); + TestHelper.doubleOnSubscribe(new BlockingSubscriber(new ArrayDeque<>())); } @Test public void cancel() { - BlockingSubscriber bq = new BlockingSubscriber(new ArrayDeque()); + BlockingSubscriber bq = new BlockingSubscriber<>(new ArrayDeque<>()); assertFalse(bq.isCancelled()); @@ -54,7 +54,7 @@ public void blockingFirstDoubleOnSubscribe() { @Test public void blockingFirstTimeout() { - BlockingFirstSubscriber bf = new BlockingFirstSubscriber(); + BlockingFirstSubscriber bf = new BlockingFirstSubscriber<>(); Thread.currentThread().interrupt(); @@ -68,7 +68,7 @@ public void blockingFirstTimeout() { @Test public void blockingFirstTimeout2() { - BlockingFirstSubscriber bf = new BlockingFirstSubscriber(); + BlockingFirstSubscriber bf = new BlockingFirstSubscriber<>(); bf.onSubscribe(new BooleanSubscription()); @@ -85,7 +85,7 @@ public void blockingFirstTimeout2() { @Test public void cancelOnRequest() { - final BlockingFirstSubscriber bf = new BlockingFirstSubscriber(); + final BlockingFirstSubscriber bf = new BlockingFirstSubscriber<>(); final AtomicBoolean b = new AtomicBoolean(); @@ -109,7 +109,7 @@ public void cancel() { @Test public void cancelUpfront() { - final BlockingFirstSubscriber bf = new BlockingFirstSubscriber(); + final BlockingFirstSubscriber bf = new BlockingFirstSubscriber<>(); final AtomicBoolean b = new AtomicBoolean(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BoundedSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BoundedSubscriberTest.java index c2d843fb23..d6b80cc873 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/BoundedSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/BoundedSubscriberTest.java @@ -33,9 +33,9 @@ public class BoundedSubscriberTest extends RxJavaTest { @Test public void onSubscribeThrows() { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - BoundedSubscriber subscriber = new BoundedSubscriber(new Consumer() { + BoundedSubscriber subscriber = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Object o) throws Exception { received.add(o); @@ -69,9 +69,9 @@ public void accept(Subscription subscription) throws Exception { @Test public void onNextThrows() { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - BoundedSubscriber subscriber = new BoundedSubscriber(new Consumer() { + BoundedSubscriber subscriber = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Object o) throws Exception { throw new TestException(); @@ -108,9 +108,9 @@ public void onErrorThrows() { List errors = TestHelper.trackPluginErrors(); try { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - BoundedSubscriber subscriber = new BoundedSubscriber(new Consumer() { + BoundedSubscriber subscriber = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Object o) throws Exception { received.add(o); @@ -154,9 +154,9 @@ public void onCompleteThrows() { List errors = TestHelper.trackPluginErrors(); try { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - BoundedSubscriber subscriber = new BoundedSubscriber(new Consumer() { + BoundedSubscriber subscriber = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Object o) throws Exception { received.add(o); @@ -196,9 +196,9 @@ public void accept(Subscription subscription) throws Exception { public void onNextThrowsCancelsUpstream() { PublishProcessor pp = PublishProcessor.create(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); - BoundedSubscriber s = new BoundedSubscriber(new Consumer() { + BoundedSubscriber s = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Integer v) throws Exception { throw new TestException(); @@ -237,9 +237,9 @@ public void accept(Subscription subscription) throws Exception { public void onSubscribeThrowsCancelsUpstream() { PublishProcessor pp = PublishProcessor.create(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); - BoundedSubscriber s = new BoundedSubscriber(new Consumer() { + BoundedSubscriber s = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Integer v) throws Exception { } @@ -285,9 +285,9 @@ public void subscribe(Subscriber s) { } }); - final List received = new ArrayList(); + final List received = new ArrayList<>(); - BoundedSubscriber subscriber = new BoundedSubscriber(new Consumer() { + BoundedSubscriber subscriber = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); @@ -330,9 +330,9 @@ public void subscribe(Subscriber s) { } }); - final List received = new ArrayList(); + final List received = new ArrayList<>(); - BoundedSubscriber subscriber = new BoundedSubscriber(new Consumer() { + BoundedSubscriber subscriber = new BoundedSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); @@ -361,7 +361,7 @@ public void accept(Subscription s) throws Exception { @Test public void onErrorMissingShouldReportNoCustomOnError() { - BoundedSubscriber subscriber = new BoundedSubscriber(Functions.emptyConsumer(), + BoundedSubscriber subscriber = new BoundedSubscriber<>(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION, Functions.boundedConsumer(128), 128); @@ -371,7 +371,7 @@ public void onErrorMissingShouldReportNoCustomOnError() { @Test public void customOnErrorShouldReportCustomOnError() { - BoundedSubscriber subscriber = new BoundedSubscriber(Functions.emptyConsumer(), + BoundedSubscriber subscriber = new BoundedSubscriber<>(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, Functions.boundedConsumer(128), 128); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/DeferredScalarSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/DeferredScalarSubscriberTest.java index fcc3a2f8ea..698fcc18ac 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/DeferredScalarSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/DeferredScalarSubscriberTest.java @@ -427,7 +427,7 @@ public void downstreamRequest(long n) { @Test public void doubleOnSubscribe() { - TestHelper.doubleOnSubscribe(new DeferredScalarSubscriber(new TestSubscriber()) { + TestHelper.doubleOnSubscribe(new DeferredScalarSubscriber(new TestSubscriber<>()) { private static final long serialVersionUID = -4445381578878059054L; @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriberTest.java index fb75eb46b0..69450f2ad0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriberTest.java @@ -34,7 +34,7 @@ public class FutureSubscriberTest extends RxJavaTest { @Before public void before() { - fs = new FutureSubscriber(); + fs = new FutureSubscriber<>(); } @Test @@ -128,7 +128,7 @@ public void onSubscribe() throws Exception { @Test public void cancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final FutureSubscriber fs = new FutureSubscriber(); + final FutureSubscriber fs = new FutureSubscriber<>(); Runnable r = new Runnable() { @Override @@ -157,7 +157,7 @@ public void run() { @Test public void onErrorCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final FutureSubscriber fs = new FutureSubscriber(); + final FutureSubscriber fs = new FutureSubscriber<>(); final TestException ex = new TestException(); @@ -182,7 +182,7 @@ public void run() { @Test public void onCompleteCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final FutureSubscriber fs = new FutureSubscriber(); + final FutureSubscriber fs = new FutureSubscriber<>(); if (i % 3 == 0) { fs.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/InnerQueuedSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/InnerQueuedSubscriberTest.java index f613bada98..e6533166ce 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/InnerQueuedSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/InnerQueuedSubscriberTest.java @@ -44,9 +44,9 @@ public void drain() { } }; - InnerQueuedSubscriber inner = new InnerQueuedSubscriber(support, 4); + InnerQueuedSubscriber inner = new InnerQueuedSubscriber<>(support, 4); - final List requests = new ArrayList(); + final List requests = new ArrayList<>(); inner.onSubscribe(new Subscription() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/LambdaSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/LambdaSubscriberTest.java index 18cde98c75..17e4fb2fc9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/LambdaSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/LambdaSubscriberTest.java @@ -34,20 +34,20 @@ public class LambdaSubscriberTest extends RxJavaTest { @Test public void onSubscribeThrows() { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaSubscriber subscriber = new LambdaSubscriber(new Consumer() { + LambdaSubscriber subscriber = new LambdaSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -71,20 +71,20 @@ public void accept(Subscription s) throws Exception { @Test public void onNextThrows() { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaSubscriber subscriber = new LambdaSubscriber(new Consumer() { + LambdaSubscriber subscriber = new LambdaSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { throw new TestException(); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -111,20 +111,20 @@ public void onErrorThrows() { List errors = TestHelper.trackPluginErrors(); try { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaSubscriber subscriber = new LambdaSubscriber(new Consumer() { + LambdaSubscriber subscriber = new LambdaSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - throw new TestException("Inner"); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + throw new TestException("Inner"); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -158,20 +158,20 @@ public void onCompleteThrows() { List errors = TestHelper.trackPluginErrors(); try { - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaSubscriber subscriber = new LambdaSubscriber(new Consumer() { + LambdaSubscriber subscriber = new LambdaSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { throw new TestException(); @@ -215,20 +215,20 @@ public void subscribe(Subscriber s) { } }); - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaSubscriber subscriber = new LambdaSubscriber(new Consumer() { + LambdaSubscriber subscriber = new LambdaSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -261,20 +261,20 @@ public void subscribe(Subscriber s) { } }); - final List received = new ArrayList(); + final List received = new ArrayList<>(); - LambdaSubscriber subscriber = new LambdaSubscriber(new Consumer() { + LambdaSubscriber subscriber = new LambdaSubscriber<>(new Consumer() { @Override public void accept(Object v) throws Exception { received.add(v); } }, - new Consumer() { - @Override - public void accept(Throwable e) throws Exception { - received.add(e); - } - }, new Action() { + new Consumer() { + @Override + public void accept(Throwable e) throws Exception { + received.add(e); + } + }, new Action() { @Override public void run() throws Exception { received.add(100); @@ -295,7 +295,7 @@ public void accept(Subscription s) throws Exception { public void onNextThrowsCancelsUpstream() { PublishProcessor pp = PublishProcessor.create(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); pp.subscribe(new Consumer() { @Override @@ -324,9 +324,9 @@ public void accept(Throwable e) throws Exception { public void onSubscribeThrowsCancelsUpstream() { PublishProcessor pp = PublishProcessor.create(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); - pp.subscribe(new LambdaSubscriber(new Consumer() { + pp.subscribe(new LambdaSubscriber<>(new Consumer() { @Override public void accept(Integer v) throws Exception { } @@ -354,7 +354,7 @@ public void accept(Subscription s) throws Exception { @Test public void onErrorMissingShouldReportNoCustomOnError() { - LambdaSubscriber subscriber = new LambdaSubscriber(Functions.emptyConsumer(), + LambdaSubscriber subscriber = new LambdaSubscriber<>(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION, FlowableInternalHelper.RequestMax.INSTANCE); @@ -364,7 +364,7 @@ public void onErrorMissingShouldReportNoCustomOnError() { @Test public void customOnErrorShouldReportCustomOnError() { - LambdaSubscriber subscriber = new LambdaSubscriber(Functions.emptyConsumer(), + LambdaSubscriber subscriber = new LambdaSubscriber<>(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, FlowableInternalHelper.RequestMax.INSTANCE); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java index 41d3178384..6d09484d55 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/QueueDrainSubscriberTest.java @@ -32,7 +32,7 @@ public class QueueDrainSubscriberTest extends RxJavaTest { static final QueueDrainSubscriber createUnordered(TestSubscriber ts, final Disposable d) { - return new QueueDrainSubscriber(ts, new SpscArrayQueue(4)) { + return new QueueDrainSubscriber(ts, new SpscArrayQueue<>(4)) { @Override public void onNext(Integer t) { fastPathEmitMax(t, false, d); @@ -60,7 +60,7 @@ public boolean accept(Subscriber a, Integer v) { } static final QueueDrainSubscriber createOrdered(TestSubscriber ts, final Disposable d) { - return new QueueDrainSubscriber(ts, new SpscArrayQueue(4)) { + return new QueueDrainSubscriber(ts, new SpscArrayQueue<>(4)) { @Override public void onNext(Integer t) { fastPathOrderedEmitMax(t, false, d); @@ -88,7 +88,7 @@ public boolean accept(Subscriber a, Integer v) { } static final QueueDrainSubscriber createUnorderedReject(TestSubscriber ts, final Disposable d) { - return new QueueDrainSubscriber(ts, new SpscArrayQueue(4)) { + return new QueueDrainSubscriber(ts, new SpscArrayQueue<>(4)) { @Override public void onNext(Integer t) { fastPathEmitMax(t, false, d); @@ -116,7 +116,7 @@ public boolean accept(Subscriber a, Integer v) { } static final QueueDrainSubscriber createOrderedReject(TestSubscriber ts, final Disposable d) { - return new QueueDrainSubscriber(ts, new SpscArrayQueue(4)) { + return new QueueDrainSubscriber(ts, new SpscArrayQueue<>(4)) { @Override public void onNext(Integer t) { fastPathOrderedEmitMax(t, false, d); @@ -145,7 +145,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void unorderedFastPathNoRequest() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createUnordered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -159,7 +159,7 @@ public void unorderedFastPathNoRequest() { @Test public void orderedFastPathNoRequest() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createOrdered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -173,7 +173,7 @@ public void orderedFastPathNoRequest() { @Test public void acceptBadRequest() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createUnordered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -191,7 +191,7 @@ public void acceptBadRequest() { @Test public void unorderedFastPathRequest1() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createUnordered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -205,7 +205,7 @@ public void unorderedFastPathRequest1() { @Test public void orderedFastPathRequest1() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createOrdered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -219,7 +219,7 @@ public void orderedFastPathRequest1() { @Test public void unorderedSlowPath() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createUnordered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -232,7 +232,7 @@ public void unorderedSlowPath() { @Test public void orderedSlowPath() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createOrdered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -245,7 +245,7 @@ public void orderedSlowPath() { @Test public void orderedSlowPathNonEmptyQueue() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createOrdered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -261,7 +261,7 @@ public void orderedSlowPathNonEmptyQueue() { public void unorderedOnNextRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); final QueueDrainSubscriber qd = createUnordered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -284,7 +284,7 @@ public void run() { public void orderedOnNextRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); final QueueDrainSubscriber qd = createOrdered(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -305,7 +305,7 @@ public void run() { @Test public void unorderedFastPathReject() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createUnorderedReject(ts, d); ts.onSubscribe(new BooleanSubscription()); @@ -321,7 +321,7 @@ public void unorderedFastPathReject() { @Test public void orderedFastPathReject() { - TestSubscriber ts = new TestSubscriber(1); + TestSubscriber ts = new TestSubscriber<>(1); Disposable d = Disposable.empty(); QueueDrainSubscriber qd = createOrderedReject(ts, d); ts.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/SinglePostCompleteSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/SinglePostCompleteSubscriberTest.java index c60ccf8920..1a9bb1310b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/SinglePostCompleteSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/SinglePostCompleteSubscriberTest.java @@ -25,7 +25,7 @@ public class SinglePostCompleteSubscriberTest extends RxJavaTest { @Test public void requestCompleteRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(0L); + final TestSubscriber ts = new TestSubscriber<>(0L); final SinglePostCompleteSubscriber spc = new SinglePostCompleteSubscriber(ts) { private static final long serialVersionUID = -2848918821531562637L; diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriberTest.java index a94b034166..1b36462c47 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriberTest.java @@ -29,7 +29,7 @@ public class StrictSubscriberTest extends RxJavaTest { @Test public void strictMode() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Subscriber sub = new Subscriber() { @Override @@ -94,8 +94,8 @@ public void onComplete() { @Test public void normalOnNext() { - TestSubscriberEx ts = new TestSubscriberEx(); - SubscriberWrapper wrapper = new SubscriberWrapper(ts); + TestSubscriberEx ts = new TestSubscriberEx<>(); + SubscriberWrapper wrapper = new SubscriberWrapper<>(ts); Flowable.range(1, 5).subscribe(wrapper); @@ -104,8 +104,8 @@ public void normalOnNext() { @Test public void normalOnNextBackpressured() { - TestSubscriberEx ts = new TestSubscriberEx(0); - SubscriberWrapper wrapper = new SubscriberWrapper(ts); + TestSubscriberEx ts = new TestSubscriberEx<>(0); + SubscriberWrapper wrapper = new SubscriberWrapper<>(ts); Flowable.range(1, 5).subscribe(wrapper); @@ -120,8 +120,8 @@ public void normalOnNextBackpressured() { @Test public void normalOnError() { - TestSubscriberEx ts = new TestSubscriberEx(); - SubscriberWrapper wrapper = new SubscriberWrapper(ts); + TestSubscriberEx ts = new TestSubscriberEx<>(); + SubscriberWrapper wrapper = new SubscriberWrapper<>(ts); Flowable.range(1, 5).concatWith(Flowable.error(new TestException())) .subscribe(wrapper); @@ -131,7 +131,7 @@ public void normalOnError() { @Test public void deferredRequest() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Subscriber sub = new Subscriber() { @Override @@ -163,7 +163,7 @@ public void onComplete() { @Test public void requestZero() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Subscriber sub = new Subscriber() { @Override @@ -195,7 +195,7 @@ public void onComplete() { @Test public void requestNegative() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Subscriber sub = new Subscriber() { @Override @@ -227,7 +227,7 @@ public void onComplete() { @Test public void cancelAfterOnComplete() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Subscriber sub = new Subscriber() { Subscription upstream; @@ -269,7 +269,7 @@ protected void subscribeActual(Subscriber s) { @Test public void cancelAfterOnError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Subscriber sub = new Subscriber() { Subscription upstream; @@ -311,8 +311,8 @@ protected void subscribeActual(Subscriber s) { @Test public void doubleOnSubscribe() { - TestSubscriberEx ts = new TestSubscriberEx(); - SubscriberWrapper wrapper = new SubscriberWrapper(ts); + TestSubscriberEx ts = new TestSubscriberEx<>(); + SubscriberWrapper wrapper = new SubscriberWrapper<>(ts); new Flowable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java index dc7c82893d..bafc579b63 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapperTest.java @@ -28,9 +28,9 @@ public class SubscriberResourceWrapperTest extends RxJavaTest { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - SubscriberResourceWrapper s = new SubscriberResourceWrapper(ts); + SubscriberResourceWrapper s = new SubscriberResourceWrapper<>(ts); @Test public void cancel() { @@ -94,7 +94,7 @@ public Flowable apply(Flowable f) throws Exception { @Override public Subscriber apply( Subscriber s) throws Exception { - return new SubscriberResourceWrapper(s); + return new SubscriberResourceWrapper<>(s); } }); } @@ -107,7 +107,7 @@ public void badRequest() { @Override public Subscriber apply( Subscriber s) throws Exception { - return new SubscriberResourceWrapper(s); + return new SubscriberResourceWrapper<>(s); } })); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscriptions/DeferredScalarSubscriptionTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscriptions/DeferredScalarSubscriptionTest.java index 0f42ba85b6..dfe9e83f89 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscriptions/DeferredScalarSubscriptionTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscriptions/DeferredScalarSubscriptionTest.java @@ -26,14 +26,14 @@ public class DeferredScalarSubscriptionTest extends RxJavaTest { @Test public void queueSubscriptionSyncRejected() { - DeferredScalarSubscription ds = new DeferredScalarSubscription(new TestSubscriber()); + DeferredScalarSubscription ds = new DeferredScalarSubscription<>(new TestSubscriber<>()); assertEquals(QueueFuseable.NONE, ds.requestFusion(QueueFuseable.SYNC)); } @Test public void clear() { - DeferredScalarSubscription ds = new DeferredScalarSubscription(new TestSubscriber()); + DeferredScalarSubscription ds = new DeferredScalarSubscription<>(new TestSubscriber<>()); ds.value = 1; @@ -45,7 +45,7 @@ public void clear() { @Test public void cancel() { - DeferredScalarSubscription ds = new DeferredScalarSubscription(new TestSubscriber()); + DeferredScalarSubscription ds = new DeferredScalarSubscription<>(new TestSubscriber<>()); assertTrue(ds.tryCancel()); @@ -55,7 +55,7 @@ public void cancel() { @Test public void completeCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final DeferredScalarSubscription ds = new DeferredScalarSubscription(new TestSubscriber()); + final DeferredScalarSubscription ds = new DeferredScalarSubscription<>(new TestSubscriber<>()); Runnable r1 = new Runnable() { @Override @@ -78,9 +78,9 @@ public void run() { @Test public void requestClearRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); - final DeferredScalarSubscription ds = new DeferredScalarSubscription(ts); + final DeferredScalarSubscription ds = new DeferredScalarSubscription<>(ts); ts.onSubscribe(ds); ds.complete(1); @@ -109,9 +109,9 @@ public void run() { @Test public void requestCancelRace() { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); - final DeferredScalarSubscription ds = new DeferredScalarSubscription(ts); + final DeferredScalarSubscription ds = new DeferredScalarSubscription<>(ts); ts.onSubscribe(ds); ds.complete(1); diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscriptions/ScalarSubscriptionTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscriptions/ScalarSubscriptionTest.java index 275b15f73c..55a564280c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscriptions/ScalarSubscriptionTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscriptions/ScalarSubscriptionTest.java @@ -26,9 +26,9 @@ public class ScalarSubscriptionTest extends RxJavaTest { @Test public void badRequest() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); - ScalarSubscription sc = new ScalarSubscription(ts, 1); + ScalarSubscription sc = new ScalarSubscription<>(ts, 1); List errors = TestHelper.trackPluginErrors(); try { @@ -42,9 +42,9 @@ public void badRequest() { @Test public void noOffer() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); - ScalarSubscription sc = new ScalarSubscription(ts, 1); + ScalarSubscription sc = new ScalarSubscription<>(ts, 1); TestHelper.assertNoOffer(sc); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelperTest.java index a28a2dd4e5..13a2dc61aa 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionHelperTest.java @@ -54,7 +54,7 @@ public void cancelNoOp() { @Test public void set() { - AtomicReference atomicSubscription = new AtomicReference(); + AtomicReference atomicSubscription = new AtomicReference<>(); BooleanSubscription bs1 = new BooleanSubscription(); @@ -71,7 +71,7 @@ public void set() { @Test public void replace() { - AtomicReference atomicSubscription = new AtomicReference(); + AtomicReference atomicSubscription = new AtomicReference<>(); BooleanSubscription bs1 = new BooleanSubscription(); @@ -89,7 +89,7 @@ public void replace() { @Test public void cancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference atomicSubscription = new AtomicReference(); + final AtomicReference atomicSubscription = new AtomicReference<>(); Runnable r = new Runnable() { @Override @@ -105,7 +105,7 @@ public void run() { @Test public void setRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference atomicSubscription = new AtomicReference(); + final AtomicReference atomicSubscription = new AtomicReference<>(); final BooleanSubscription bs1 = new BooleanSubscription(); final BooleanSubscription bs2 = new BooleanSubscription(); @@ -133,7 +133,7 @@ public void run() { @Test public void replaceRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference atomicSubscription = new AtomicReference(); + final AtomicReference atomicSubscription = new AtomicReference<>(); final BooleanSubscription bs1 = new BooleanSubscription(); final BooleanSubscription bs2 = new BooleanSubscription(); @@ -161,7 +161,7 @@ public void run() { @Test public void cancelAndChange() { - AtomicReference atomicSubscription = new AtomicReference(); + AtomicReference atomicSubscription = new AtomicReference<>(); SubscriptionHelper.cancel(atomicSubscription); @@ -180,7 +180,7 @@ public void cancelAndChange() { @Test public void invalidDeferredRequest() { - AtomicReference atomicSubscription = new AtomicReference(); + AtomicReference atomicSubscription = new AtomicReference<>(); AtomicLong r = new AtomicLong(); List errors = TestHelper.trackPluginErrors(); @@ -196,7 +196,7 @@ public void invalidDeferredRequest() { @Test public void deferredRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference atomicSubscription = new AtomicReference(); + final AtomicReference atomicSubscription = new AtomicReference<>(); final AtomicLong r = new AtomicLong(); final AtomicLong q = new AtomicLong(); @@ -237,7 +237,7 @@ public void run() { @Test public void setOnceAndRequest() { - AtomicReference ref = new AtomicReference(); + AtomicReference ref = new AtomicReference<>(); Subscription sub = mock(Subscription.class); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java index 9b033bffdd..efe3b5c7b2 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/AtomicThrowableTest.java @@ -91,7 +91,7 @@ public void tryTerminateAndReportHasError() { @Test public void tryTerminateConsumerSubscriberNoError() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); AtomicThrowable ex = new AtomicThrowable(); @@ -101,7 +101,7 @@ public void tryTerminateConsumerSubscriberNoError() { @Test public void tryTerminateConsumerSubscriberError() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); AtomicThrowable ex = new AtomicThrowable(); @@ -112,7 +112,7 @@ public void tryTerminateConsumerSubscriberError() { @Test public void tryTerminateConsumerSubscriberTerminated() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); AtomicThrowable ex = new AtomicThrowable(); @@ -123,7 +123,7 @@ public void tryTerminateConsumerSubscriberTerminated() { @Test public void tryTerminateConsumerObserverNoError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -133,7 +133,7 @@ public void tryTerminateConsumerObserverNoError() { @Test public void tryTerminateConsumerObserverError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -144,7 +144,7 @@ public void tryTerminateConsumerObserverError() { @Test public void tryTerminateConsumerObserverTerminated() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -155,7 +155,7 @@ public void tryTerminateConsumerObserverTerminated() { @Test public void tryTerminateConsumerMaybeObserverNoError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -165,7 +165,7 @@ public void tryTerminateConsumerMaybeObserverNoError() { @Test public void tryTerminateConsumerMaybeObserverError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -176,7 +176,7 @@ public void tryTerminateConsumerMaybeObserverError() { @Test public void tryTerminateConsumerMaybeObserverTerminated() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -187,7 +187,7 @@ public void tryTerminateConsumerMaybeObserverTerminated() { @Test public void tryTerminateConsumerSingleNoError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -197,7 +197,7 @@ public void tryTerminateConsumerSingleNoError() { @Test public void tryTerminateConsumerSingleError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -208,7 +208,7 @@ public void tryTerminateConsumerSingleError() { @Test public void tryTerminateConsumerSingleTerminated() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -219,7 +219,7 @@ public void tryTerminateConsumerSingleTerminated() { @Test public void tryTerminateConsumerCompletableObserverNoError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -229,7 +229,7 @@ public void tryTerminateConsumerCompletableObserverNoError() { @Test public void tryTerminateConsumerCompletableObserverError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -240,7 +240,7 @@ public void tryTerminateConsumerCompletableObserverError() { @Test public void tryTerminateConsumerCompletableObserverTerminated() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -270,7 +270,7 @@ public void onComplete() { @Test public void tryTerminateConsumerEmitterNoError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -280,7 +280,7 @@ public void tryTerminateConsumerEmitterNoError() { @Test public void tryTerminateConsumerEmitterError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); @@ -291,7 +291,7 @@ public void tryTerminateConsumerEmitterError() { @Test public void tryTerminateConsumerEmitterTerminated() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); AtomicThrowable ex = new AtomicThrowable(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/CrashingMappedIterable.java b/src/test/java/io/reactivex/rxjava3/internal/util/CrashingMappedIterable.java index 53089322ea..caf09b372e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/CrashingMappedIterable.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/CrashingMappedIterable.java @@ -45,7 +45,7 @@ public Iterator iterator() { if (--crashOnIterator <= 0) { throw new TestException("iterator()"); } - return new CrashingMapperIterator(crashOnHasNext, crashOnNext, mapper); + return new CrashingMapperIterator<>(crashOnHasNext, crashOnNext, mapper); } static final class CrashingMapperIterator implements Iterator { diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/ExceptionHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/ExceptionHelperTest.java index 666718a1f5..70b70aea1f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/ExceptionHelperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/ExceptionHelperTest.java @@ -33,7 +33,7 @@ public void utilityClass() { public void addRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); final TestException ex = new TestException(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java index 050e58dc64..cbb66e6425 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerObserverTest.java @@ -208,7 +208,7 @@ public void onNextOnCompleteRace() { final AtomicInteger wip = new AtomicInteger(); final AtomicThrowable error = new AtomicThrowable(); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); Runnable r1 = new Runnable() { @@ -240,7 +240,7 @@ public void onErrorOnCompleteRace() { final AtomicInteger wip = new AtomicInteger(); final AtomicThrowable error = new AtomicThrowable(); - final TestObserverEx to = new TestObserverEx(); + final TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerSubscriberTest.java index d7c472e660..44eea4907c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/HalfSerializerSubscriberTest.java @@ -214,7 +214,7 @@ public void onNextOnCompleteRace() { final AtomicInteger wip = new AtomicInteger(); final AtomicThrowable error = new AtomicThrowable(); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); Runnable r1 = new Runnable() { @@ -246,7 +246,7 @@ public void onErrorOnCompleteRace() { final AtomicInteger wip = new AtomicInteger(); final AtomicThrowable error = new AtomicThrowable(); - final TestSubscriberEx ts = new TestSubscriberEx(); + final TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/MergerBiFunctionTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/MergerBiFunctionTest.java index c8a0968729..dce15c4b28 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/MergerBiFunctionTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/MergerBiFunctionTest.java @@ -25,7 +25,7 @@ public class MergerBiFunctionTest extends RxJavaTest { @Test public void firstEmpty() throws Exception { - MergerBiFunction merger = new MergerBiFunction(new Comparator() { + MergerBiFunction merger = new MergerBiFunction<>(new Comparator() { @Override public int compare(Integer o1, Integer o2) { return o1.compareTo(o2); @@ -38,7 +38,7 @@ public int compare(Integer o1, Integer o2) { @Test public void bothEmpty() throws Exception { - MergerBiFunction merger = new MergerBiFunction(new Comparator() { + MergerBiFunction merger = new MergerBiFunction<>(new Comparator() { @Override public int compare(Integer o1, Integer o2) { return o1.compareTo(o2); @@ -51,7 +51,7 @@ public int compare(Integer o1, Integer o2) { @Test public void secondEmpty() throws Exception { - MergerBiFunction merger = new MergerBiFunction(new Comparator() { + MergerBiFunction merger = new MergerBiFunction<>(new Comparator() { @Override public int compare(Integer o1, Integer o2) { return o1.compareTo(o2); @@ -64,7 +64,7 @@ public int compare(Integer o1, Integer o2) { @Test public void sameSize() throws Exception { - MergerBiFunction merger = new MergerBiFunction(new Comparator() { + MergerBiFunction merger = new MergerBiFunction<>(new Comparator() { @Override public int compare(Integer o1, Integer o2) { return o1.compareTo(o2); @@ -77,7 +77,7 @@ public int compare(Integer o1, Integer o2) { @Test public void sameSizeReverse() throws Exception { - MergerBiFunction merger = new MergerBiFunction(new Comparator() { + MergerBiFunction merger = new MergerBiFunction<>(new Comparator() { @Override public int compare(Integer o1, Integer o2) { return o1.compareTo(o2); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/MiscUtilTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/MiscUtilTest.java index d0b627afaa..b27279eb43 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/MiscUtilTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/MiscUtilTest.java @@ -68,13 +68,13 @@ public void linkedArrayList() { @Test public void appendOnlyLinkedArrayListForEachWhile() throws Exception { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(2); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(2); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(new NonThrowingPredicate() { @Override @@ -89,13 +89,13 @@ public boolean test(Integer t2) { @Test public void appendOnlyLinkedArrayListForEachWhileBi() throws Throwable { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(2); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(2); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(2, new BiPredicate() { @Override @@ -110,13 +110,13 @@ public boolean test(Integer t1, Integer t2) throws Throwable { @Test public void appendOnlyLinkedArrayListForEachWhilePreGrow() throws Exception { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(12); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(12); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(new NonThrowingPredicate() { @Override @@ -131,13 +131,13 @@ public boolean test(Integer t2) { @Test public void appendOnlyLinkedArrayListForEachWhileExact() throws Exception { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(3); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(3); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(new NonThrowingPredicate() { @Override @@ -152,13 +152,13 @@ public boolean test(Integer t2) { @Test public void appendOnlyLinkedArrayListForEachWhileAll() throws Exception { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(2); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(2); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(new NonThrowingPredicate() { @Override @@ -173,13 +173,13 @@ public boolean test(Integer t2) { @Test public void appendOnlyLinkedArrayListForEachWhileBigger() throws Exception { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(4); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(4); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(new NonThrowingPredicate() { @Override @@ -194,13 +194,13 @@ public boolean test(Integer t2) { @Test public void appendOnlyLinkedArrayListForEachWhileBiPreGrow() throws Throwable { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(12); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(12); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(2, new BiPredicate() { @Override @@ -215,13 +215,13 @@ public boolean test(Integer t1, Integer t2) throws Throwable { @Test public void appendOnlyLinkedArrayListForEachWhileBiExact() throws Throwable { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(3); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(3); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(2, new BiPredicate() { @Override @@ -236,13 +236,13 @@ public boolean test(Integer t1, Integer t2) throws Exception { @Test public void appendOnlyLinkedArrayListForEachWhileBiAll() throws Throwable { - AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList(2); + AppendOnlyLinkedArrayList list = new AppendOnlyLinkedArrayList<>(2); list.add(1); list.add(2); list.add(3); - final List out = new ArrayList(); + final List out = new ArrayList<>(); list.forEachWhile(3, new BiPredicate() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java index 6fafc2d5db..7e701ff5b9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/NotificationLiteTest.java @@ -26,7 +26,7 @@ public class NotificationLiteTest extends RxJavaTest { @Test public void acceptFullObserver() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Disposable d = Disposable.empty(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/ObservableToFlowabeTestSync.java b/src/test/java/io/reactivex/rxjava3/internal/util/ObservableToFlowabeTestSync.java index 02b64e2772..fbe13a185e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/ObservableToFlowabeTestSync.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/ObservableToFlowabeTestSync.java @@ -26,7 +26,7 @@ private ObservableToFlowabeTestSync() { } static List readAllLines(File f) { - List result = new ArrayList(); + List result = new ArrayList<>(); try { BufferedReader in = new BufferedReader(new FileReader(f)); try { @@ -65,7 +65,7 @@ static void list(String basepath, String basepackage) throws Exception { Class clazz2 = Class.forName(basepackage + "flowable." + cn); - Set methods2 = new HashSet(); + Set methods2 = new HashSet<>(); for (Method m : clazz2.getMethods()) { methods2.add(m.getName()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/OpenHashSetTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/OpenHashSetTest.java index e424298e46..c87bd13192 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/OpenHashSetTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/OpenHashSetTest.java @@ -39,7 +39,7 @@ public void addRemoveCollision() { Value v1 = new Value(); Value v2 = new Value(); - OpenHashSet set = new OpenHashSet(); + OpenHashSet set = new OpenHashSet<>(); assertTrue(set.add(v1)); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java index f2ace8d235..5fac86182a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/QueueDrainHelperTest.java @@ -89,8 +89,8 @@ public void cancel() { @Test public void postCompleteEmpty() { - TestSubscriber ts = new TestSubscriber(); - ArrayDeque queue = new ArrayDeque(); + TestSubscriber ts = new TestSubscriber<>(); + ArrayDeque queue = new ArrayDeque<>(); AtomicLong state = new AtomicLong(); BooleanSupplier isCancelled = new BooleanSupplier() { @Override @@ -108,8 +108,8 @@ public boolean getAsBoolean() throws Exception { @Test public void postCompleteWithRequest() { - TestSubscriber ts = new TestSubscriber(); - ArrayDeque queue = new ArrayDeque(); + TestSubscriber ts = new TestSubscriber<>(); + ArrayDeque queue = new ArrayDeque<>(); AtomicLong state = new AtomicLong(); BooleanSupplier isCancelled = new BooleanSupplier() { @Override @@ -130,8 +130,8 @@ public boolean getAsBoolean() throws Exception { @Test public void completeRequestRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(); - final ArrayDeque queue = new ArrayDeque(); + final TestSubscriber ts = new TestSubscriber<>(); + final ArrayDeque queue = new ArrayDeque<>(); final AtomicLong state = new AtomicLong(); final BooleanSupplier isCancelled = new BooleanSupplier() { @Override @@ -165,8 +165,8 @@ public void run() { @Test public void postCompleteCancelled() { - final TestSubscriber ts = new TestSubscriber(); - ArrayDeque queue = new ArrayDeque(); + final TestSubscriber ts = new TestSubscriber<>(); + ArrayDeque queue = new ArrayDeque<>(); AtomicLong state = new AtomicLong(); BooleanSupplier isCancelled = new BooleanSupplier() { @Override @@ -194,7 +194,7 @@ public void onNext(Integer t) { cancel(); } }; - ArrayDeque queue = new ArrayDeque(); + ArrayDeque queue = new ArrayDeque<>(); AtomicLong state = new AtomicLong(); BooleanSupplier isCancelled = new BooleanSupplier() { @Override @@ -214,7 +214,7 @@ public boolean getAsBoolean() throws Exception { @Test public void drainMaxLoopMissingBackpressure() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); QueueDrain qd = new QueueDrain() { @@ -259,7 +259,7 @@ public boolean accept(Subscriber a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); q.offer(1); QueueDrainHelper.drainMaxLoop(q, ts, false, null, qd); @@ -269,7 +269,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void drainMaxLoopMissingBackpressureWithResource() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); QueueDrain qd = new QueueDrain() { @@ -314,7 +314,7 @@ public boolean accept(Subscriber a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); q.offer(1); Disposable d = Disposable.empty(); @@ -328,7 +328,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void drainMaxLoopDontAccept() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); QueueDrain qd = new QueueDrain() { @@ -373,7 +373,7 @@ public boolean accept(Subscriber a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); q.offer(1); QueueDrainHelper.drainMaxLoop(q, ts, false, null, qd); @@ -383,7 +383,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void checkTerminatedDelayErrorEmpty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); QueueDrain qd = new QueueDrain() { @@ -428,7 +428,7 @@ public boolean accept(Subscriber a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, true, ts, true, q, qd); @@ -437,7 +437,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void checkTerminatedDelayErrorNonEmpty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); QueueDrain qd = new QueueDrain() { @@ -482,7 +482,7 @@ public boolean accept(Subscriber a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, false, ts, true, q, qd); @@ -491,7 +491,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void checkTerminatedDelayErrorEmptyError() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); QueueDrain qd = new QueueDrain() { @@ -536,7 +536,7 @@ public boolean accept(Subscriber a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, true, ts, true, q, qd); @@ -545,7 +545,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void checkTerminatedNonDelayErrorError() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); QueueDrain qd = new QueueDrain() { @@ -590,7 +590,7 @@ public boolean accept(Subscriber a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, false, ts, false, q, qd); @@ -599,7 +599,7 @@ public boolean accept(Subscriber a, Integer v) { @Test public void observerCheckTerminatedDelayErrorEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); ObservableQueueDrain qd = new ObservableQueueDrain() { @@ -633,7 +633,7 @@ public void accept(Observer a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, true, to, true, q, null, qd); @@ -642,7 +642,7 @@ public void accept(Observer a, Integer v) { @Test public void observerCheckTerminatedDelayErrorEmptyResource() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); ObservableQueueDrain qd = new ObservableQueueDrain() { @@ -676,7 +676,7 @@ public void accept(Observer a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); Disposable d = Disposable.empty(); @@ -689,7 +689,7 @@ public void accept(Observer a, Integer v) { @Test public void observerCheckTerminatedDelayErrorNonEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); ObservableQueueDrain qd = new ObservableQueueDrain() { @@ -723,7 +723,7 @@ public void accept(Observer a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, false, to, true, q, null, qd); @@ -732,7 +732,7 @@ public void accept(Observer a, Integer v) { @Test public void observerCheckTerminatedDelayErrorEmptyError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); ObservableQueueDrain qd = new ObservableQueueDrain() { @@ -766,7 +766,7 @@ public void accept(Observer a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, true, to, true, q, null, qd); @@ -775,7 +775,7 @@ public void accept(Observer a, Integer v) { @Test public void observerCheckTerminatedNonDelayErrorError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); ObservableQueueDrain qd = new ObservableQueueDrain() { @@ -809,7 +809,7 @@ public void accept(Observer a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); QueueDrainHelper.checkTerminated(true, false, to, false, q, null, qd); @@ -818,7 +818,7 @@ public void accept(Observer a, Integer v) { @Test public void observerCheckTerminatedNonDelayErrorErrorResource() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); ObservableQueueDrain qd = new ObservableQueueDrain() { @@ -852,7 +852,7 @@ public void accept(Observer a, Integer v) { } }; - SpscArrayQueue q = new SpscArrayQueue(32); + SpscArrayQueue q = new SpscArrayQueue<>(32); Disposable d = Disposable.empty(); @@ -866,9 +866,9 @@ public void accept(Observer a, Integer v) { @Test public void postCompleteAlreadyComplete() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - Queue q = new ArrayDeque(); + Queue q = new ArrayDeque<>(); q.offer(1); AtomicLong state = new AtomicLong(QueueDrainHelper.COMPLETED_MASK); diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/TestingHelper.java b/src/test/java/io/reactivex/rxjava3/internal/util/TestingHelper.java index 867f518d60..976757acd7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/TestingHelper.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/TestingHelper.java @@ -40,7 +40,7 @@ public static Supplier> supplierListCreator() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }; } diff --git a/src/test/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayListTest.java b/src/test/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayListTest.java index e087542ff3..c59537c988 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayListTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayListTest.java @@ -25,14 +25,14 @@ public class VolatileSizeArrayListTest extends RxJavaTest { @Test public void normal() { - List list = new VolatileSizeArrayList(); + List list = new VolatileSizeArrayList<>(); assertTrue(list.isEmpty()); assertEquals(0, list.size()); assertFalse(list.contains(1)); assertFalse(list.remove((Integer)1)); - list = new VolatileSizeArrayList(16); + list = new VolatileSizeArrayList<>(16); assertTrue(list.add(1)); assertTrue(list.addAll(Arrays.asList(3, 4, 7))); list.add(1, 2); @@ -81,7 +81,7 @@ public void normal() { assertEquals(Arrays.asList(3, 4, 5), list.subList(2, 5)); - VolatileSizeArrayList list2 = new VolatileSizeArrayList(); + VolatileSizeArrayList list2 = new VolatileSizeArrayList<>(); list2.addAll(Arrays.asList(1, 2, 3, 4, 5, 6)); assertNotEquals(list2, list); @@ -91,7 +91,7 @@ public void normal() { assertEquals(list2, list); assertEquals(list, list2); - List list3 = new ArrayList(); + List list3 = new ArrayList<>(); list3.addAll(Arrays.asList(1, 2, 3, 4, 5, 6)); assertNotEquals(list3, list); From 0bd2a0152b64723ac3daf3e2fbad3d59c7f3bbb8 Mon Sep 17 00:00:00 2001 From: Dmitry Volkov <42734936+dvolkovv@users.noreply.github.com> Date: Sun, 22 Dec 2019 14:17:37 +0300 Subject: [PATCH 019/665] diamond operators in unit tests, small packages (#6789) --- .../rxjava3/completable/CompletableTest.java | 104 +++++----- .../disposables/CompositeDisposableTest.java | 4 +- .../rxjava3/disposables/DisposableTest.java | 2 +- .../disposables/FutureDisposableTest.java | 6 +- .../disposables/SequentialDisposableTest.java | 6 +- .../disposables/SerialDisposableTests.java | 6 +- .../exceptions/CompositeExceptionTest.java | 6 +- .../io/reactivex/rxjava3/flowable/Burst.java | 6 +- .../flowable/FlowableBackpressureTests.java | 36 ++-- .../rxjava3/flowable/FlowableCollectTest.java | 24 +-- .../flowable/FlowableConversionTest.java | 18 +- .../flowable/FlowableCovarianceTest.java | 8 +- .../rxjava3/flowable/FlowableDoOnTest.java | 4 +- .../flowable/FlowableErrorHandlingTests.java | 4 +- .../rxjava3/flowable/FlowableEventStream.java | 2 +- .../rxjava3/flowable/FlowableNullTests.java | 16 +- .../flowable/FlowableStartWithTests.java | 4 +- .../flowable/FlowableSubscriberTest.java | 46 ++--- .../rxjava3/flowable/FlowableTests.java | 26 +-- .../rxjava3/flowable/FlowableWindowTests.java | 2 +- .../rxjava3/flowable/FlowableZipTests.java | 2 +- .../io/reactivex/rxjava3/maybe/MaybeTest.java | 14 +- .../observable/ObservableCovarianceTest.java | 8 +- .../observable/ObservableDoOnTest.java | 4 +- .../ObservableErrorHandlingTests.java | 4 +- .../observable/ObservableEventStream.java | 2 +- .../observable/ObservableNullTests.java | 16 +- .../observable/ObservableScanTests.java | 2 +- .../observable/ObservableStartWithTests.java | 4 +- .../observable/ObservableSubscriberTest.java | 8 +- .../rxjava3/observable/ObservableTest.java | 26 +-- .../observable/ObservableWindowTests.java | 2 +- .../observable/ObservableZipTests.java | 2 +- .../DisposableCompletableObserverTest.java | 2 +- .../DisposableMaybeObserverTest.java | 10 +- .../observers/DisposableObserverTest.java | 10 +- .../DisposableSingleObserverTest.java | 10 +- .../ResourceCompletableObserverTest.java | 2 +- .../observers/ResourceMaybeObserverTest.java | 22 +- .../observers/ResourceObserverTest.java | 20 +- .../observers/ResourceSingleObserverTest.java | 18 +- .../rxjava3/observers/SafeObserverTest.java | 40 ++-- .../observers/SerializedObserverTest.java | 52 ++--- .../rxjava3/observers/TestObserverTest.java | 60 +++--- .../rxjava3/parallel/ParallelCollectTest.java | 10 +- .../parallel/ParallelFlowableTest.java | 72 +++---- .../parallel/ParallelFromPublisherTest.java | 12 +- .../rxjava3/parallel/ParallelReduceTest.java | 10 +- .../rxjava3/parallel/ParallelRunOnTest.java | 4 +- .../rxjava3/plugins/RxJavaPluginsTest.java | 16 +- .../processors/AsyncProcessorTest.java | 16 +- .../processors/BehaviorProcessorTest.java | 28 +-- .../processors/MulticastProcessorTest.java | 14 +- .../processors/PublishProcessorTest.java | 16 +- ...ReplayProcessorBoundedConcurrencyTest.java | 14 +- .../ReplayProcessorConcurrencyTest.java | 14 +- .../processors/ReplayProcessorTest.java | 16 +- .../processors/SerializedProcessorTest.java | 8 +- .../processors/UnicastProcessorTest.java | 8 +- .../schedulers/AbstractSchedulerTests.java | 10 +- .../schedulers/ComputationSchedulerTests.java | 2 +- .../schedulers/ExecutorSchedulerFairTest.java | 2 +- .../ExecutorSchedulerInterruptibleTest.java | 2 +- .../schedulers/ExecutorSchedulerTest.java | 2 +- .../schedulers/SchedulerLifecycleTest.java | 6 +- .../schedulers/SchedulerTestHelper.java | 2 +- .../schedulers/SchedulerWorkerTest.java | 4 +- .../rxjava3/schedulers/TimedTest.java | 20 +- .../schedulers/TrampolineSchedulerTest.java | 4 +- .../rxjava3/single/SingleCacheTest.java | 4 +- .../rxjava3/single/SingleNullTests.java | 10 +- .../reactivex/rxjava3/single/SingleTest.java | 32 +-- .../rxjava3/subjects/AsyncSubjectTest.java | 16 +- .../rxjava3/subjects/BehaviorSubjectTest.java | 24 +-- .../rxjava3/subjects/PublishSubjectTest.java | 18 +- .../ReplaySubjectBoundedConcurrencyTest.java | 14 +- .../ReplaySubjectConcurrencyTest.java | 14 +- .../rxjava3/subjects/ReplaySubjectTest.java | 10 +- .../subjects/SerializedSubjectTest.java | 8 +- .../rxjava3/subjects/UnicastSubjectTest.java | 22 +- .../subscribers/DefaultSubscriberTest.java | 2 +- .../subscribers/DisposableSubscriberTest.java | 10 +- .../subscribers/ResourceSubscriberTest.java | 20 +- .../subscribers/SafeSubscriberTest.java | 48 ++--- .../subscribers/SerializedSubscriberTest.java | 52 ++--- .../subscribers/TestSubscriberTest.java | 126 ++++++------ .../rxjava3/tck/FromFutureTckTest.java | 2 +- .../tck/MulticastProcessorTckTest.java | 2 +- .../rxjava3/tck/RefCountProcessor.java | 4 +- .../rxjava3/tck/UnicastProcessorTckTest.java | 2 +- .../rxjava3/testsupport/TestHelper.java | 6 +- .../rxjava3/testsupport/TestObserverEx.java | 2 +- .../testsupport/TestObserverExTest.java | 134 ++++++------ .../rxjava3/testsupport/TestSubscriberEx.java | 2 +- .../testsupport/TestSubscriberExTest.java | 194 +++++++++--------- .../rxjava3/validators/BaseTypeParser.java | 2 +- .../CheckLocalVariablesInTests.java | 2 +- .../rxjava3/validators/FixLicenseHeaders.java | 4 +- .../validators/InternalWrongNaming.java | 2 +- .../JavadocFindUnescapedAngleBrackets.java | 2 +- .../validators/NewLinesBeforeAnnotation.java | 4 +- .../NoAnonymousInnerClassesTest.java | 2 +- .../ParamValidationCheckerTest.java | 18 +- .../validators/TestPrefixInMethodName.java | 4 +- .../rxjava3/validators/TextualAorAn.java | 4 +- .../validators/TooManyEmptyNewLines.java | 4 +- 106 files changed, 905 insertions(+), 903 deletions(-) diff --git a/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java b/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java index e18d884f54..d319e48861 100644 --- a/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java +++ b/src/test/java/io/reactivex/rxjava3/completable/CompletableTest.java @@ -349,7 +349,7 @@ public void concatObservableManyOneThrows() { @Test public void concatObservablePrefetch() { - final List requested = new ArrayList(); + final List requested = new ArrayList<>(); Flowable cs = Flowable .just(normal.completable) .repeat(10) @@ -859,7 +859,7 @@ public void mergeObservableManyOneThrows() { @Test public void mergeObservableMaxConcurrent() { - final List requested = new ArrayList(); + final List requested = new ArrayList<>(); Flowable cs = Flowable .just(normal.completable) .repeat(10) @@ -1079,7 +1079,7 @@ public void mergeDelayErrorObservableManyOneThrows() { @Test public void mergeDelayErrorObservableMaxConcurrent() { - final List requested = new ArrayList(); + final List requested = new ArrayList<>(); Flowable cs = Flowable .just(normal.completable) .repeat(10) @@ -1236,7 +1236,7 @@ public void accept(Integer d) { }); final AtomicBoolean disposedFirst = new AtomicBoolean(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); c.subscribe(new CompletableObserver() { @Override @@ -1282,7 +1282,7 @@ public void accept(Integer d) { }, false); final AtomicBoolean disposedFirst = new AtomicBoolean(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); c.subscribe(new CompletableObserver() { @Override @@ -1571,7 +1571,7 @@ public void delayNormal() throws InterruptedException { Completable c = normal.completable.delay(250, TimeUnit.MILLISECONDS); final AtomicBoolean done = new AtomicBoolean(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); c.subscribe(new CompletableObserver() { @Override @@ -1611,7 +1611,7 @@ public void delayErrorImmediately() throws InterruptedException { final Completable c = error.completable.delay(250, TimeUnit.MILLISECONDS, scheduler); final AtomicBoolean done = new AtomicBoolean(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); c.subscribe(new CompletableObserver() { @Override @@ -1645,7 +1645,7 @@ public void delayErrorToo() throws InterruptedException { Completable c = error.completable.delay(250, TimeUnit.MILLISECONDS, Schedulers.computation(), true); final AtomicBoolean done = new AtomicBoolean(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); c.subscribe(new CompletableObserver() { @Override @@ -1833,7 +1833,7 @@ public void onComplete() { @Test public void doOnErrorNoError() { - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); Completable c = normal.completable.doOnError(new Consumer() { @Override @@ -1849,7 +1849,7 @@ public void accept(Throwable e) { @Test public void doOnErrorHasError() { - final AtomicReference err = new AtomicReference(); + final AtomicReference err = new AtomicReference<>(); Completable c = error.completable.doOnError(new Consumer() { @Override @@ -2036,8 +2036,8 @@ public void observeOnNull() { @Test public void observeOnNormal() throws InterruptedException { - final AtomicReference name = new AtomicReference(); - final AtomicReference err = new AtomicReference(); + final AtomicReference name = new AtomicReference<>(); + final AtomicReference err = new AtomicReference<>(); final CountDownLatch cdl = new CountDownLatch(1); Completable c = normal.completable.observeOn(Schedulers.computation()); @@ -2069,8 +2069,8 @@ public void onError(Throwable e) { @Test public void observeOnError() throws InterruptedException { - final AtomicReference name = new AtomicReference(); - final AtomicReference err = new AtomicReference(); + final AtomicReference name = new AtomicReference<>(); + final AtomicReference err = new AtomicReference<>(); final CountDownLatch cdl = new CountDownLatch(1); Completable c = error.completable.observeOn(Schedulers.computation()); @@ -2195,7 +2195,7 @@ public Completable apply(Throwable v) { @Test public void repeatNormal() { - final AtomicReference err = new AtomicReference(); + final AtomicReference err = new AtomicReference<>(); final AtomicInteger calls = new AtomicInteger(); Completable c = Completable.fromCallable(new Callable() { @@ -2500,7 +2500,7 @@ public void run() { @Test public void subscribeTwoCallbacksNormal() { - final AtomicReference err = new AtomicReference(); + final AtomicReference err = new AtomicReference<>(); final AtomicBoolean complete = new AtomicBoolean(); normal.completable.subscribe(new Action() { @Override @@ -2520,7 +2520,7 @@ public void accept(Throwable e) { @Test public void subscribeTwoCallbacksError() { - final AtomicReference err = new AtomicReference(); + final AtomicReference err = new AtomicReference<>(); final AtomicBoolean complete = new AtomicBoolean(); error.completable.subscribe(new Action() { @Override @@ -2558,7 +2558,7 @@ public void run() { } public void subscribeTwoCallbacksCompleteThrows() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference err = new AtomicReference(); + final AtomicReference err = new AtomicReference<>(); normal.completable.subscribe(new Action() { @Override public void run() { throw new TestException(); } @@ -2596,7 +2596,7 @@ public void run() { } @Test public void subscribeObserverNormal() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); normal.completable.toObservable().subscribe(to); @@ -2607,7 +2607,7 @@ public void subscribeObserverNormal() { @Test public void subscribeObserverError() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); error.completable.toObservable().subscribe(to); @@ -2673,7 +2673,7 @@ public void subscribeCompletableSubscriberNull() { @Test public void subscribeSubscriberNormal() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); normal.completable.toFlowable().subscribe(ts); @@ -2684,7 +2684,7 @@ public void subscribeSubscriberNormal() { @Test public void subscribeSubscriberError() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); error.completable.toFlowable().subscribe(ts); @@ -2700,7 +2700,7 @@ public void subscribeOnNull() { @Test public void subscribeOnNormal() { - final AtomicReference name = new AtomicReference(); + final AtomicReference name = new AtomicReference<>(); Completable c = Completable.unsafeCreate(new CompletableSource() { @Override @@ -2717,7 +2717,7 @@ public void subscribe(CompletableObserver observer) { @Test public void subscribeOnError() { - final AtomicReference name = new AtomicReference(); + final AtomicReference name = new AtomicReference<>(); Completable c = Completable.unsafeCreate(new CompletableSource() { @Override @@ -2903,7 +2903,7 @@ public void toSingleDefaultNull() { @Test public void unsubscribeOnNormal() throws InterruptedException { - final AtomicReference name = new AtomicReference(); + final AtomicReference name = new AtomicReference<>(); final CountDownLatch cdl = new CountDownLatch(1); normal.completable.delay(1, TimeUnit.SECONDS) @@ -3010,7 +3010,7 @@ public void ambArrayOneFiresError() { Completable c = Completable.ambArray(c1, c2); - final AtomicReference complete = new AtomicReference(); + final AtomicReference complete = new AtomicReference<>(); c.subscribe(Functions.EMPTY_ACTION, new Consumer() { @Override @@ -3072,7 +3072,7 @@ public void ambArraySecondFiresError() { Completable c = Completable.ambArray(c1, c2); - final AtomicReference complete = new AtomicReference(); + final AtomicReference complete = new AtomicReference<>(); c.subscribe(Functions.EMPTY_ACTION, new Consumer() { @Override @@ -3231,7 +3231,7 @@ public void ambWithArrayOneFiresError() { Completable c = c1.ambWith(c2); - final AtomicReference complete = new AtomicReference(); + final AtomicReference complete = new AtomicReference<>(); c.subscribe(Functions.EMPTY_ACTION, new Consumer() { @Override @@ -3293,7 +3293,7 @@ public void ambWithArraySecondFiresError() { Completable c = c1.ambWith(c2); - final AtomicReference complete = new AtomicReference(); + final AtomicReference complete = new AtomicReference<>(); c.subscribe(Functions.EMPTY_ACTION, new Consumer() { @Override @@ -3356,7 +3356,7 @@ public Object call() throws Exception { } })); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); c.subscribe(ts); @@ -3373,7 +3373,7 @@ public void startWithFlowableError() { Flowable c = normal.completable .startWith(Flowable.error(new TestException())); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); c.subscribe(ts); @@ -3396,7 +3396,7 @@ public Object call() throws Exception { } })); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); o.subscribe(to); @@ -3413,7 +3413,7 @@ public void startWithObservableError() { Observable o = normal.completable .startWith(Observable.error(new TestException())); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); o.subscribe(to); @@ -3441,14 +3441,14 @@ public void startWithObservableNull() { @Test public void andThen() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Completable.complete().andThen(Flowable.just("foo")).subscribe(ts); ts.request(1); ts.assertValue("foo"); ts.assertComplete(); ts.assertNoErrors(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Completable.complete().andThen(Observable.just("foo")).subscribe(to); to.assertValue("foo"); to.assertComplete(); @@ -3543,7 +3543,7 @@ public void accept(Integer t) { } }; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Completable.using(new Supplier() { @Override @@ -3613,7 +3613,7 @@ public void subscribeActionReportsUnsubscribedAfter() { PublishSubject stringSubject = PublishSubject.create(); Completable completable = stringSubject.ignoreElements(); - final AtomicReference disposableRef = new AtomicReference(); + final AtomicReference disposableRef = new AtomicReference<>(); Disposable completableSubscription = completable.subscribe(new Action() { @Override public void run() { @@ -3695,7 +3695,7 @@ public void accept(Throwable e) { } @Test public void andThenSubscribeOn() { - TestSubscriberEx ts = new TestSubscriberEx(0); + TestSubscriberEx ts = new TestSubscriberEx<>(0); TestScheduler scheduler = new TestScheduler(); Completable.complete().andThen(Flowable.just("foo").delay(1, TimeUnit.SECONDS, scheduler)).subscribe(ts); @@ -3712,7 +3712,7 @@ public void andThenSubscribeOn() { @Test public void andThenSingleNever() { - TestSubscriberEx ts = new TestSubscriberEx(0); + TestSubscriberEx ts = new TestSubscriberEx<>(0); Completable.never().andThen(Single.just("foo")).toFlowable().subscribe(ts); ts.request(1); ts.assertNoValues(); @@ -3721,7 +3721,7 @@ public void andThenSingleNever() { @Test public void andThenSingleError() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); final AtomicBoolean hasRun = new AtomicBoolean(false); final Exception e = new Exception(); Completable.error(e) @@ -3740,7 +3740,7 @@ public void subscribeActual(SingleObserver observer) { @Test public void andThenSingleSubscribeOn() { - TestSubscriberEx ts = new TestSubscriberEx(0); + TestSubscriberEx ts = new TestSubscriberEx<>(0); TestScheduler scheduler = new TestScheduler(); Completable.complete().andThen(Single.just("foo").delay(1, TimeUnit.SECONDS, scheduler)).toFlowable().subscribe(ts); @@ -3992,7 +3992,7 @@ public Object call() throws Exception { } })); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); c.subscribe(ts); @@ -4009,7 +4009,7 @@ public void andThenFlowableError() { Flowable c = normal.completable .andThen(Flowable.error(new TestException())); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); c.subscribe(ts); @@ -4025,7 +4025,7 @@ public void usingFactoryThrows() throws Throwable { @SuppressWarnings("unchecked") Consumer onDispose = mock(Consumer.class); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Completable.using(new Supplier() { @Override @@ -4056,7 +4056,7 @@ public void accept(Integer t) { } }; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Completable.using(new Supplier() { @Override @@ -4126,7 +4126,7 @@ public void subscribeReportsUnsubscribed() { @Test public void hookSubscribeStart() throws Throwable { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Completable completable = Completable.unsafeCreate(new CompletableSource() { @Override public void subscribe(CompletableObserver observer) { @@ -4156,7 +4156,7 @@ public void onStart() { @Test public void onErrorCompleteFunctionThrows() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); error.completable.onErrorComplete(new Predicate() { @Override @@ -4185,7 +4185,7 @@ public void subscribeAction2ReportsUnsubscribedAfter() { PublishSubject stringSubject = PublishSubject.create(); Completable completable = stringSubject.ignoreElements(); - final AtomicReference disposableRef = new AtomicReference(); + final AtomicReference disposableRef = new AtomicReference<>(); Disposable completableSubscription = completable.subscribe(new Action() { @Override public void run() { @@ -4207,7 +4207,7 @@ public void subscribeAction2ReportsUnsubscribedOnErrorAfter() { PublishSubject stringSubject = PublishSubject.create(); Completable completable = stringSubject.ignoreElements(); - final AtomicReference disposableRef = new AtomicReference(); + final AtomicReference disposableRef = new AtomicReference<>(); Disposable completableSubscription = completable.subscribe(Functions.EMPTY_ACTION, new Consumer() { @Override @@ -4242,7 +4242,7 @@ public void accept(Integer integer) { @Test public void andThenNever() { - TestSubscriberEx ts = new TestSubscriberEx(0); + TestSubscriberEx ts = new TestSubscriberEx<>(0); Completable.never().andThen(Flowable.just("foo")).subscribe(ts); ts.request(1); ts.assertNoValues(); @@ -4251,7 +4251,7 @@ public void andThenNever() { @Test public void andThenError() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); final AtomicBoolean hasRun = new AtomicBoolean(false); final Exception e = new Exception(); Completable.unsafeCreate(new CompletableSource() { @@ -4278,7 +4278,7 @@ public void subscribe(Subscriber s) { @Test public void andThenSingle() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Completable.complete().andThen(Single.just("foo")).toFlowable().subscribe(ts); ts.request(1); ts.assertValue("foo"); diff --git a/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java index b219e8f9d7..73d605c2cc 100644 --- a/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java +++ b/src/test/java/io/reactivex/rxjava3/disposables/CompositeDisposableTest.java @@ -72,7 +72,7 @@ public void run() { })); } - final List threads = new ArrayList(); + final List threads = new ArrayList<>(); for (int i = 0; i < count; i++) { final Thread t = new Thread() { @Override @@ -253,7 +253,7 @@ public void run() { })); - final List threads = new ArrayList(); + final List threads = new ArrayList<>(); for (int i = 0; i < count; i++) { final Thread t = new Thread() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java index 69bd6a8cc2..66abc6a280 100644 --- a/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java +++ b/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java @@ -160,7 +160,7 @@ public void setOnceTwice() { List errors = TestHelper.trackPluginErrors(); try { - AtomicReference target = new AtomicReference(); + AtomicReference target = new AtomicReference<>(); Disposable d = Disposable.empty(); DisposableHelper.setOnce(target, d); diff --git a/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java index a15f018c9b..3ae97dd306 100644 --- a/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java +++ b/src/test/java/io/reactivex/rxjava3/disposables/FutureDisposableTest.java @@ -26,7 +26,7 @@ public class FutureDisposableTest extends RxJavaTest { @Test public void normal() { - FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask ft = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); Disposable d = Disposable.fromFuture(ft); assertFalse(d.isDisposed()); @@ -43,7 +43,7 @@ public void normal() { @Test public void interruptible() { - FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask ft = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); Disposable d = Disposable.fromFuture(ft, true); assertFalse(d.isDisposed()); @@ -60,7 +60,7 @@ public void interruptible() { @Test public void normalDone() { - FutureTask ft = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask ft = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); FutureDisposable d = new FutureDisposable(ft, false); assertFalse(d.isDisposed()); diff --git a/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java index dc92b1f1d3..62248db45e 100644 --- a/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java +++ b/src/test/java/io/reactivex/rxjava3/disposables/SequentialDisposableTest.java @@ -129,7 +129,7 @@ public void settingUnderlyingWhenUnsubscribedCausesImmediateUnsubscriptionConcur final int count = 10; final CountDownLatch end = new CountDownLatch(count); - final List threads = new ArrayList(); + final List threads = new ArrayList<>(); for (int i = 0; i < count; i++) { final Thread t = new Thread() { @Override @@ -164,12 +164,12 @@ public void run() { public void concurrentSetDisposableShouldNotInterleave() throws InterruptedException { final int count = 10; - final List subscriptions = new ArrayList(); + final List subscriptions = new ArrayList<>(); final CountDownLatch start = new CountDownLatch(1); final CountDownLatch end = new CountDownLatch(count); - final List threads = new ArrayList(); + final List threads = new ArrayList<>(); for (int i = 0; i < count; i++) { final Disposable subscription = mock(Disposable.class); subscriptions.add(subscription); diff --git a/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java b/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java index 0205d6a85d..aa53f2ecb0 100644 --- a/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java +++ b/src/test/java/io/reactivex/rxjava3/disposables/SerialDisposableTests.java @@ -129,7 +129,7 @@ public void settingUnderlyingWhenUnsubscribedCausesImmediateUnsubscriptionConcur final int count = 10; final CountDownLatch end = new CountDownLatch(count); - final List threads = new ArrayList(); + final List threads = new ArrayList<>(); for (int i = 0; i < count; i++) { final Thread t = new Thread() { @Override @@ -164,12 +164,12 @@ public void run() { public void concurrentSetDisposableShouldNotInterleave() throws InterruptedException { final int count = 10; - final List subscriptions = new ArrayList(); + final List subscriptions = new ArrayList<>(); final CountDownLatch start = new CountDownLatch(1); final CountDownLatch end = new CountDownLatch(count); - final List threads = new ArrayList(); + final List threads = new ArrayList<>(); for (int i = 0; i < count; i++) { final Disposable subscription = mock(Disposable.class); subscriptions.add(subscription); diff --git a/src/test/java/io/reactivex/rxjava3/exceptions/CompositeExceptionTest.java b/src/test/java/io/reactivex/rxjava3/exceptions/CompositeExceptionTest.java index 6f8ab6ff55..331a7dce9c 100644 --- a/src/test/java/io/reactivex/rxjava3/exceptions/CompositeExceptionTest.java +++ b/src/test/java/io/reactivex/rxjava3/exceptions/CompositeExceptionTest.java @@ -31,7 +31,7 @@ public class CompositeExceptionTest extends RxJavaTest { private final Throwable ex3 = new Throwable("Ex3", ex2); private CompositeException getNewCompositeExceptionWithEx123() { - List throwables = new ArrayList(); + List throwables = new ArrayList<>(); throwables.add(ex1); throwables.add(ex2); throwables.add(ex3); @@ -65,7 +65,7 @@ public void emptyErrors() { assertEquals("errors is empty", e.getMessage()); } try { - new CompositeException(new ArrayList()); + new CompositeException(new ArrayList<>()); fail("CompositeException should fail if errors is empty"); } catch (IllegalArgumentException e) { assertEquals("errors is empty", e.getMessage()); @@ -134,7 +134,7 @@ public void compositeExceptionFromCompositeAndChild() { @Test public void compositeExceptionFromTwoDuplicateComposites() { - List exs = new ArrayList(); + List exs = new ArrayList<>(); exs.add(getNewCompositeExceptionWithEx123()); exs.add(getNewCompositeExceptionWithEx123()); CompositeException cex = new CompositeException(exs); diff --git a/src/test/java/io/reactivex/rxjava3/flowable/Burst.java b/src/test/java/io/reactivex/rxjava3/flowable/Burst.java index 1e6c255a30..791ec79a2f 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/Burst.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/Burst.java @@ -58,12 +58,12 @@ public static Builder item(T item) { @SafeVarargs public static Builder items(T... items) { - return new Builder(Arrays.asList(items)); + return new Builder<>(Arrays.asList(items)); } final class BurstSubscription implements Subscription { private final Subscriber subscriber; - final Queue q = new ConcurrentLinkedQueue(items); + final Queue q = new ConcurrentLinkedQueue<>(items); final AtomicLong requested = new AtomicLong(); volatile boolean cancelled; @@ -121,7 +121,7 @@ public Flowable error(Throwable e) { } public Flowable create() { - return new Burst(error, items); + return new Burst<>(error, items); } } diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableBackpressureTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableBackpressureTests.java index e110b410c2..7d0687dd72 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableBackpressureTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableBackpressureTests.java @@ -82,7 +82,7 @@ public void doAfterTest() { public void observeOn() { int num = (int) (Flowable.bufferSize() * 2.1); AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); incrementingIntegers(c).observeOn(Schedulers.computation()).take(num).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertNoErrors(); @@ -95,7 +95,7 @@ public void observeOn() { public void observeOnWithSlowConsumer() { int num = (int) (Flowable.bufferSize() * 0.2); AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); incrementingIntegers(c).observeOn(Schedulers.computation()).map( new Function() { @Override @@ -121,7 +121,7 @@ public void mergeSync() { int num = (int) (Flowable.bufferSize() * 4.1); AtomicInteger c1 = new AtomicInteger(); AtomicInteger c2 = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable merged = Flowable.merge(incrementingIntegers(c1), incrementingIntegers(c2)); merged.take(num).subscribe(ts); @@ -142,7 +142,7 @@ public void mergeAsync() { int num = (int) (Flowable.bufferSize() * 4.1); AtomicInteger c1 = new AtomicInteger(); AtomicInteger c2 = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable merged = Flowable.merge( incrementingIntegers(c1).subscribeOn(Schedulers.computation()), incrementingIntegers(c2).subscribeOn(Schedulers.computation())); @@ -171,7 +171,7 @@ public void mergeAsyncThenObserveOnLoop() { AtomicInteger c1 = new AtomicInteger(); AtomicInteger c2 = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable merged = Flowable.merge( incrementingIntegers(c1).subscribeOn(Schedulers.computation()), incrementingIntegers(c2).subscribeOn(Schedulers.computation())); @@ -194,7 +194,7 @@ public void mergeAsyncThenObserveOn() { int num = (int) (Flowable.bufferSize() * 4.1); AtomicInteger c1 = new AtomicInteger(); AtomicInteger c2 = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable merged = Flowable.merge( incrementingIntegers(c1).subscribeOn(Schedulers.computation()), incrementingIntegers(c2).subscribeOn(Schedulers.computation())); @@ -216,7 +216,7 @@ public void mergeAsyncThenObserveOn() { public void flatMapSync() { int num = (int) (Flowable.bufferSize() * 2.1); AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); incrementingIntegers(c) .flatMap(new Function>() { @@ -240,7 +240,7 @@ public void zipSync() { int num = (int) (Flowable.bufferSize() * 4.1); AtomicInteger c1 = new AtomicInteger(); AtomicInteger c2 = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable zipped = Flowable.zip( incrementingIntegers(c1), @@ -268,7 +268,7 @@ public void zipAsync() { int num = (int) (Flowable.bufferSize() * 2.1); AtomicInteger c1 = new AtomicInteger(); AtomicInteger c2 = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable zipped = Flowable.zip( incrementingIntegers(c1).subscribeOn(Schedulers.computation()), incrementingIntegers(c2).subscribeOn(Schedulers.computation()), @@ -295,8 +295,8 @@ public void subscribeOnScheduling() { for (int i = 0; i < 100; i++) { int num = (int) (Flowable.bufferSize() * 2.1); AtomicInteger c = new AtomicInteger(); - ConcurrentLinkedQueue threads = new ConcurrentLinkedQueue(); - TestSubscriber ts = new TestSubscriber(); + ConcurrentLinkedQueue threads = new ConcurrentLinkedQueue<>(); + TestSubscriber ts = new TestSubscriber<>(); // observeOn is there to make it async and need backpressure incrementingIntegers(c, threads).subscribeOn(Schedulers.computation()).observeOn(Schedulers.computation()).take(num).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -325,7 +325,7 @@ public void subscribeOnScheduling() { public void takeFilterSkipChainAsync() { int num = (int) (Flowable.bufferSize() * 2.1); AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); incrementingIntegers(c).observeOn(Schedulers.computation()) .skip(10000) .filter(new Predicate() { @@ -452,7 +452,7 @@ public void onNext(Integer t) { @Test public void firehoseFailsAsExpected() { AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); firehose(c).observeOn(Schedulers.computation()) .map(new Function() { @@ -496,7 +496,7 @@ public void onBackpressureDrop() { } int num = (int) (Flowable.bufferSize() * 1.1); // > 1 so that take doesn't prevent buffer overflow AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); firehose(c).onBackpressureDrop() .observeOn(Schedulers.computation()) .map(SLOW_PASS_THRU).take(num).subscribe(ts); @@ -521,7 +521,7 @@ public void onBackpressureDropWithAction() { final AtomicInteger dropCount = new AtomicInteger(); final AtomicInteger passCount = new AtomicInteger(); final int num = Flowable.bufferSize() * 3; // > 1 so that take doesn't prevent buffer overflow - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); firehose(emitCount) .onBackpressureDrop(new Consumer() { @@ -561,7 +561,7 @@ public void onBackpressureDropSynchronous() { for (int i = 0; i < 100; i++) { int num = (int) (Flowable.bufferSize() * 1.1); // > 1 so that take doesn't prevent buffer overflow AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); firehose(c).onBackpressureDrop() .map(SLOW_PASS_THRU).take(num).subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); @@ -584,7 +584,7 @@ public void onBackpressureDropSynchronousWithAction() { final AtomicInteger dropCount = new AtomicInteger(); int num = (int) (Flowable.bufferSize() * 1.1); // > 1 so that take doesn't prevent buffer overflow AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); firehose(c).onBackpressureDrop(new Consumer() { @Override public void accept(Integer j) { @@ -613,7 +613,7 @@ public void accept(Integer j) { public void onBackpressureBuffer() { int num = (int) (Flowable.bufferSize() * 1.1); // > 1 so that take doesn't prevent buffer overflow AtomicInteger c = new AtomicInteger(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); firehose(c).takeWhile(new Predicate() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java index f0f93c6453..ae3cd0c906 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java @@ -36,7 +36,7 @@ public void collectToListFlowable() { .collect(new Supplier>() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -109,7 +109,7 @@ public void accept(List list, Integer t) { @Test public void collectorFailureDoesNotResultInTwoErrorEmissionsFlowable() { try { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); RxJavaPlugins.setErrorHandler(addToList(list)); final RuntimeException e1 = new RuntimeException(); final RuntimeException e2 = new RuntimeException(); @@ -171,7 +171,7 @@ public void accept(Object o, Integer t) { @Test public void collectIntoFlowable() { Flowable.just(1, 1, 1, 1, 2) - .collectInto(new HashSet(), new BiConsumer, Integer>() { + .collectInto(new HashSet<>(), new BiConsumer, Integer>() { @Override public void accept(HashSet s, Integer v) throws Exception { s.add(v); @@ -179,7 +179,7 @@ public void accept(HashSet s, Integer v) throws Exception { }) .toFlowable() .test() - .assertResult(new HashSet(Arrays.asList(1, 2))); + .assertResult(new HashSet<>(Arrays.asList(1, 2))); } @Test @@ -188,7 +188,7 @@ public void collectToList() { .collect(new Supplier>() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -261,7 +261,7 @@ public void accept(List list, Integer t) { @Test public void collectorFailureDoesNotResultInTwoErrorEmissions() { try { - final List list = new CopyOnWriteArrayList(); + final List list = new CopyOnWriteArrayList<>(); RxJavaPlugins.setErrorHandler(addToList(list)); final RuntimeException e1 = new RuntimeException(); final RuntimeException e2 = new RuntimeException(); @@ -320,20 +320,20 @@ public void accept(Object o, Integer t) { @Test public void collectInto() { Flowable.just(1, 1, 1, 1, 2) - .collectInto(new HashSet(), new BiConsumer, Integer>() { + .collectInto(new HashSet<>(), new BiConsumer, Integer>() { @Override public void accept(HashSet s, Integer v) throws Exception { s.add(v); } }) .test() - .assertResult(new HashSet(Arrays.asList(1, 2))); + .assertResult(new HashSet<>(Arrays.asList(1, 2))); } @Test public void dispose() { TestHelper.checkDisposed(Flowable.just(1, 2) - .collect(Functions.justSupplier(new ArrayList()), new BiConsumer, Integer>() { + .collect(Functions.justSupplier(new ArrayList<>()), new BiConsumer, Integer>() { @Override public void accept(ArrayList a, Integer b) throws Exception { a.add(b); @@ -341,7 +341,7 @@ public void accept(ArrayList a, Integer b) throws Exception { })); TestHelper.checkDisposed(Flowable.just(1, 2) - .collect(Functions.justSupplier(new ArrayList()), new BiConsumer, Integer>() { + .collect(Functions.justSupplier(new ArrayList<>()), new BiConsumer, Integer>() { @Override public void accept(ArrayList a, Integer b) throws Exception { a.add(b); @@ -354,7 +354,7 @@ public void doubleOnSubscribe() { TestHelper.checkDoubleOnSubscribeFlowable(new Function, Flowable>>() { @Override public Flowable> apply(Flowable f) throws Exception { - return f.collect(Functions.justSupplier(new ArrayList()), + return f.collect(Functions.justSupplier(new ArrayList<>()), new BiConsumer, Integer>() { @Override public void accept(ArrayList a, Integer b) throws Exception { @@ -366,7 +366,7 @@ public void accept(ArrayList a, Integer b) throws Exception { TestHelper.checkDoubleOnSubscribeFlowableToSingle(new Function, Single>>() { @Override public Single> apply(Flowable f) throws Exception { - return f.collect(Functions.justSupplier(new ArrayList()), + return f.collect(Functions.justSupplier(new ArrayList<>()), new BiConsumer, Integer>() { @Override public void accept(ArrayList a, Integer b) throws Exception { diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java index 23b484e369..c5b75a9b39 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java @@ -44,7 +44,7 @@ public static class CylonDetectorObservable { protected Publisher onSubscribe; public static CylonDetectorObservable create(Publisher onSubscribe) { - return new CylonDetectorObservable(onSubscribe); + return new CylonDetectorObservable<>(onSubscribe); } protected CylonDetectorObservable(Publisher onSubscribe) { @@ -56,7 +56,7 @@ public void subscribe(Subscriber subscriber) { } public CylonDetectorObservable lift(FlowableOperator operator) { - return x(new RobotConversionFunc(operator)); + return x(new RobotConversionFunc<>(operator)); } public O x(Function, O> operator) { @@ -76,11 +76,11 @@ public CylonDetectorObservable compose(Function beep(Predicate predicate) { - return new CylonDetectorObservable(new FlowableFilter(Flowable.fromPublisher(onSubscribe), predicate)); + return new CylonDetectorObservable<>(new FlowableFilter<>(Flowable.fromPublisher(onSubscribe), predicate)); } public final CylonDetectorObservable boop(Function func) { - return new CylonDetectorObservable(new FlowableMap(Flowable.fromPublisher(onSubscribe), func)); + return new CylonDetectorObservable<>(new FlowableMap<>(Flowable.fromPublisher(onSubscribe), func)); } public CylonDetectorObservable DESTROY() { @@ -147,7 +147,7 @@ public Flowable apply(final Publisher onSubscribe) { @Test public void conversionBetweenObservableClasses() { - final TestObserver to = new TestObserver(new DefaultObserver() { + final TestObserver to = new TestObserver<>(new DefaultObserver() { @Override public void onComplete() { @@ -175,7 +175,7 @@ public void accept(Object pv) { System.out.println(pv); } }) - .to(new ConvertToCylonDetector()) + .to(new ConvertToCylonDetector<>()) .beep(new Predicate() { @Override public boolean test(Object t) { @@ -189,7 +189,7 @@ public Object apply(Object cylon) { } }) .DESTROY() - .x(new ConvertToObservable()) + .x(new ConvertToObservable<>()) .reduce("Cylon Detector finished. Report:\n", new BiFunction() { @Override public String apply(String a, String n) { @@ -204,7 +204,7 @@ public String apply(String a, String n) { @Test public void convertToConcurrentQueue() { - final AtomicReference thrown = new AtomicReference(null); + final AtomicReference thrown = new AtomicReference<>(null); final AtomicBoolean isFinished = new AtomicBoolean(false); ConcurrentLinkedQueue queue = Flowable.range(0, 5) .flatMap(new Function>() { @@ -228,7 +228,7 @@ public Integer apply(Integer k) { .to(new FlowableConverter>() { @Override public ConcurrentLinkedQueue apply(Flowable onSubscribe) { - final ConcurrentLinkedQueue q = new ConcurrentLinkedQueue(); + final ConcurrentLinkedQueue q = new ConcurrentLinkedQueue<>(); onSubscribe.subscribe(new DefaultSubscriber() { @Override public void onComplete() { diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableCovarianceTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableCovarianceTest.java index a14462e905..ae50092ce9 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableCovarianceTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableCovarianceTest.java @@ -66,7 +66,7 @@ public int compare(Media t1, Media t2) { @Test public void groupByCompose() { Flowable movies = Flowable.just(new HorrorMovie(), new ActionMovie(), new Movie()); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); movies .groupBy(new Function() { @@ -188,9 +188,9 @@ public Flowable apply(List> listOfLists) { } else { // diff the two List newList = listOfLists.get(1); - List oldList = new ArrayList(listOfLists.get(0)); + List oldList = new ArrayList<>(listOfLists.get(0)); - Set delta = new LinkedHashSet(); + Set delta = new LinkedHashSet<>(); delta.addAll(newList); // remove all that match in old delta.removeAll(oldList); @@ -212,7 +212,7 @@ public Flowable apply(List> listOfLists) { @Override public Publisher apply(Flowable> movieList) { return movieList - .startWithItem(new ArrayList()) + .startWithItem(new ArrayList<>()) .buffer(2, 1) .skip(1) .flatMap(calculateDelta); diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableDoOnTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableDoOnTest.java index 2655f8c4fd..b1b12c5577 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableDoOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableDoOnTest.java @@ -27,7 +27,7 @@ public class FlowableDoOnTest extends RxJavaTest { @Test public void doOnEach() { - final AtomicReference r = new AtomicReference(); + final AtomicReference r = new AtomicReference<>(); String output = Flowable.just("one").doOnNext(new Consumer() { @Override public void accept(String v) { @@ -41,7 +41,7 @@ public void accept(String v) { @Test public void doOnError() { - final AtomicReference r = new AtomicReference(); + final AtomicReference r = new AtomicReference<>(); Throwable t = null; try { Flowable. error(new RuntimeException("an error")) diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableErrorHandlingTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableErrorHandlingTests.java index 694dedded4..df123ee304 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableErrorHandlingTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableErrorHandlingTests.java @@ -35,7 +35,7 @@ public class FlowableErrorHandlingTests extends RxJavaTest { @Test public void onNextError() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference caughtError = new AtomicReference(); + final AtomicReference caughtError = new AtomicReference<>(); Flowable f = Flowable.interval(50, TimeUnit.MILLISECONDS); Subscriber subscriber = new DefaultSubscriber() { @@ -72,7 +72,7 @@ public void onNext(Long args) { @Test public void onNextErrorAcrossThread() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference caughtError = new AtomicReference(); + final AtomicReference caughtError = new AtomicReference<>(); Flowable f = Flowable.interval(50, TimeUnit.MILLISECONDS); Subscriber subscriber = new DefaultSubscriber() { diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableEventStream.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableEventStream.java index 7c64da9ada..861d67c68d 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableEventStream.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableEventStream.java @@ -33,7 +33,7 @@ public static Flowable getEventStream(final String type, final int numIns } public static Event randomEvent(String type, int numInstances) { - Map values = new LinkedHashMap(); + Map values = new LinkedHashMap<>(); values.put("count200", randomIntFrom0to(4000)); values.put("count4xx", randomIntFrom0to(300)); values.put("count5xx", randomIntFrom0to(500)); diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java index ed520c494c..0e85aa8a40 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java @@ -239,10 +239,10 @@ public void fromFutureNull() { @Test public void fromFutureReturnsNull() { - FutureTask f = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); f.run(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.fromFuture(f).subscribe(ts); ts.assertNoValues(); ts.assertNotComplete(); @@ -256,24 +256,24 @@ public void fromFutureTimedFutureNull() { @Test(expected = NullPointerException.class) public void fromFutureTimedUnitNull() { - Flowable.fromFuture(new FutureTask(Functions.EMPTY_RUNNABLE, null), 1, null); + Flowable.fromFuture(new FutureTask<>(Functions.EMPTY_RUNNABLE, null), 1, null); } @Test(expected = NullPointerException.class) public void fromFutureTimedSchedulerNull() { - Flowable.fromFuture(new FutureTask(Functions.EMPTY_RUNNABLE, null), 1, TimeUnit.SECONDS, null); + Flowable.fromFuture(new FutureTask<>(Functions.EMPTY_RUNNABLE, null), 1, TimeUnit.SECONDS, null); } @Test(expected = NullPointerException.class) public void fromFutureTimedReturnsNull() { - FutureTask f = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); f.run(); Flowable.fromFuture(f, 1, TimeUnit.SECONDS).blockingLast(); } @Test(expected = NullPointerException.class) public void fromFutureSchedulerNull() { - Flowable.fromFuture(new FutureTask(Functions.EMPTY_RUNNABLE, null), null); + Flowable.fromFuture(new FutureTask<>(Functions.EMPTY_RUNNABLE, null), null); } @Test(expected = NullPointerException.class) @@ -2392,7 +2392,7 @@ public Integer apply(Integer v) { }, new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }, null); } @@ -2412,7 +2412,7 @@ public Integer apply(Integer v) { }, new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }, new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableStartWithTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableStartWithTests.java index e4ea973b5b..7ac13a4bd6 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableStartWithTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableStartWithTests.java @@ -34,7 +34,7 @@ public void startWith1() { @Test public void startWithIterable() { - List li = new ArrayList(); + List li = new ArrayList<>(); li.add("alpha"); li.add("beta"); List values = Flowable.just("one", "two").startWithIterable(li).toList().blockingGet(); @@ -47,7 +47,7 @@ public void startWithIterable() { @Test public void startWithObservable() { - List li = new ArrayList(); + List li = new ArrayList<>(); li.add("alpha"); li.add("beta"); List values = Flowable.just("one", "two") diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java index 2a94f73b8e..2a27aaada0 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableSubscriberTest.java @@ -40,7 +40,7 @@ public class FlowableSubscriberTest { */ @Test public void requestFromFinalSubscribeWithRequestValue() { - TestSubscriber s = new TestSubscriber(0L); + TestSubscriber s = new TestSubscriber<>(0L); s.request(10); final AtomicLong r = new AtomicLong(); s.onSubscribe(new Subscription() { @@ -64,7 +64,7 @@ public void cancel() { */ @Test public void requestFromFinalSubscribeWithoutRequestValue() { - TestSubscriber s = new TestSubscriber(); + TestSubscriber s = new TestSubscriber<>(); final AtomicLong r = new AtomicLong(); s.onSubscribe(new Subscription() { @@ -84,7 +84,7 @@ public void cancel() { @Test public void requestFromChainedOperator() throws Throwable { - TestSubscriber s = new TestSubscriber(10L); + TestSubscriber s = new TestSubscriber<>(10L); FlowableOperator o = new FlowableOperator() { @Override public Subscriber apply(final Subscriber s1) { @@ -136,7 +136,7 @@ public void cancel() { @Test public void requestFromDecoupledOperator() throws Throwable { - TestSubscriber s = new TestSubscriber(0L); + TestSubscriber s = new TestSubscriber<>(0L); FlowableOperator o = new FlowableOperator() { @Override public Subscriber apply(final Subscriber s1) { @@ -189,7 +189,7 @@ public void cancel() { @Test public void requestFromDecoupledOperatorThatRequestsN() throws Throwable { - TestSubscriber s = new TestSubscriber(10L); + TestSubscriber s = new TestSubscriber<>(10L); final AtomicLong innerR = new AtomicLong(); FlowableOperator o = new FlowableOperator() { @Override @@ -260,7 +260,7 @@ public void cancel() { @Test public void requestToFlowable() { - TestSubscriber ts = new TestSubscriber(3L); + TestSubscriber ts = new TestSubscriber<>(3L); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @Override @@ -284,7 +284,7 @@ public void cancel() { @Test public void requestThroughMap() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); ts.request(3); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -309,7 +309,7 @@ public void cancel() { @Test public void requestThroughTakeThatReducesRequest() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); ts.request(3); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -336,7 +336,7 @@ public void cancel() { @Test public void requestThroughTakeWhereRequestIsSmallerThanTake() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); ts.request(3); final AtomicLong requested = new AtomicLong(); Flowable.unsafeCreate(new Publisher() { @@ -463,7 +463,7 @@ public void onNext(Integer t) { @Test public void onStartRequestsAreAdditive() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.just(1, 2, 3, 4, 5) .subscribe(new DefaultSubscriber() { @Override @@ -491,7 +491,7 @@ public void onNext(Integer t) { @Test public void onStartRequestsAreAdditiveAndOverflowBecomesMaxValue() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.just(1, 2, 3, 4, 5).subscribe(new DefaultSubscriber() { @Override public void onStart() { @@ -520,7 +520,7 @@ public void onNext(Integer t) { public void forEachWhile() { PublishProcessor pp = PublishProcessor.create(); - final List list = new ArrayList(); + final List list = new ArrayList<>(); Disposable d = pp.forEachWhile(new Predicate() { @Override @@ -543,7 +543,7 @@ public boolean test(Integer v) throws Exception { @Test public void doubleSubscribe() { - ForEachWhileSubscriber s = new ForEachWhileSubscriber(new Predicate() { + ForEachWhileSubscriber s = new ForEachWhileSubscriber<>(new Predicate() { @Override public boolean test(Integer v) throws Exception { return true; @@ -570,10 +570,10 @@ public boolean test(Integer v) throws Exception { public void suppressAfterCompleteEvents() { List errors = TestHelper.trackPluginErrors(); try { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); - ForEachWhileSubscriber s = new ForEachWhileSubscriber(new Predicate() { + ForEachWhileSubscriber s = new ForEachWhileSubscriber<>(new Predicate() { @Override public boolean test(Integer v) throws Exception { ts.onNext(v); @@ -606,10 +606,10 @@ public void run() throws Exception { @Test public void onNextCrashes() { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); - ForEachWhileSubscriber s = new ForEachWhileSubscriber(new Predicate() { + ForEachWhileSubscriber s = new ForEachWhileSubscriber<>(new Predicate() { @Override public boolean test(Integer v) throws Exception { throw new TestException(); @@ -637,7 +637,7 @@ public void run() throws Exception { @Test public void onErrorThrows() { - ForEachWhileSubscriber s = new ForEachWhileSubscriber(new Predicate() { + ForEachWhileSubscriber s = new ForEachWhileSubscriber<>(new Predicate() { @Override public boolean test(Integer v) throws Exception { return true; @@ -672,7 +672,7 @@ public void run() throws Exception { @Test public void onCompleteThrows() { - ForEachWhileSubscriber s = new ForEachWhileSubscriber(new Predicate() { + ForEachWhileSubscriber s = new ForEachWhileSubscriber<>(new Predicate() { @Override public boolean test(Integer v) throws Exception { return true; @@ -703,7 +703,7 @@ public void run() throws Exception { @Test public void subscribeConsumerConsumerWithError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.error(new TestException()).subscribe(new Consumer() { @Override @@ -731,8 +731,8 @@ public void methodTestCancelled() { @Test public void safeSubscriberAlreadySafe() { - TestSubscriber ts = new TestSubscriber(); - Flowable.just(1).safeSubscribe(new SafeSubscriber(ts)); + TestSubscriber ts = new TestSubscriber<>(); + Flowable.just(1).safeSubscribe(new SafeSubscriber<>(ts)); ts.assertResult(1); } @@ -748,7 +748,7 @@ public void methodTestNoCancel() { @Test public void subscribeConsumerConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Flowable.just(1).subscribe(new Consumer() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java index e61e106f1a..616140c6b9 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java @@ -71,7 +71,7 @@ public void fromArray() { @Test public void fromIterable() { - ArrayList items = new ArrayList(); + ArrayList items = new ArrayList<>(); items.add("one"); items.add("two"); items.add("three"); @@ -350,7 +350,7 @@ public void materializeDematerializeChaining() { public void customObservableWithErrorInObserverAsynchronous() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); // FIXME custom built??? Flowable.just("1", "2", "three", "4") @@ -398,7 +398,7 @@ public void onNext(String v) { @Test public void customObservableWithErrorInObserverSynchronous() { final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); // FIXME custom built??? Flowable.just("1", "2", "three", "4") @@ -440,7 +440,7 @@ public void onNext(String v) { @Test public void customObservableWithErrorInObservableSynchronous() { final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); // FIXME custom built??? Flowable.just("1", "2").concatWith(Flowable.error(new Supplier() { @Override @@ -660,7 +660,7 @@ public void accept(String v) { @Test public void takeWithErrorInObserver() { final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); Flowable.just("1", "2", "three", "4").take(3) .safeSubscribe(new DefaultSubscriber() { @@ -710,9 +710,9 @@ public void ofType() { @Test public void ofTypeWithPolymorphism() { - ArrayList l1 = new ArrayList(); + ArrayList l1 = new ArrayList<>(); l1.add(1); - LinkedList l2 = new LinkedList(); + LinkedList l2 = new LinkedList<>(); l2.add(2); @SuppressWarnings("rawtypes") @@ -898,21 +898,21 @@ public void rangeWithScheduler() { @Test public void mergeWith() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).mergeWith(Flowable.just(2)).subscribe(ts); ts.assertValues(1, 2); } @Test public void concatWith() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).concatWith(Flowable.just(2)).subscribe(ts); ts.assertValues(1, 2); } @Test public void ambWith() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).ambWith(Flowable.just(2)).subscribe(ts); ts.assertValue(1); } @@ -944,7 +944,7 @@ public void accept(List booleans) { @Test public void compose() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1, 2, 3).compose(new FlowableTransformer() { @Override public Publisher apply(Flowable t1) { @@ -1008,7 +1008,7 @@ public void forEachWithNull() { @Test public void extend() { - final TestSubscriber subscriber = new TestSubscriber(); + final TestSubscriber subscriber = new TestSubscriber<>(); final Object value = new Object(); Object returned = Flowable.just(value).to(new FlowableConverter() { @Override @@ -1025,7 +1025,7 @@ public Object apply(Flowable onSubscribe) { @Test public void asExtend() { - final TestSubscriber subscriber = new TestSubscriber(); + final TestSubscriber subscriber = new TestSubscriber<>(); final Object value = new Object(); Object returned = Flowable.just(value).to(new FlowableConverter() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableWindowTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableWindowTests.java index d43b96f17f..f96a010728 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableWindowTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableWindowTests.java @@ -30,7 +30,7 @@ public class FlowableWindowTests extends RxJavaTest { @Test public void window() { - final ArrayList> lists = new ArrayList>(); + final ArrayList> lists = new ArrayList<>(); Flowable.concat( Flowable.just(1, 2, 3, 4, 5, 6) diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableZipTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableZipTests.java index e022e02c82..e1b2e2809b 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableZipTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableZipTests.java @@ -41,7 +41,7 @@ public String apply(Event e) { .flatMap(new Function, Publisher>>() { @Override public Publisher> apply(final GroupedFlowable ge) { - return ge.scan(new HashMap(), new BiFunction, Event, HashMap>() { + return ge.scan(new HashMap<>(), new BiFunction, Event, HashMap>() { @Override public HashMap apply(HashMap accum, Event perInstanceEvent) { diff --git a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java index 69ea889363..e4d64764d8 100644 --- a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java @@ -896,7 +896,7 @@ public void run() throws Exception { @Test public void observeOnDispose() throws Exception { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final CountDownLatch cdl = new CountDownLatch(1); @@ -2282,7 +2282,7 @@ public void subscribeZeroError() { @Test public void subscribeToOnSuccess() { - final List values = new ArrayList(); + final List values = new ArrayList<>(); Consumer onSuccess = new Consumer() { @Override @@ -2302,7 +2302,7 @@ public void accept(Integer e) throws Exception { @Test public void subscribeToOnError() { - final List values = new ArrayList(); + final List values = new ArrayList<>(); Consumer onError = new Consumer() { @Override @@ -2323,7 +2323,7 @@ public void accept(Throwable e) throws Exception { @Test public void subscribeToOnComplete() { - final List values = new ArrayList(); + final List values = new ArrayList<>(); Action onComplete = new Action() { @Override @@ -2370,7 +2370,7 @@ public void onComplete() { @Test public void doOnEventSuccess() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); assertTrue(Maybe.just(1) .doOnEvent(new BiConsumer() { @@ -2389,7 +2389,7 @@ public void accept(Integer v, Throwable e) throws Exception { public void doOnEventError() { List errors = TestHelper.trackPluginErrors(); try { - final List list = new ArrayList(); + final List list = new ArrayList<>(); TestException ex = new TestException(); @@ -2413,7 +2413,7 @@ public void accept(Integer v, Throwable e) throws Exception { @Test public void doOnEventComplete() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); assertTrue(Maybe.empty() .doOnEvent(new BiConsumer() { diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableCovarianceTest.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableCovarianceTest.java index 904af1d8a4..e7e3302fd8 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableCovarianceTest.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableCovarianceTest.java @@ -66,7 +66,7 @@ public int compare(Media t1, Media t2) { @Test public void groupByCompose() { Observable movies = Observable.just(new HorrorMovie(), new ActionMovie(), new Movie()); - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); movies .groupBy(new Function() { @Override @@ -187,9 +187,9 @@ public Observable apply(List> listOfLists) { } else { // diff the two List newList = listOfLists.get(1); - List oldList = new ArrayList(listOfLists.get(0)); + List oldList = new ArrayList<>(listOfLists.get(0)); - Set delta = new LinkedHashSet(); + Set delta = new LinkedHashSet<>(); delta.addAll(newList); // remove all that match in old delta.removeAll(oldList); @@ -211,7 +211,7 @@ public Observable apply(List> listOfLists) { @Override public Observable apply(Observable> movieList) { return movieList - .startWithItem(new ArrayList()) + .startWithItem(new ArrayList<>()) .buffer(2, 1) .skip(1) .flatMap(calculateDelta); diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableDoOnTest.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableDoOnTest.java index 1e28d19b24..6c3b5f54ab 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableDoOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableDoOnTest.java @@ -27,7 +27,7 @@ public class ObservableDoOnTest extends RxJavaTest { @Test public void doOnEach() { - final AtomicReference r = new AtomicReference(); + final AtomicReference r = new AtomicReference<>(); String output = Observable.just("one").doOnNext(new Consumer() { @Override public void accept(String v) { @@ -41,7 +41,7 @@ public void accept(String v) { @Test public void doOnError() { - final AtomicReference r = new AtomicReference(); + final AtomicReference r = new AtomicReference<>(); Throwable t = null; try { Observable. error(new RuntimeException("an error")) diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableErrorHandlingTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableErrorHandlingTests.java index bbbc80daed..c524261032 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableErrorHandlingTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableErrorHandlingTests.java @@ -33,7 +33,7 @@ public class ObservableErrorHandlingTests extends RxJavaTest { @Test public void onNextError() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference caughtError = new AtomicReference(); + final AtomicReference caughtError = new AtomicReference<>(); Observable o = Observable.interval(50, TimeUnit.MILLISECONDS); Observer observer = new DefaultObserver() { @@ -69,7 +69,7 @@ public void onNext(Long args) { @Test public void onNextErrorAcrossThread() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference caughtError = new AtomicReference(); + final AtomicReference caughtError = new AtomicReference<>(); Observable o = Observable.interval(50, TimeUnit.MILLISECONDS); Observer observer = new DefaultObserver() { diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableEventStream.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableEventStream.java index c60f694209..6b161dcb0a 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableEventStream.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableEventStream.java @@ -33,7 +33,7 @@ public static Observable getEventStream(final String type, final int numI } public static Event randomEvent(String type, int numInstances) { - Map values = new LinkedHashMap(); + Map values = new LinkedHashMap<>(); values.put("count200", randomIntFrom0to(4000)); values.put("count4xx", randomIntFrom0to(300)); values.put("count5xx", randomIntFrom0to(500)); diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java index 211053b0c7..5e02ecc3e7 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java @@ -289,10 +289,10 @@ public void fromFutureNull() { @Test public void fromFutureReturnsNull() { - FutureTask f = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); f.run(); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.fromFuture(f).subscribe(to); to.assertNoValues(); to.assertNotComplete(); @@ -306,24 +306,24 @@ public void fromFutureTimedFutureNull() { @Test(expected = NullPointerException.class) public void fromFutureTimedUnitNull() { - Observable.fromFuture(new FutureTask(Functions.EMPTY_RUNNABLE, null), 1, null); + Observable.fromFuture(new FutureTask<>(Functions.EMPTY_RUNNABLE, null), 1, null); } @Test(expected = NullPointerException.class) public void fromFutureTimedSchedulerNull() { - Observable.fromFuture(new FutureTask(Functions.EMPTY_RUNNABLE, null), 1, TimeUnit.SECONDS, null); + Observable.fromFuture(new FutureTask<>(Functions.EMPTY_RUNNABLE, null), 1, TimeUnit.SECONDS, null); } @Test(expected = NullPointerException.class) public void fromFutureTimedReturnsNull() { - FutureTask f = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); f.run(); Observable.fromFuture(f, 1, TimeUnit.SECONDS).blockingLast(); } @Test(expected = NullPointerException.class) public void fromFutureSchedulerNull() { - FutureTask f = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); Observable.fromFuture(f, null); } @@ -2385,7 +2385,7 @@ public Integer apply(Integer v) { }, new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }, null); } @@ -2405,7 +2405,7 @@ public Integer apply(Integer v) { }, new Supplier>>() { @Override public Map> get() { - return new HashMap>(); + return new HashMap<>(); } }, new Function>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableScanTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableScanTests.java index 195c7d5865..bc73a886b2 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableScanTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableScanTests.java @@ -27,7 +27,7 @@ public class ObservableScanTests extends RxJavaTest { public void unsubscribeScan() throws Exception { ObservableEventStream.getEventStream("HTTP-ClusterB", 20) - .scan(new HashMap(), new BiFunction, Event, HashMap>() { + .scan(new HashMap<>(), new BiFunction, Event, HashMap>() { @Override public HashMap apply(HashMap accum, Event perInstanceEvent) { accum.put("instance", perInstanceEvent.instanceId); diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableStartWithTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableStartWithTests.java index c8a195d826..9c9261273c 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableStartWithTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableStartWithTests.java @@ -35,7 +35,7 @@ public void startWith1() { @Test public void startWithIterable() { - List li = new ArrayList(); + List li = new ArrayList<>(); li.add("alpha"); li.add("beta"); List values = Observable.just("one", "two").startWithIterable(li).toList().blockingGet(); @@ -48,7 +48,7 @@ public void startWithIterable() { @Test public void startWithObservable() { - List li = new ArrayList(); + List li = new ArrayList<>(); li.add("alpha"); li.add("beta"); List values = Observable.just("one", "two") diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableSubscriberTest.java index 6fc73f4468..60e8470ca4 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableSubscriberTest.java @@ -128,7 +128,7 @@ public void onNext(Integer t) { @Test public void subscribeConsumerConsumer() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.just(1).subscribe(new Consumer() { @Override @@ -147,7 +147,7 @@ public void accept(Throwable e) throws Exception { @Test public void subscribeConsumerConsumerWithError() { - final List list = new ArrayList(); + final List list = new ArrayList<>(); Observable.error(new TestException()).subscribe(new Consumer() { @Override @@ -175,8 +175,8 @@ public void methodTestCancelled() { @Test public void safeSubscriberAlreadySafe() { - TestObserver to = new TestObserver(); - Observable.just(1).safeSubscribe(new SafeObserver(to)); + TestObserver to = new TestObserver<>(); + Observable.just(1).safeSubscribe(new SafeObserver<>(to)); to.assertResult(1); } diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java index a3d79a426c..4b888d4ed5 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java @@ -66,7 +66,7 @@ public void fromArray() { @Test public void fromIterable() { - ArrayList items = new ArrayList(); + ArrayList items = new ArrayList<>(); items.add("one"); items.add("two"); items.add("three"); @@ -367,7 +367,7 @@ public void materializeDematerializeChaining() { public void customObservableWithErrorInObserverAsynchronous() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); // FIXME custom built??? Observable.just("1", "2", "three", "4") @@ -415,7 +415,7 @@ public void onNext(String v) { @Test public void customObservableWithErrorInObserverSynchronous() { final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); // FIXME custom built??? Observable.just("1", "2", "three", "4") @@ -458,7 +458,7 @@ public void onNext(String v) { @Test public void customObservableWithErrorInObservableSynchronous() { final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); // FIXME custom built??? Observable.just("1", "2").concatWith(Observable.error(new Supplier() { @Override @@ -678,7 +678,7 @@ public void accept(String v) { @Test public void takeWithErrorInObserver() { final AtomicInteger count = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); Observable.just("1", "2", "three", "4").take(3) .safeSubscribe(new DefaultObserver() { @@ -729,9 +729,9 @@ public void ofType() { @Test public void ofTypeWithPolymorphism() { - ArrayList l1 = new ArrayList(); + ArrayList l1 = new ArrayList<>(); l1.add(1); - LinkedList l2 = new LinkedList(); + LinkedList l2 = new LinkedList<>(); l2.add(2); @SuppressWarnings("rawtypes") @@ -921,21 +921,21 @@ public void rangeWithScheduler() { @Test public void mergeWith() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1).mergeWith(Observable.just(2)).subscribe(to); to.assertValues(1, 2); } @Test public void concatWith() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1).concatWith(Observable.just(2)).subscribe(to); to.assertValues(1, 2); } @Test public void ambWith() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1).ambWith(Observable.just(2)).subscribe(to); to.assertValue(1); } @@ -967,7 +967,7 @@ public void accept(List booleans) { @Test public void compose() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1, 2, 3).compose(new ObservableTransformer() { @Override @@ -1048,7 +1048,7 @@ public void forEachWithNull() { @Test public void extend() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); final Object value = new Object(); Object returned = Observable.just(value).to(new ObservableConverter() { @Override @@ -1065,7 +1065,7 @@ public Object apply(Observable onSubscribe) { @Test public void asExtend() { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); final Object value = new Object(); Object returned = Observable.just(value).to(new ObservableConverter() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableWindowTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableWindowTests.java index 1d4a62f7f3..9e1076017a 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableWindowTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableWindowTests.java @@ -31,7 +31,7 @@ public class ObservableWindowTests extends RxJavaTest { @Test public void window() { - final ArrayList> lists = new ArrayList>(); + final ArrayList> lists = new ArrayList<>(); Observable.concat( Observable.just(1, 2, 3, 4, 5, 6) diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableZipTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableZipTests.java index b630807758..1f67113348 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableZipTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableZipTests.java @@ -41,7 +41,7 @@ public String apply(Event e) { .flatMap(new Function, Observable>>() { @Override public Observable> apply(final GroupedObservable ge) { - return ge.scan(new HashMap(), new BiFunction, Event, HashMap>() { + return ge.scan(new HashMap<>(), new BiFunction, Event, HashMap>() { @Override public HashMap apply(HashMap accum, Event perInstanceEvent) { diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java index 84db3819ae..19ee3e7f06 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableCompletableObserverTest.java @@ -33,7 +33,7 @@ static final class TestCompletable extends DisposableCompletableObserver { int complete; - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); @Override protected void onStart() { diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java index 42cbd25c5d..36dc6a6d3b 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableMaybeObserverTest.java @@ -31,9 +31,9 @@ static final class TestMaybe extends DisposableMaybeObserver { int start; - final List values = new ArrayList(); + final List values = new ArrayList<>(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int complete; @@ -62,7 +62,7 @@ public void onComplete() { @Test public void normal() { - TestMaybe tc = new TestMaybe(); + TestMaybe tc = new TestMaybe<>(); assertFalse(tc.isDisposed()); assertEquals(0, tc.start); @@ -85,7 +85,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestMaybe tc = new TestMaybe(); + TestMaybe tc = new TestMaybe<>(); tc.onSubscribe(Disposable.empty()); @@ -105,7 +105,7 @@ public void startOnce() { @Test public void dispose() { - TestMaybe tc = new TestMaybe(); + TestMaybe tc = new TestMaybe<>(); tc.dispose(); assertTrue(tc.isDisposed()); diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java index b223e47b48..c2b766e761 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableObserverTest.java @@ -32,9 +32,9 @@ static final class TestDisposableObserver extends DisposableObserver { int start; - final List values = new ArrayList(); + final List values = new ArrayList<>(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int completions; @@ -63,7 +63,7 @@ public void onComplete() { @Test public void normal() { - TestDisposableObserver tc = new TestDisposableObserver(); + TestDisposableObserver tc = new TestDisposableObserver<>(); assertFalse(tc.isDisposed()); assertEquals(0, tc.start); @@ -84,7 +84,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestDisposableObserver tc = new TestDisposableObserver(); + TestDisposableObserver tc = new TestDisposableObserver<>(); tc.onSubscribe(Disposable.empty()); @@ -104,7 +104,7 @@ public void startOnce() { @Test public void dispose() { - TestDisposableObserver tc = new TestDisposableObserver(); + TestDisposableObserver tc = new TestDisposableObserver<>(); assertFalse(tc.isDisposed()); diff --git a/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java index 6bf84e21d3..b6239d65ee 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/DisposableSingleObserverTest.java @@ -31,9 +31,9 @@ static final class TestSingle extends DisposableSingleObserver { int start; - final List values = new ArrayList(); + final List values = new ArrayList<>(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); @Override protected void onStart() { @@ -56,7 +56,7 @@ public void onError(Throwable e) { @Test public void normal() { - TestSingle tc = new TestSingle(); + TestSingle tc = new TestSingle<>(); assertFalse(tc.isDisposed()); assertEquals(0, tc.start); @@ -77,7 +77,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestSingle tc = new TestSingle(); + TestSingle tc = new TestSingle<>(); tc.onSubscribe(Disposable.empty()); @@ -97,7 +97,7 @@ public void startOnce() { @Test public void dispose() { - TestSingle tc = new TestSingle(); + TestSingle tc = new TestSingle<>(); tc.dispose(); assertTrue(tc.isDisposed()); diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java index 6564ae4d16..f007940592 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceCompletableObserverTest.java @@ -28,7 +28,7 @@ public class ResourceCompletableObserverTest extends RxJavaTest { static final class TestResourceCompletableObserver extends ResourceCompletableObserver { - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int complete; diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java index 04a093d753..08d06044e7 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceMaybeObserverTest.java @@ -30,7 +30,7 @@ public class ResourceMaybeObserverTest extends RxJavaTest { static final class TestResourceMaybeObserver extends ResourceMaybeObserver { T value; - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int complete; @@ -67,13 +67,13 @@ public void onError(Throwable e) { @Test(expected = NullPointerException.class) public void nullResource() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); rmo.add(null); } @Test public void addResources() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); assertFalse(rmo.isDisposed()); @@ -98,7 +98,7 @@ public void addResources() { @Test public void onCompleteCleansUp() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); assertFalse(rmo.isDisposed()); @@ -117,7 +117,7 @@ public void onCompleteCleansUp() { @Test public void onSuccessCleansUp() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); assertFalse(rmo.isDisposed()); @@ -136,7 +136,7 @@ public void onSuccessCleansUp() { @Test public void onErrorCleansUp() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); assertFalse(rmo.isDisposed()); @@ -155,7 +155,7 @@ public void onErrorCleansUp() { @Test public void normal() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); assertFalse(rmo.isDisposed()); assertEquals(0, rmo.start); @@ -173,7 +173,7 @@ public void normal() { @Test public void empty() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); assertFalse(rmo.isDisposed()); assertEquals(0, rmo.start); @@ -191,7 +191,7 @@ public void empty() { @Test public void error() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); assertFalse(rmo.isDisposed()); assertEquals(0, rmo.start); @@ -215,7 +215,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); rmo.onSubscribe(Disposable.empty()); @@ -235,7 +235,7 @@ public void startOnce() { @Test public void dispose() { - TestResourceMaybeObserver rmo = new TestResourceMaybeObserver(); + TestResourceMaybeObserver rmo = new TestResourceMaybeObserver<>(); rmo.dispose(); Disposable d = Disposable.empty(); diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java index e0ff866e75..94934d82df 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceObserverTest.java @@ -30,9 +30,9 @@ public class ResourceObserverTest extends RxJavaTest { static final class TestResourceObserver extends ResourceObserver { - final List values = new ArrayList(); + final List values = new ArrayList<>(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int complete; @@ -67,13 +67,13 @@ public void onComplete() { @Test(expected = NullPointerException.class) public void nullResource() { - TestResourceObserver ro = new TestResourceObserver(); + TestResourceObserver ro = new TestResourceObserver<>(); ro.add(null); } @Test public void addResources() { - TestResourceObserver ro = new TestResourceObserver(); + TestResourceObserver ro = new TestResourceObserver<>(); assertFalse(ro.isDisposed()); @@ -98,7 +98,7 @@ public void addResources() { @Test public void onCompleteCleansUp() { - TestResourceObserver ro = new TestResourceObserver(); + TestResourceObserver ro = new TestResourceObserver<>(); assertFalse(ro.isDisposed()); @@ -117,7 +117,7 @@ public void onCompleteCleansUp() { @Test public void onErrorCleansUp() { - TestResourceObserver ro = new TestResourceObserver(); + TestResourceObserver ro = new TestResourceObserver<>(); assertFalse(ro.isDisposed()); @@ -136,7 +136,7 @@ public void onErrorCleansUp() { @Test public void normal() { - TestResourceObserver tc = new TestResourceObserver(); + TestResourceObserver tc = new TestResourceObserver<>(); assertFalse(tc.isDisposed()); assertEquals(0, tc.start); @@ -153,7 +153,7 @@ public void normal() { @Test public void error() { - TestResourceObserver tc = new TestResourceObserver(); + TestResourceObserver tc = new TestResourceObserver<>(); assertFalse(tc.isDisposed()); assertEquals(0, tc.start); @@ -176,7 +176,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestResourceObserver tc = new TestResourceObserver(); + TestResourceObserver tc = new TestResourceObserver<>(); tc.onSubscribe(Disposable.empty()); @@ -196,7 +196,7 @@ public void startOnce() { @Test public void dispose() { - TestResourceObserver tc = new TestResourceObserver(); + TestResourceObserver tc = new TestResourceObserver<>(); tc.dispose(); Disposable d = Disposable.empty(); diff --git a/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java index 054f189bc5..b720931088 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/ResourceSingleObserverTest.java @@ -30,7 +30,7 @@ public class ResourceSingleObserverTest extends RxJavaTest { static final class TestResourceSingleObserver extends ResourceSingleObserver { T value; - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int start; @@ -58,13 +58,13 @@ public void onError(Throwable e) { @Test(expected = NullPointerException.class) public void nullResource() { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); rso.add(null); } @Test public void addResources() { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); assertFalse(rso.isDisposed()); @@ -89,7 +89,7 @@ public void addResources() { @Test public void onSuccessCleansUp() { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); assertFalse(rso.isDisposed()); @@ -108,7 +108,7 @@ public void onSuccessCleansUp() { @Test public void onErrorCleansUp() { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); assertFalse(rso.isDisposed()); @@ -127,7 +127,7 @@ public void onErrorCleansUp() { @Test public void normal() { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); assertFalse(rso.isDisposed()); assertEquals(0, rso.start); @@ -144,7 +144,7 @@ public void normal() { @Test public void error() { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); assertFalse(rso.isDisposed()); assertEquals(0, rso.start); @@ -167,7 +167,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); rso.onSubscribe(Disposable.empty()); @@ -187,7 +187,7 @@ public void startOnce() { @Test public void dispose() { - TestResourceSingleObserver rso = new TestResourceSingleObserver(); + TestResourceSingleObserver rso = new TestResourceSingleObserver<>(); rso.dispose(); Disposable d = Disposable.empty(); diff --git a/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java index 446dfe20df..3afced2d40 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/SafeObserverTest.java @@ -30,7 +30,7 @@ public class SafeObserverTest extends RxJavaTest { @Test public void onNextFailure() { - AtomicReference onError = new AtomicReference(); + AtomicReference onError = new AtomicReference<>(); try { OBSERVER_ONNEXT_FAIL(onError).onNext("one"); fail("expects exception to be thrown"); @@ -43,9 +43,9 @@ public void onNextFailure() { @Test public void onNextFailureSafe() { - AtomicReference onError = new AtomicReference(); + AtomicReference onError = new AtomicReference<>(); try { - SafeObserver safeObserver = new SafeObserver(OBSERVER_ONNEXT_FAIL(onError)); + SafeObserver safeObserver = new SafeObserver<>(OBSERVER_ONNEXT_FAIL(onError)); safeObserver.onSubscribe(Disposable.empty()); safeObserver.onNext("one"); assertNotNull(onError.get()); @@ -58,7 +58,7 @@ public void onNextFailureSafe() { @Test public void onCompleteFailure() { - AtomicReference onError = new AtomicReference(); + AtomicReference onError = new AtomicReference<>(); try { OBSERVER_ONCOMPLETED_FAIL(onError).onComplete(); fail("expects exception to be thrown"); @@ -198,16 +198,16 @@ public void onError(Throwable e) { public void onComplete() { } }; - SafeObserver observer = new SafeObserver(actual); + SafeObserver observer = new SafeObserver<>(actual); assertSame(actual, observer.downstream); } @Test public void dispose() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - SafeObserver so = new SafeObserver(to); + SafeObserver so = new SafeObserver<>(to); Disposable d = Disposable.empty(); @@ -222,9 +222,9 @@ public void dispose() { @Test public void onNextAfterComplete() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - SafeObserver so = new SafeObserver(to); + SafeObserver so = new SafeObserver<>(to); Disposable d = Disposable.empty(); @@ -243,9 +243,9 @@ public void onNextAfterComplete() { @Test public void onNextNull() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - SafeObserver so = new SafeObserver(to); + SafeObserver so = new SafeObserver<>(to); Disposable d = Disposable.empty(); @@ -258,9 +258,9 @@ public void onNextNull() { @Test public void onNextWithoutOnSubscribe() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); - SafeObserver so = new SafeObserver(to); + SafeObserver so = new SafeObserver<>(to); so.onNext(1); @@ -269,9 +269,9 @@ public void onNextWithoutOnSubscribe() { @Test public void onErrorWithoutOnSubscribe() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); - SafeObserver so = new SafeObserver(to); + SafeObserver so = new SafeObserver<>(to); so.onError(new TestException()); @@ -283,9 +283,9 @@ public void onErrorWithoutOnSubscribe() { @Test public void onCompleteWithoutOnSubscribe() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); - SafeObserver so = new SafeObserver(to); + SafeObserver so = new SafeObserver<>(to); so.onComplete(); @@ -294,9 +294,9 @@ public void onCompleteWithoutOnSubscribe() { @Test public void onNextNormal() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - SafeObserver so = new SafeObserver(to); + SafeObserver so = new SafeObserver<>(to); Disposable d = Disposable.empty(); @@ -369,7 +369,7 @@ public void onComplete() { } public SafeObserver toSafe() { - return new SafeObserver(this); + return new SafeObserver<>(this); } public CrashDummy assertError(Class clazz) { diff --git a/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java index 84fa434879..e581012d9b 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/SerializedObserverTest.java @@ -41,7 +41,7 @@ public void before() { } private Observer serializedObserver(Observer o) { - return new SerializedObserver(o); + return new SerializedObserver<>(o); } @Test @@ -163,7 +163,7 @@ public void runOutOfOrderConcurrencyTest() { try { TestConcurrencySubscriber tw = new TestConcurrencySubscriber(); // we need Synchronized + SafeObserver to handle synchronization plus life-cycle - Observer w = serializedObserver(new SafeObserver(tw)); + Observer w = serializedObserver(new SafeObserver<>(tw)); Future f1 = tp.submit(new OnNextThread(w, 12000)); Future f2 = tp.submit(new OnNextThread(w, 5000)); @@ -219,7 +219,7 @@ public void runConcurrencyTest() { try { TestConcurrencySubscriber tw = new TestConcurrencySubscriber(); // we need Synchronized + SafeObserver to handle synchronization plus life-cycle - Observer w = serializedObserver(new SafeObserver(tw)); + Observer w = serializedObserver(new SafeObserver<>(tw)); w.onSubscribe(Disposable.empty()); Future f1 = tp.submit(new OnNextThread(w, 12000)); @@ -275,7 +275,7 @@ public void notificationDelay() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch running = new CountDownLatch(2); - TestObserverEx to = new TestObserverEx(new DefaultObserver() { + TestObserverEx to = new TestObserverEx<>(new DefaultObserver() { @Override public void onComplete() { @@ -356,7 +356,7 @@ public void onNext(String t) { @Test public void threadStarvation() throws InterruptedException { - TestObserver to = new TestObserver(new DefaultObserver() { + TestObserver to = new TestObserver<>(new DefaultObserver() { @Override public void onComplete() { @@ -552,7 +552,7 @@ private static class TestConcurrencySubscriber extends DefaultObserver { /** * used to store the order and number of events received. */ - private final LinkedBlockingQueue events = new LinkedBlockingQueue(); + private final LinkedBlockingQueue events = new LinkedBlockingQueue<>(); private final int waitTime; @SuppressWarnings("unused") @@ -848,7 +848,7 @@ protected void captureMaxThreads() { public void errorReentry() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> serial = new AtomicReference>(); + final AtomicReference> serial = new AtomicReference<>(); TestObserver to = new TestObserver() { @Override @@ -858,7 +858,7 @@ public void onNext(Integer v) { super.onNext(v); } }; - SerializedObserver sobs = new SerializedObserver(to); + SerializedObserver sobs = new SerializedObserver<>(to); sobs.onSubscribe(Disposable.empty()); serial.set(sobs); @@ -875,7 +875,7 @@ public void onNext(Integer v) { @Test public void completeReentry() { - final AtomicReference> serial = new AtomicReference>(); + final AtomicReference> serial = new AtomicReference<>(); TestObserver to = new TestObserver() { @Override @@ -885,7 +885,7 @@ public void onNext(Integer v) { super.onNext(v); } }; - SerializedObserver sobs = new SerializedObserver(to); + SerializedObserver sobs = new SerializedObserver<>(to); sobs.onSubscribe(Disposable.empty()); serial.set(sobs); @@ -898,9 +898,9 @@ public void onNext(Integer v) { @Test public void dispose() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - SerializedObserver so = new SerializedObserver(to); + SerializedObserver so = new SerializedObserver<>(to); Disposable d = Disposable.empty(); @@ -918,9 +918,9 @@ public void dispose() { @Test public void onCompleteRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final SerializedObserver so = new SerializedObserver(to); + final SerializedObserver so = new SerializedObserver<>(to); Disposable d = Disposable.empty(); @@ -944,9 +944,9 @@ public void run() { @Test public void onNextOnCompleteRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final SerializedObserver so = new SerializedObserver(to); + final SerializedObserver so = new SerializedObserver<>(to); Disposable d = Disposable.empty(); @@ -980,9 +980,9 @@ public void run() { @Test public void onNextOnErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final SerializedObserver so = new SerializedObserver(to); + final SerializedObserver so = new SerializedObserver<>(to); Disposable d = Disposable.empty(); @@ -1018,9 +1018,9 @@ public void run() { @Test public void onNextOnErrorRaceDelayError() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final SerializedObserver so = new SerializedObserver(to, true); + final SerializedObserver so = new SerializedObserver<>(to, true); Disposable d = Disposable.empty(); @@ -1059,9 +1059,9 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final SerializedObserver so = new SerializedObserver(to); + final SerializedObserver so = new SerializedObserver<>(to); so.onSubscribe(Disposable.empty()); @@ -1083,9 +1083,9 @@ public void onCompleteOnErrorRace() { List errors = TestHelper.trackPluginErrors(); try { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); - final SerializedObserver so = new SerializedObserver(to); + final SerializedObserver so = new SerializedObserver<>(to); Disposable d = Disposable.empty(); @@ -1130,9 +1130,9 @@ public void run() { @Test public void nullOnNext() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); - final SerializedObserver so = new SerializedObserver(to); + final SerializedObserver so = new SerializedObserver<>(to); Disposable d = Disposable.empty(); diff --git a/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java b/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java index 581aa0b0b1..1e4a287cee 100644 --- a/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java +++ b/src/test/java/io/reactivex/rxjava3/observers/TestObserverTest.java @@ -46,7 +46,7 @@ public class TestObserverTest extends RxJavaTest { @Test public void assertTestObserver() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); oi.subscribe(subscriber); subscriber.assertValues(1, 2); @@ -57,7 +57,7 @@ public void assertTestObserver() { @Test public void assertNotMatchCount() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); oi.subscribe(subscriber); thrown.expect(AssertionError.class); @@ -72,7 +72,7 @@ public void assertNotMatchCount() { @Test public void assertNotMatchValue() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); oi.subscribe(subscriber); thrown.expect(AssertionError.class); @@ -87,7 +87,7 @@ public void assertNotMatchValue() { @Test public void assertTerminalEventNotReceived() { PublishProcessor p = PublishProcessor.create(); - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); p.subscribe(subscriber); p.onNext(1); @@ -108,7 +108,7 @@ public void wrappingMock() { Subscriber mockSubscriber = TestHelper.mockSubscriber(); - oi.subscribe(new TestSubscriber(mockSubscriber)); + oi.subscribe(new TestSubscriber<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -121,7 +121,7 @@ public void wrappingMock() { public void wrappingMockWhenUnsubscribeInvolved() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)).take(2); Subscriber mockSubscriber = TestHelper.mockSubscriber(); - oi.subscribe(new TestSubscriber(mockSubscriber)); + oi.subscribe(new TestSubscriber<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -132,12 +132,12 @@ public void wrappingMockWhenUnsubscribeInvolved() { @Test public void errorSwallowed() { - Flowable.error(new RuntimeException()).subscribe(new TestSubscriber()); + Flowable.error(new RuntimeException()).subscribe(new TestSubscriber<>()); } @Test public void nullExpected() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onNext(1); try { @@ -151,7 +151,7 @@ public void nullExpected() { @Test public void nullActual() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onNext(null); try { @@ -334,7 +334,7 @@ public void assertFuseable() { to = TestObserver.create(); - to.onSubscribe(new ScalarDisposable(to, 1)); + to.onSubscribe(new ScalarDisposable<>(to, 1)); } @Test @@ -645,7 +645,7 @@ public void assertValueSequence() { @Test public void assertEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); try { to.assertEmpty(); @@ -670,7 +670,7 @@ public void assertEmpty() { @Test public void awaitDoneTimed() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Thread.currentThread().interrupt(); @@ -683,7 +683,7 @@ public void awaitDoneTimed() { @Test public void assertErrorMultiple() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); TestException e = new TestException(); to.onError(e); @@ -711,7 +711,7 @@ public void assertErrorMultiple() { @Test public void errorInPredicate() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onError(new RuntimeException()); try { to.assertError(new Predicate() { @@ -729,7 +729,7 @@ public boolean test(Throwable throwable) throws Exception { @Test public void assertComplete() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onSubscribe(Disposable.empty()); @@ -756,7 +756,7 @@ public void assertComplete() { @Test public void completeWithoutOnSubscribe() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); to.onComplete(); @@ -765,7 +765,7 @@ public void completeWithoutOnSubscribe() { @Test public void completeDelegateThrows() { - TestObserver to = new TestObserver(new Observer() { + TestObserver to = new TestObserver<>(new Observer() { @Override public void onSubscribe(Disposable d) { @@ -801,7 +801,7 @@ public void onComplete() { @Test public void errorDelegateThrows() { - TestObserver to = new TestObserver(new Observer() { + TestObserver to = new TestObserver<>(new Observer() { @Override public void onSubscribe(Disposable d) { @@ -853,7 +853,7 @@ public void errorMeansDisposed() { @Test public void assertValuePredicateEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.empty().subscribe(to); @@ -868,7 +868,7 @@ public void assertValuePredicateEmpty() { @Test public void assertValuePredicateMatch() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1).subscribe(to); @@ -881,7 +881,7 @@ public void assertValuePredicateMatch() { @Test public void assertValuePredicateNoMatch() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1).subscribe(to); @@ -896,7 +896,7 @@ public void assertValuePredicateNoMatch() { @Test public void assertValuePredicateMatchButMore() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1, 2).subscribe(to); @@ -911,7 +911,7 @@ public void assertValuePredicateMatchButMore() { @Test public void assertValueAtPredicateEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.empty().subscribe(to); @@ -926,7 +926,7 @@ public void assertValueAtPredicateEmpty() { @Test public void assertValueAtPredicateMatch() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1, 2).subscribe(to); @@ -939,7 +939,7 @@ public void assertValueAtPredicateMatch() { @Test public void assertValueAtPredicateNoMatch() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1, 2, 3).subscribe(to); @@ -954,7 +954,7 @@ public void assertValueAtPredicateNoMatch() { @Test public void assertValueAtInvalidIndex() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just(1, 2).subscribe(to); @@ -969,7 +969,7 @@ public void assertValueAtInvalidIndex() { @Test public void assertValueAtIndexEmpty() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.empty().subscribe(to); @@ -980,7 +980,7 @@ public void assertValueAtIndexEmpty() { @Test public void assertValueAtIndexMatch() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just("a", "b").subscribe(to); @@ -989,7 +989,7 @@ public void assertValueAtIndexMatch() { @Test public void assertValueAtIndexNoMatch() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just("a", "b", "c").subscribe(to); @@ -1000,7 +1000,7 @@ public void assertValueAtIndexNoMatch() { @Test public void assertValueAtIndexInvalidIndex() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); Observable.just("a", "b").subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java index 044c429d6c..26d5af8628 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java @@ -35,7 +35,7 @@ public void subscriberCount() { .collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -74,7 +74,7 @@ public void reducerCrash() { .collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -99,7 +99,7 @@ public void cancel() { .collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -125,7 +125,7 @@ public void error() { .collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Integer>() { @Override @@ -147,7 +147,7 @@ public void doubleError() { .collect(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiConsumer, Object>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java index 40afd9a511..1e048a1dcc 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java @@ -49,7 +49,7 @@ public Integer apply(Integer v) throws Exception { .sequential() ; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -77,7 +77,7 @@ public Integer apply(Integer v) throws Exception { .sequential() ; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -113,7 +113,7 @@ public Integer apply(Integer v) throws Exception { .sequential() ; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -154,7 +154,7 @@ public Integer apply(Integer v) throws Exception { .sequential() ; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); result.subscribe(ts); @@ -176,7 +176,7 @@ public Integer apply(Integer v) throws Exception { @Test public void reduceFull() { for (int i = 1; i <= Runtime.getRuntime().availableProcessors() * 2; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 10) .parallel(i) @@ -205,7 +205,7 @@ public void parallelReduceFull() { Scheduler scheduler = Schedulers.from(exec); try { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, n) .map(new Function() { @@ -239,7 +239,7 @@ public Long apply(Long a, Long b) throws Exception { @SuppressWarnings("unchecked") @Test public void toSortedList() { - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); Flowable.fromArray(10, 9, 8, 7, 6, 5, 4, 3, 2, 1) .parallel() @@ -251,7 +251,7 @@ public void toSortedList() { @Test public void sorted() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.fromArray(10, 9, 8, 7, 6, 5, 4, 3, 2, 1) .parallel() @@ -278,11 +278,11 @@ public void collect() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(1, 10) .parallel() .collect(as, new BiConsumer, Integer>() { @@ -310,7 +310,7 @@ public Iterable apply(List v) throws Exception { @SuppressWarnings("unchecked") @Test public void from() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ParallelFlowable.fromArray(Flowable.range(1, 5), Flowable.range(6, 5)) .sequential() @@ -325,7 +325,7 @@ public void from() { @Test public void concatMapUnordered() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(1, 5) .parallel() @@ -347,7 +347,7 @@ public Publisher apply(Integer v) throws Exception { @Test public void flatMapUnordered() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.range(1, 5) .parallel() @@ -377,10 +377,10 @@ public void collectAsyncFused() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); Flowable.range(1, 100000) .parallel(3) @@ -424,10 +424,10 @@ public void collectAsync() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); Flowable.range(1, 100000).hide() .parallel(3) @@ -471,10 +471,10 @@ public void collectAsync2() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); Flowable.range(1, 100000).hide() .observeOn(s) @@ -519,10 +519,10 @@ public void collectAsync3() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); Flowable.range(1, 100000).hide() .observeOn(s) @@ -567,10 +567,10 @@ public void collectAsync3Fused() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); Flowable.range(1, 100000) .observeOn(s) @@ -615,10 +615,10 @@ public void collectAsync3Take() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); Flowable.range(1, 100000) .take(1000) @@ -664,10 +664,10 @@ public void collectAsync4Take() { Supplier> as = new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }; - TestSubscriber> ts = new TestSubscriber>(); + TestSubscriber> ts = new TestSubscriber<>(); UnicastProcessor up = UnicastProcessor.create(); @@ -711,7 +711,7 @@ public void accept(List v) throws Exception { @Test public void emptySourceZeroRequest() { - TestSubscriber ts = new TestSubscriber(0); + TestSubscriber ts = new TestSubscriber<>(0); Flowable.range(1, 3).parallel(3).sequential().subscribe(ts); @@ -762,7 +762,7 @@ public void parallelismAndPrefetchAsync() { @SuppressWarnings("unchecked") @Test public void badParallelismStage() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.range(1, 10) .parallel(2) @@ -774,9 +774,9 @@ public void badParallelismStage() { @SuppressWarnings("unchecked") @Test public void badParallelismStage2() { - TestSubscriber ts1 = new TestSubscriber(); - TestSubscriber ts2 = new TestSubscriber(); - TestSubscriber ts3 = new TestSubscriber(); + TestSubscriber ts1 = new TestSubscriber<>(); + TestSubscriber ts2 = new TestSubscriber<>(); + TestSubscriber ts3 = new TestSubscriber<>(); Flowable.range(1, 10) .parallel(2) @@ -1305,7 +1305,7 @@ public static void checkSubscriberCount(ParallelFlowable source) { TestSubscriber[] consumers = new TestSubscriber[n + 1]; for (int i = 0; i <= n; i++) { - consumers[i] = new TestSubscriber(); + consumers[i] = new TestSubscriber<>(); } source.subscribe(consumers); @@ -1323,7 +1323,7 @@ public void checkAddBiConsumer() { @Test public void mergeBiFunction() throws Exception { - MergerBiFunction f = new MergerBiFunction(Functions.naturalComparator()); + MergerBiFunction f = new MergerBiFunction<>(Functions.naturalComparator()); assertEquals(0, f.apply(Collections.emptyList(), Collections.emptyList()).size()); @@ -1331,12 +1331,12 @@ public void mergeBiFunction() throws Exception { for (int i = 0; i < 4; i++) { int k = 0; - List list1 = new ArrayList(); + List list1 = new ArrayList<>(); for (int j = 0; j < i; j++) { list1.add(k++); } - List list2 = new ArrayList(); + List list2 = new ArrayList<>(); for (int j = i; j < 4; j++) { list2.add(k++); } diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFromPublisherTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFromPublisherTest.java index 0d889d21cb..c790201d98 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFromPublisherTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFromPublisherTest.java @@ -71,12 +71,12 @@ static final class StripBoundary extends Flowable implements FlowableTrans @Override public Publisher apply(Flowable upstream) { - return new StripBoundary(upstream); + return new StripBoundary<>(upstream); } @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new StripBoundarySubscriber(s)); + source.subscribe(new StripBoundarySubscriber<>(s)); } static final class StripBoundarySubscriber extends BasicFuseableSubscriber { @@ -117,7 +117,7 @@ public Object apply(Integer v) throws Exception { throw new TestException(); } }) - .compose(new StripBoundary(null)) + .compose(new StripBoundary<>(null)) .parallel() .sequential() .test() @@ -137,7 +137,7 @@ public Object apply(Integer v) throws Exception { throw new TestException(); } }) - .compose(new StripBoundary(null)) + .compose(new StripBoundary<>(null)) .parallel() .sequential() .test() @@ -148,8 +148,8 @@ public Object apply(Integer v) throws Exception { @Test public void boundaryConfinement() { - final Set between = new HashSet(); - final ConcurrentHashMap processing = new ConcurrentHashMap(); + final Set between = new HashSet<>(); + final ConcurrentHashMap processing = new ConcurrentHashMap<>(); TestSubscriberEx ts = Flowable.range(1, 10) .observeOn(Schedulers.single(), false, 1) diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java index 61ef50c1f6..aac8c0af62 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java @@ -35,7 +35,7 @@ public void subscriberCount() { .reduce(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiFunction, Integer, List>() { @Override @@ -76,7 +76,7 @@ public void reducerCrash() { .reduce(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiFunction, Integer, List>() { @Override @@ -102,7 +102,7 @@ public void cancel() { .reduce(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiFunction, Integer, List>() { @Override @@ -129,7 +129,7 @@ public void error() { .reduce(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiFunction, Integer, List>() { @Override @@ -152,7 +152,7 @@ public void doubleError() { .reduce(new Supplier>() { @Override public List get() throws Exception { - return new ArrayList(); + return new ArrayList<>(); } }, new BiFunction, Object, List>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelRunOnTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelRunOnTest.java index 4b19afdc66..907b0bb889 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelRunOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelRunOnTest.java @@ -135,7 +135,7 @@ public void errorConditional() { @SuppressWarnings("unchecked") @Test public void errorConditionalBackpressured() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.error(new TestException()) .parallel(1) @@ -150,7 +150,7 @@ public void errorConditionalBackpressured() { @SuppressWarnings("unchecked") @Test public void emptyConditionalBackpressured() { - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); Flowable.empty() .parallel(1) diff --git a/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java b/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java index 090ff6b766..defc141800 100644 --- a/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java +++ b/src/test/java/io/reactivex/rxjava3/plugins/RxJavaPluginsTest.java @@ -599,7 +599,7 @@ public void singleCreate() { RxJavaPlugins.setOnSingleAssembly(new Function() { @Override public Single apply(Single t) { - return new SingleJust(10); + return new SingleJust<>(10); } }); @@ -806,7 +806,7 @@ public void onScheduleNewThread() throws InterruptedException { @Test public void onError() { try { - final List list = new ArrayList(); + final List list = new ArrayList<>(); RxJavaPlugins.setErrorHandler(new Consumer() { @Override @@ -827,7 +827,7 @@ public void accept(Throwable t) { @Test public void onErrorNoHandler() { try { - final List list = new ArrayList(); + final List list = new ArrayList<>(); RxJavaPlugins.setErrorHandler(null); @@ -858,7 +858,7 @@ public void uncaughtException(Thread t, Throwable e) { @Test public void onErrorCrashes() { try { - final List list = new ArrayList(); + final List list = new ArrayList<>(); RxJavaPlugins.setErrorHandler(new Consumer() { @Override @@ -893,7 +893,7 @@ public void uncaughtException(Thread t, Throwable e) { @Test public void onErrorWithNull() { try { - final List list = new ArrayList(); + final List list = new ArrayList<>(); RxJavaPlugins.setErrorHandler(new Consumer() { @Override @@ -1523,7 +1523,7 @@ public void onComplete() { @Test public void onErrorNull() { try { - final AtomicReference t = new AtomicReference(); + final AtomicReference t = new AtomicReference<>(); RxJavaPlugins.setErrorHandler(new Consumer() { @Override @@ -1547,7 +1547,7 @@ private static void verifyThread(Scheduler scheduler, String expectedThreadName) assertNotNull(scheduler); Worker w = scheduler.createWorker(); try { - final AtomicReference value = new AtomicReference(); + final AtomicReference value = new AtomicReference<>(); final CountDownLatch cdl = new CountDownLatch(1); w.schedule(new Runnable() { @@ -1703,7 +1703,7 @@ public void onParallelAssembly() { RxJavaPlugins.setOnParallelAssembly(new Function() { @Override public ParallelFlowable apply(ParallelFlowable pf) throws Exception { - return new ParallelFromPublisher(Flowable.just(2), 2, 2); + return new ParallelFromPublisher<>(Flowable.just(2), 2, 2); } }); diff --git a/src/test/java/io/reactivex/rxjava3/processors/AsyncProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/AsyncProcessorTest.java index 421aee51a4..6daaf15245 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/AsyncProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/AsyncProcessorTest.java @@ -136,7 +136,7 @@ public void unsubscribeBeforeCompleted() { AsyncProcessor processor = AsyncProcessor.create(); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); processor.subscribe(ts); processor.onNext("one"); @@ -185,7 +185,7 @@ public void subscribeCompletionRaceCondition() { */ for (int i = 0; i < 50; i++) { final AsyncProcessor processor = AsyncProcessor.create(); - final AtomicReference value1 = new AtomicReference(); + final AtomicReference value1 = new AtomicReference<>(); processor.subscribe(new Consumer() { @@ -243,7 +243,7 @@ public void run() { private static class SubjectSubscriberThread extends Thread { private final AsyncProcessor processor; - private final AtomicReference value = new AtomicReference(); + private final AtomicReference value = new AtomicReference<>(); SubjectSubscriberThread(AsyncProcessor processor) { this.processor = processor; @@ -327,7 +327,7 @@ public void currentStateMethodsError() { @Test public void fusionLive() { - AsyncProcessor ap = new AsyncProcessor(); + AsyncProcessor ap = new AsyncProcessor<>(); TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); @@ -350,7 +350,7 @@ public void fusionLive() { @Test public void fusionOfflie() { - AsyncProcessor ap = new AsyncProcessor(); + AsyncProcessor ap = new AsyncProcessor<>(); ap.onNext(1); ap.onComplete(); @@ -461,7 +461,7 @@ public void run() { public void onNextCrossCancel() { AsyncProcessor p = AsyncProcessor.create(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); TestSubscriber ts1 = new TestSubscriber() { @Override public void onNext(Object t) { @@ -484,7 +484,7 @@ public void onNext(Object t) { public void onErrorCrossCancel() { AsyncProcessor p = AsyncProcessor.create(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); TestSubscriber ts1 = new TestSubscriber() { @Override public void onError(Throwable t) { @@ -506,7 +506,7 @@ public void onError(Throwable t) { public void onCompleteCrossCancel() { AsyncProcessor p = AsyncProcessor.create(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); TestSubscriber ts1 = new TestSubscriber() { @Override public void onComplete() { diff --git a/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java index 0d61998b30..0f04c4baea 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java @@ -137,7 +137,7 @@ public void completedStopsEmittingData() { Subscriber observerB = TestHelper.mockSubscriber(); Subscriber observerC = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(observerA); + TestSubscriber ts = new TestSubscriber<>(observerA); channel.subscribe(ts); channel.subscribe(observerB); @@ -389,7 +389,7 @@ public void run() { } }); - final AtomicReference o = new AtomicReference(); + final AtomicReference o = new AtomicReference<>(); rs.subscribeOn(s).observeOn(Schedulers.io()) .subscribe(new DefaultSubscriber() { @@ -715,7 +715,7 @@ public void completeSubscribeRace() throws Exception { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final BehaviorProcessor p = BehaviorProcessor.create(); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -742,7 +742,7 @@ public void errorSubscribeRace() throws Exception { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final BehaviorProcessor p = BehaviorProcessor.create(); - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final TestException ex = new TestException(); @@ -804,9 +804,9 @@ public void behaviorDisposableDisposeState() { BehaviorProcessor bp = BehaviorProcessor.create(); bp.onNext(1); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - BehaviorSubscription bs = new BehaviorSubscription(ts, bp); + BehaviorSubscription bs = new BehaviorSubscription<>(ts, bp); ts.onSubscribe(bs); assertFalse(bs.cancelled); @@ -835,9 +835,9 @@ public void emitFirstDisposeRace() { BehaviorProcessor bp = BehaviorProcessor.create(); bp.onNext(1); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final BehaviorSubscription bs = new BehaviorSubscription(ts, bp); + final BehaviorSubscription bs = new BehaviorSubscription<>(ts, bp); ts.onSubscribe(bs); Runnable r1 = new Runnable() { @@ -864,9 +864,9 @@ public void emitNextDisposeRace() { BehaviorProcessor bp = BehaviorProcessor.create(); bp.onNext(1); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final BehaviorSubscription bs = new BehaviorSubscription(ts, bp); + final BehaviorSubscription bs = new BehaviorSubscription<>(ts, bp); ts.onSubscribe(bs); Runnable r1 = new Runnable() { @@ -891,9 +891,9 @@ public void emittingEmitNext() { BehaviorProcessor bp = BehaviorProcessor.create(); bp.onNext(1); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final BehaviorSubscription bs = new BehaviorSubscription(ts, bp); + final BehaviorSubscription bs = new BehaviorSubscription<>(ts, bp); ts.onSubscribe(bs); bs.emitting = true; @@ -910,9 +910,9 @@ public void badRequest() { BehaviorProcessor bp = BehaviorProcessor.create(); bp.onNext(1); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final BehaviorSubscription bs = new BehaviorSubscription(ts, bp); + final BehaviorSubscription bs = new BehaviorSubscription<>(ts, bp); ts.onSubscribe(bs); bs.request(-1); diff --git a/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java index d34fbabcb4..4e71919b09 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java @@ -229,7 +229,7 @@ public void takeRefCountExact() { @Test public void crossCancel() { - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override @@ -257,7 +257,7 @@ public void onNext(Integer t) { @Test public void crossCancelError() { - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override @@ -285,7 +285,7 @@ public void onError(Throwable t) { @Test public void crossCancelComplete() { - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override @@ -314,7 +314,7 @@ public void onComplete() { @Test public void crossCancel1() { - final TestSubscriber ts1 = new TestSubscriber(1); + final TestSubscriber ts1 = new TestSubscriber<>(1); TestSubscriber ts2 = new TestSubscriber(1) { @Override @@ -536,7 +536,7 @@ public void addRemoveRaceNoRefCount() { final MulticastProcessor mp = MulticastProcessor.create(); final TestSubscriber ts = mp.test(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -565,7 +565,7 @@ public void addRemoveRaceNoRefCountNonEmpty() { mp.test(); final TestSubscriber ts = mp.test(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override @@ -593,7 +593,7 @@ public void addRemoveRaceWitRefCount() { final MulticastProcessor mp = MulticastProcessor.create(true); final TestSubscriber ts = mp.test(); - final TestSubscriber ts2 = new TestSubscriber(); + final TestSubscriber ts2 = new TestSubscriber<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java index 3398e1bc58..277ac40c78 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java @@ -69,7 +69,7 @@ public void completedStopsEmittingData() { Subscriber observerB = TestHelper.mockSubscriber(); Subscriber observerC = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(observerA); + TestSubscriber ts = new TestSubscriber<>(observerA); channel.subscribe(ts); channel.subscribe(observerB); @@ -178,7 +178,7 @@ public void unsubscribeFirstSubscriber() { PublishProcessor processor = PublishProcessor.create(); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); processor.subscribe(ts); processor.onNext("one"); @@ -213,7 +213,7 @@ public void nestedSubscribe() { final AtomicInteger countChildren = new AtomicInteger(); final AtomicInteger countTotal = new AtomicInteger(); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); s.flatMap(new Function>() { @@ -264,7 +264,7 @@ public void reSubscribe() { final PublishProcessor pp = PublishProcessor.create(); Subscriber subscriber1 = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber1); + TestSubscriber ts = new TestSubscriber<>(subscriber1); pp.subscribe(ts); // emit @@ -282,7 +282,7 @@ public void reSubscribe() { pp.onNext(2); Subscriber subscriber2 = TestHelper.mockSubscriber(); - TestSubscriber ts2 = new TestSubscriber(subscriber2); + TestSubscriber ts2 = new TestSubscriber<>(subscriber2); pp.subscribe(ts2); // emit @@ -412,7 +412,7 @@ public void requestValidation() { @Test public void crossCancel() { - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override public void onNext(Integer t) { @@ -435,7 +435,7 @@ public void onNext(Integer t) { @Test public void crossCancelOnError() { - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override public void onError(Throwable t) { @@ -458,7 +458,7 @@ public void onError(Throwable t) { @Test public void crossCancelOnComplete() { - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override public void onComplete() { diff --git a/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorBoundedConcurrencyTest.java b/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorBoundedConcurrencyTest.java index 8f733ae6ed..4590a6dec9 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorBoundedConcurrencyTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorBoundedConcurrencyTest.java @@ -162,8 +162,8 @@ public void subscribe(Subscriber subscriber) { }); // used to collect results of each thread - final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList>()); - final List threads = Collections.synchronizedList(new ArrayList()); + final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList<>()); + final List threads = Collections.synchronizedList(new ArrayList<>()); for (int i = 1; i <= 200; i++) { final int count = i; @@ -196,7 +196,7 @@ public void run() { } // assert all threads got the same results - List sums = new ArrayList(); + List sums = new ArrayList<>(); for (List values : listOfListsOfValues) { long v = 0; for (long l : values) { @@ -229,7 +229,7 @@ public void run() { public void subscribeCompletionRaceCondition() { for (int i = 0; i < 50; i++) { final ReplayProcessor processor = ReplayProcessor.createUnbounded(); - final AtomicReference value1 = new AtomicReference(); + final AtomicReference value1 = new AtomicReference<>(); processor.subscribe(new Consumer() { @@ -292,7 +292,7 @@ public void run() { public void raceForTerminalState() { final List expected = Arrays.asList(1); for (int i = 0; i < 100000; i++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1).subscribeOn(Schedulers.computation()).cache().subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertValueSequence(expected); @@ -303,7 +303,7 @@ public void raceForTerminalState() { private static class SubjectObserverThread extends Thread { private final ReplayProcessor processor; - private final AtomicReference value = new AtomicReference(); + private final AtomicReference value = new AtomicReference<>(); SubjectObserverThread(ReplayProcessor processor) { this.processor = processor; @@ -350,7 +350,7 @@ public void run() { } }); - final AtomicReference o = new AtomicReference(); + final AtomicReference o = new AtomicReference<>(); rs // .doOnSubscribe(v -> System.out.println("!! " + j)) diff --git a/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorConcurrencyTest.java b/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorConcurrencyTest.java index f9754a49ee..3cacc0d67a 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorConcurrencyTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorConcurrencyTest.java @@ -162,8 +162,8 @@ public void subscribe(Subscriber subscriber) { }); // used to collect results of each thread - final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList>()); - final List threads = Collections.synchronizedList(new ArrayList()); + final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList<>()); + final List threads = Collections.synchronizedList(new ArrayList<>()); for (int i = 1; i <= 200; i++) { final int count = i; @@ -196,7 +196,7 @@ public void run() { } // assert all threads got the same results - List sums = new ArrayList(); + List sums = new ArrayList<>(); for (List values : listOfListsOfValues) { long v = 0; for (long l : values) { @@ -229,7 +229,7 @@ public void run() { public void subscribeCompletionRaceCondition() { for (int i = 0; i < 50; i++) { final ReplayProcessor processor = ReplayProcessor.create(); - final AtomicReference value1 = new AtomicReference(); + final AtomicReference value1 = new AtomicReference<>(); processor.subscribe(new Consumer() { @@ -292,7 +292,7 @@ public void run() { public void raceForTerminalState() { final List expected = Arrays.asList(1); for (int i = 0; i < 100000; i++) { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1).subscribeOn(Schedulers.computation()).cache().subscribe(ts); ts.awaitDone(5, TimeUnit.SECONDS); ts.assertValueSequence(expected); @@ -303,7 +303,7 @@ public void raceForTerminalState() { static class SubjectObserverThread extends Thread { private final ReplayProcessor processor; - private final AtomicReference value = new AtomicReference(); + private final AtomicReference value = new AtomicReference<>(); SubjectObserverThread(ReplayProcessor processor) { this.processor = processor; @@ -347,7 +347,7 @@ public void run() { } }); - final AtomicReference o = new AtomicReference(); + final AtomicReference o = new AtomicReference<>(); rs.subscribeOn(s).observeOn(Schedulers.io()) .subscribe(new DefaultSubscriber() { diff --git a/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorTest.java index 80fe3af7b7..bb4b97e68b 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/ReplayProcessorTest.java @@ -76,7 +76,7 @@ public void completedStopsEmittingData() { Subscriber observerB = TestHelper.mockSubscriber(); Subscriber observerC = TestHelper.mockSubscriber(); Subscriber observerD = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(observerA); + TestSubscriber ts = new TestSubscriber<>(observerA); channel.subscribe(ts); channel.subscribe(observerB); @@ -227,7 +227,7 @@ public void unsubscribeFirstSubscriber() { ReplayProcessor processor = ReplayProcessor.create(); Subscriber subscriber = TestHelper.mockSubscriber(); - TestSubscriber ts = new TestSubscriber(subscriber); + TestSubscriber ts = new TestSubscriber<>(subscriber); processor.subscribe(ts); processor.onNext("one"); @@ -258,7 +258,7 @@ private void assertObservedUntilTwo(Subscriber subscriber) { @Test public void newSubscriberDoesntBlockExisting() throws InterruptedException { - final AtomicReference lastValueForSubscriber1 = new AtomicReference(); + final AtomicReference lastValueForSubscriber1 = new AtomicReference<>(); Subscriber subscriber1 = new DefaultSubscriber() { @Override @@ -279,7 +279,7 @@ public void onNext(String v) { }; - final AtomicReference lastValueForSubscriber2 = new AtomicReference(); + final AtomicReference lastValueForSubscriber2 = new AtomicReference<>(); final CountDownLatch oneReceived = new CountDownLatch(1); final CountDownLatch makeSlow = new CountDownLatch(1); final CountDownLatch completed = new CountDownLatch(1); @@ -806,7 +806,7 @@ public void backpressureHonored() { rs.onNext(3); rs.onComplete(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); rs.subscribe(ts); @@ -834,7 +834,7 @@ public void backpressureHonoredSizeBound() { rs.onNext(3); rs.onComplete(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); rs.subscribe(ts); @@ -862,7 +862,7 @@ public void backpressureHonoredTimeBound() { rs.onNext(3); rs.onComplete(); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); rs.subscribe(ts); @@ -1026,7 +1026,7 @@ public void capacityHint() { @Test public void subscribeCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestSubscriber ts = new TestSubscriber(); + final TestSubscriber ts = new TestSubscriber<>(); final ReplayProcessor rp = ReplayProcessor.create(); diff --git a/src/test/java/io/reactivex/rxjava3/processors/SerializedProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/SerializedProcessorTest.java index 0582689830..fc50ade3ef 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/SerializedProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/SerializedProcessorTest.java @@ -31,8 +31,8 @@ public class SerializedProcessorTest extends RxJavaTest { @Test public void basic() { - SerializedProcessor processor = new SerializedProcessor(PublishProcessor. create()); - TestSubscriber ts = new TestSubscriber(); + SerializedProcessor processor = new SerializedProcessor<>(PublishProcessor.create()); + TestSubscriber ts = new TestSubscriber<>(); processor.subscribe(ts); processor.onNext("hello"); processor.onComplete(); @@ -416,7 +416,7 @@ public void normal() { @Test public void onNextOnNextRace() { - Set expectedSet = new HashSet(Arrays.asList(1, 2)); + Set expectedSet = new HashSet<>(Arrays.asList(1, 2)); for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final FlowableProcessor s = PublishProcessor.create().toSerialized(); @@ -445,7 +445,7 @@ public void run() { .assertValueCount(2) ; - Set actualSet = new HashSet(ts.values()); + Set actualSet = new HashSet<>(ts.values()); assertEquals("" + actualSet, expectedSet, actualSet); } } diff --git a/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java index a552783994..53b7062028 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java @@ -268,7 +268,7 @@ public void onErrorStatePeeking() { public void rejectSyncFusion() { UnicastProcessor p = UnicastProcessor.create(); - TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.SYNC); + TestSubscriberEx ts = new TestSubscriberEx<>().setInitialFusionMode(QueueFuseable.SYNC); p.subscribe(ts); @@ -302,7 +302,7 @@ public void fusedDrainCancel() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final UnicastProcessor p = UnicastProcessor.create(); - final TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); + final TestSubscriberEx ts = new TestSubscriberEx<>().setInitialFusionMode(QueueFuseable.ANY); p.subscribe(ts); @@ -329,8 +329,8 @@ public void subscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final UnicastProcessor us = UnicastProcessor.create(); - final TestSubscriberEx ts1 = new TestSubscriberEx(); - final TestSubscriberEx ts2 = new TestSubscriberEx(); + final TestSubscriberEx ts1 = new TestSubscriberEx<>(); + final TestSubscriberEx ts2 = new TestSubscriberEx<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/AbstractSchedulerTests.java b/src/test/java/io/reactivex/rxjava3/schedulers/AbstractSchedulerTests.java index 4163c71ec6..34d8ac1b39 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/AbstractSchedulerTests.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/AbstractSchedulerTests.java @@ -395,7 +395,7 @@ public void run() { } }); - ConcurrentObserverValidator observer = new ConcurrentObserverValidator(); + ConcurrentObserverValidator observer = new ConcurrentObserverValidator<>(); // this should call onNext concurrently f.subscribe(observer); @@ -414,7 +414,7 @@ public final void observeOn() throws InterruptedException { Flowable f = Flowable.fromArray("one", "two", "three", "four", "five", "six", "seven", "eight", "nine", "ten"); - ConcurrentObserverValidator observer = new ConcurrentObserverValidator(); + ConcurrentObserverValidator observer = new ConcurrentObserverValidator<>(); f.observeOn(scheduler).subscribe(observer); @@ -449,7 +449,7 @@ public void subscribe(Subscriber subscriber) { } }); - ConcurrentObserverValidator observer = new ConcurrentObserverValidator(); + ConcurrentObserverValidator observer = new ConcurrentObserverValidator<>(); f.subscribe(observer); @@ -471,7 +471,7 @@ public void subscribe(Subscriber subscriber) { private static class ConcurrentObserverValidator extends DefaultSubscriber { final AtomicInteger concurrentCounter = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); final CountDownLatch completed = new CountDownLatch(1); @Override @@ -685,7 +685,7 @@ public void schedulePeriodicallyDirectDecoratesRunnable() throws InterruptedExce return; } - final AtomicReference disposable = new AtomicReference(); + final AtomicReference disposable = new AtomicReference<>(); try { assertRunnableDecorated(new Runnable() { diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java b/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java index a963f24665..0e6248cc89 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/ComputationSchedulerTests.java @@ -39,7 +39,7 @@ public void threadSafetyWhenSchedulerIsHoppingBetweenThreads() { final int NUM = 1000000; final CountDownLatch latch = new CountDownLatch(1); - final HashMap map = new HashMap(); + final HashMap map = new HashMap<>(); final Scheduler.Worker inner = Schedulers.computation().createWorker(); diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerFairTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerFairTest.java index 97fd1d3269..128a3bd5b5 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerFairTest.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerFairTest.java @@ -72,7 +72,7 @@ public void cancelledTaskRetention() throws InterruptedException { /** 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(); + final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); @Override public void execute(Runnable command) { queue.offer(command); diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerInterruptibleTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerInterruptibleTest.java index ecf50c3e99..e9c953b049 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerInterruptibleTest.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerInterruptibleTest.java @@ -69,7 +69,7 @@ public void cancelledTaskRetention() throws InterruptedException { /** 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(); + final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); @Override public void execute(Runnable command) { queue.offer(command); diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerTest.java index b24fa65f90..b2e90cdb00 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerTest.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/ExecutorSchedulerTest.java @@ -155,7 +155,7 @@ public void cancelledTaskRetention() throws InterruptedException { /** 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(); + final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); @Override public void execute(Runnable command) { queue.offer(command); diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerLifecycleTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerLifecycleTest.java index 6a8eb53344..c378448220 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerLifecycleTest.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerLifecycleTest.java @@ -32,7 +32,7 @@ public void shutdown() throws InterruptedException { System.out.println("testShutdown >> Giving time threads to spin-up"); Thread.sleep(500); - Set rxThreads = new HashSet(); + Set rxThreads = new HashSet<>(); for (Thread t : Thread.getAllStackTraces().keySet()) { if (t.getName().startsWith("Rx")) { rxThreads.add(t); @@ -108,7 +108,7 @@ public void startIdempotence() throws InterruptedException { System.out.println("testStartIdempotence >> giving some time"); Thread.sleep(500); - Set rxThreadsBefore = new HashSet(); + Set rxThreadsBefore = new HashSet<>(); for (Thread t : Thread.getAllStackTraces().keySet()) { if (t.getName().startsWith("Rx")) { rxThreadsBefore.add(t); @@ -120,7 +120,7 @@ public void startIdempotence() throws InterruptedException { System.out.println("testStartIdempotence >> giving some time again"); Thread.sleep(500); - Set rxThreadsAfter = new HashSet(); + Set rxThreadsAfter = new HashSet<>(); for (Thread t : Thread.getAllStackTraces().keySet()) { if (t.getName().startsWith("Rx")) { rxThreadsAfter.add(t); diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerTestHelper.java b/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerTestHelper.java index 33c41e1189..05dda6a058 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerTestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerTestHelper.java @@ -33,7 +33,7 @@ static void handledErrorIsNotDeliveredToThreadHandler(Scheduler scheduler) throw Thread.UncaughtExceptionHandler originalHandler = Thread.getDefaultUncaughtExceptionHandler(); try { CapturingUncaughtExceptionHandler handler = new CapturingUncaughtExceptionHandler(); - CapturingObserver observer = new CapturingObserver(); + CapturingObserver observer = new CapturingObserver<>(); Thread.setDefaultUncaughtExceptionHandler(handler); IllegalStateException error = new IllegalStateException("Should be delivered to handler"); Flowable.error(error) diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerWorkerTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerWorkerTest.java index 688e5fc429..87ad3698ad 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerWorkerTest.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/SchedulerWorkerTest.java @@ -76,7 +76,7 @@ public void currentTimeDriftBackwards() throws Exception { Scheduler.Worker w = s.createWorker(); try { - final List times = new ArrayList(); + final List times = new ArrayList<>(); Disposable d = w.schedulePeriodically(new Runnable() { @Override @@ -118,7 +118,7 @@ public void currentTimeDriftForwards() throws Exception { Scheduler.Worker w = s.createWorker(); try { - final List times = new ArrayList(); + final List times = new ArrayList<>(); Disposable d = w.schedulePeriodically(new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/TimedTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/TimedTest.java index 0fb0ce9767..d7a5eb752e 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/TimedTest.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/TimedTest.java @@ -25,7 +25,7 @@ public class TimedTest extends RxJavaTest { @Test public void properties() { - Timed timed = new Timed(1, 5, TimeUnit.SECONDS); + Timed timed = new Timed<>(1, 5, TimeUnit.SECONDS); assertEquals(1, timed.value().intValue()); assertEquals(5, timed.time()); @@ -35,22 +35,22 @@ public void properties() { @Test public void hashCodeOf() { - Timed t1 = new Timed(1, 5, TimeUnit.SECONDS); + Timed t1 = new Timed<>(1, 5, TimeUnit.SECONDS); assertEquals(TimeUnit.SECONDS.hashCode() + 31 * (5 + 31 * 1), t1.hashCode()); - Timed t2 = new Timed(null, 5, TimeUnit.SECONDS); + Timed t2 = new Timed<>(null, 5, TimeUnit.SECONDS); assertEquals(TimeUnit.SECONDS.hashCode() + 31 * (5 + 31 * 0), t2.hashCode()); } @Test public void equalsWith() { - Timed t1 = new Timed(1, 5, TimeUnit.SECONDS); - Timed t2 = new Timed(1, 5, TimeUnit.SECONDS); - Timed t3 = new Timed(2, 5, TimeUnit.SECONDS); - Timed t4 = new Timed(1, 4, TimeUnit.SECONDS); - Timed t5 = new Timed(1, 5, TimeUnit.MINUTES); + Timed t1 = new Timed<>(1, 5, TimeUnit.SECONDS); + Timed t2 = new Timed<>(1, 5, TimeUnit.SECONDS); + Timed t3 = new Timed<>(2, 5, TimeUnit.SECONDS); + Timed t4 = new Timed<>(1, 4, TimeUnit.SECONDS); + Timed t5 = new Timed<>(1, 5, TimeUnit.MINUTES); assertEquals(t1, t1); assertEquals(t1, t2); @@ -83,13 +83,13 @@ public void equalsWith() { @Test public void toStringOf() { - Timed t1 = new Timed(1, 5, TimeUnit.SECONDS); + Timed t1 = new Timed<>(1, 5, TimeUnit.SECONDS); assertEquals("Timed[time=5, unit=SECONDS, value=1]", t1.toString()); } @Test(expected = NullPointerException.class) public void timeUnitNullFail() throws Exception { - new Timed(1, 5, null); + new Timed<>(1, 5, null); } } diff --git a/src/test/java/io/reactivex/rxjava3/schedulers/TrampolineSchedulerTest.java b/src/test/java/io/reactivex/rxjava3/schedulers/TrampolineSchedulerTest.java index bfd7240db7..b409aa143a 100644 --- a/src/test/java/io/reactivex/rxjava3/schedulers/TrampolineSchedulerTest.java +++ b/src/test/java/io/reactivex/rxjava3/schedulers/TrampolineSchedulerTest.java @@ -63,7 +63,7 @@ public void accept(String t) { @Test public void nestedTrampolineWithUnsubscribe() { - final ArrayList workDone = new ArrayList(); + final ArrayList workDone = new ArrayList<>(); final CompositeDisposable workers = new CompositeDisposable(); Worker worker = Schedulers.trampoline().createWorker(); try { @@ -107,7 +107,7 @@ public void run() { public void trampolineWorkerHandlesConcurrentScheduling() { final Worker trampolineWorker = Schedulers.trampoline().createWorker(); final Subscriber subscriber = TestHelper.mockSubscriber(); - final TestSubscriber ts = new TestSubscriber(subscriber); + final TestSubscriber ts = new TestSubscriber<>(subscriber); // Spam the trampoline with actions. Flowable.range(0, 50) diff --git a/src/test/java/io/reactivex/rxjava3/single/SingleCacheTest.java b/src/test/java/io/reactivex/rxjava3/single/SingleCacheTest.java index 16221a253f..1851c15e1f 100644 --- a/src/test/java/io/reactivex/rxjava3/single/SingleCacheTest.java +++ b/src/test/java/io/reactivex/rxjava3/single/SingleCacheTest.java @@ -89,7 +89,7 @@ public void crossCancel() { PublishSubject ps = PublishSubject.create(); Single cache = ps.single(-99).cache(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override @@ -114,7 +114,7 @@ public void crossCancelOnError() { PublishSubject ps = PublishSubject.create(); Single cache = ps.single(-99).cache(); - final TestSubscriber ts1 = new TestSubscriber(); + final TestSubscriber ts1 = new TestSubscriber<>(); TestSubscriber ts2 = new TestSubscriber() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java b/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java index 8210632567..7da2cdf95f 100644 --- a/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java @@ -177,7 +177,7 @@ public void fromFutureNull() { @Test(expected = NullPointerException.class) public void fromFutureReturnsNull() { - FutureTask f = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); f.run(); Single.fromFuture(f).blockingGet(); } @@ -189,7 +189,7 @@ public void fromFutureTimedFutureNull() { @Test(expected = NullPointerException.class) public void fromFutureTimedUnitNull() { - Single.fromFuture(new FutureTask(new Callable() { + Single.fromFuture(new FutureTask<>(new Callable() { @Override public Object call() throws Exception { return null; @@ -199,7 +199,7 @@ public Object call() throws Exception { @Test(expected = NullPointerException.class) public void fromFutureTimedSchedulerNull() { - Single.fromFuture(new FutureTask(new Callable() { + Single.fromFuture(new FutureTask<>(new Callable() { @Override public Object call() throws Exception { return null; @@ -209,14 +209,14 @@ public Object call() throws Exception { @Test(expected = NullPointerException.class) public void fromFutureTimedReturnsNull() { - FutureTask f = new FutureTask(Functions.EMPTY_RUNNABLE, null); + FutureTask f = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); f.run(); Single.fromFuture(f, 1, TimeUnit.SECONDS).blockingGet(); } @Test(expected = NullPointerException.class) public void fromFutureSchedulerNull() { - Single.fromFuture(new FutureTask(new Callable() { + Single.fromFuture(new FutureTask<>(new Callable() { @Override public Object call() throws Exception { return null; diff --git a/src/test/java/io/reactivex/rxjava3/single/SingleTest.java b/src/test/java/io/reactivex/rxjava3/single/SingleTest.java index 81b5038719..92974c0ef2 100644 --- a/src/test/java/io/reactivex/rxjava3/single/SingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/single/SingleTest.java @@ -35,14 +35,14 @@ public class SingleTest extends RxJavaTest { @Test public void helloWorld() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single.just("Hello World!").toFlowable().subscribe(ts); ts.assertValueSequence(Arrays.asList("Hello World!")); } @Test public void helloWorld2() { - final AtomicReference v = new AtomicReference(); + final AtomicReference v = new AtomicReference<>(); Single.just("Hello World!").subscribe(new SingleObserver() { @Override @@ -66,7 +66,7 @@ public void onError(Throwable error) { @Test public void map() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single.just("A") .map(new Function() { @Override @@ -80,7 +80,7 @@ public String apply(String s) { @Test public void zip() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single a = Single.just("A"); Single b = Single.just("B"); @@ -96,7 +96,7 @@ public String apply(String a1, String b1) { @Test public void zipWith() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single.just("A").zipWith(Single.just("B"), new BiFunction() { @Override @@ -110,7 +110,7 @@ public String apply(String a1, String b1) { @Test public void merge() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single a = Single.just("A"); Single b = Single.just("B"); @@ -120,7 +120,7 @@ public void merge() { @Test public void mergeWith() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single.just("A").mergeWith(Single.just("B")).subscribe(ts); ts.assertValueSequence(Arrays.asList("A", "B")); @@ -128,7 +128,7 @@ public void mergeWith() { @Test public void createSuccess() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single.unsafeCreate(new SingleSource() { @Override @@ -143,7 +143,7 @@ public void subscribe(SingleObserver observer) { @Test public void createError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Single.unsafeCreate(new SingleSource() { @Override public void subscribe(SingleObserver observer) { @@ -158,7 +158,7 @@ public void subscribe(SingleObserver observer) { @Test public void async() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single.just("Hello") .subscribeOn(Schedulers.io()) .map(new Function() { @@ -183,7 +183,7 @@ public String apply(String v) { @Test public void flatMap() throws InterruptedException { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Single.just("Hello").flatMap(new Function>() { @Override public Single apply(String s) { @@ -200,7 +200,7 @@ public Single apply(String s) { @Test public void timeout() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single s1 = Single.unsafeCreate(new SingleSource() { @Override public void subscribe(SingleObserver observer) { @@ -222,7 +222,7 @@ public void subscribe(SingleObserver observer) { @Test public void timeoutWithFallback() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Single s1 = Single.unsafeCreate(new SingleSource() { @Override public void subscribe(SingleObserver observer) { @@ -245,7 +245,7 @@ public void subscribe(SingleObserver observer) { @Test public void unsubscribe() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicBoolean unsubscribed = new AtomicBoolean(); final AtomicBoolean interrupted = new AtomicBoolean(); final CountDownLatch latch = new CountDownLatch(2); @@ -436,7 +436,7 @@ public void subscribe(SingleObserver t) { } }); - TestSubscriber ts = new TestSubscriber(0L); + TestSubscriber ts = new TestSubscriber<>(0L); s.toFlowable().subscribe(ts); @@ -450,7 +450,7 @@ public void subscribe(SingleObserver t) { @Test public void toObservable() { Flowable a = Single.just("a").toFlowable(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); a.subscribe(ts); ts.assertValue("a"); ts.assertNoErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java index e6f8245d22..395300e292 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/AsyncSubjectTest.java @@ -136,7 +136,7 @@ public void unsubscribeBeforeCompleted() { AsyncSubject subject = AsyncSubject.create(); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); subject.subscribe(to); subject.onNext("one"); @@ -185,7 +185,7 @@ public void subscribeCompletionRaceCondition() { */ for (int i = 0; i < 50; i++) { final AsyncSubject subject = AsyncSubject.create(); - final AtomicReference value1 = new AtomicReference(); + final AtomicReference value1 = new AtomicReference<>(); subject.subscribe(new Consumer() { @@ -243,7 +243,7 @@ public void run() { private static class SubjectSubscriberThread extends Thread { private final AsyncSubject subject; - private final AtomicReference value = new AtomicReference(); + private final AtomicReference value = new AtomicReference<>(); SubjectSubscriberThread(AsyncSubject subject) { this.subject = subject; @@ -327,7 +327,7 @@ public void currentStateMethodsError() { @Test public void fusionLive() { - AsyncSubject ap = new AsyncSubject(); + AsyncSubject ap = new AsyncSubject<>(); TestObserverEx to = ap.to(TestHelper.testConsumer(false, QueueFuseable.ANY)); @@ -347,7 +347,7 @@ public void fusionLive() { @Test public void fusionOfflie() { - AsyncSubject ap = new AsyncSubject(); + AsyncSubject ap = new AsyncSubject<>(); ap.onNext(1); ap.onComplete(); @@ -455,7 +455,7 @@ public void run() { public void onNextCrossCancel() { AsyncSubject p = AsyncSubject.create(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); TestObserver to1 = new TestObserver() { @Override public void onNext(Object t) { @@ -478,7 +478,7 @@ public void onNext(Object t) { public void onErrorCrossCancel() { AsyncSubject p = AsyncSubject.create(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); TestObserver to1 = new TestObserver() { @Override public void onError(Throwable t) { @@ -500,7 +500,7 @@ public void onError(Throwable t) { public void onCompleteCrossCancel() { AsyncSubject p = AsyncSubject.create(); - final TestObserver to2 = new TestObserver(); + final TestObserver to2 = new TestObserver<>(); TestObserver to1 = new TestObserver() { @Override public void onComplete() { diff --git a/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java index 0d14802814..8e472ec4c5 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/BehaviorSubjectTest.java @@ -136,7 +136,7 @@ public void completedStopsEmittingData() { Observer observerB = TestHelper.mockObserver(); Observer observerC = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observerA); + TestObserver to = new TestObserver<>(observerA); channel.subscribe(to); channel.subscribe(observerB); @@ -389,7 +389,7 @@ public void run() { } }); - final AtomicReference o = new AtomicReference(); + final AtomicReference o = new AtomicReference<>(); rs.subscribeOn(s).observeOn(Schedulers.io()) .subscribe(new DefaultObserver() { @@ -679,7 +679,7 @@ public void completeSubscribeRace() throws Exception { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final BehaviorSubject p = BehaviorSubject.create(); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Runnable r1 = new Runnable() { @Override @@ -706,7 +706,7 @@ public void errorSubscribeRace() throws Exception { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final BehaviorSubject p = BehaviorSubject.create(); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); final TestException ex = new TestException(); @@ -735,9 +735,9 @@ public void behaviorDisposableDisposeState() { BehaviorSubject bs = BehaviorSubject.create(); bs.onNext(1); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - BehaviorDisposable bd = new BehaviorDisposable(to, bs); + BehaviorDisposable bd = new BehaviorDisposable<>(to, bs); to.onSubscribe(bd); assertFalse(bd.isDisposed()); @@ -766,9 +766,9 @@ public void emitFirstDisposeRace() { BehaviorSubject bs = BehaviorSubject.create(); bs.onNext(1); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final BehaviorDisposable bd = new BehaviorDisposable(to, bs); + final BehaviorDisposable bd = new BehaviorDisposable<>(to, bs); to.onSubscribe(bd); Runnable r1 = new Runnable() { @@ -795,9 +795,9 @@ public void emitNextDisposeRace() { BehaviorSubject bs = BehaviorSubject.create(); bs.onNext(1); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final BehaviorDisposable bd = new BehaviorDisposable(to, bs); + final BehaviorDisposable bd = new BehaviorDisposable<>(to, bs); to.onSubscribe(bd); Runnable r1 = new Runnable() { @@ -822,9 +822,9 @@ public void emittingEmitNext() { BehaviorSubject bs = BehaviorSubject.create(); bs.onNext(1); - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); - final BehaviorDisposable bd = new BehaviorDisposable(to, bs); + final BehaviorDisposable bd = new BehaviorDisposable<>(to, bs); to.onSubscribe(bd); bd.emitting = true; diff --git a/src/test/java/io/reactivex/rxjava3/subjects/PublishSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/PublishSubjectTest.java index 43ebb533d5..7c2c333a2b 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/PublishSubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/PublishSubjectTest.java @@ -68,7 +68,7 @@ public void completedStopsEmittingData() { Observer observerB = TestHelper.mockObserver(); Observer observerC = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observerA); + TestObserver to = new TestObserver<>(observerA); channel.subscribe(to); channel.subscribe(observerB); @@ -177,7 +177,7 @@ public void unsubscribeFirstSubscriber() { PublishSubject subject = PublishSubject.create(); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); subject.subscribe(to); subject.onNext("one"); @@ -212,7 +212,7 @@ public void nestedSubscribe() { final AtomicInteger countChildren = new AtomicInteger(); final AtomicInteger countTotal = new AtomicInteger(); - final ArrayList list = new ArrayList(); + final ArrayList list = new ArrayList<>(); s.flatMap(new Function>() { @@ -263,7 +263,7 @@ public void reSubscribe() { final PublishSubject ps = PublishSubject.create(); Observer o1 = TestHelper.mockObserver(); - TestObserver to = new TestObserver(o1); + TestObserver to = new TestObserver<>(o1); ps.subscribe(to); // emit @@ -281,7 +281,7 @@ public void reSubscribe() { ps.onNext(2); Observer o2 = TestHelper.mockObserver(); - TestObserver to2 = new TestObserver(o2); + TestObserver to2 = new TestObserver<>(o2); ps.subscribe(to2); // emit @@ -392,7 +392,7 @@ public void currentStateMethodsError() { @Test public void crossCancel() { - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); TestObserver to2 = new TestObserver() { @Override public void onNext(Integer t) { @@ -415,7 +415,7 @@ public void onNext(Integer t) { @Test public void crossCancelOnError() { - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); TestObserver to2 = new TestObserver() { @Override public void onError(Throwable t) { @@ -438,7 +438,7 @@ public void onError(Throwable t) { @Test public void crossCancelOnComplete() { - final TestObserver to1 = new TestObserver(); + final TestObserver to1 = new TestObserver<>(); TestObserver to2 = new TestObserver() { @Override public void onComplete() { @@ -572,7 +572,7 @@ public void addCompleteRance() throws Exception { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final PublishSubject ps = PublishSubject.create(); - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java index c1cef763fe..369fcfc59f 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectBoundedConcurrencyTest.java @@ -166,8 +166,8 @@ public void subscribe(Observer o) { }); // used to collect results of each thread - final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList>()); - final List threads = Collections.synchronizedList(new ArrayList()); + final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList<>()); + final List threads = Collections.synchronizedList(new ArrayList<>()); for (int i = 1; i <= 200; i++) { final int count = i; @@ -200,7 +200,7 @@ public void run() { } // assert all threads got the same results - List sums = new ArrayList(); + List sums = new ArrayList<>(); for (List values : listOfListsOfValues) { long v = 0; for (long l : values) { @@ -233,7 +233,7 @@ public void run() { public void subscribeCompletionRaceCondition() { for (int i = 0; i < 50; i++) { final ReplaySubject subject = ReplaySubject.createUnbounded(); - final AtomicReference value1 = new AtomicReference(); + final AtomicReference value1 = new AtomicReference<>(); subject.subscribe(new Consumer() { @@ -296,7 +296,7 @@ public void run() { public void raceForTerminalState() { final List expected = Arrays.asList(1); for (int i = 0; i < 100000; i++) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1).subscribeOn(Schedulers.computation()).cache().subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertValueSequence(expected); @@ -307,7 +307,7 @@ public void raceForTerminalState() { static class SubjectObserverThread extends Thread { private final ReplaySubject subject; - private final AtomicReference value = new AtomicReference(); + private final AtomicReference value = new AtomicReference<>(); SubjectObserverThread(ReplaySubject subject) { this.subject = subject; @@ -354,7 +354,7 @@ public void run() { } }); - final AtomicReference o = new AtomicReference(); + final AtomicReference o = new AtomicReference<>(); rs // .doOnSubscribe(v -> System.out.println("!! " + j)) diff --git a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java index c608875071..767553dc59 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectConcurrencyTest.java @@ -166,8 +166,8 @@ public void subscribe(Observer o) { }); // used to collect results of each thread - final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList>()); - final List threads = Collections.synchronizedList(new ArrayList()); + final List> listOfListsOfValues = Collections.synchronizedList(new ArrayList<>()); + final List threads = Collections.synchronizedList(new ArrayList<>()); for (int i = 1; i <= 200; i++) { final int count = i; @@ -200,7 +200,7 @@ public void run() { } // assert all threads got the same results - List sums = new ArrayList(); + List sums = new ArrayList<>(); for (List values : listOfListsOfValues) { long v = 0; for (long l : values) { @@ -233,7 +233,7 @@ public void run() { public void subscribeCompletionRaceCondition() { for (int i = 0; i < 50; i++) { final ReplaySubject subject = ReplaySubject.create(); - final AtomicReference value1 = new AtomicReference(); + final AtomicReference value1 = new AtomicReference<>(); subject.subscribe(new Consumer() { @@ -296,7 +296,7 @@ public void run() { public void raceForTerminalState() { final List expected = Arrays.asList(1); for (int i = 0; i < 100000; i++) { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1).subscribeOn(Schedulers.computation()).cache().subscribe(to); to.awaitDone(5, TimeUnit.SECONDS); to.assertValueSequence(expected); @@ -307,7 +307,7 @@ public void raceForTerminalState() { static class SubjectObserverThread extends Thread { private final ReplaySubject subject; - private final AtomicReference value = new AtomicReference(); + private final AtomicReference value = new AtomicReference<>(); SubjectObserverThread(ReplaySubject subject) { this.subject = subject; @@ -351,7 +351,7 @@ public void run() { } }); - final AtomicReference o = new AtomicReference(); + final AtomicReference o = new AtomicReference<>(); rs.subscribeOn(s).observeOn(Schedulers.io()) .subscribe(new DefaultObserver() { diff --git a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java index e8d36a8c62..eb06a1a0ea 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/ReplaySubjectTest.java @@ -74,7 +74,7 @@ public void completedStopsEmittingData() { Observer observerB = TestHelper.mockObserver(); Observer observerC = TestHelper.mockObserver(); Observer observerD = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observerA); + TestObserver to = new TestObserver<>(observerA); channel.subscribe(to); channel.subscribe(observerB); @@ -225,7 +225,7 @@ public void unsubscribeFirstSubscriber() { ReplaySubject subject = ReplaySubject.create(); Observer observer = TestHelper.mockObserver(); - TestObserver to = new TestObserver(observer); + TestObserver to = new TestObserver<>(observer); subject.subscribe(to); subject.onNext("one"); @@ -256,7 +256,7 @@ private void assertObservedUntilTwo(Observer observer) { @Test public void newSubscriberDoesntBlockExisting() throws InterruptedException { - final AtomicReference lastValueForSubscriber1 = new AtomicReference(); + final AtomicReference lastValueForSubscriber1 = new AtomicReference<>(); Observer observer1 = new DefaultObserver() { @Override @@ -277,7 +277,7 @@ public void onNext(String v) { }; - final AtomicReference lastValueForSubscriber2 = new AtomicReference(); + final AtomicReference lastValueForSubscriber2 = new AtomicReference<>(); final CountDownLatch oneReceived = new CountDownLatch(1); final CountDownLatch makeSlow = new CountDownLatch(1); final CountDownLatch completed = new CountDownLatch(1); @@ -940,7 +940,7 @@ public void capacityHint() { @Test public void subscribeCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final TestObserver to = new TestObserver(); + final TestObserver to = new TestObserver<>(); final ReplaySubject rp = ReplaySubject.create(); diff --git a/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java index 2d9999380b..bf74ac5998 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/SerializedSubjectTest.java @@ -32,8 +32,8 @@ public class SerializedSubjectTest extends RxJavaTest { @Test public void basic() { - SerializedSubject subject = new SerializedSubject(PublishSubject. create()); - TestObserver to = new TestObserver(); + SerializedSubject subject = new SerializedSubject<>(PublishSubject.create()); + TestObserver to = new TestObserver<>(); subject.subscribe(to); subject.onNext("hello"); subject.onComplete(); @@ -417,7 +417,7 @@ public void normal() { @Test public void onNextOnNextRace() { - Set expectedSet = new HashSet(Arrays.asList(1, 2)); + Set expectedSet = new HashSet<>(Arrays.asList(1, 2)); for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final Subject s = PublishSubject.create().toSerialized(); @@ -446,7 +446,7 @@ public void run() { .assertValueCount(2) ; - Set actualSet = new HashSet(to.values()); + Set actualSet = new HashSet<>(to.values()); assertEquals("" + actualSet, expectedSet, actualSet); } } diff --git a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java index a65544532b..9c9301176c 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java @@ -43,7 +43,7 @@ protected Subject create() { public void fusionLive() { UnicastSubject ap = UnicastSubject.create(); - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); ap.subscribe(to); @@ -68,7 +68,7 @@ public void fusionOfflie() { ap.onNext(1); ap.onComplete(); - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); ap.subscribe(to); @@ -124,7 +124,7 @@ public void fusionOfflineFailFast() { UnicastSubject ap = UnicastSubject.create(false); ap.onNext(1); ap.onError(new RuntimeException()); - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); ap.subscribe(to); to @@ -139,7 +139,7 @@ public void fusionOfflineFailFastMultipleEvents() { ap.onNext(2); ap.onNext(3); ap.onComplete(); - TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); ap.subscribe(to); to @@ -303,7 +303,7 @@ public void onErrorStatePeeking() { public void rejectSyncFusion() { UnicastSubject p = UnicastSubject.create(); - TestObserverEx to = new TestObserverEx(QueueFuseable.SYNC); + TestObserverEx to = new TestObserverEx<>(QueueFuseable.SYNC); p.subscribe(to); @@ -337,7 +337,7 @@ public void fusedDrainCancel() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final UnicastSubject p = UnicastSubject.create(); - final TestObserverEx to = new TestObserverEx(QueueFuseable.ANY); + final TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); p.subscribe(to); @@ -363,9 +363,11 @@ public void run() { public void dispose() { final int[] calls = { 0 }; - UnicastSubject us = new UnicastSubject(128, new Runnable() { + UnicastSubject us = new UnicastSubject<>(128, new Runnable() { @Override - public void run() { calls[0]++; } + public void run() { + calls[0]++; + } }); TestHelper.checkDisposed(us); @@ -393,8 +395,8 @@ public void subscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final UnicastSubject us = UnicastSubject.create(); - final TestObserverEx to1 = new TestObserverEx(); - final TestObserverEx to2 = new TestObserverEx(); + final TestObserverEx to1 = new TestObserverEx<>(); + final TestObserverEx to2 = new TestObserverEx<>(); Runnable r1 = new Runnable() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/subscribers/DefaultSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/subscribers/DefaultSubscriberTest.java index ead17680fc..562759247c 100644 --- a/src/test/java/io/reactivex/rxjava3/subscribers/DefaultSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/subscribers/DefaultSubscriberTest.java @@ -25,7 +25,7 @@ public class DefaultSubscriberTest extends RxJavaTest { static final class RequestEarly extends DefaultSubscriber { - final List events = new ArrayList(); + final List events = new ArrayList<>(); RequestEarly() { request(5); diff --git a/src/test/java/io/reactivex/rxjava3/subscribers/DisposableSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/subscribers/DisposableSubscriberTest.java index 5958110afb..b58b231467 100644 --- a/src/test/java/io/reactivex/rxjava3/subscribers/DisposableSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/subscribers/DisposableSubscriberTest.java @@ -31,9 +31,9 @@ static final class TestDisposableSubscriber extends DisposableSubscriber { int start; - final List values = new ArrayList(); + final List values = new ArrayList<>(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int completions; @@ -62,7 +62,7 @@ public void onComplete() { @Test public void normal() { - TestDisposableSubscriber tc = new TestDisposableSubscriber(); + TestDisposableSubscriber tc = new TestDisposableSubscriber<>(); assertFalse(tc.isDisposed()); assertEquals(0, tc.start); @@ -83,7 +83,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestDisposableSubscriber tc = new TestDisposableSubscriber(); + TestDisposableSubscriber tc = new TestDisposableSubscriber<>(); tc.onSubscribe(new BooleanSubscription()); @@ -103,7 +103,7 @@ public void startOnce() { @Test public void dispose() { - TestDisposableSubscriber tc = new TestDisposableSubscriber(); + TestDisposableSubscriber tc = new TestDisposableSubscriber<>(); assertFalse(tc.isDisposed()); diff --git a/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java index 5a444ee9a9..5b135a6ed2 100644 --- a/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/subscribers/ResourceSubscriberTest.java @@ -30,9 +30,9 @@ public class ResourceSubscriberTest extends RxJavaTest { static class TestResourceSubscriber extends ResourceSubscriber { - final List values = new ArrayList(); + final List values = new ArrayList<>(); - final List errors = new ArrayList(); + final List errors = new ArrayList<>(); int complete; @@ -71,13 +71,13 @@ void requestMore(long n) { @Test(expected = NullPointerException.class) public void nullResource() { - TestResourceSubscriber ro = new TestResourceSubscriber(); + TestResourceSubscriber ro = new TestResourceSubscriber<>(); ro.add(null); } @Test public void addResources() { - TestResourceSubscriber ro = new TestResourceSubscriber(); + TestResourceSubscriber ro = new TestResourceSubscriber<>(); assertFalse(ro.isDisposed()); @@ -102,7 +102,7 @@ public void addResources() { @Test public void onCompleteCleansUp() { - TestResourceSubscriber ro = new TestResourceSubscriber(); + TestResourceSubscriber ro = new TestResourceSubscriber<>(); assertFalse(ro.isDisposed()); @@ -121,7 +121,7 @@ public void onCompleteCleansUp() { @Test public void onErrorCleansUp() { - TestResourceSubscriber ro = new TestResourceSubscriber(); + TestResourceSubscriber ro = new TestResourceSubscriber<>(); assertFalse(ro.isDisposed()); @@ -140,7 +140,7 @@ public void onErrorCleansUp() { @Test public void normal() { - TestResourceSubscriber tc = new TestResourceSubscriber(); + TestResourceSubscriber tc = new TestResourceSubscriber<>(); assertFalse(tc.isDisposed()); assertEquals(0, tc.start); @@ -161,7 +161,7 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestResourceSubscriber tc = new TestResourceSubscriber(); + TestResourceSubscriber tc = new TestResourceSubscriber<>(); tc.onSubscribe(new BooleanSubscription()); @@ -181,7 +181,7 @@ public void startOnce() { @Test public void dispose() { - TestResourceSubscriber tc = new TestResourceSubscriber(); + TestResourceSubscriber tc = new TestResourceSubscriber<>(); tc.dispose(); BooleanSubscription bs = new BooleanSubscription(); @@ -219,7 +219,7 @@ protected void onStart() { static final class RequestEarly extends ResourceSubscriber { - final List events = new ArrayList(); + final List events = new ArrayList<>(); RequestEarly() { request(5); diff --git a/src/test/java/io/reactivex/rxjava3/subscribers/SafeSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/subscribers/SafeSubscriberTest.java index 0dccdc1306..e2bc1b1bbd 100644 --- a/src/test/java/io/reactivex/rxjava3/subscribers/SafeSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/subscribers/SafeSubscriberTest.java @@ -41,7 +41,7 @@ public void onNextAfterOnError() { Flowable st = Flowable.unsafeCreate(t); Subscriber w = TestHelper.mockSubscriber(); - st.subscribe(new SafeSubscriber(new TestSubscriber(w))); + st.subscribe(new SafeSubscriber<>(new TestSubscriber<>(w))); t.sendOnNext("one"); t.sendOnError(new RuntimeException("bad")); @@ -62,7 +62,7 @@ public void onCompletedAfterOnError() { Subscriber w = TestHelper.mockSubscriber(); - st.subscribe(new SafeSubscriber(new TestSubscriber(w))); + st.subscribe(new SafeSubscriber<>(new TestSubscriber<>(w))); t.sendOnNext("one"); t.sendOnError(new RuntimeException("bad")); @@ -82,7 +82,7 @@ public void onNextAfterOnCompleted() { Flowable st = Flowable.unsafeCreate(t); Subscriber w = TestHelper.mockSubscriber(); - st.subscribe(new SafeSubscriber(new TestSubscriber(w))); + st.subscribe(new SafeSubscriber<>(new TestSubscriber<>(w))); t.sendOnNext("one"); t.sendOnCompleted(); @@ -103,7 +103,7 @@ public void onErrorAfterOnCompleted() { Flowable st = Flowable.unsafeCreate(t); Subscriber w = TestHelper.mockSubscriber(); - st.subscribe(new SafeSubscriber(new TestSubscriber(w))); + st.subscribe(new SafeSubscriber<>(new TestSubscriber<>(w))); t.sendOnNext("one"); t.sendOnCompleted(); @@ -159,7 +159,7 @@ public void request(long n) { @Test public void onNextFailure() { - AtomicReference onError = new AtomicReference(); + AtomicReference onError = new AtomicReference<>(); try { OBSERVER_ONNEXT_FAIL(onError).onNext("one"); fail("expects exception to be thrown"); @@ -172,9 +172,9 @@ public void onNextFailure() { @Test public void onNextFailureSafe() { - AtomicReference onError = new AtomicReference(); + AtomicReference onError = new AtomicReference<>(); try { - SafeSubscriber safeObserver = new SafeSubscriber(OBSERVER_ONNEXT_FAIL(onError)); + SafeSubscriber safeObserver = new SafeSubscriber<>(OBSERVER_ONNEXT_FAIL(onError)); safeObserver.onSubscribe(new BooleanSubscription()); safeObserver.onNext("one"); assertNotNull(onError.get()); @@ -187,7 +187,7 @@ public void onNextFailureSafe() { @Test public void onCompleteFailure() { - AtomicReference onError = new AtomicReference(); + AtomicReference onError = new AtomicReference<>(); try { OBSERVER_ONCOMPLETED_FAIL(onError).onComplete(); fail("expects exception to be thrown"); @@ -341,16 +341,16 @@ public void onError(Throwable e) { public void onComplete() { } }; - SafeSubscriber s = new SafeSubscriber(actual); + SafeSubscriber s = new SafeSubscriber<>(actual); assertSame(actual, s.downstream); } @Test public void dispose() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - SafeSubscriber so = new SafeSubscriber(ts); + SafeSubscriber so = new SafeSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -365,9 +365,9 @@ public void dispose() { @Test public void onNextAfterComplete() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - SafeSubscriber so = new SafeSubscriber(ts); + SafeSubscriber so = new SafeSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -386,9 +386,9 @@ public void onNextAfterComplete() { @Test public void onNextNull() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - SafeSubscriber so = new SafeSubscriber(ts); + SafeSubscriber so = new SafeSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -401,9 +401,9 @@ public void onNextNull() { @Test public void onNextWithoutOnSubscribe() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); - SafeSubscriber so = new SafeSubscriber(ts); + SafeSubscriber so = new SafeSubscriber<>(ts); so.onNext(1); @@ -412,9 +412,9 @@ public void onNextWithoutOnSubscribe() { @Test public void onErrorWithoutOnSubscribe() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); - SafeSubscriber so = new SafeSubscriber(ts); + SafeSubscriber so = new SafeSubscriber<>(ts); so.onError(new TestException()); @@ -426,9 +426,9 @@ public void onErrorWithoutOnSubscribe() { @Test public void onCompleteWithoutOnSubscribe() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); - SafeSubscriber so = new SafeSubscriber(ts); + SafeSubscriber so = new SafeSubscriber<>(ts); so.onComplete(); @@ -437,9 +437,9 @@ public void onCompleteWithoutOnSubscribe() { @Test public void onNextNormal() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - SafeSubscriber so = new SafeSubscriber(ts); + SafeSubscriber so = new SafeSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -520,7 +520,7 @@ public void onComplete() { } public SafeSubscriber toSafe() { - return new SafeSubscriber(this); + return new SafeSubscriber<>(this); } public CrashDummy assertError(Class clazz) { diff --git a/src/test/java/io/reactivex/rxjava3/subscribers/SerializedSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/subscribers/SerializedSubscriberTest.java index 09b800d43e..00c7ae2103 100644 --- a/src/test/java/io/reactivex/rxjava3/subscribers/SerializedSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/subscribers/SerializedSubscriberTest.java @@ -42,7 +42,7 @@ public void before() { } private Subscriber serializedSubscriber(Subscriber subscriber) { - return new SerializedSubscriber(subscriber); + return new SerializedSubscriber<>(subscriber); } @Test @@ -164,7 +164,7 @@ public void runOutOfOrderConcurrencyTest() { try { TestConcurrencySubscriber tw = new TestConcurrencySubscriber(); // we need Synchronized + SafeSubscriber to handle synchronization plus life-cycle - Subscriber w = serializedSubscriber(new SafeSubscriber(tw)); + Subscriber w = serializedSubscriber(new SafeSubscriber<>(tw)); Future f1 = tp.submit(new OnNextThread(w, 12000)); Future f2 = tp.submit(new OnNextThread(w, 5000)); @@ -220,7 +220,7 @@ public void runConcurrencyTest() { try { TestConcurrencySubscriber tw = new TestConcurrencySubscriber(); // we need Synchronized + SafeSubscriber to handle synchronization plus life-cycle - Subscriber w = serializedSubscriber(new SafeSubscriber(tw)); + Subscriber w = serializedSubscriber(new SafeSubscriber<>(tw)); w.onSubscribe(new BooleanSubscription()); Future f1 = tp.submit(new OnNextThread(w, 12000)); @@ -276,7 +276,7 @@ public void notificationDelay() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch running = new CountDownLatch(2); - TestSubscriberEx ts = new TestSubscriberEx(new DefaultSubscriber() { + TestSubscriberEx ts = new TestSubscriberEx<>(new DefaultSubscriber() { @Override public void onComplete() { @@ -357,7 +357,7 @@ public void onNext(String t) { @Test public void threadStarvation() throws InterruptedException { - TestSubscriber ts = new TestSubscriber(new DefaultSubscriber() { + TestSubscriber ts = new TestSubscriber<>(new DefaultSubscriber() { @Override public void onComplete() { @@ -553,7 +553,7 @@ private static class TestConcurrencySubscriber extends DefaultSubscriber /** * Used to store the order and number of events received. */ - private final LinkedBlockingQueue events = new LinkedBlockingQueue(); + private final LinkedBlockingQueue events = new LinkedBlockingQueue<>(); private final int waitTime; @SuppressWarnings("unused") @@ -849,7 +849,7 @@ protected void captureMaxThreads() { public void errorReentry() { List errors = TestHelper.trackPluginErrors(); try { - final AtomicReference> serial = new AtomicReference>(); + final AtomicReference> serial = new AtomicReference<>(); TestSubscriber ts = new TestSubscriber() { @Override @@ -859,7 +859,7 @@ public void onNext(Integer v) { super.onNext(v); } }; - SerializedSubscriber sobs = new SerializedSubscriber(ts); + SerializedSubscriber sobs = new SerializedSubscriber<>(ts); sobs.onSubscribe(new BooleanSubscription()); serial.set(sobs); @@ -876,7 +876,7 @@ public void onNext(Integer v) { @Test public void completeReentry() { - final AtomicReference> serial = new AtomicReference>(); + final AtomicReference> serial = new AtomicReference<>(); TestSubscriber ts = new TestSubscriber() { @Override @@ -886,7 +886,7 @@ public void onNext(Integer v) { super.onNext(v); } }; - SerializedSubscriber sobs = new SerializedSubscriber(ts); + SerializedSubscriber sobs = new SerializedSubscriber<>(ts); sobs.onSubscribe(new BooleanSubscription()); serial.set(sobs); @@ -899,9 +899,9 @@ public void onNext(Integer v) { @Test public void dispose() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - SerializedSubscriber so = new SerializedSubscriber(ts); + SerializedSubscriber so = new SerializedSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -915,9 +915,9 @@ public void dispose() { @Test public void onCompleteRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final SerializedSubscriber so = new SerializedSubscriber(ts); + final SerializedSubscriber so = new SerializedSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -941,9 +941,9 @@ public void run() { @Test public void onNextOnCompleteRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final SerializedSubscriber so = new SerializedSubscriber(ts); + final SerializedSubscriber so = new SerializedSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -977,9 +977,9 @@ public void run() { @Test public void onNextOnErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final SerializedSubscriber so = new SerializedSubscriber(ts); + final SerializedSubscriber so = new SerializedSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -1015,9 +1015,9 @@ public void run() { @Test public void onNextOnErrorRaceDelayError() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final SerializedSubscriber so = new SerializedSubscriber(ts, true); + final SerializedSubscriber so = new SerializedSubscriber<>(ts, true); BooleanSubscription bs = new BooleanSubscription(); @@ -1056,9 +1056,9 @@ public void startOnce() { List error = TestHelper.trackPluginErrors(); try { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); - final SerializedSubscriber so = new SerializedSubscriber(ts); + final SerializedSubscriber so = new SerializedSubscriber<>(ts); so.onSubscribe(new BooleanSubscription()); @@ -1079,9 +1079,9 @@ public void onCompleteOnErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { List errors = TestHelper.trackPluginErrors(); try { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); - final SerializedSubscriber so = new SerializedSubscriber(ts); + final SerializedSubscriber so = new SerializedSubscriber<>(ts); BooleanSubscription bs = new BooleanSubscription(); @@ -1124,9 +1124,9 @@ public void run() { @Test public void nullOnNext() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); - final SerializedSubscriber so = new SerializedSubscriber(ts); + final SerializedSubscriber so = new SerializedSubscriber<>(ts); so.onSubscribe(new BooleanSubscription()); diff --git a/src/test/java/io/reactivex/rxjava3/subscribers/TestSubscriberTest.java b/src/test/java/io/reactivex/rxjava3/subscribers/TestSubscriberTest.java index 33d95bea88..6127d31ff5 100644 --- a/src/test/java/io/reactivex/rxjava3/subscribers/TestSubscriberTest.java +++ b/src/test/java/io/reactivex/rxjava3/subscribers/TestSubscriberTest.java @@ -44,7 +44,7 @@ public class TestSubscriberTest extends RxJavaTest { @Test public void assertTestSubscriber() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); oi.subscribe(ts); ts.assertValues(1, 2); @@ -56,7 +56,7 @@ public void assertTestSubscriber() { @Test public void assertNotMatchCount() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); oi.subscribe(ts); thrown.expect(AssertionError.class); @@ -72,7 +72,7 @@ public void assertNotMatchCount() { @Test public void assertNotMatchValue() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); oi.subscribe(ts); thrown.expect(AssertionError.class); @@ -88,7 +88,7 @@ public void assertNotMatchValue() { @Test public void assertTerminalEventNotReceived() { PublishProcessor p = PublishProcessor.create(); - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); p.subscribe(ts); p.onNext(1); @@ -109,7 +109,7 @@ public void wrappingMock() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); Subscriber mockSubscriber = TestHelper.mockSubscriber(); - oi.subscribe(new TestSubscriber(mockSubscriber)); + oi.subscribe(new TestSubscriber<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -122,7 +122,7 @@ public void wrappingMock() { public void wrappingMockWhenUnsubscribeInvolved() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)).take(2); Subscriber mockSubscriber = TestHelper.mockSubscriber(); - oi.subscribe(new TestSubscriber(mockSubscriber)); + oi.subscribe(new TestSubscriber<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -134,14 +134,14 @@ public void wrappingMockWhenUnsubscribeInvolved() { @Test public void assertError() { RuntimeException e = new RuntimeException("Oops"); - TestSubscriber subscriber = new TestSubscriber(); + TestSubscriber subscriber = new TestSubscriber<>(); Flowable.error(e).subscribe(subscriber); subscriber.assertError(e); } @Test public void awaitTerminalEventWithDuration() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).subscribe(ts); ts.awaitDone(1, TimeUnit.SECONDS); ts.assertComplete(); @@ -150,7 +150,7 @@ public void awaitTerminalEventWithDuration() { @Test public void awaitTerminalEventWithDurationAndUnsubscribeOnTimeout() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final AtomicBoolean unsub = new AtomicBoolean(false); Flowable.just(1) // @@ -169,28 +169,28 @@ public void run() { @Test(expected = NullPointerException.class) public void nullDelegate1() { - TestSubscriber ts = new TestSubscriber(null); + TestSubscriber ts = new TestSubscriber<>(null); ts.onComplete(); } @Test(expected = NullPointerException.class) public void nullDelegate2() { - TestSubscriber ts = new TestSubscriber(null); + TestSubscriber ts = new TestSubscriber<>(null); ts.onComplete(); } @Test(expected = NullPointerException.class) public void nullDelegate3() { - TestSubscriber ts = new TestSubscriber(null, 0L); + TestSubscriber ts = new TestSubscriber<>(null, 0L); ts.onComplete(); } @Test public void delegate1() { - TestSubscriber ts0 = new TestSubscriber(); + TestSubscriber ts0 = new TestSubscriber<>(); ts0.onSubscribe(EmptySubscription.INSTANCE); - TestSubscriber ts = new TestSubscriber(ts0); + TestSubscriber ts = new TestSubscriber<>(ts0); ts.onComplete(); ts0.assertComplete(); @@ -199,8 +199,8 @@ public void delegate1() { @Test public void delegate2() { - TestSubscriber ts1 = new TestSubscriber(); - TestSubscriber ts2 = new TestSubscriber(ts1); + TestSubscriber ts1 = new TestSubscriber<>(); + TestSubscriber ts2 = new TestSubscriber<>(ts1); ts2.onComplete(); ts1.assertComplete(); @@ -208,21 +208,21 @@ public void delegate2() { @Test public void delegate3() { - TestSubscriber ts1 = new TestSubscriber(); - TestSubscriber ts2 = new TestSubscriber(ts1, 0L); + TestSubscriber ts1 = new TestSubscriber<>(); + TestSubscriber ts2 = new TestSubscriber<>(ts1, 0L); ts2.onComplete(); ts1.assertComplete(); } @Test public void unsubscribed() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); assertFalse(ts.isCancelled()); } @Test public void noErrors() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onError(new TestException()); try { ts.assertNoErrors(); @@ -235,7 +235,7 @@ public void noErrors() { @Test public void notCompleted() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { ts.assertComplete(); } catch (AssertionError ex) { @@ -247,7 +247,7 @@ public void notCompleted() { @Test public void multipleCompletions() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onComplete(); ts.onComplete(); try { @@ -261,7 +261,7 @@ public void multipleCompletions() { @Test public void completed() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onComplete(); try { ts.assertNotComplete(); @@ -274,7 +274,7 @@ public void completed() { @Test public void multipleCompletions2() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onComplete(); ts.onComplete(); try { @@ -288,7 +288,7 @@ public void multipleCompletions2() { @Test public void multipleErrors() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -312,7 +312,7 @@ public void multipleErrors() { @Test public void multipleErrors2() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -333,7 +333,7 @@ public void multipleErrors2() { @Test public void multipleErrors3() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -354,7 +354,7 @@ public void multipleErrors3() { @Test public void multipleErrors4() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -375,7 +375,7 @@ public void multipleErrors4() { @Test public void differentError() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onError(new TestException()); try { ts.assertError(new TestException()); @@ -388,7 +388,7 @@ public void differentError() { @Test public void differentError2() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onError(new RuntimeException()); try { ts.assertError(new TestException()); @@ -401,7 +401,7 @@ public void differentError2() { @Test public void differentError3() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onError(new RuntimeException()); try { ts.assertError(TestException.class); @@ -415,7 +415,7 @@ public void differentError3() { @Test public void differentError4() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onError(new RuntimeException()); try { ts.assertError(Functions.alwaysFalse()); @@ -428,7 +428,7 @@ public void differentError4() { @Test public void errorInPredicate() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onError(new RuntimeException()); try { ts.assertError(new Predicate() { @@ -446,7 +446,7 @@ public boolean test(Throwable throwable) throws Exception { @Test public void noError() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { ts.assertError(TestException.class); } catch (AssertionError ex) { @@ -458,7 +458,7 @@ public void noError() { @Test public void noError2() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { ts.assertError(new TestException()); } catch (AssertionError ex) { @@ -470,7 +470,7 @@ public void noError2() { @Test public void noError3() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { ts.assertError(Functions.alwaysTrue()); } catch (AssertionError ex) { @@ -482,7 +482,7 @@ public void noError3() { @Test public void interruptTerminalEventAwait() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Thread t0 = Thread.currentThread(); Worker w = Schedulers.computation().createWorker(); @@ -508,7 +508,7 @@ public void run() { @Test public void interruptTerminalEventAwaitTimed() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Thread t0 = Thread.currentThread(); Worker w = Schedulers.computation().createWorker(); @@ -535,7 +535,7 @@ public void run() { @Test public void interruptTerminalEventAwaitAndUnsubscribe() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); final Thread t0 = Thread.currentThread(); Worker w = Schedulers.computation().createWorker(); @@ -563,7 +563,7 @@ public void run() { @Test public void noTerminalEventBut1Completed() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onComplete(); @@ -577,7 +577,7 @@ public void noTerminalEventBut1Completed() { @Test public void noTerminalEventBut1Error() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onError(new TestException()); @@ -591,7 +591,7 @@ public void noTerminalEventBut1Error() { @Test public void noTerminalEventBut1Error1Complete() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onComplete(); ts.onError(new TestException()); @@ -613,7 +613,7 @@ public void noTerminalEventBut1Error1Complete() { @Test public void noTerminalEventBut2Errors() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); @@ -635,7 +635,7 @@ public void noTerminalEventBut2Errors() { @Test public void noValues() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onNext(1); try { @@ -648,7 +648,7 @@ public void noValues() { @Test public void valueCount() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onNext(1); ts.onNext(2); @@ -668,7 +668,7 @@ public void onComplete() { throw new TestException(); } }; - TestSubscriber ts = new TestSubscriber(ts0); + TestSubscriber ts = new TestSubscriber<>(ts0); try { ts.onComplete(); @@ -687,7 +687,7 @@ public void onError(Throwable e) { throw new TestException(); } }; - TestSubscriber ts = new TestSubscriber(ts0); + TestSubscriber ts = new TestSubscriber<>(ts0); try { ts.onError(new RuntimeException()); @@ -1207,7 +1207,7 @@ public void assertValueSequence() { @Test public void assertEmpty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); try { ts.assertEmpty(); @@ -1232,7 +1232,7 @@ public void assertEmpty() { @Test public void awaitDoneTimed() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Thread.currentThread().interrupt(); @@ -1245,7 +1245,7 @@ public void awaitDoneTimed() { @Test public void assertErrorMultiple() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); TestException e = new TestException(); ts.onError(e); @@ -1267,7 +1267,7 @@ public void assertErrorMultiple() { @Test public void assertComplete() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1294,7 +1294,7 @@ public void assertComplete() { @Test public void completeWithoutOnSubscribe() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.onComplete(); @@ -1303,7 +1303,7 @@ public void completeWithoutOnSubscribe() { @Test public void completeDelegateThrows() throws Exception { - TestSubscriber ts = new TestSubscriber(new FlowableSubscriber() { + TestSubscriber ts = new TestSubscriber<>(new FlowableSubscriber() { @Override public void onSubscribe(Subscription s) { @@ -1339,7 +1339,7 @@ public void onComplete() { @Test public void errorDelegateThrows() throws Exception { - TestSubscriber ts = new TestSubscriber(new FlowableSubscriber() { + TestSubscriber ts = new TestSubscriber<>(new FlowableSubscriber() { @Override public void onSubscribe(Subscription s) { @@ -1375,7 +1375,7 @@ public void onComplete() { @Test public void assertValuePredicateEmpty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.empty().subscribe(ts); @@ -1390,7 +1390,7 @@ public void assertValuePredicateEmpty() { @Test public void assertValuePredicateMatch() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).subscribe(ts); @@ -1403,7 +1403,7 @@ public void assertValuePredicateMatch() { @Test public void assertValuePredicateNoMatch() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1).subscribe(ts); @@ -1418,7 +1418,7 @@ public void assertValuePredicateNoMatch() { @Test public void assertValuePredicateMatchButMore() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2).subscribe(ts); @@ -1433,7 +1433,7 @@ public void assertValuePredicateMatchButMore() { @Test public void assertValueAtPredicateEmpty() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.empty().subscribe(ts); @@ -1448,7 +1448,7 @@ public void assertValueAtPredicateEmpty() { @Test public void assertValueAtPredicateMatch() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2).subscribe(ts); @@ -1461,7 +1461,7 @@ public void assertValueAtPredicateMatch() { @Test public void assertValueAtPredicateNoMatch() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2, 3).subscribe(ts); @@ -1476,7 +1476,7 @@ public void assertValueAtPredicateNoMatch() { @Test public void assertValueAtInvalidIndex() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); Flowable.just(1, 2).subscribe(ts); @@ -1563,7 +1563,7 @@ public void timeoutIndicated3() throws InterruptedException { @Test public void disposeIndicated() { - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); ts.cancel(); try { diff --git a/src/test/java/io/reactivex/rxjava3/tck/FromFutureTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/FromFutureTckTest.java index ee0dd0695a..7b47ac3192 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/FromFutureTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/FromFutureTckTest.java @@ -25,7 +25,7 @@ public class FromFutureTckTest extends BaseTck { @Override public Publisher createPublisher(final long elements) { - FutureTask ft = new FutureTask(new Callable() { + FutureTask ft = new FutureTask<>(new Callable() { @Override public Long call() throws Exception { return 1L; diff --git a/src/test/java/io/reactivex/rxjava3/tck/MulticastProcessorTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/MulticastProcessorTckTest.java index a1089b7370..ed6de68012 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/MulticastProcessorTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/MulticastProcessorTckTest.java @@ -32,7 +32,7 @@ public MulticastProcessorTckTest() { @Override public Processor createIdentityProcessor(int bufferSize) { MulticastProcessor mp = MulticastProcessor.create(); - return new RefCountProcessor(mp); + return new RefCountProcessor<>(mp); } @Override diff --git a/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java b/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java index 908f3e010c..77bdfef211 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java +++ b/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java @@ -45,7 +45,7 @@ @SuppressWarnings("unchecked") RefCountProcessor(FlowableProcessor actual) { this.actual = actual; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.subscribers = new AtomicReference[]>(EMPTY); } @@ -75,7 +75,7 @@ public void onComplete() { @Override protected void subscribeActual(Subscriber s) { - RefCountSubscriber rcs = new RefCountSubscriber(s, this); + RefCountSubscriber rcs = new RefCountSubscriber<>(s, this); if (!add(rcs)) { EmptySubscription.error(new IllegalStateException("RefCountProcessor terminated"), s); return; diff --git a/src/test/java/io/reactivex/rxjava3/tck/UnicastProcessorTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/UnicastProcessorTckTest.java index f97b6ee83a..079c349245 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/UnicastProcessorTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/UnicastProcessorTckTest.java @@ -32,7 +32,7 @@ public UnicastProcessorTckTest() { @Override public Processor createIdentityProcessor(int bufferSize) { UnicastProcessor up = UnicastProcessor.create(); - return new RefCountProcessor(up); + return new RefCountProcessor<>(up); } @Override diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index 886728709d..9bfb21668e 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -152,7 +152,7 @@ public static void checkUtilityClass(Class clazz) { } public static List trackPluginErrors() { - final List list = Collections.synchronizedList(new ArrayList()); + final List list = Collections.synchronizedList(new ArrayList<>()); RxJavaPlugins.setErrorHandler(new Consumer() { @Override @@ -2901,7 +2901,7 @@ public Flowable apply(Flowable upstream) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new StripBoundarySubscriber(s)); + source.subscribe(new StripBoundarySubscriber<>(s)); } static final class StripBoundarySubscriber implements FlowableSubscriber, QueueSubscription { @@ -3017,7 +3017,7 @@ public Observable apply(Observable upstream) { @Override protected void subscribeActual(Observer observer) { - source.subscribe(new StripBoundaryObserver(observer)); + source.subscribe(new StripBoundaryObserver<>(observer)); } static final class StripBoundaryObserver implements Observer, QueueDisposable { diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverEx.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverEx.java index 9747d034bb..ff7df2d8b3 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverEx.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverEx.java @@ -36,7 +36,7 @@ public class TestObserverEx private final Observer downstream; /** Holds the current subscription if any. */ - private final AtomicReference upstream = new AtomicReference(); + private final AtomicReference upstream = new AtomicReference<>(); private QueueDisposable qd; diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java index e374aa67a2..9ee8a1cfc5 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java @@ -45,7 +45,7 @@ public class TestObserverExTest extends RxJavaTest { @Test public void assertTestObserverEx() { Observable oi = Observable.fromIterable(Arrays.asList(1, 2)); - TestObserverEx subscriber = new TestObserverEx(); + TestObserverEx subscriber = new TestObserverEx<>(); oi.subscribe(subscriber); subscriber.assertValues(1, 2); @@ -56,7 +56,7 @@ public void assertTestObserverEx() { @Test public void assertNotMatchCount() { Observable oi = Observable.fromIterable(Arrays.asList(1, 2)); - TestObserverEx subscriber = new TestObserverEx(); + TestObserverEx subscriber = new TestObserverEx<>(); oi.subscribe(subscriber); thrown.expect(AssertionError.class); @@ -71,7 +71,7 @@ public void assertNotMatchCount() { @Test public void assertNotMatchValue() { Observable oi = Observable.fromIterable(Arrays.asList(1, 2)); - TestObserverEx subscriber = new TestObserverEx(); + TestObserverEx subscriber = new TestObserverEx<>(); oi.subscribe(subscriber); thrown.expect(AssertionError.class); @@ -86,7 +86,7 @@ public void assertNotMatchValue() { @Test public void assertNeverAtNotMatchingValue() { Observable oi = Observable.fromIterable(Arrays.asList(1, 2)); - TestObserverEx subscriber = new TestObserverEx(); + TestObserverEx subscriber = new TestObserverEx<>(); oi.subscribe(subscriber); subscriber.assertNever(3); @@ -97,7 +97,7 @@ public void assertNeverAtNotMatchingValue() { @Test public void assertNeverAtMatchingValue() { Observable oi = Observable.fromIterable(Arrays.asList(1, 2)); - TestObserverEx subscriber = new TestObserverEx(); + TestObserverEx subscriber = new TestObserverEx<>(); oi.subscribe(subscriber); subscriber.assertValues(1, 2); @@ -111,7 +111,7 @@ public void assertNeverAtMatchingValue() { @Test public void assertNeverAtMatchingPredicate() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1, 2).subscribe(to); @@ -129,7 +129,7 @@ public boolean test(final Integer o) throws Exception { @Test public void assertNeverAtNotMatchingPredicate() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(2, 3).subscribe(to); @@ -144,7 +144,7 @@ public boolean test(final Integer o) throws Exception { @Test public void assertTerminalEventNotReceived() { PublishSubject p = PublishSubject.create(); - TestObserverEx subscriber = new TestObserverEx(); + TestObserverEx subscriber = new TestObserverEx<>(); p.subscribe(subscriber); p.onNext(1); @@ -165,7 +165,7 @@ public void wrappingMock() { Observer mockSubscriber = TestHelper.mockObserver(); - oi.subscribe(new TestObserverEx(mockSubscriber)); + oi.subscribe(new TestObserverEx<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -178,7 +178,7 @@ public void wrappingMock() { public void wrappingMockWhenUnsubscribeInvolved() { Observable oi = Observable.fromIterable(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)).take(2); Observer mockSubscriber = TestHelper.mockObserver(); - oi.subscribe(new TestObserverEx(mockSubscriber)); + oi.subscribe(new TestObserverEx<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -189,12 +189,12 @@ public void wrappingMockWhenUnsubscribeInvolved() { @Test public void errorSwallowed() { - Observable.error(new RuntimeException()).subscribe(new TestObserverEx()); + Observable.error(new RuntimeException()).subscribe(new TestObserverEx<>()); } @Test public void nullExpected() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onNext(1); try { @@ -208,7 +208,7 @@ public void nullExpected() { @Test public void nullActual() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onNext(null); try { @@ -222,7 +222,7 @@ public void nullActual() { @Test public void terminalErrorOnce() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onError(new TestException()); to.onError(new TestException()); @@ -237,7 +237,7 @@ public void terminalErrorOnce() { @Test public void terminalCompletedOnce() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onComplete(); to.onComplete(); @@ -252,7 +252,7 @@ public void terminalCompletedOnce() { @Test public void terminalOneKind() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onError(new TestException()); to.onComplete(); @@ -267,9 +267,9 @@ public void terminalOneKind() { @Test public void createDelegate() { - TestObserverEx to1 = new TestObserverEx(); + TestObserverEx to1 = new TestObserverEx<>(); - TestObserverEx to = new TestObserverEx(to1); + TestObserverEx to = new TestObserverEx<>(to1); to.assertNotSubscribed(); @@ -326,7 +326,7 @@ public void createDelegate() { @Test public void assertError() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); try { to.assertError(TestException.class); @@ -451,7 +451,7 @@ public void valueAndClass() { @Test public void assertFailure() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -474,7 +474,7 @@ public void assertFailure() { @Test public void assertFuseable() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -494,10 +494,10 @@ public void assertFuseable() { // expected } - to = new TestObserverEx(); + to = new TestObserverEx<>(); to.setInitialFusionMode(QueueFuseable.ANY); - to.onSubscribe(new ScalarDisposable(to, 1)); + to.onSubscribe(new ScalarDisposable<>(to, 1)); to.assertFuseable(); @@ -521,7 +521,7 @@ public void assertFuseable() { @Test public void assertTerminated() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.assertNotTerminated(); @@ -537,7 +537,7 @@ public void assertTerminated() { @Test public void assertResult() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -574,7 +574,7 @@ public void assertResult() { @Test public void await() throws Exception { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -596,7 +596,7 @@ public void await() throws Exception { to.assertNoErrors().assertComplete(); - final TestObserverEx to1 = new TestObserverEx(); + final TestObserverEx to1 = new TestObserverEx<>(); to1.onSubscribe(Disposable.empty()); @@ -612,7 +612,7 @@ public void run() { @Test public void errors() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -627,7 +627,7 @@ public void errors() { @Test public void onNext() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -658,7 +658,7 @@ public void fusionModeToString() { @Test public void multipleTerminals() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -701,7 +701,7 @@ public void multipleTerminals() { @Test public void assertValue() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -735,13 +735,13 @@ public void assertValue() { @Test public void onNextMisbehave() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onNext(1); to.assertError(IllegalStateException.class); - to = new TestObserverEx(); + to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -752,7 +752,7 @@ public void onNextMisbehave() { @Test public void assertTerminated2() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -778,7 +778,7 @@ public void assertTerminated2() { // expected } - to = new TestObserverEx(); + to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -795,13 +795,13 @@ public void assertTerminated2() { @Test public void onSubscribe() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(null); to.assertError(NullPointerException.class); - to = new TestObserverEx(); + to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -813,7 +813,7 @@ public void onSubscribe() { to.assertError(IllegalStateException.class); - to = new TestObserverEx(); + to = new TestObserverEx<>(); to.dispose(); d1 = Disposable.empty(); @@ -826,7 +826,7 @@ public void onSubscribe() { @Test public void assertValueSequence() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -859,7 +859,7 @@ public void assertValueSequence() { @Test public void assertEmpty() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); try { to.assertEmpty(); @@ -884,7 +884,7 @@ public void assertEmpty() { @Test public void awaitDoneTimed() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Thread.currentThread().interrupt(); @@ -897,7 +897,7 @@ public void awaitDoneTimed() { @Test public void assertNotSubscribed() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.assertNotSubscribed(); @@ -913,7 +913,7 @@ public void assertNotSubscribed() { @Test public void assertErrorMultiple() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); TestException e = new TestException(); to.errors().add(e); @@ -947,7 +947,7 @@ public void assertErrorMultiple() { @Test public void errorInPredicate() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onError(new RuntimeException()); try { to.assertError(new Predicate() { @@ -965,7 +965,7 @@ public boolean test(Throwable throwable) throws Exception { @Test public void assertComplete() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); @@ -992,7 +992,7 @@ public void assertComplete() { @Test public void completeWithoutOnSubscribe() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onComplete(); @@ -1001,7 +1001,7 @@ public void completeWithoutOnSubscribe() { @Test public void completeDelegateThrows() { - TestObserverEx to = new TestObserverEx(new Observer() { + TestObserverEx to = new TestObserverEx<>(new Observer() { @Override public void onSubscribe(Disposable d) { @@ -1037,7 +1037,7 @@ public void onComplete() { @Test public void errorDelegateThrows() { - TestObserverEx to = new TestObserverEx(new Observer() { + TestObserverEx to = new TestObserverEx<>(new Observer() { @Override public void onSubscribe(Disposable d) { @@ -1073,7 +1073,7 @@ public void onComplete() { @Test public void syncQueueThrows() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.setInitialFusionMode(QueueFuseable.SYNC); Observable.range(1, 5) @@ -1091,7 +1091,7 @@ public void syncQueueThrows() { @Test public void asyncQueueThrows() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.setInitialFusionMode(QueueFuseable.ANY); UnicastSubject up = UnicastSubject.create(); @@ -1129,7 +1129,7 @@ public void errorMeansDisposed() { @Test public void asyncFusion() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.setInitialFusionMode(QueueFuseable.ANY); UnicastSubject up = UnicastSubject.create(); @@ -1148,7 +1148,7 @@ public void asyncFusion() { @Test public void assertValuePredicateEmpty() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.empty().subscribe(to); @@ -1163,7 +1163,7 @@ public void assertValuePredicateEmpty() { @Test public void assertValuePredicateMatch() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1).subscribe(to); @@ -1176,7 +1176,7 @@ public void assertValuePredicateMatch() { @Test public void assertValuePredicateNoMatch() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1).subscribe(to); @@ -1191,7 +1191,7 @@ public void assertValuePredicateNoMatch() { @Test public void assertValuePredicateMatchButMore() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1, 2).subscribe(to); @@ -1206,7 +1206,7 @@ public void assertValuePredicateMatchButMore() { @Test public void assertValueAtPredicateEmpty() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.empty().subscribe(to); @@ -1221,7 +1221,7 @@ public void assertValueAtPredicateEmpty() { @Test public void assertValueAtPredicateMatch() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1, 2).subscribe(to); @@ -1234,7 +1234,7 @@ public void assertValueAtPredicateMatch() { @Test public void assertValueAtPredicateNoMatch() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1, 2, 3).subscribe(to); @@ -1249,7 +1249,7 @@ public void assertValueAtPredicateNoMatch() { @Test public void assertValueAtInvalidIndex() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just(1, 2).subscribe(to); @@ -1264,7 +1264,7 @@ public void assertValueAtInvalidIndex() { @Test public void assertValueAtIndexEmpty() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.empty().subscribe(to); @@ -1275,7 +1275,7 @@ public void assertValueAtIndexEmpty() { @Test public void assertValueAtIndexMatch() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just("a", "b").subscribe(to); @@ -1284,7 +1284,7 @@ public void assertValueAtIndexMatch() { @Test public void assertValueAtIndexNoMatch() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just("a", "b", "c").subscribe(to); @@ -1295,7 +1295,7 @@ public void assertValueAtIndexNoMatch() { @Test public void assertValueAtIndexInvalidIndex() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); Observable.just("a", "b").subscribe(to); @@ -1322,7 +1322,7 @@ public void withTag() { @Test public void assertValuesOnly() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); to.assertValuesOnly(); @@ -1335,7 +1335,7 @@ public void assertValuesOnly() { @Test public void assertValuesOnlyThrowsOnUnexpectedValue() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); to.assertValuesOnly(); @@ -1354,7 +1354,7 @@ public void assertValuesOnlyThrowsOnUnexpectedValue() { @Test public void assertValuesOnlyThrowsWhenCompleted() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); to.onComplete(); @@ -1369,7 +1369,7 @@ public void assertValuesOnlyThrowsWhenCompleted() { @Test public void assertValuesOnlyThrowsWhenErrored() { - TestObserverEx to = new TestObserverEx(); + TestObserverEx to = new TestObserverEx<>(); to.onSubscribe(Disposable.empty()); to.onError(new TestException()); diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java index 23297689b8..43d4087c3e 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberEx.java @@ -88,7 +88,7 @@ public TestSubscriberEx(Subscriber actual, long initialRequest) { throw new IllegalArgumentException("Negative initial request not allowed"); } this.downstream = actual; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.missedRequested = new AtomicLong(initialRequest); } diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberExTest.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberExTest.java index 24a9e84c9b..308c10e041 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberExTest.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestSubscriberExTest.java @@ -44,7 +44,7 @@ public class TestSubscriberExTest extends RxJavaTest { @Test public void assertTestSubscriberEx() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); oi.subscribe(ts); ts.assertValues(1, 2); @@ -55,7 +55,7 @@ public void assertTestSubscriberEx() { @Test public void assertNotMatchCount() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); oi.subscribe(ts); thrown.expect(AssertionError.class); @@ -68,7 +68,7 @@ public void assertNotMatchCount() { @Test public void assertNotMatchValue() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); oi.subscribe(ts); thrown.expect(AssertionError.class); @@ -81,7 +81,7 @@ public void assertNotMatchValue() { @Test public void assertNeverAtNotMatchingValue() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); oi.subscribe(ts); ts.assertNever(3); @@ -92,7 +92,7 @@ public void assertNeverAtNotMatchingValue() { @Test public void assertNeverAtMatchingValue() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); oi.subscribe(ts); ts.assertValues(1, 2); @@ -106,7 +106,7 @@ public void assertNeverAtMatchingValue() { @Test public void assertNeverAtMatchingPredicate() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1, 2).subscribe(ts); @@ -124,7 +124,7 @@ public boolean test(final Integer o) throws Exception { @Test public void assertNeverAtNotMatchingPredicate() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(2, 3).subscribe(ts); @@ -139,7 +139,7 @@ public boolean test(final Integer o) throws Exception { @Test public void assertTerminalEventNotReceived() { PublishProcessor p = PublishProcessor.create(); - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); p.subscribe(ts); p.onNext(1); @@ -157,7 +157,7 @@ public void wrappingMock() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2)); Subscriber mockSubscriber = TestHelper.mockSubscriber(); - oi.subscribe(new TestSubscriberEx(mockSubscriber)); + oi.subscribe(new TestSubscriberEx<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -170,7 +170,7 @@ public void wrappingMock() { public void wrappingMockWhenUnsubscribeInvolved() { Flowable oi = Flowable.fromIterable(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)).take(2); Subscriber mockSubscriber = TestHelper.mockSubscriber(); - oi.subscribe(new TestSubscriberEx(mockSubscriber)); + oi.subscribe(new TestSubscriberEx<>(mockSubscriber)); InOrder inOrder = inOrder(mockSubscriber); inOrder.verify(mockSubscriber, times(1)).onNext(1); @@ -182,14 +182,14 @@ public void wrappingMockWhenUnsubscribeInvolved() { @Test public void assertError() { RuntimeException e = new RuntimeException("Oops"); - TestSubscriberEx subscriber = new TestSubscriberEx(); + TestSubscriberEx subscriber = new TestSubscriberEx<>(); Flowable.error(e).subscribe(subscriber); subscriber.assertError(e); } @Test public void awaitTerminalEventWithDurationAndUnsubscribeOnTimeout() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final AtomicBoolean unsub = new AtomicBoolean(false); Flowable.just(1) // @@ -208,28 +208,28 @@ public void run() { @Test(expected = NullPointerException.class) public void nullDelegate1() { - TestSubscriberEx ts = new TestSubscriberEx(null); + TestSubscriberEx ts = new TestSubscriberEx<>(null); ts.onComplete(); } @Test(expected = NullPointerException.class) public void nullDelegate2() { - TestSubscriberEx ts = new TestSubscriberEx(null); + TestSubscriberEx ts = new TestSubscriberEx<>(null); ts.onComplete(); } @Test(expected = NullPointerException.class) public void nullDelegate3() { - TestSubscriberEx ts = new TestSubscriberEx(null, 0L); + TestSubscriberEx ts = new TestSubscriberEx<>(null, 0L); ts.onComplete(); } @Test public void delegate1() { - TestSubscriberEx ts0 = new TestSubscriberEx(); + TestSubscriberEx ts0 = new TestSubscriberEx<>(); ts0.onSubscribe(EmptySubscription.INSTANCE); - TestSubscriberEx ts = new TestSubscriberEx(ts0); + TestSubscriberEx ts = new TestSubscriberEx<>(ts0); ts.onComplete(); ts0.assertTerminated(); @@ -237,8 +237,8 @@ public void delegate1() { @Test public void delegate2() { - TestSubscriberEx ts1 = new TestSubscriberEx(); - TestSubscriberEx ts2 = new TestSubscriberEx(ts1); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(ts1); ts2.onComplete(); ts1.assertComplete(); @@ -246,21 +246,21 @@ public void delegate2() { @Test public void delegate3() { - TestSubscriberEx ts1 = new TestSubscriberEx(); - TestSubscriberEx ts2 = new TestSubscriberEx(ts1, 0L); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); + TestSubscriberEx ts2 = new TestSubscriberEx<>(ts1, 0L); ts2.onComplete(); ts1.assertComplete(); } @Test public void unsubscribed() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); assertFalse(ts.isCancelled()); } @Test public void noErrors() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onError(new TestException()); try { ts.assertNoErrors(); @@ -273,7 +273,7 @@ public void noErrors() { @Test public void notCompleted() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); try { ts.assertComplete(); } catch (AssertionError ex) { @@ -285,7 +285,7 @@ public void notCompleted() { @Test public void multipleCompletions() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onComplete(); ts.onComplete(); try { @@ -299,7 +299,7 @@ public void multipleCompletions() { @Test public void completed() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onComplete(); try { ts.assertNotComplete(); @@ -312,7 +312,7 @@ public void completed() { @Test public void multipleCompletions2() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onComplete(); ts.onComplete(); try { @@ -326,7 +326,7 @@ public void multipleCompletions2() { @Test public void multipleErrors() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -350,7 +350,7 @@ public void multipleErrors() { @Test public void multipleErrors2() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -371,7 +371,7 @@ public void multipleErrors2() { @Test public void multipleErrors3() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -392,7 +392,7 @@ public void multipleErrors3() { @Test public void multipleErrors4() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); ts.onError(new TestException()); @@ -413,7 +413,7 @@ public void multipleErrors4() { @Test public void differentError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onError(new TestException()); try { ts.assertError(new TestException()); @@ -426,7 +426,7 @@ public void differentError() { @Test public void differentError2() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onError(new RuntimeException()); try { ts.assertError(new TestException()); @@ -439,7 +439,7 @@ public void differentError2() { @Test public void differentError3() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onError(new RuntimeException()); try { ts.assertError(TestException.class); @@ -453,7 +453,7 @@ public void differentError3() { @Test public void differentError4() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onError(new RuntimeException()); try { ts.assertError(Functions.alwaysFalse()); @@ -466,7 +466,7 @@ public void differentError4() { @Test public void errorInPredicate() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onError(new RuntimeException()); try { ts.assertError(new Predicate() { @@ -484,7 +484,7 @@ public boolean test(Throwable throwable) throws Exception { @Test public void noError() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); try { ts.assertError(TestException.class); } catch (AssertionError ex) { @@ -496,7 +496,7 @@ public void noError() { @Test public void noError2() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); try { ts.assertError(new TestException()); } catch (AssertionError ex) { @@ -508,7 +508,7 @@ public void noError2() { @Test public void noError3() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); try { ts.assertError(Functions.alwaysTrue()); } catch (AssertionError ex) { @@ -520,7 +520,7 @@ public void noError3() { @Test public void interruptTerminalEventAwait() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final Thread t0 = Thread.currentThread(); Worker w = Schedulers.computation().createWorker(); @@ -546,7 +546,7 @@ public void run() { @Test public void interruptTerminalEventAwaitTimed() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final Thread t0 = Thread.currentThread(); Worker w = Schedulers.computation().createWorker(); @@ -573,7 +573,7 @@ public void run() { @Test public void interruptTerminalEventAwaitAndUnsubscribe() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); final Thread t0 = Thread.currentThread(); Worker w = Schedulers.computation().createWorker(); @@ -602,7 +602,7 @@ public void run() { @Test public void noTerminalEventBut1Completed() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onComplete(); @@ -616,7 +616,7 @@ public void noTerminalEventBut1Completed() { @Test public void noTerminalEventBut1Error() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onError(new TestException()); @@ -630,7 +630,7 @@ public void noTerminalEventBut1Error() { @Test public void noTerminalEventBut1Error1Completed() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onComplete(); ts.onError(new TestException()); @@ -645,7 +645,7 @@ public void noTerminalEventBut1Error1Completed() { @Test public void noTerminalEventBut2Errors() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(EmptySubscription.INSTANCE); ts.onError(new TestException()); @@ -667,7 +667,7 @@ public void noTerminalEventBut2Errors() { @Test public void noValues() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onNext(1); try { @@ -680,7 +680,7 @@ public void noValues() { @Test public void valueCount() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onNext(1); ts.onNext(2); @@ -700,7 +700,7 @@ public void onComplete() { throw new TestException(); } }; - TestSubscriberEx ts = new TestSubscriberEx(ts0); + TestSubscriberEx ts = new TestSubscriberEx<>(ts0); try { ts.onComplete(); @@ -719,7 +719,7 @@ public void onError(Throwable e) { throw new TestException(); } }; - TestSubscriberEx ts = new TestSubscriberEx(ts0); + TestSubscriberEx ts = new TestSubscriberEx<>(ts0); try { ts.onError(new RuntimeException()); @@ -732,9 +732,9 @@ public void onError(Throwable e) { @Test public void createDelegate() { - TestSubscriberEx ts1 = new TestSubscriberEx(); + TestSubscriberEx ts1 = new TestSubscriberEx<>(); - TestSubscriberEx ts = new TestSubscriberEx(ts1); + TestSubscriberEx ts = new TestSubscriberEx<>(ts1); ts.assertNotSubscribed(); @@ -791,7 +791,7 @@ public void createDelegate() { @Test public void assertError2() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); try { ts.assertError(TestException.class); @@ -916,7 +916,7 @@ public void valueAndClass() { @Test public void assertFailure() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -939,7 +939,7 @@ public void assertFailure() { @Test public void assertFuseable() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -958,10 +958,10 @@ public void assertFuseable() { } catch (AssertionError ex) { // expected } - ts = new TestSubscriberEx(); + ts = new TestSubscriberEx<>(); ts.setInitialFusionMode(QueueFuseable.ANY); - ts.onSubscribe(new ScalarSubscription(ts, 1)); + ts.onSubscribe(new ScalarSubscription<>(ts, 1)); ts.assertFuseable(); @@ -985,7 +985,7 @@ public void assertFuseable() { @Test public void assertTerminated() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.assertNotTerminated(); @@ -1001,7 +1001,7 @@ public void assertTerminated() { @Test public void assertResult() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1038,7 +1038,7 @@ public void assertResult() { @Test public void await() throws Exception { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1064,7 +1064,7 @@ public void await() throws Exception { assertTrue(ts.await(5, TimeUnit.SECONDS)); - final TestSubscriberEx ts1 = new TestSubscriberEx(); + final TestSubscriberEx ts1 = new TestSubscriberEx<>(); ts1.onSubscribe(new BooleanSubscription()); @@ -1080,7 +1080,7 @@ public void run() { @Test public void errors() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1095,7 +1095,7 @@ public void errors() { @Test public void onNext() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1127,7 +1127,7 @@ public void fusionModeToString() { @Test public void multipleTerminals() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1170,7 +1170,7 @@ public void multipleTerminals() { @Test public void assertValue() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1204,13 +1204,13 @@ public void assertValue() { @Test public void onNextMisbehave() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onNext(1); ts.assertError(IllegalStateException.class); - ts = new TestSubscriberEx(); + ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1221,7 +1221,7 @@ public void onNextMisbehave() { @Test public void awaitTerminalEventInterrupt() { - final TestSubscriberEx ts = new TestSubscriberEx(); + final TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1250,7 +1250,7 @@ public void awaitTerminalEventInterrupt() { @Test public void assertTerminated2() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1276,7 +1276,7 @@ public void assertTerminated2() { // expected } - ts = new TestSubscriberEx(); + ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1293,13 +1293,13 @@ public void assertTerminated2() { @Test public void onSubscribe() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(null); ts.assertError(NullPointerException.class); - ts = new TestSubscriberEx(); + ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1311,7 +1311,7 @@ public void onSubscribe() { ts.assertError(IllegalStateException.class); - ts = new TestSubscriberEx(); + ts = new TestSubscriberEx<>(); ts.dispose(); bs1 = new BooleanSubscription(); @@ -1324,7 +1324,7 @@ public void onSubscribe() { @Test public void assertValueSequence() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1357,7 +1357,7 @@ public void assertValueSequence() { @Test public void assertEmpty() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); try { ts.assertEmpty(); @@ -1382,7 +1382,7 @@ public void assertEmpty() { @Test public void awaitDoneTimed() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Thread.currentThread().interrupt(); @@ -1395,7 +1395,7 @@ public void awaitDoneTimed() { @Test public void assertNotSubscribed() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.assertNotSubscribed(); @@ -1411,7 +1411,7 @@ public void assertNotSubscribed() { @Test public void assertErrorMultiple() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); TestException e = new TestException(); ts.errors().add(e); @@ -1439,7 +1439,7 @@ public void assertErrorMultiple() { @Test public void assertComplete() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); @@ -1466,7 +1466,7 @@ public void assertComplete() { @Test public void completeWithoutOnSubscribe() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onComplete(); @@ -1475,7 +1475,7 @@ public void completeWithoutOnSubscribe() { @Test public void completeDelegateThrows() { - TestSubscriberEx ts = new TestSubscriberEx(new FlowableSubscriber() { + TestSubscriberEx ts = new TestSubscriberEx<>(new FlowableSubscriber() { @Override public void onSubscribe(Subscription s) { @@ -1511,7 +1511,7 @@ public void onComplete() { @Test public void errorDelegateThrows() { - TestSubscriberEx ts = new TestSubscriberEx(new FlowableSubscriber() { + TestSubscriberEx ts = new TestSubscriberEx<>(new FlowableSubscriber() { @Override public void onSubscribe(Subscription s) { @@ -1547,7 +1547,7 @@ public void onComplete() { @Test public void syncQueueThrows() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.setInitialFusionMode(QueueFuseable.SYNC); Flowable.range(1, 5) @@ -1565,7 +1565,7 @@ public void syncQueueThrows() { @Test public void asyncQueueThrows() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.setInitialFusionMode(QueueFuseable.ANY); UnicastProcessor up = UnicastProcessor.create(); @@ -1587,7 +1587,7 @@ public void asyncQueueThrows() { @Test public void assertValuePredicateEmpty() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.empty().subscribe(ts); @@ -1602,7 +1602,7 @@ public void assertValuePredicateEmpty() { @Test public void assertValuePredicateMatch() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1).subscribe(ts); @@ -1615,7 +1615,7 @@ public void assertValuePredicateMatch() { @Test public void assertValuePredicateNoMatch() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1).subscribe(ts); @@ -1630,7 +1630,7 @@ public void assertValuePredicateNoMatch() { @Test public void assertValuePredicateMatchButMore() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1, 2).subscribe(ts); @@ -1645,7 +1645,7 @@ public void assertValuePredicateMatchButMore() { @Test public void assertValueAtPredicateEmpty() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.empty().subscribe(ts); @@ -1660,7 +1660,7 @@ public void assertValueAtPredicateEmpty() { @Test public void assertValueAtPredicateMatch() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1, 2).subscribe(ts); @@ -1673,7 +1673,7 @@ public void assertValueAtPredicateMatch() { @Test public void assertValueAtPredicateNoMatch() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1, 2, 3).subscribe(ts); @@ -1688,7 +1688,7 @@ public void assertValueAtPredicateNoMatch() { @Test public void assertValueAtInvalidIndex() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); Flowable.just(1, 2).subscribe(ts); @@ -1775,7 +1775,7 @@ public void timeoutIndicated3() throws InterruptedException { @Test public void disposeIndicated() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.cancel(); try { @@ -1824,7 +1824,7 @@ public boolean test(Integer t) throws Exception { @Test public void assertValuesOnly() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); ts.assertValuesOnly(); @@ -1837,7 +1837,7 @@ public void assertValuesOnly() { @Test public void assertValuesOnlyThrowsOnUnexpectedValue() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); ts.assertValuesOnly(); @@ -1856,7 +1856,7 @@ public void assertValuesOnlyThrowsOnUnexpectedValue() { @Test public void assertValuesOnlyThrowsWhenCompleted() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); ts.onComplete(); @@ -1871,7 +1871,7 @@ public void assertValuesOnlyThrowsWhenCompleted() { @Test public void assertValuesOnlyThrowsWhenErrored() { - TestSubscriberEx ts = new TestSubscriberEx(); + TestSubscriberEx ts = new TestSubscriberEx<>(); ts.onSubscribe(new BooleanSubscription()); ts.onError(new TestException()); diff --git a/src/test/java/io/reactivex/rxjava3/validators/BaseTypeParser.java b/src/test/java/io/reactivex/rxjava3/validators/BaseTypeParser.java index 1dd9f1c1ce..39704aba14 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/BaseTypeParser.java +++ b/src/test/java/io/reactivex/rxjava3/validators/BaseTypeParser.java @@ -48,7 +48,7 @@ public static class RxMethod { } public static List parse(File f, String baseClassName) throws Exception { - List list = new ArrayList(); + List list = new ArrayList<>(); StringBuilder b = JavadocForAnnotations.readFile(f); diff --git a/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java b/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java index 221896fea2..40a976a165 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java +++ b/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java @@ -49,7 +49,7 @@ static void findPattern(String pattern, boolean checkMain) throws Exception { return; } - Queue dirs = new ArrayDeque(); + Queue dirs = new ArrayDeque<>(); StringBuilder fail = new StringBuilder(); fail.append("The following code pattern was found: ").append(pattern).append("\n"); diff --git a/src/test/java/io/reactivex/rxjava3/validators/FixLicenseHeaders.java b/src/test/java/io/reactivex/rxjava3/validators/FixLicenseHeaders.java index f069ec43cf..c5e601d017 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/FixLicenseHeaders.java +++ b/src/test/java/io/reactivex/rxjava3/validators/FixLicenseHeaders.java @@ -52,7 +52,7 @@ public void checkAndUpdateLicenses() throws Exception { return; } - Queue dirs = new ArrayDeque(); + Queue dirs = new ArrayDeque<>(); File parent = f.getParentFile().getParentFile(); dirs.offer(parent); @@ -73,7 +73,7 @@ public void checkAndUpdateLicenses() throws Exception { } else { if (u.getName().endsWith(".java")) { - List lines = new ArrayList(); + List lines = new ArrayList<>(); BufferedReader in = new BufferedReader(new FileReader(u)); try { for (;;) { diff --git a/src/test/java/io/reactivex/rxjava3/validators/InternalWrongNaming.java b/src/test/java/io/reactivex/rxjava3/validators/InternalWrongNaming.java index 984d6bbbd5..14d937502f 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/InternalWrongNaming.java +++ b/src/test/java/io/reactivex/rxjava3/validators/InternalWrongNaming.java @@ -81,7 +81,7 @@ static void checkInternalOperatorNaming(String baseClassName, String consumerCla } static List readFile(File u) throws Exception { - List lines = new ArrayList(); + List lines = new ArrayList<>(); BufferedReader in = new BufferedReader(new FileReader(u)); try { diff --git a/src/test/java/io/reactivex/rxjava3/validators/JavadocFindUnescapedAngleBrackets.java b/src/test/java/io/reactivex/rxjava3/validators/JavadocFindUnescapedAngleBrackets.java index 7ebdd1e8a1..083da2902c 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/JavadocFindUnescapedAngleBrackets.java +++ b/src/test/java/io/reactivex/rxjava3/validators/JavadocFindUnescapedAngleBrackets.java @@ -31,7 +31,7 @@ public void find() throws Exception { base = base.getParentFile().getParentFile(); - Queue files = new ArrayDeque(); + Queue files = new ArrayDeque<>(); files.offer(base.listFiles()); diff --git a/src/test/java/io/reactivex/rxjava3/validators/NewLinesBeforeAnnotation.java b/src/test/java/io/reactivex/rxjava3/validators/NewLinesBeforeAnnotation.java index 1e4177c4cd..36a0297a43 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/NewLinesBeforeAnnotation.java +++ b/src/test/java/io/reactivex/rxjava3/validators/NewLinesBeforeAnnotation.java @@ -72,7 +72,7 @@ static void findPattern(int newLines) throws Exception { return; } - Queue dirs = new ArrayDeque(); + Queue dirs = new ArrayDeque<>(); StringBuilder fail = new StringBuilder(); fail.append("The following code pattern was found: "); @@ -102,7 +102,7 @@ static void findPattern(int newLines) throws Exception { String fname = u.getName(); if (fname.endsWith(".java")) { - List lines = new ArrayList(); + List lines = new ArrayList<>(); BufferedReader in = new BufferedReader(new FileReader(u)); try { for (;;) { diff --git a/src/test/java/io/reactivex/rxjava3/validators/NoAnonymousInnerClassesTest.java b/src/test/java/io/reactivex/rxjava3/validators/NoAnonymousInnerClassesTest.java index d744c2a48c..47e5d1dfd6 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/NoAnonymousInnerClassesTest.java +++ b/src/test/java/io/reactivex/rxjava3/validators/NoAnonymousInnerClassesTest.java @@ -39,7 +39,7 @@ public void verify() throws Exception { StringBuilder b = new StringBuilder("Anonymous inner classes found:"); - Queue queue = new ArrayDeque(); + Queue queue = new ArrayDeque<>(); queue.offer(f); diff --git a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java index b796904cdb..5bc4306492 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java +++ b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java @@ -82,7 +82,7 @@ public void checkParallelFlowable() { static Map, List> defaultInstances; static { - overrides = new HashMap>(); + overrides = new HashMap<>(); // *********************************************************************************************************************** @@ -514,7 +514,7 @@ public void checkParallelFlowable() { // ----------------------------------------------------------------------------------- - ignores = new HashMap>(); + ignores = new HashMap<>(); // needs special param validation due to (long)start + end - 1 <= Integer.MAX_VALUE addIgnore(new ParamIgnore(Flowable.class, "range", Integer.TYPE, Integer.TYPE)); @@ -544,7 +544,7 @@ public void checkParallelFlowable() { // ----------------------------------------------------------------------------------- - defaultValues = new HashMap, Object>(); + defaultValues = new HashMap<>(); defaultValues.put(Publisher.class, new NeverPublisher()); defaultValues.put(Flowable.class, new NeverPublisher()); @@ -645,7 +645,7 @@ public Object apply(Flowable upstream) { // ----------------------------------------------------------------------------------- - defaultInstances = new HashMap, List>(); + defaultInstances = new HashMap<>(); // addDefaultInstance(Flowable.class, Flowable.empty(), "Empty()"); // addDefaultInstance(Flowable.class, Flowable.empty().hide(), "Empty().Hide()"); @@ -677,7 +677,7 @@ static void addIgnore(ParamIgnore ignore) { String key = ignore.toString(); List list = ignores.get(key); if (list == null) { - list = new ArrayList(); + list = new ArrayList<>(); ignores.put(key, list); } list.add(ignore); @@ -687,7 +687,7 @@ static void addOverride(ParamOverride ignore) { String key = ignore.toString(); List list = overrides.get(key); if (list == null) { - list = new ArrayList(); + list = new ArrayList<>(); overrides.put(key, list); } list.add(ignore); @@ -696,7 +696,7 @@ static void addOverride(ParamOverride ignore) { static void addDefaultInstance(Class clazz, Object o, String tag) { List list = defaultInstances.get(clazz); if (list == null) { - list = new ArrayList(); + list = new ArrayList<>(); defaultInstances.put(clazz, list); } list.add(o); @@ -786,7 +786,7 @@ void checkClass(Class clazz) { List overrideList = overrides.get(key); - List baseObjects = new ArrayList(); + List baseObjects = new ArrayList<>(); if ((m.getModifiers() & Modifier.STATIC) != 0) { baseObjects.add(null); @@ -853,7 +853,7 @@ void checkClass(Class clazz) { } } - List entryValues = new ArrayList(); + List entryValues = new ArrayList<>(); if (entryClass.isPrimitive()) { addCheckPrimitive(params[i], overrideEntry, entryValues); diff --git a/src/test/java/io/reactivex/rxjava3/validators/TestPrefixInMethodName.java b/src/test/java/io/reactivex/rxjava3/validators/TestPrefixInMethodName.java index aedcc603f7..2fb3460a73 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/TestPrefixInMethodName.java +++ b/src/test/java/io/reactivex/rxjava3/validators/TestPrefixInMethodName.java @@ -37,7 +37,7 @@ public void checkAndUpdateTestMethodNames() throws Exception { return; } - Queue dirs = new ArrayDeque(); + Queue dirs = new ArrayDeque<>(); StringBuilder fail = new StringBuilder(); fail.append("The following code pattern was found: ").append(pattern).append("\n"); @@ -66,7 +66,7 @@ public void checkAndUpdateTestMethodNames() throws Exception { if (fname.endsWith(".java")) { int lineNum = 0; - List lines = new ArrayList(); + List lines = new ArrayList<>(); BufferedReader in = new BufferedReader(new FileReader(u)); //boolean found = false; try { diff --git a/src/test/java/io/reactivex/rxjava3/validators/TextualAorAn.java b/src/test/java/io/reactivex/rxjava3/validators/TextualAorAn.java index 4f693bd02c..6aff9cc10e 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/TextualAorAn.java +++ b/src/test/java/io/reactivex/rxjava3/validators/TextualAorAn.java @@ -32,7 +32,7 @@ public void checkFiles() throws Exception { return; } - Queue dirs = new ArrayDeque(); + Queue dirs = new ArrayDeque<>(); File parent = f.getParentFile().getParentFile(); dirs.offer(parent); @@ -53,7 +53,7 @@ public void checkFiles() throws Exception { } else { if (u.getName().endsWith(".java")) { - List lines = new ArrayList(); + List lines = new ArrayList<>(); BufferedReader in = new BufferedReader(new FileReader(u)); try { for (;;) { diff --git a/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java b/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java index 22ab721a21..c87f19f10e 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java +++ b/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java @@ -52,7 +52,7 @@ static void findPattern(int newLines) throws Exception { return; } - Queue dirs = new ArrayDeque(); + Queue dirs = new ArrayDeque<>(); StringBuilder fail = new StringBuilder(); fail.append("The following code pattern was found: "); @@ -82,7 +82,7 @@ static void findPattern(int newLines) throws Exception { String fname = u.getName(); if (fname.endsWith(".java")) { - List lines = new ArrayList(); + List lines = new ArrayList<>(); BufferedReader in = new BufferedReader(new FileReader(u)); try { for (;;) { From 0b92c6f0464e4b1edcd92cfbb72270f28d820aa8 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Sun, 22 Dec 2019 22:05:32 +0100 Subject: [PATCH 020/665] 3.x: Remove now unnecessary unchecked warning suppressions (#6790) --- .../io/reactivex/rxjava3/core/MemoryPerf.java | 4 +- .../rxjava3/core/OperatorMergePerf.java | 2 +- .../rxjava3/flowable/FlowableCollectTest.java | 2 - .../rxjava3/flowable/FlowableConcatTests.java | 1 - .../rxjava3/flowable/FlowableNullTests.java | 16 ------ .../rxjava3/flowable/FlowableTests.java | 2 - .../CompletableMaterializeTest.java | 2 - .../operators/flowable/FlowableAmbTest.java | 22 -------- .../flowable/FlowableBufferTest.java | 51 ------------------- .../flowable/FlowableCombineLatestTest.java | 13 ----- .../FlowableConcatDelayErrorTest.java | 2 - .../flowable/FlowableConcatMapEagerTest.java | 17 ------- .../flowable/FlowableConcatTest.java | 20 ++------ .../flowable/FlowableGroupByTest.java | 1 - .../flowable/FlowableMergeDelayErrorTest.java | 6 --- .../operators/flowable/FlowableMergeTest.java | 2 - .../flowable/FlowablePublishTest.java | 2 - .../flowable/FlowableTimeIntervalTest.java | 1 - .../flowable/FlowableToListTest.java | 8 --- .../flowable/FlowableToSortedListTest.java | 4 -- .../FlowableWindowWithFlowableTest.java | 2 - .../flowable/FlowableWindowWithSizeTest.java | 4 -- ...lowableWindowWithStartEndFlowableTest.java | 2 - .../flowable/FlowableWindowWithTimeTest.java | 4 -- .../flowable/FlowableWithLatestFromTest.java | 1 - .../operators/flowable/FlowableZipTest.java | 2 - .../operators/maybe/MaybeAmbTest.java | 4 -- .../maybe/MaybeConcatIterableTest.java | 5 -- .../operators/maybe/MaybeMaterializeTest.java | 3 -- .../operators/maybe/MaybeOnErrorXTest.java | 3 -- .../operators/maybe/MaybeZipIterableTest.java | 10 ---- .../observable/ObservableAmbTest.java | 2 - .../observable/ObservableBufferTest.java | 40 --------------- .../observable/ObservableCollectTest.java | 2 - .../ObservableCombineLatestTest.java | 6 --- .../ObservableConcatMapEagerTest.java | 3 -- .../observable/ObservableConcatTest.java | 11 ++-- .../ObservableMergeDelayErrorTest.java | 4 -- .../ObservableTimeIntervalTest.java | 1 - .../observable/ObservableToListTest.java | 8 --- .../ObservableToSortedListTest.java | 4 -- .../ObservableWindowWithSizeTest.java | 3 -- ...vableWindowWithStartEndObservableTest.java | 2 - .../ObservableWithLatestFromTest.java | 1 - .../observable/ObservableZipTest.java | 1 - .../operators/single/SingleAmbTest.java | 4 -- .../operators/single/SingleConcatTest.java | 2 - .../single/SingleMaterializeTest.java | 2 - .../operators/single/SingleMergeTest.java | 1 - .../single/SingleZipIterableTest.java | 11 ---- .../operators/single/SingleZipTest.java | 1 - .../io/reactivex/rxjava3/maybe/MaybeTest.java | 20 -------- .../observable/ObservableConcatTests.java | 1 - .../observable/ObservableNullTests.java | 14 ----- .../rxjava3/observable/ObservableTest.java | 2 - .../rxjava3/parallel/ParallelCollectTest.java | 4 -- .../parallel/ParallelDoOnNextTryTest.java | 2 - .../parallel/ParallelFilterTryTest.java | 2 - .../parallel/ParallelFlowableTest.java | 1 - .../rxjava3/parallel/ParallelMapTryTest.java | 2 - .../rxjava3/parallel/ParallelReduceTest.java | 4 -- .../rxjava3/single/SingleNullTests.java | 7 --- .../reactivex/rxjava3/single/SingleTest.java | 2 - .../rxjava3/tck/AmbArrayTckTest.java | 1 - .../io/reactivex/rxjava3/tck/AmbTckTest.java | 1 - ...ombineLatestIterableDelayErrorTckTest.java | 1 - .../tck/CombineLatestIterableTckTest.java | 1 - .../rxjava3/tck/ConcatArrayEagerTckTest.java | 1 - .../tck/ConcatIterableEagerTckTest.java | 1 - .../rxjava3/tck/MergeIterableTckTest.java | 1 - .../rxjava3/tck/RefCountProcessor.java | 2 +- .../rxjava3/tck/ZipIterableTckTest.java | 1 - .../testsupport/BaseTestConsumerEx.java | 2 + 73 files changed, 12 insertions(+), 388 deletions(-) diff --git a/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java index 7679ec5346..b7ed7e4ae1 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java @@ -184,7 +184,7 @@ public Object call() throws Exception { checkMemory(new Callable() { @Override public Object call() throws Exception { - return new io.reactivex.rxjava3.observers.TestObserver(); + return new io.reactivex.rxjava3.observers.TestObserver<>(); } }, "test-consumer", "Rx2Observable"); @@ -374,7 +374,7 @@ public Object call() throws Exception { checkMemory(new Callable() { @Override public Object call() throws Exception { - return new io.reactivex.rxjava3.observers.TestObserver(); + return new io.reactivex.rxjava3.observers.TestObserver<>(); } }, "test-consumer", "Rx2Flowable"); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/OperatorMergePerf.java b/src/jmh/java/io/reactivex/rxjava3/core/OperatorMergePerf.java index 7d68316c83..560736c86c 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/OperatorMergePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/OperatorMergePerf.java @@ -134,7 +134,7 @@ public static class InputForMergeN { @Setup public void setup(final Blackhole bh) { this.bh = bh; - observables = new ArrayList>(); + observables = new ArrayList<>(); for (int i = 0; i < size; i++) { observables.add(Flowable.just(i)); } diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java index ae3cd0c906..ea93877cdc 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableCollectTest.java @@ -167,7 +167,6 @@ public void accept(Object o, Integer t) { assertFalse(added.get()); } - @SuppressWarnings("unchecked") @Test public void collectIntoFlowable() { Flowable.just(1, 1, 1, 1, 2) @@ -316,7 +315,6 @@ public void accept(Object o, Integer t) { assertFalse(added.get()); } - @SuppressWarnings("unchecked") @Test public void collectInto() { Flowable.just(1, 1, 1, 1, 2) diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableConcatTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableConcatTests.java index e800a9173c..5d54f9df5f 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableConcatTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableConcatTests.java @@ -61,7 +61,6 @@ public void concatWithIterableOfFlowable() { Flowable f2 = Flowable.just("three", "four"); Flowable f3 = Flowable.just("five", "six"); - @SuppressWarnings("unchecked") Iterable> is = Arrays.asList(f1, f2, f3); List values = Flowable.concat(Flowable.fromIterable(is)).toList().blockingGet(); diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java index 0e85aa8a40..668a7acf92 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java @@ -47,7 +47,6 @@ public void ambVarargsNull() { Flowable.ambArray((Publisher[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void ambVarargsOneIsNull() { Flowable.ambArray(Flowable.never(), null).blockingLast(); @@ -68,7 +67,6 @@ public Iterator> iterator() { }).test().assertError(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void ambIterableOneIsNull() { Flowable.amb(Arrays.asList(Flowable.never(), null)) @@ -101,7 +99,6 @@ public Object apply(Object[] v) { }).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestIterableOneIsNull() { Flowable.combineLatestDelayError(Arrays.asList(Flowable.never(), null), new Function() { @@ -112,13 +109,11 @@ public Object apply(Object[] v) { }).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestIterableFunctionNull() { Flowable.combineLatestDelayError(Arrays.asList(just1), null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestIterableFunctionReturnsNull() { Flowable.combineLatestDelayError(Arrays.asList(just1), new Function() { @@ -144,7 +139,6 @@ public Iterator> iterator() { }).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void concatIterableOneIsNull() { Flowable.concat(Arrays.asList(just1, null)).blockingLast(); @@ -161,7 +155,6 @@ public void concatArrayNull() { Flowable.concatArray((Publisher[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void concatArrayOneIsNull() { Flowable.concatArray(just1, null).blockingLast(); @@ -480,7 +473,6 @@ public Iterator> iterator() { }, 128, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeIterableOneIsNull() { Flowable.merge(Arrays.asList(just1, null), 128, 128).blockingLast(); @@ -512,7 +504,6 @@ public Iterator> iterator() { }, 128, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeDelayErrorIterableOneIsNull() { Flowable.mergeDelayError(Arrays.asList(just1, null), 128, 128).blockingLast(); @@ -634,13 +625,11 @@ public Object apply(Object[] v) { }).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableFunctionNull() { Flowable.zip(Arrays.asList(just1, just1), null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableFunctionReturnsNull() { Flowable.zip(Arrays.asList(just1, just1), new Function() { @@ -676,13 +665,11 @@ public Object apply(Object[] a) { }, true, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterable2FunctionNull() { Flowable.zip(Arrays.asList(just1, just1), null, true, 128); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterable2FunctionReturnsNull() { Flowable.zip(Arrays.asList(just1, just1), new Function() { @@ -2679,7 +2666,6 @@ public void unsafeSubscribeNull() { just1.subscribe((FlowableSubscriber)null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestDelayErrorIterableFunctionReturnsNull() { Flowable.combineLatestDelayError(Arrays.asList(just1), new Function() { @@ -2690,7 +2676,6 @@ public Object apply(Object[] v) { }, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestDelayErrorIterableFunctionNull() { Flowable.combineLatestDelayError(Arrays.asList(just1), null, 128); @@ -2731,7 +2716,6 @@ public void doOnDisposeNull() { just1.doOnCancel(null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestDelayErrorIterableOneIsNull() { Flowable.combineLatestDelayError(Arrays.asList(Flowable.never(), null), new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java index 616140c6b9..3d462efe0b 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableTests.java @@ -1074,7 +1074,6 @@ public void toObservableError() { @Test public void zipIterableObject() { - @SuppressWarnings("unchecked") final List> flowables = Arrays.asList(Flowable.just(1, 2, 3), Flowable.just(1, 2, 3)); Flowable.zip(flowables, new Function() { @Override @@ -1090,7 +1089,6 @@ public Object apply(Object[] o) throws Exception { @Test public void combineLatestObject() { - @SuppressWarnings("unchecked") final List> flowables = Arrays.asList(Flowable.just(1, 2, 3), Flowable.just(1, 2, 3)); Flowable.combineLatest(flowables, new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterializeTest.java index 23b4e25731..aa77e6a975 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterializeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterializeTest.java @@ -24,7 +24,6 @@ public class CompletableMaterializeTest extends RxJavaTest { @Test - @SuppressWarnings("unchecked") public void error() { TestException ex = new TestException(); Completable.error(ex) @@ -34,7 +33,6 @@ public void error() { } @Test - @SuppressWarnings("unchecked") public void empty() { Completable.complete() .materialize() diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java index 42cba044db..0657a15474 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmbTest.java @@ -102,7 +102,6 @@ public void amb() { Flowable flowable3 = createFlowable(new String[] { "3", "33", "333", "3333" }, 3000, null); - @SuppressWarnings("unchecked") Flowable f = Flowable.ambArray(flowable1, flowable2, flowable3); @@ -131,7 +130,6 @@ public void amb2() { Flowable flowable3 = createFlowable(new String[] {}, 3000, new IOException("fake exception")); - @SuppressWarnings("unchecked") Flowable f = Flowable.ambArray(flowable1, flowable2, flowable3); @@ -158,7 +156,6 @@ public void amb3() { Flowable flowable3 = createFlowable(new String[] { "3" }, 3000, null); - @SuppressWarnings("unchecked") Flowable f = Flowable.ambArray(flowable1, flowable2, flowable3); @@ -171,7 +168,6 @@ public void amb3() { inOrder.verifyNoMoreInteractions(); } - @SuppressWarnings("unchecked") @Test public void producerRequestThroughAmb() { TestSubscriber ts = new TestSubscriber<>(0L); @@ -237,7 +233,6 @@ public void backpressure() { assertEquals(Flowable.bufferSize() * 2, ts.values().size()); } - @SuppressWarnings("unchecked") @Test public void subscriptionOnlyHappensOnce() throws InterruptedException { final AtomicLong count = new AtomicLong(); @@ -262,7 +257,6 @@ public void accept(Subscription s) { assertEquals(2, count.get()); } - @SuppressWarnings("unchecked") @Test public void secondaryRequestsPropagatedToChildren() throws InterruptedException { //this aync stream should emit first @@ -302,7 +296,6 @@ public void accept(Integer t) { assertEquals(1, result); } - @SuppressWarnings("unchecked") @Test public void ambCancelsOthers() { PublishProcessor source1 = PublishProcessor.create(); @@ -347,7 +340,6 @@ public void multipleUse() { ts2.assertNoErrors(); } - @SuppressWarnings("unchecked") @Test public void ambIterable() { PublishProcessor pp1 = PublishProcessor.create(); @@ -370,7 +362,6 @@ public void ambIterable() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void ambIterable2() { PublishProcessor pp1 = PublishProcessor.create(); @@ -393,19 +384,16 @@ public void ambIterable2() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void ambArrayEmpty() { assertSame(Flowable.empty(), Flowable.ambArray()); } - @SuppressWarnings("unchecked") @Test public void ambArraySingleElement() { assertSame(Flowable.never(), Flowable.ambArray(Flowable.never())); } - @SuppressWarnings("unchecked") @Test public void disposed() { TestHelper.checkDisposed(Flowable.ambArray(Flowable.never(), Flowable.never())); @@ -442,7 +430,6 @@ public void onNextRace() { final PublishProcessor pp1 = PublishProcessor.create(); final PublishProcessor pp2 = PublishProcessor.create(); - @SuppressWarnings("unchecked") TestSubscriberEx ts = Flowable.ambArray(pp1, pp2).to(TestHelper.testConsumer()); Runnable r1 = new Runnable() { @@ -471,7 +458,6 @@ public void onCompleteRace() { final PublishProcessor pp1 = PublishProcessor.create(); final PublishProcessor pp2 = PublishProcessor.create(); - @SuppressWarnings("unchecked") TestSubscriber ts = Flowable.ambArray(pp1, pp2).test(); Runnable r1 = new Runnable() { @@ -499,7 +485,6 @@ public void onErrorRace() { final PublishProcessor pp1 = PublishProcessor.create(); final PublishProcessor pp2 = PublishProcessor.create(); - @SuppressWarnings("unchecked") TestSubscriber ts = Flowable.ambArray(pp1, pp2).test(); final Throwable ex = new TestException(); @@ -531,7 +516,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void nullIterableElement() { Flowable.amb(Arrays.asList(Flowable.never(), null, Flowable.never())) @@ -575,21 +559,18 @@ public void ambWithOrder() { Flowable.just(1).ambWith(error).test().assertValue(1).assertComplete(); } - @SuppressWarnings("unchecked") @Test public void ambIterableOrder() { Flowable error = Flowable.error(new RuntimeException()); Flowable.amb(Arrays.asList(Flowable.just(1), error)).test().assertValue(1).assertComplete(); } - @SuppressWarnings("unchecked") @Test public void ambArrayOrder() { Flowable error = Flowable.error(new RuntimeException()); Flowable.ambArray(Flowable.just(1), error).test().assertValue(1).assertComplete(); } - @SuppressWarnings("unchecked") @Test public void noWinnerSuccessDispose() throws Exception { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { @@ -615,7 +596,6 @@ public void accept(Object v) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void noWinnerErrorDispose() throws Exception { final TestException ex = new TestException(); @@ -642,7 +622,6 @@ public void accept(Throwable e) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void noWinnerCompleteDispose() throws Exception { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { @@ -668,7 +647,6 @@ public void run() throws Exception { } } - @SuppressWarnings("unchecked") @Test public void publishersInIterable() { Publisher source = new Publisher() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java index 707d98e0d4..87a0eeaf38 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTest.java @@ -988,7 +988,6 @@ public void onComplete() { assertFalse(s.isDisposed()); } - @SuppressWarnings("unchecked") @Test public void postCompleteBackpressure() { Flowable> source = Flowable.range(1, 10).buffer(3, 1); @@ -1064,7 +1063,6 @@ public void postCompleteBackpressure() { ts.assertNoErrors(); } - @SuppressWarnings("unchecked") @Test public void timeAndSkipOverlap() { @@ -1104,7 +1102,6 @@ public void timeAndSkipOverlap() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void timeAndSkipSkip() { @@ -1141,7 +1138,6 @@ public void timeAndSkipSkip() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void timeAndSkipOverlapScheduler() { @@ -1192,7 +1188,6 @@ public Scheduler apply(Scheduler t) { } } - @SuppressWarnings("unchecked") @Test public void timeAndSkipSkipDefaultScheduler() { RxJavaPlugins.setComputationSchedulerHandler(new Function() { @@ -1240,7 +1235,6 @@ public Scheduler apply(Scheduler t) { } } - @SuppressWarnings("unchecked") @Test public void bufferBoundaryHint() { Flowable.range(1, 5).buffer(Flowable.timer(1, TimeUnit.MINUTES), 2) @@ -1252,7 +1246,6 @@ static HashSet set(Integer... values) { return new HashSet<>(Arrays.asList(values)); } - @SuppressWarnings("unchecked") @Test public void bufferIntoCustomCollection() { Flowable.just(1, 1, 2, 2, 3, 3, 4, 4) @@ -1266,7 +1259,6 @@ public Collection get() throws Exception { .assertResult(set(1, 2), set(2, 3), set(4)); } - @SuppressWarnings("unchecked") @Test public void bufferSkipIntoCustomCollection() { Flowable.just(1, 1, 2, 2, 3, 3, 4, 4) @@ -1280,7 +1272,6 @@ public Collection get() throws Exception { .assertResult(set(1, 2), set(2, 3), set(4)); } - @SuppressWarnings("unchecked") @Test public void bufferTimeSkipDefault() { Flowable.range(1, 5).buffer(1, 1, TimeUnit.MINUTES) @@ -1307,7 +1298,6 @@ public void dispose() { } @Test - @SuppressWarnings("unchecked") public void supplierReturnsNull() { Flowable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), Integer.MAX_VALUE, new Supplier>() { @@ -1327,7 +1317,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierReturnsNull2() { Flowable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), 10, new Supplier>() { @@ -1347,7 +1336,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierReturnsNull3() { Flowable.never() .buffer(2, 1, TimeUnit.MILLISECONDS, Schedulers.single(), new Supplier>() { @@ -1367,7 +1355,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows() { Flowable.just(1) .buffer(1, TimeUnit.SECONDS, Schedulers.single(), Integer.MAX_VALUE, new Supplier>() { @@ -1381,7 +1368,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows2() { Flowable.just(1) .buffer(1, TimeUnit.SECONDS, Schedulers.single(), 10, new Supplier>() { @@ -1395,7 +1381,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows3() { Flowable.just(1) .buffer(2, 1, TimeUnit.SECONDS, Schedulers.single(), new Supplier>() { @@ -1409,7 +1394,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows4() { Flowable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), Integer.MAX_VALUE, new Supplier>() { @@ -1429,7 +1413,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows5() { Flowable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), 10, new Supplier>() { @@ -1449,7 +1432,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows6() { Flowable.never() .buffer(2, 1, TimeUnit.MILLISECONDS, Schedulers.single(), new Supplier>() { @@ -1468,7 +1450,6 @@ public Collection get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void restartTimer() { Flowable.range(1, 5) @@ -1477,7 +1458,6 @@ public void restartTimer() { .assertResult(Arrays.asList(1, 2), Arrays.asList(3, 4), Arrays.asList(5)); } - @SuppressWarnings("unchecked") @Test public void bufferSkipError() { Flowable.error(new TestException()) @@ -1486,7 +1466,6 @@ public void bufferSkipError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferSupplierCrash2() { Flowable.range(1, 2) @@ -1504,7 +1483,6 @@ public List get() throws Exception { .assertFailure(TestException.class, Arrays.asList(1)); } - @SuppressWarnings("unchecked") @Test public void bufferSkipSupplierCrash2() { Flowable.range(1, 2) @@ -1522,7 +1500,6 @@ public List get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferOverlapSupplierCrash2() { Flowable.range(1, 2) @@ -1540,7 +1517,6 @@ public List get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferSkipOverlap() { Flowable.range(1, 5) @@ -1555,7 +1531,6 @@ public void bufferSkipOverlap() { ); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactError() { Flowable.error(new TestException()) @@ -1564,7 +1539,6 @@ public void bufferTimedExactError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferTimedSkipError() { Flowable.error(new TestException()) @@ -1573,7 +1547,6 @@ public void bufferTimedSkipError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferTimedOverlapError() { Flowable.error(new TestException()) @@ -1582,7 +1555,6 @@ public void bufferTimedOverlapError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactEmpty() { Flowable.empty() @@ -1591,7 +1563,6 @@ public void bufferTimedExactEmpty() { .assertResult(Collections.emptyList()); } - @SuppressWarnings("unchecked") @Test public void bufferTimedSkipEmpty() { Flowable.empty() @@ -1600,7 +1571,6 @@ public void bufferTimedSkipEmpty() { .assertResult(Collections.emptyList()); } - @SuppressWarnings("unchecked") @Test public void bufferTimedOverlapEmpty() { Flowable.empty() @@ -1609,7 +1579,6 @@ public void bufferTimedOverlapEmpty() { .assertResult(Collections.emptyList()); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactSupplierCrash() { TestScheduler scheduler = new TestScheduler(); @@ -1639,7 +1608,6 @@ public List get() throws Exception { .assertFailure(TestException.class, Arrays.asList(1)); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactBoundedError() { TestScheduler scheduler = new TestScheduler(); @@ -1713,7 +1681,6 @@ public void badRequest() { TestHelper.assertBadRequestReported(PublishProcessor.create().buffer(2, 1)); } - @SuppressWarnings("unchecked") @Test public void skipError() { Flowable.error(new TestException()) @@ -1722,7 +1689,6 @@ public void skipError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void skipSingleResult() { Flowable.just(1) @@ -1731,7 +1697,6 @@ public void skipSingleResult() { .assertResult(Arrays.asList(1)); } - @SuppressWarnings("unchecked") @Test public void skipBackpressure() { Flowable.range(1, 10) @@ -1782,7 +1747,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void noCompletionCancelExact() { final AtomicInteger counter = new AtomicInteger(); @@ -1802,7 +1766,6 @@ public void run() throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void noCompletionCancelSkip() { final AtomicInteger counter = new AtomicInteger(); @@ -1822,7 +1785,6 @@ public void run() throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void noCompletionCancelOverlap() { final AtomicInteger counter = new AtomicInteger(); @@ -1843,7 +1805,6 @@ public void run() throws Exception { } @Test - @SuppressWarnings("unchecked") public void boundaryOpenCloseDisposedOnComplete() { PublishProcessor source = PublishProcessor.create(); @@ -1919,7 +1880,6 @@ public Publisher apply(Long a) { } @Test - @SuppressWarnings("unchecked") public void openClosemainError() { Flowable.error(new TestException()) .buffer(Flowable.never(), Functions.justFunction(Flowable.never())) @@ -1928,7 +1888,6 @@ public void openClosemainError() { } @Test - @SuppressWarnings("unchecked") public void openClosebadSource() { List errors = TestHelper.trackPluginErrors(); try { @@ -1966,7 +1925,6 @@ protected void subscribeActual(Subscriber s) { } @Test - @SuppressWarnings("unchecked") public void openCloseOpenCompletes() { PublishProcessor source = PublishProcessor.create(); @@ -1995,7 +1953,6 @@ public void openCloseOpenCompletes() { } @Test - @SuppressWarnings("unchecked") public void openCloseOpenCompletesNoBuffers() { PublishProcessor source = PublishProcessor.create(); @@ -2024,7 +1981,6 @@ public void openCloseOpenCompletesNoBuffers() { } @Test - @SuppressWarnings("unchecked") public void openCloseTake() { PublishProcessor source = PublishProcessor.create(); @@ -2048,7 +2004,6 @@ public void openCloseTake() { } @Test - @SuppressWarnings("unchecked") public void openCloseEmptyBackpressure() { PublishProcessor source = PublishProcessor.create(); @@ -2069,7 +2024,6 @@ public void openCloseEmptyBackpressure() { } @Test - @SuppressWarnings("unchecked") public void openCloseErrorBackpressure() { PublishProcessor source = PublishProcessor.create(); @@ -2090,7 +2044,6 @@ public void openCloseErrorBackpressure() { } @Test - @SuppressWarnings("unchecked") public void openCloseBadOpen() { List errors = TestHelper.trackPluginErrors(); try { @@ -2134,7 +2087,6 @@ protected void subscribeActual(Subscriber s) { } @Test - @SuppressWarnings("unchecked") public void openCloseBadClose() { List errors = TestHelper.trackPluginErrors(); try { @@ -2191,7 +2143,6 @@ public Flowable> apply(Flowable f) ); } - @SuppressWarnings("unchecked") @Test public void bufferExactBoundarySecondBufferCrash() { PublishProcessor pp = PublishProcessor.create(); @@ -2213,7 +2164,6 @@ public List get() throws Exception { ts.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferExactBoundaryBadSource() { Flowable pp = new Flowable() { @@ -2425,7 +2375,6 @@ public void timedSizeBufferAlreadyCleared() { } @Test - @SuppressWarnings("unchecked") public void bufferExactFailingSupplier() { Flowable.empty() .buffer(1, TimeUnit.SECONDS, Schedulers.computation(), 10, new Supplier>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java index 8ed8197138..c8e041b951 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java @@ -802,7 +802,6 @@ public Long apply(Long t1, Integer t2) { @Test public void combineLatestRequestOverflow() throws InterruptedException { - @SuppressWarnings("unchecked") List> sources = Arrays.asList(Flowable.fromArray(1, 2, 3, 4), Flowable.fromArray(5, 6, 7, 8)); Flowable f = Flowable.combineLatest(sources, new Function() { @@ -864,7 +863,6 @@ public void accept(Throwable t) { assertFalse(errorOccurred.get()); } - @SuppressWarnings("unchecked") @Test public void combineLatestIterable() { Flowable source = Flowable.just(1); @@ -916,7 +914,6 @@ public String apply(Object[] args) { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void firstJustError() { TestSubscriber ts = TestSubscriber.create(); @@ -936,7 +933,6 @@ public Integer apply(Object[] args) { ts.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void secondJustError() { TestSubscriber ts = TestSubscriber.create(); @@ -956,7 +952,6 @@ public Integer apply(Object[] args) { ts.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void oneErrors() { TestSubscriber ts = TestSubscriber.create(); @@ -976,7 +971,6 @@ public Integer apply(Object[] args) { ts.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void twoErrors() { TestSubscriber ts = TestSubscriber.create(); @@ -996,7 +990,6 @@ public Integer apply(Object[] args) { ts.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void bothError() { TestSubscriber ts = TestSubscriber.create(); @@ -1140,7 +1133,6 @@ public Object apply(Object[] a) throws Exception { } @Test - @SuppressWarnings("unchecked") public void combineLatestDelayErrorIterableOfSources() { Flowable.combineLatestDelayError(Arrays.asList( @@ -1156,7 +1148,6 @@ public Object apply(Object[] a) throws Exception { } @Test - @SuppressWarnings("unchecked") public void combineLatestDelayErrorIterableOfSourcesWithError() { Flowable.combineLatestDelayError(Arrays.asList( @@ -1366,7 +1357,6 @@ public Object apply(Object a, Object b) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void dontSubscribeIfDone2() { List errors = TestHelper.trackPluginErrors(); @@ -1400,7 +1390,6 @@ public Object apply(Object[] a) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void combine2Flowable2Errors() throws Exception { List errors = TestHelper.trackPluginErrors(); @@ -1534,7 +1523,6 @@ public Integer apply(Integer t1, Integer t2) throws Exception { } @Test - @SuppressWarnings("unchecked") public void syncFirstErrorsAfterItemDelayError() { Flowable.combineLatestDelayError(Arrays.asList( Flowable.just(21).concatWith(Flowable.error(new TestException())), @@ -1552,7 +1540,6 @@ public Object apply(Object[] a) throws Exception { .assertFailure(TestException.class, 42); } - @SuppressWarnings("unchecked") @Test public void publishersInIterable() { Publisher source = new Publisher() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java index 0543eb1b0f..22e23a4391 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatDelayErrorTest.java @@ -252,7 +252,6 @@ public void concatDelayErrorFlowableError() { assertTrue(cex.get(2).toString(), cex.get(2) instanceof TestException); } - @SuppressWarnings("unchecked") @Test public void concatDelayErrorIterable() { TestSubscriber ts = TestSubscriber.create(); @@ -266,7 +265,6 @@ public void concatDelayErrorIterable() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void concatDelayErrorIterableError() { TestSubscriberEx ts = new TestSubscriberEx<>(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java index 5c052d616c..9289288ccc 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java @@ -298,7 +298,6 @@ public void simple2() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness2() { final AtomicInteger count = new AtomicInteger(); @@ -323,7 +322,6 @@ public void accept(Integer t) { tsBp.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness3() { final AtomicInteger count = new AtomicInteger(); @@ -348,7 +346,6 @@ public void accept(Integer t) { tsBp.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness4() { final AtomicInteger count = new AtomicInteger(); @@ -373,7 +370,6 @@ public void accept(Integer t) { tsBp.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness5() { final AtomicInteger count = new AtomicInteger(); @@ -398,7 +394,6 @@ public void accept(Integer t) { tsBp.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness6() { final AtomicInteger count = new AtomicInteger(); @@ -423,7 +418,6 @@ public void accept(Integer t) { tsBp.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness7() { final AtomicInteger count = new AtomicInteger(); @@ -448,7 +442,6 @@ public void accept(Integer t) { tsBp.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness8() { final AtomicInteger count = new AtomicInteger(); @@ -473,7 +466,6 @@ public void accept(Integer t) { tsBp.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness9() { final AtomicInteger count = new AtomicInteger(); @@ -507,7 +499,6 @@ public void mainError() { ts.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void innerError() { Flowable.concatArrayEager(Flowable.just(1), Flowable.error(new TestException())).subscribe(ts); @@ -517,7 +508,6 @@ public void innerError() { ts.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void innerEmpty() { Flowable.concatArrayEager(Flowable.empty(), Flowable.empty()).subscribe(ts); @@ -552,7 +542,6 @@ public void invalidCapacityHint() { } @Test - @SuppressWarnings("unchecked") public void backpressure() { Flowable.concatArrayEager(Flowable.just(1), Flowable.just(1)).subscribe(tsBp); @@ -662,7 +651,6 @@ public void accept(long reqCount) { Assert.assertEquals(1, (long) requests.get(5)); } - @SuppressWarnings("unchecked") @Test public void capacityHint() { Flowable source = Flowable.just(1); @@ -699,7 +687,6 @@ public void flowableCapacityHint() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void badCapacityHint() throws Exception { Flowable source = Flowable.just(1); @@ -730,7 +717,6 @@ public void concatEagerZero() { .assertResult(); } - @SuppressWarnings("unchecked") @Test public void concatEagerOne() { Flowable.concatEager(Arrays.asList(Flowable.just(1))) @@ -738,7 +724,6 @@ public void concatEagerOne() { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void concatEagerTwo() { Flowable.concatEager(Arrays.asList(Flowable.just(1), Flowable.just(2))) @@ -770,7 +755,6 @@ public void ObservableCapacityHint() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void concatEagerIterable() { Flowable.concatEager(Arrays.asList(Flowable.just(1), Flowable.just(2))) @@ -1193,7 +1177,6 @@ public void arrayDelayErrorDefault() { PublishProcessor pp2 = PublishProcessor.create(); PublishProcessor pp3 = PublishProcessor.create(); - @SuppressWarnings("unchecked") TestSubscriber ts = Flowable.concatArrayEagerDelayError(pp1, pp2, pp3) .test(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java index 67451d6097..208f7b1b51 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatTest.java @@ -283,7 +283,7 @@ public void blockedObservableOfObservables() { final Flowable even = Flowable.fromArray(e); final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); - @SuppressWarnings("unchecked") + TestObservable> observableOfObservables = new TestObservable<>(callOnce, okToContinue, odds, even); Flowable concatF = Flowable.concat(Flowable.unsafeCreate(observableOfObservables)); concatF.subscribe(subscriber); @@ -322,7 +322,6 @@ public void concatConcurrentWithInfinity() { Subscriber subscriber = TestHelper.mockSubscriber(); - @SuppressWarnings("unchecked") TestObservable> observableOfObservables = new TestObservable<>(Flowable.unsafeCreate(w1), Flowable.unsafeCreate(w2)); Flowable concatF = Flowable.concat(Flowable.unsafeCreate(observableOfObservables)); @@ -450,7 +449,6 @@ public void concatUnsubscribeConcurrent() { Subscriber subscriber = TestHelper.mockSubscriber(); TestSubscriber ts = new TestSubscriber<>(subscriber, 0L); - @SuppressWarnings("unchecked") TestObservable> observableOfObservables = new TestObservable<>(Flowable.unsafeCreate(w1), Flowable.unsafeCreate(w2)); Flowable concatF = Flowable.concat(Flowable.unsafeCreate(observableOfObservables)); @@ -506,10 +504,12 @@ public void cancel() { private final T seed; private final int size; + @SafeVarargs TestObservable(T... values) { this(null, null, values); } + @SafeVarargs TestObservable(CountDownLatch once, CountDownLatch okToContinue, T... values) { this.values = Arrays.asList(values); this.size = this.values.size(); @@ -1055,7 +1055,6 @@ public void concat4() { .assertResult(1, 1, 1, 1); } - @SuppressWarnings("unchecked") @Test public void concatArrayDelayError() { Flowable.concatArrayDelayError(Flowable.just(1), Flowable.just(2), @@ -1064,7 +1063,6 @@ public void concatArrayDelayError() { .assertResult(1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatArrayDelayErrorWithError() { Flowable.concatArrayDelayError(Flowable.just(1), Flowable.just(2), @@ -1074,7 +1072,6 @@ public void concatArrayDelayErrorWithError() { .assertFailure(TestException.class, 1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatIterableDelayError() { Flowable.concatDelayError( @@ -1084,7 +1081,6 @@ public void concatIterableDelayError() { .assertResult(1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatIterableDelayErrorWithError() { Flowable.concatDelayError( @@ -1163,13 +1159,11 @@ public Iterable apply(Integer v) throws Exception { .assertResult(1, 2, 3, 4, 5, 1, 2, 3, 4, 5); } - @SuppressWarnings("unchecked") @Test public void emptyArray() { assertSame(Flowable.empty(), Flowable.concatArrayDelayError()); } - @SuppressWarnings("unchecked") @Test public void singleElementArray() { assertSame(Flowable.never(), Flowable.concatArrayDelayError(Flowable.never())); @@ -1200,13 +1194,11 @@ public Flowable apply(Object v) throws Exception { } - @SuppressWarnings("unchecked") @Test public void concatArrayEmpty() { assertSame(Flowable.empty(), Flowable.concatArray()); } - @SuppressWarnings("unchecked") @Test public void concatArraySingleElement() { assertSame(Flowable.never(), Flowable.concatArray(Flowable.never())); @@ -1537,7 +1529,6 @@ public Publisher apply(Integer v) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscription() { final int[] calls = { 0 }; @@ -1558,7 +1549,6 @@ public void subscribe(FlowableEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionDelayError() { final int[] calls = { 0 }; @@ -1579,7 +1569,6 @@ public void subscribe(FlowableEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionIterable() { final int[] calls = { 0 }; @@ -1600,7 +1589,6 @@ public void subscribe(FlowableEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionDelayErrorIterable() { final int[] calls = { 0 }; @@ -1621,7 +1609,6 @@ public void subscribe(FlowableEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noCancelPreviousArray() { final AtomicInteger counter = new AtomicInteger(); @@ -1640,7 +1627,6 @@ public void run() throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void noCancelPreviousIterable() { final AtomicInteger counter = new AtomicInteger(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java index a14372b959..f20f0ae0c7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupByTest.java @@ -2390,7 +2390,6 @@ public Publisher apply(GroupedFlowable g) { } @Test - @SuppressWarnings("unchecked") public void valueSelectorCrashAndMissingBackpressure() { PublishProcessor pp = PublishProcessor.create(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java index 7a528e8cb2..118f249091 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java @@ -544,7 +544,6 @@ public void mergeIterable() { verify(stringSubscriber, times(2)).onNext("hello"); } - @SuppressWarnings("unchecked") @Test public void iterableMaxConcurrent() { TestSubscriber ts = TestSubscriber.create(); @@ -571,7 +570,6 @@ public void iterableMaxConcurrent() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void iterableMaxConcurrentError() { TestSubscriberEx ts = new TestSubscriberEx<>(); @@ -632,7 +630,6 @@ public void mergeArrayDelayError() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayErrorWithError() { Flowable.mergeDelayError( @@ -678,7 +675,6 @@ public void mergeDelayErrorWithErrorMaxConcurrency() { .assertFailure(TestException.class, 1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayErrorMaxConcurrency() { Flowable.mergeDelayError( @@ -688,7 +684,6 @@ public void mergeIterableDelayErrorMaxConcurrency() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayErrorWithErrorMaxConcurrency() { Flowable.mergeDelayError( @@ -720,7 +715,6 @@ public void mergeDelayError3WithError() { .assertFailure(TestException.class, 1, 2, 3); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayError() { Flowable.mergeDelayError(Arrays.asList(Flowable.just(1), Flowable.just(2))) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java index 6b3f72e2f9..19d0ac86a0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java @@ -1343,7 +1343,6 @@ public void onNext(Integer t) { } } - @SuppressWarnings("unchecked") @Test public void negativeMaxConcurrent() { try { @@ -1354,7 +1353,6 @@ public void negativeMaxConcurrent() { } } - @SuppressWarnings("unchecked") @Test public void zeroMaxConcurrent() { try { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java index ec7c88da09..0dffd13798 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishTest.java @@ -1368,7 +1368,6 @@ public void badRequest() { } @Test - @SuppressWarnings("unchecked") public void splitCombineSubscriberChangeAfterOnNext() { Flowable source = Flowable.range(0, 20) .doOnSubscribe(new Consumer() { @@ -1436,7 +1435,6 @@ public boolean test(List v) throws Exception { } @Test - @SuppressWarnings("unchecked") public void splitCombineSubscriberChangeAfterOnNextFused() { Flowable source = Flowable.range(0, 20) .publish(10) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java index 496bce4a38..4153c92a22 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeIntervalTest.java @@ -129,7 +129,6 @@ public void dispose() { TestHelper.checkDisposed(Flowable.just(1).timeInterval()); } - @SuppressWarnings("unchecked") @Test public void error() { Flowable.error(new TestException()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java index 73d1455d5b..cad29e3fc8 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListTest.java @@ -113,7 +113,6 @@ public void backpressureHonoredFlowable() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void capacityHintFlowable() { Flowable.range(1, 10) @@ -182,7 +181,6 @@ static void await(CyclicBarrier cb) { } } - @SuppressWarnings("unchecked") @Test public void capacityHint() { Flowable.range(1, 10) @@ -198,7 +196,6 @@ public void dispose() { TestHelper.checkDisposed(Flowable.just(1).toList()); } - @SuppressWarnings("unchecked") @Test public void error() { Flowable.error(new TestException()) @@ -208,7 +205,6 @@ public void error() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorSingle() { Flowable.error(new TestException()) @@ -217,7 +213,6 @@ public void errorSingle() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void collectionSupplierThrows() { Flowable.just(1) @@ -232,7 +227,6 @@ public Collection get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void collectionSupplierReturnsNull() { Flowable.just(1) @@ -248,7 +242,6 @@ public Collection get() throws Exception { .assertErrorMessage(ExceptionHelper.nullWarning("The collectionSupplier returned a null Collection.")); } - @SuppressWarnings("unchecked") @Test public void singleCollectionSupplierThrows() { Flowable.just(1) @@ -262,7 +255,6 @@ public Collection get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void singleCollectionSupplierReturnsNull() { Flowable.just(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java index 6012db0add..5c7278da54 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToSortedListTest.java @@ -112,7 +112,6 @@ public int compare(Integer a, Integer b) { .assertResult(5, 4, 3, 2, 1); } - @SuppressWarnings("unchecked") @Test public void toSortedListCapacityFlowable() { Flowable.just(5, 1, 2, 4, 3).toSortedList(4).toFlowable() @@ -120,7 +119,6 @@ public void toSortedListCapacityFlowable() { .assertResult(Arrays.asList(1, 2, 3, 4, 5)); } - @SuppressWarnings("unchecked") @Test public void toSortedListComparatorCapacityFlowable() { Flowable.just(5, 1, 2, 4, 3).toSortedList(new Comparator() { @@ -178,7 +176,6 @@ static void await(CyclicBarrier cb) { } } - @SuppressWarnings("unchecked") @Test public void toSortedListCapacity() { Flowable.just(5, 1, 2, 4, 3).toSortedList(4) @@ -186,7 +183,6 @@ public void toSortedListCapacity() { .assertResult(Arrays.asList(1, 2, 3, 4, 5)); } - @SuppressWarnings("unchecked") @Test public void toSortedListComparatorCapacity() { Flowable.just(5, 1, 2, 4, 3).toSortedList(new Comparator() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java index c17e2ccfae..66273656f9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithFlowableTest.java @@ -328,7 +328,6 @@ public Flowable apply(Flowable v) throws Exception { }, false, 1, 1, 1); } - @SuppressWarnings("unchecked") @Test public void boundaryDirectMissingBackpressure() { List errors = TestHelper.trackPluginErrors(); @@ -344,7 +343,6 @@ public void boundaryDirectMissingBackpressure() { } } - @SuppressWarnings("unchecked") @Test public void boundaryDirectMissingBackpressureNoNullPointerException() { List errors = TestHelper.trackPluginErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java index e9a7c77518..fbefcc953e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithSizeTest.java @@ -290,7 +290,6 @@ public Flowable apply(Flowable w) { ts.assertValueCount(22); } - @SuppressWarnings("unchecked") @Test public void backpressureOuterInexact() { TestSubscriber> ts = new TestSubscriber<>(0L); @@ -364,7 +363,6 @@ public Flowable> apply(Flowable f) throws Exception { }); } - @SuppressWarnings("unchecked") @Test public void errorExact() { Flowable.error(new TestException()) @@ -373,7 +371,6 @@ public void errorExact() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorSkip() { Flowable.error(new TestException()) @@ -382,7 +379,6 @@ public void errorSkip() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorOverlap() { Flowable.error(new TestException()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java index 55db25f654..023f6848d5 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithStartEndFlowableTest.java @@ -434,7 +434,6 @@ public void accept(Flowable w) throws Throwable { } @Test - @SuppressWarnings("unchecked") public void mainWindowMissingBackpressure() { PublishProcessor source = PublishProcessor.create(); PublishProcessor boundary = PublishProcessor.create(); @@ -515,7 +514,6 @@ public void accept(Flowable v) throws Throwable { inner.get().test().assertResult(); } - @SuppressWarnings("unchecked") @Test public void closingIndicatorFunctionCrash() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java index 86da1bb09c..e3d9466b3e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowWithTimeTest.java @@ -434,7 +434,6 @@ public void skipOnError() { } @Test - @SuppressWarnings("unchecked") public void exactBackpressure() { TestScheduler scheduler = new TestScheduler(); @@ -449,7 +448,6 @@ public void exactBackpressure() { } @Test - @SuppressWarnings("unchecked") public void skipBackpressure() { TestScheduler scheduler = new TestScheduler(); @@ -464,7 +462,6 @@ public void skipBackpressure() { } @Test - @SuppressWarnings("unchecked") public void overlapBackpressure() { TestScheduler scheduler = new TestScheduler(); @@ -866,7 +863,6 @@ public Publisher> apply(Flowable f) }); } - @SuppressWarnings("unchecked") @Test public void firstWindowMissingBackpressure() { Flowable.never() diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java index 544ca11bc6..655452057b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromTest.java @@ -704,7 +704,6 @@ public Object apply(Integer a, Integer b) throws Exception { .assertFailure(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void combineToNull2() { Flowable.just(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java index d40b3908c9..e4b41c8bea 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java @@ -1422,7 +1422,6 @@ public Object apply(Integer a, Integer b) throws Exception { .assertResult("929"); } - @SuppressWarnings("unchecked") @Test public void zipArrayEmpty() { assertSame(Flowable.empty(), Flowable.zipArray(Functions.identity(), false, 16)); @@ -1897,7 +1896,6 @@ public Object apply(Object[] a) throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void publishersInIterable() { Publisher source = new Publisher() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java index af49d5353b..dec723d651 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java @@ -50,7 +50,6 @@ public void ambLots() { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void ambFirstDone() { Maybe.amb(Arrays.asList(Maybe.just(1), Maybe.just(2))) @@ -58,7 +57,6 @@ public void ambFirstDone() { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void dispose() { PublishProcessor pp1 = PublishProcessor.create(); @@ -76,7 +74,6 @@ public void dispose() { assertFalse(pp2.hasSubscribers()); } - @SuppressWarnings("unchecked") @Test public void innerErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -255,7 +252,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void maybeSourcesInIterable() { MaybeSource source = new MaybeSource() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java index c693e0296a..7450dc993b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterableTest.java @@ -29,7 +29,6 @@ public class MaybeConcatIterableTest extends RxJavaTest { - @SuppressWarnings("unchecked") @Test public void take() { Maybe.concat(Arrays.asList(Maybe.just(1), Maybe.just(2), Maybe.just(3))) @@ -50,7 +49,6 @@ public Iterator> iterator() { .assertFailureAndMessage(TestException.class, "iterator()"); } - @SuppressWarnings("unchecked") @Test public void error() { Maybe.concat(Arrays.asList(Maybe.just(1), Maybe.error(new TestException()), Maybe.just(3))) @@ -58,7 +56,6 @@ public void error() { .assertFailure(TestException.class, 1); } - @SuppressWarnings("unchecked") @Test public void successCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -124,7 +121,6 @@ public Maybe apply(Integer v) throws Exception { .assertFailure(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscription() { final int[] calls = { 0 }; @@ -144,7 +140,6 @@ public void subscribe(MaybeEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionDelayError() { final int[] calls = { 0 }; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterializeTest.java index 57a0ed11ae..57a478d112 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterializeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterializeTest.java @@ -24,7 +24,6 @@ public class MaybeMaterializeTest extends RxJavaTest { @Test - @SuppressWarnings("unchecked") public void success() { Maybe.just(1) .materialize() @@ -33,7 +32,6 @@ public void success() { } @Test - @SuppressWarnings("unchecked") public void error() { TestException ex = new TestException(); Maybe.error(ex) @@ -43,7 +41,6 @@ public void error() { } @Test - @SuppressWarnings("unchecked") public void empty() { Maybe.empty() .materialize() diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorXTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorXTest.java index e7d5e4b31b..f3a9b9345c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorXTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorXTest.java @@ -66,7 +66,6 @@ public void onErrorCompleteFalse() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void onErrorReturnFunctionThrows() { TestHelper.assertCompositeExceptions(Maybe.error(new TestException()) @@ -79,7 +78,6 @@ public Object apply(Throwable v) throws Exception { .to(TestHelper.testConsumer()), TestException.class, IOException.class); } - @SuppressWarnings("unchecked") @Test public void onErrorCompletePredicateThrows() { TestHelper.assertCompositeExceptions(Maybe.error(new TestException()) @@ -116,7 +114,6 @@ public void onExceptionResumeNextPassthrough() { .assertFailure(AssertionError.class); } - @SuppressWarnings("unchecked") @Test public void onErrorResumeNextFunctionThrows() { TestHelper.assertCompositeExceptions(Maybe.error(new TestException()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java index f06be6be57..aae11b288e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java @@ -38,7 +38,6 @@ public Object apply(Object[] a) throws Exception { } }; - @SuppressWarnings("unchecked") @Test public void firstError() { Maybe.zip(Arrays.asList(Maybe.error(new TestException()), Maybe.just(1)), addString) @@ -46,7 +45,6 @@ public void firstError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void secondError() { Maybe.zip(Arrays.asList(Maybe.just(1), Maybe.error(new TestException())), addString) @@ -54,7 +52,6 @@ public void secondError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void dispose() { PublishProcessor pp = PublishProcessor.create(); @@ -69,7 +66,6 @@ public void dispose() { assertFalse(pp.hasSubscribers()); } - @SuppressWarnings("unchecked") @Test public void zipperThrows() { Maybe.zip(Arrays.asList(Maybe.just(1), Maybe.just(2)), new Function() { @@ -82,7 +78,6 @@ public Object apply(Object[] b) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void zipperReturnsNull() { Maybe.zip(Arrays.asList(Maybe.just(1), Maybe.just(2)), new Function() { @@ -95,7 +90,6 @@ public Object apply(Object[] a) throws Exception { .assertFailure(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void middleError() { PublishProcessor pp0 = PublishProcessor.create(); @@ -112,7 +106,6 @@ public void middleError() { to.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void innerErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -190,7 +183,6 @@ public Maybe apply(Integer v) throws Exception { .assertFailureAndMessage(TestException.class, "next()"); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableOneIsNull() { Maybe.zip(Arrays.asList(null, Maybe.just(1)), new Function() { @@ -202,7 +194,6 @@ public Object apply(Object[] v) { .blockingGet(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableTwoIsNull() { Maybe.zip(Arrays.asList(Maybe.just(1), null), new Function() { @@ -222,7 +213,6 @@ public void singleSourceZipperReturnsNull() { .assertFailureAndMessage(NullPointerException.class, "The zipper returned a null value"); } - @SuppressWarnings("unchecked") @Test public void maybeSourcesInIterable() { MaybeSource source = new MaybeSource() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java index 239d8339ea..4e8cbd34b6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java @@ -374,7 +374,6 @@ public void ambWithOrder() { Observable.just(1).ambWith(error).test().assertValue(1).assertComplete(); } - @SuppressWarnings("unchecked") @Test public void ambIterableOrder() { Observable error = Observable.error(new RuntimeException()); @@ -467,7 +466,6 @@ public void run() throws Exception { } } - @SuppressWarnings("unchecked") @Test public void observableSourcesInIterable() { ObservableSource source = new ObservableSource() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java index eeee099b0d..af4e7a9415 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTest.java @@ -764,7 +764,6 @@ public void onComplete() { assertFalse(observer.isDisposed()); } - @SuppressWarnings("unchecked") @Test public void bufferTimeSkipDefault() { Observable.range(1, 5).buffer(1, 1, TimeUnit.MINUTES) @@ -772,7 +771,6 @@ public void bufferTimeSkipDefault() { .assertResult(Arrays.asList(1, 2, 3, 4, 5)); } - @SuppressWarnings("unchecked") @Test public void bufferBoundaryHint() { Observable.range(1, 5).buffer(Observable.timer(1, TimeUnit.MINUTES), 2) @@ -784,7 +782,6 @@ static HashSet set(Integer... values) { return new HashSet<>(Arrays.asList(values)); } - @SuppressWarnings("unchecked") @Test public void bufferIntoCustomCollection() { Observable.just(1, 1, 2, 2, 3, 3, 4, 4) @@ -798,7 +795,6 @@ public Collection get() throws Exception { .assertResult(set(1, 2), set(2, 3), set(4)); } - @SuppressWarnings("unchecked") @Test public void bufferSkipIntoCustomCollection() { Observable.just(1, 1, 2, 2, 3, 3, 4, 4) @@ -813,7 +809,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows() { Observable.just(1) .buffer(1, TimeUnit.SECONDS, Schedulers.single(), Integer.MAX_VALUE, new Supplier>() { @@ -827,7 +822,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows2() { Observable.just(1) .buffer(1, TimeUnit.SECONDS, Schedulers.single(), 10, new Supplier>() { @@ -841,7 +835,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows3() { Observable.just(1) .buffer(2, 1, TimeUnit.SECONDS, Schedulers.single(), new Supplier>() { @@ -855,7 +848,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows4() { Observable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), Integer.MAX_VALUE, new Supplier>() { @@ -875,7 +867,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows5() { Observable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), 10, new Supplier>() { @@ -895,7 +886,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierThrows6() { Observable.never() .buffer(2, 1, TimeUnit.MILLISECONDS, Schedulers.single(), new Supplier>() { @@ -915,7 +905,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierReturnsNull() { Observable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), Integer.MAX_VALUE, new Supplier>() { @@ -935,7 +924,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierReturnsNull2() { Observable.never() .buffer(1, TimeUnit.MILLISECONDS, Schedulers.single(), 10, new Supplier>() { @@ -955,7 +943,6 @@ public Collection get() throws Exception { } @Test - @SuppressWarnings("unchecked") public void supplierReturnsNull3() { Observable.never() .buffer(2, 1, TimeUnit.MILLISECONDS, Schedulers.single(), new Supplier>() { @@ -996,7 +983,6 @@ public void dispose() { TestHelper.checkDisposed(PublishSubject.create().buffer(Observable.never(), Functions.justFunction(Observable.never()))); } - @SuppressWarnings("unchecked") @Test public void restartTimer() { Observable.range(1, 5) @@ -1005,7 +991,6 @@ public void restartTimer() { .assertResult(Arrays.asList(1, 2), Arrays.asList(3, 4), Arrays.asList(5)); } - @SuppressWarnings("unchecked") @Test public void bufferSupplierCrash2() { Observable.range(1, 2) @@ -1023,7 +1008,6 @@ public List get() throws Exception { .assertFailure(TestException.class, Arrays.asList(1)); } - @SuppressWarnings("unchecked") @Test public void bufferSkipSupplierCrash2() { Observable.range(1, 2) @@ -1041,7 +1025,6 @@ public List get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferSkipError() { Observable.error(new TestException()) @@ -1050,7 +1033,6 @@ public void bufferSkipError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferSkipOverlap() { Observable.range(1, 5) @@ -1065,7 +1047,6 @@ public void bufferSkipOverlap() { ); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactError() { Observable.error(new TestException()) @@ -1074,7 +1055,6 @@ public void bufferTimedExactError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferTimedSkipError() { Observable.error(new TestException()) @@ -1083,7 +1063,6 @@ public void bufferTimedSkipError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferTimedOverlapError() { Observable.error(new TestException()) @@ -1092,7 +1071,6 @@ public void bufferTimedOverlapError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactEmpty() { Observable.empty() @@ -1101,7 +1079,6 @@ public void bufferTimedExactEmpty() { .assertResult(Collections.emptyList()); } - @SuppressWarnings("unchecked") @Test public void bufferTimedSkipEmpty() { Observable.empty() @@ -1110,7 +1087,6 @@ public void bufferTimedSkipEmpty() { .assertResult(Collections.emptyList()); } - @SuppressWarnings("unchecked") @Test public void bufferTimedOverlapEmpty() { Observable.empty() @@ -1119,7 +1095,6 @@ public void bufferTimedOverlapEmpty() { .assertResult(Collections.emptyList()); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactSupplierCrash() { TestScheduler scheduler = new TestScheduler(); @@ -1149,7 +1124,6 @@ public List get() throws Exception { .assertFailure(TestException.class, Arrays.asList(1)); } - @SuppressWarnings("unchecked") @Test public void bufferTimedExactBoundedError() { TestScheduler scheduler = new TestScheduler(); @@ -1207,7 +1181,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void noCompletionCancelExact() { final AtomicInteger counter = new AtomicInteger(); @@ -1227,7 +1200,6 @@ public void run() throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void noCompletionCancelSkip() { final AtomicInteger counter = new AtomicInteger(); @@ -1247,7 +1219,6 @@ public void run() throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void noCompletionCancelOverlap() { final AtomicInteger counter = new AtomicInteger(); @@ -1268,7 +1239,6 @@ public void run() throws Exception { } @Test - @SuppressWarnings("unchecked") public void boundaryOpenCloseDisposedOnComplete() { PublishSubject source = PublishSubject.create(); @@ -1344,7 +1314,6 @@ public Observable apply(Long a) { } @Test - @SuppressWarnings("unchecked") public void openClosemainError() { Observable.error(new TestException()) .buffer(Observable.never(), Functions.justFunction(Observable.never())) @@ -1353,7 +1322,6 @@ public void openClosemainError() { } @Test - @SuppressWarnings("unchecked") public void openClosebadSource() { List errors = TestHelper.trackPluginErrors(); try { @@ -1391,7 +1359,6 @@ protected void subscribeActual(Observer observer) { } @Test - @SuppressWarnings("unchecked") public void openCloseOpenCompletes() { PublishSubject source = PublishSubject.create(); @@ -1420,7 +1387,6 @@ public void openCloseOpenCompletes() { } @Test - @SuppressWarnings("unchecked") public void openCloseOpenCompletesNoBuffers() { PublishSubject source = PublishSubject.create(); @@ -1449,7 +1415,6 @@ public void openCloseOpenCompletesNoBuffers() { } @Test - @SuppressWarnings("unchecked") public void openCloseTake() { PublishSubject source = PublishSubject.create(); @@ -1473,7 +1438,6 @@ public void openCloseTake() { } @Test - @SuppressWarnings("unchecked") public void openCloseBadOpen() { List errors = TestHelper.trackPluginErrors(); try { @@ -1517,7 +1481,6 @@ protected void subscribeActual(Observer observer) { } @Test - @SuppressWarnings("unchecked") public void openCloseBadClose() { List errors = TestHelper.trackPluginErrors(); try { @@ -1574,7 +1537,6 @@ public ObservableSource> apply(Observable f) ); } - @SuppressWarnings("unchecked") @Test public void bufferExactBoundarySecondBufferCrash() { PublishSubject ps = PublishSubject.create(); @@ -1596,7 +1558,6 @@ public List get() throws Exception { to.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void bufferExactBoundaryBadSource() { Observable ps = new Observable() { @@ -1814,7 +1775,6 @@ public ObservableSource> apply(Observable o) } @Test - @SuppressWarnings("unchecked") public void bufferExactFailingSupplier() { Observable.empty() .buffer(1, TimeUnit.SECONDS, Schedulers.computation(), 10, new Supplier>() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java index d28bd20e97..0568b0d474 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectTest.java @@ -142,7 +142,6 @@ public void accept(Object o, Integer t) { assertFalse(added.get()); } - @SuppressWarnings("unchecked") @Test public void collectIntoObservable() { Observable.just(1, 1, 1, 1, 2) @@ -266,7 +265,6 @@ public void accept(Object o, Integer t) { assertFalse(added.get()); } - @SuppressWarnings("unchecked") @Test public void collectInto() { Observable.just(1, 1, 1, 1, 2) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java index c5f93d4222..3096cdabd7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatestTest.java @@ -818,7 +818,6 @@ public Object apply(Object[] a) throws Exception { } @Test - @SuppressWarnings("unchecked") public void combineLatestDelayErrorIterableOfSources() { Observable.combineLatestDelayError(Arrays.asList( @@ -834,7 +833,6 @@ public Object apply(Object[] a) throws Exception { } @Test - @SuppressWarnings("unchecked") public void combineLatestDelayErrorIterableOfSourcesWithError() { Observable.combineLatestDelayError(Arrays.asList( @@ -1044,7 +1042,6 @@ public Object apply(Object a, Object b) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void dontSubscribeIfDone2() { List errors = TestHelper.trackPluginErrors(); @@ -1078,7 +1075,6 @@ public Object apply(Object[] a) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void combine2Observable2Errors() throws Exception { List errors = TestHelper.trackPluginErrors(); @@ -1195,7 +1191,6 @@ public Integer apply(Integer t1, Integer t2) throws Exception { } @Test - @SuppressWarnings("unchecked") public void syncFirstErrorsAfterItemDelayError() { Observable.combineLatestDelayError(Arrays.asList( Observable.just(21).concatWith(Observable.error(new TestException())), @@ -1213,7 +1208,6 @@ public Object apply(Object[] a) throws Exception { .assertFailure(TestException.class, 42); } - @SuppressWarnings("unchecked") @Test public void observableSourcesInIterable() { ObservableSource source = new ObservableSource() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java index fe1cafe56b..6ad8b9c649 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java @@ -503,7 +503,6 @@ public void concatArrayEager() throws Exception { } } - @SuppressWarnings("unchecked") @Test public void capacityHint() { Observable source = Observable.just(1); @@ -540,7 +539,6 @@ public void ObservableCapacityHint() { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void badCapacityHint() throws Exception { Observable source = Observable.just(1); @@ -564,7 +562,6 @@ public void mappingBadCapacityHint() throws Exception { } - @SuppressWarnings("unchecked") @Test public void concatEagerIterable() { Observable.concatEager(Arrays.asList(Observable.just(1), Observable.just(2))) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java index c5b00fdf7c..05df845a44 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java @@ -271,7 +271,7 @@ public void blockedObservableOfObservables() { final Observable even = Observable.fromArray(e); final CountDownLatch callOnce = new CountDownLatch(1); final CountDownLatch okToContinue = new CountDownLatch(1); - @SuppressWarnings("unchecked") + TestObservable> observableOfObservables = new TestObservable<>(callOnce, okToContinue, odds, even); Observable concatF = Observable.concat(Observable.unsafeCreate(observableOfObservables)); concatF.subscribe(observer); @@ -310,7 +310,6 @@ public void concatConcurrentWithInfinity() { Observer observer = TestHelper.mockObserver(); - @SuppressWarnings("unchecked") TestObservable> observableOfObservables = new TestObservable<>(Observable.unsafeCreate(w1), Observable.unsafeCreate(w2)); Observable concatF = Observable.concat(Observable.unsafeCreate(observableOfObservables)); @@ -438,7 +437,6 @@ public void concatUnsubscribeConcurrent() { Observer observer = TestHelper.mockObserver(); TestObserver to = new TestObserver<>(observer); - @SuppressWarnings("unchecked") TestObservable> observableOfObservables = new TestObservable<>(Observable.unsafeCreate(w1), Observable.unsafeCreate(w2)); Observable concatF = Observable.concat(Observable.unsafeCreate(observableOfObservables)); @@ -493,10 +491,12 @@ public boolean isDisposed() { private final T seed; private final int size; + @SafeVarargs TestObservable(T... values) { this(null, null, values); } + @SafeVarargs TestObservable(CountDownLatch once, CountDownLatch okToContinue, T... values) { this.values = Arrays.asList(values); this.size = this.values.size(); @@ -799,7 +799,6 @@ public void concatArrayDelayErrorWithError() { .assertFailure(TestException.class, 1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatIterableDelayError() { Observable.concatDelayError( @@ -809,7 +808,6 @@ public void concatIterableDelayError() { .assertResult(1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatIterableDelayErrorWithError() { Observable.concatDelayError( @@ -1004,7 +1002,6 @@ public void subscribe(ObservableEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionIterable() { final int[] calls = { 0 }; @@ -1025,7 +1022,6 @@ public void subscribe(ObservableEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionDelayErrorIterable() { final int[] calls = { 0 }; @@ -1179,7 +1175,6 @@ public void run() throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void noCancelPreviousIterable() { final AtomicInteger counter = new AtomicInteger(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java index db6450d097..c4d24a69c7 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java @@ -499,7 +499,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayError() { Observable.mergeDelayError(Arrays.asList(Observable.just(1), Observable.just(2))) @@ -515,7 +514,6 @@ public void mergeArrayDelayError() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayErrorWithError() { Observable.mergeDelayError( @@ -561,7 +559,6 @@ public void mergeDelayErrorWithErrorMaxConcurrency() { .assertFailure(TestException.class, 1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayErrorMaxConcurrency() { Observable.mergeDelayError( @@ -571,7 +568,6 @@ public void mergeIterableDelayErrorMaxConcurrency() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayErrorWithErrorMaxConcurrency() { Observable.mergeDelayError( diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java index 3b849ab978..5f935791b1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeIntervalTest.java @@ -128,7 +128,6 @@ public void dispose() { TestHelper.checkDisposed(Observable.just(1).timeInterval()); } - @SuppressWarnings("unchecked") @Test public void error() { Observable.error(new TestException()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListTest.java index 1981c2b5af..020183f9fa 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListTest.java @@ -85,7 +85,6 @@ public void listWithBlockingFirstObservable() { Assert.assertEquals(Arrays.asList("one", "two", "three"), actual); } - @SuppressWarnings("unchecked") @Test public void capacityHintObservable() { Observable.range(1, 10) @@ -154,7 +153,6 @@ static void await(CyclicBarrier cb) { } } - @SuppressWarnings("unchecked") @Test public void capacityHint() { Observable.range(1, 10) @@ -170,7 +168,6 @@ public void dispose() { TestHelper.checkDisposed(Observable.just(1).toList()); } - @SuppressWarnings("unchecked") @Test public void error() { Observable.error(new TestException()) @@ -180,7 +177,6 @@ public void error() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorSingle() { Observable.error(new TestException()) @@ -189,7 +185,6 @@ public void errorSingle() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void collectionSupplierThrows() { Observable.just(1) @@ -204,7 +199,6 @@ public Collection get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void collectionSupplierReturnsNull() { Observable.just(1) @@ -220,7 +214,6 @@ public Collection get() throws Exception { .assertErrorMessage(ExceptionHelper.nullWarning("The collectionSupplier returned a null Collection.")); } - @SuppressWarnings("unchecked") @Test public void singleCollectionSupplierThrows() { Observable.just(1) @@ -234,7 +227,6 @@ public Collection get() throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void singleCollectionSupplierReturnsNull() { Observable.just(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToSortedListTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToSortedListTest.java index 7d01bcc3b1..19c10c6622 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToSortedListTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToSortedListTest.java @@ -96,7 +96,6 @@ public int compare(Integer a, Integer b) { .assertResult(5, 4, 3, 2, 1); } - @SuppressWarnings("unchecked") @Test public void toSortedListCapacityObservable() { Observable.just(5, 1, 2, 4, 3).toSortedList(4).toObservable() @@ -104,7 +103,6 @@ public void toSortedListCapacityObservable() { .assertResult(Arrays.asList(1, 2, 3, 4, 5)); } - @SuppressWarnings("unchecked") @Test public void toSortedListComparatorCapacityObservable() { Observable.just(5, 1, 2, 4, 3).toSortedList(new Comparator() { @@ -152,7 +150,6 @@ public void withFollowingFirst() { assertEquals(Arrays.asList(1, 2, 3, 4, 5), o.toSortedList().blockingGet()); } - @SuppressWarnings("unchecked") @Test public void toSortedListCapacity() { Observable.just(5, 1, 2, 4, 3).toSortedList(4) @@ -160,7 +157,6 @@ public void toSortedListCapacity() { .assertResult(Arrays.asList(1, 2, 3, 4, 5)); } - @SuppressWarnings("unchecked") @Test public void toSortedListComparatorCapacity() { Observable.just(5, 1, 2, 4, 3).toSortedList(new Comparator() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java index a897430ab7..04099c31cb 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithSizeTest.java @@ -293,7 +293,6 @@ public ObservableSource> apply(Observable o) throws E }); } - @SuppressWarnings("unchecked") @Test public void errorExact() { Observable.error(new TestException()) @@ -302,7 +301,6 @@ public void errorExact() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorSkip() { Observable.error(new TestException()) @@ -311,7 +309,6 @@ public void errorSkip() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorOverlap() { Observable.error(new TestException()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java index 0c7dc8056b..af67c36bf2 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowWithStartEndObservableTest.java @@ -498,7 +498,6 @@ public void accept(Observable v) throws Throwable { inner.get().test().assertResult(); } - @SuppressWarnings("unchecked") @Test public void closingIndicatorFunctionCrash() { @@ -524,7 +523,6 @@ public Observable apply(Integer end) throws Throwable { assertFalse(boundary.hasObservers()); } - @SuppressWarnings("unchecked") @Test public void mainError() { Observable.error(new TestException()) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java index a754b21b8c..c23c41fde1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromTest.java @@ -579,7 +579,6 @@ public Object apply(Integer a, Integer b) throws Exception { .assertFailure(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void combineToNull2() { Observable.just(1) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java index d8c153a285..1de6190f05 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java @@ -1430,7 +1430,6 @@ public Object apply(Object[] a) throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void observableSourcesInIterable() { ObservableSource source = new ObservableSource() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java index a6caec6171..7faf26454e 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java @@ -71,7 +71,6 @@ public void ambWithSecondFires() { to.assertResult(2); } - @SuppressWarnings("unchecked") @Test public void ambIterableWithFirstFires() { PublishProcessor pp1 = PublishProcessor.create(); @@ -93,7 +92,6 @@ public void ambIterableWithFirstFires() { } - @SuppressWarnings("unchecked") @Test public void ambIterableWithSecondFires() { PublishProcessor pp1 = PublishProcessor.create(); @@ -272,7 +270,6 @@ public void ambWithOrder() { Single.just(1).ambWith(error).test().assertValue(1); } - @SuppressWarnings("unchecked") @Test public void ambIterableOrder() { Single error = Single.error(new RuntimeException()); @@ -343,7 +340,6 @@ public void accept(Object v, Throwable e) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void singleSourcesInIterable() { SingleSource source = new SingleSource() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java index eebe0c1020..348810be1f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java @@ -89,7 +89,6 @@ public void concatArrayEagerTest() { ts.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void concatEagerIterableTest() { PublishProcessor pp1 = PublishProcessor.create(); @@ -163,7 +162,6 @@ public void subscribe(SingleEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionIterable() { final int[] calls = { 0 }; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterializeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterializeTest.java index 00b5f8d712..4bc05ea67d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterializeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterializeTest.java @@ -24,7 +24,6 @@ public class SingleMaterializeTest extends RxJavaTest { @Test - @SuppressWarnings("unchecked") public void success() { Single.just(1) .materialize() @@ -33,7 +32,6 @@ public void success() { } @Test - @SuppressWarnings("unchecked") public void error() { TestException ex = new TestException(); Maybe.error(ex) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMergeTest.java index ee991b431b..7aba476c86 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMergeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleMergeTest.java @@ -72,7 +72,6 @@ public void mergeErrors() { } } - @SuppressWarnings("unchecked") @Test public void mergeDelayErrorIterable() { Single.mergeDelayError(Arrays.asList( diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java index 948972dbcb..6fd6744479 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterableTest.java @@ -38,7 +38,6 @@ public Object apply(Object[] a) throws Exception { } }; - @SuppressWarnings("unchecked") @Test public void firstError() { Single.zip(Arrays.asList(Single.error(new TestException()), Single.just(1)), addString) @@ -46,7 +45,6 @@ public void firstError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void secondError() { Single.zip(Arrays.asList(Single.just(1), Single.error(new TestException())), addString) @@ -54,7 +52,6 @@ public void secondError() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void dispose() { PublishProcessor pp = PublishProcessor.create(); @@ -69,7 +66,6 @@ public void dispose() { assertFalse(pp.hasSubscribers()); } - @SuppressWarnings("unchecked") @Test public void zipperThrows() { Single.zip(Arrays.asList(Single.just(1), Single.just(2)), new Function() { @@ -82,7 +78,6 @@ public Object apply(Object[] b) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void zipperReturnsNull() { Single.zip(Arrays.asList(Single.just(1), Single.just(2)), new Function() { @@ -95,7 +90,6 @@ public Object apply(Object[] a) throws Exception { .assertFailure(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void middleError() { PublishProcessor pp0 = PublishProcessor.create(); @@ -112,7 +106,6 @@ public void middleError() { to.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void innerErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -190,7 +183,6 @@ public Single apply(Integer v) throws Exception { .assertFailureAndMessage(TestException.class, "next()"); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableOneIsNull() { Single.zip(Arrays.asList(null, Single.just(1)), new Function() { @@ -202,7 +194,6 @@ public Object apply(Object[] v) { .blockingGet(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableTwoIsNull() { Single.zip(Arrays.asList(Single.just(1), null), new Function() { @@ -238,7 +229,6 @@ public Object apply(Object[] a) throws Exception { .assertResult(2); } - @SuppressWarnings("unchecked") @Test public void singleSourceZipperReturnsNull() { Single.zip(Arrays.asList(Single.just(1)), Functions.justFunction(null)) @@ -246,7 +236,6 @@ public void singleSourceZipperReturnsNull() { .assertFailureAndMessage(NullPointerException.class, "The zipper returned a null value"); } - @SuppressWarnings("unchecked") @Test public void singleSourcesInIterable() { SingleSource source = new SingleSource() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipTest.java index f5ae16d0bc..79380cfa5f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipTest.java @@ -163,7 +163,6 @@ public Integer apply(Integer a, Integer b) throws Exception { assertEquals(0, counter.get()); } - @SuppressWarnings("unchecked") @Test public void noDisposeOnAllSuccess2() { final AtomicInteger counter = new AtomicInteger(); diff --git a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java index e4d64764d8..c3335c0bab 100644 --- a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java @@ -1401,7 +1401,6 @@ public void concat4() { .assertResult(1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatIterable() { Maybe.concat(Arrays.asList(Maybe.just(1), Maybe.just(2))) @@ -1409,7 +1408,6 @@ public void concatIterable() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void concatIterableEmpty() { Maybe.concat(Arrays.asList(Maybe.empty(), Maybe.empty())) @@ -1417,7 +1415,6 @@ public void concatIterableEmpty() { .assertResult(); } - @SuppressWarnings("unchecked") @Test public void concatIterableBackpressured() { TestSubscriber ts = Maybe.concat(Arrays.asList(Maybe.just(1), Maybe.just(2))) @@ -1434,7 +1431,6 @@ public void concatIterableBackpressured() { ts.assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void concatIterableBackpressuredNonEager() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1624,7 +1620,6 @@ public void ambWithOrder() { Maybe.just(1).ambWith(error).test().assertValue(1); } - @SuppressWarnings("unchecked") @Test public void ambIterableOrder() { Maybe error = Maybe.error(new RuntimeException()); @@ -1772,7 +1767,6 @@ public void ambArray2SignalsComplete() { to.assertResult(); } - @SuppressWarnings("unchecked") @Test public void ambIterable1SignalsSuccess() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1795,7 +1789,6 @@ public void ambIterable1SignalsSuccess() { to.assertResult(1); } - @SuppressWarnings("unchecked") @Test public void ambIterable2SignalsSuccess() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1818,7 +1811,6 @@ public void ambIterable2SignalsSuccess() { to.assertResult(2); } - @SuppressWarnings("unchecked") @Test public void ambIterable2SignalsSuccessWithOverlap() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1842,7 +1834,6 @@ public void ambIterable2SignalsSuccessWithOverlap() { to.assertResult(2); } - @SuppressWarnings("unchecked") @Test public void ambIterable1SignalsError() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1864,7 +1855,6 @@ public void ambIterable1SignalsError() { to.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void ambIterable2SignalsError() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1886,7 +1876,6 @@ public void ambIterable2SignalsError() { to.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void ambIterable2SignalsErrorWithOverlap() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1909,7 +1898,6 @@ public void ambIterable2SignalsErrorWithOverlap() { to.assertFailureAndMessage(TestException.class, "2"); } - @SuppressWarnings("unchecked") @Test public void ambIterable1SignalsComplete() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1931,7 +1919,6 @@ public void ambIterable1SignalsComplete() { to.assertResult(); } - @SuppressWarnings("unchecked") @Test public void ambIterable2SignalsComplete() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1968,7 +1955,6 @@ public Iterator> iterator() { }).test().assertError(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void ambIterableOneIsNull() { Maybe.amb(Arrays.asList(null, Maybe.just(1))) @@ -2188,7 +2174,6 @@ public void mergeMaybe() { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void mergeIterable() { Maybe.merge(Arrays.asList(Maybe.just(1), Maybe.just(2), Maybe.just(3))) @@ -2493,7 +2478,6 @@ public void concatArrayDelayError() { assertFalse(Maybe.concatArrayDelayError(Maybe.never()) instanceof MaybeConcatArrayDelayError); } - @SuppressWarnings("unchecked") @Test public void concatIterableDelayError() { Maybe.concatDelayError(Arrays.asList(Maybe.empty(), Maybe.just(1), Maybe.error(new TestException()))) @@ -2539,7 +2523,6 @@ public void concatEagerArray() { } - @SuppressWarnings("unchecked") @Test public void concatEagerIterable() { PublishProcessor pp1 = PublishProcessor.create(); @@ -2638,7 +2621,6 @@ public void mergeArrayDelayError() { assertSame(Flowable.empty(), Maybe.mergeArrayDelayError()); } - @SuppressWarnings("unchecked") @Test public void mergeIterableDelayError() { Maybe.mergeDelayError(Arrays.asList(Maybe.empty(), Maybe.just(1), Maybe.error(new TestException()))) @@ -2900,7 +2882,6 @@ public void zipArray() { .assertResult("[1]"); } - @SuppressWarnings("unchecked") @Test public void zipIterable() { Maybe.zip( @@ -3048,7 +3029,6 @@ public void ambWith2SignalsSuccess() { @Test public void zipIterableObject() { - @SuppressWarnings("unchecked") final List> maybes = Arrays.asList(Maybe.just(1), Maybe.just(4)); Maybe.zip(maybes, new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableConcatTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableConcatTests.java index a9d1afca12..f867a165d2 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableConcatTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableConcatTests.java @@ -62,7 +62,6 @@ public void concatWithIterableOfObservable() { Observable o2 = Observable.just("three", "four"); Observable o3 = Observable.just("five", "six"); - @SuppressWarnings("unchecked") Iterable> is = Arrays.asList(o1, o2, o3); List values = Observable.concat(Observable.fromIterable(is)).toList().blockingGet(); diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java index 5e02ecc3e7..9c112b0f7d 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java @@ -66,7 +66,6 @@ public Iterator> iterator() { }).test().assertError(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void ambIterableOneIsNull() { Observable.amb(Arrays.asList(Observable.never(), null)) @@ -99,7 +98,6 @@ public Object apply(Object[] v) { }, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestIterableOneIsNull() { Observable.combineLatest(Arrays.asList(Observable.never(), null), new Function() { @@ -110,13 +108,11 @@ public Object apply(Object[] v) { }, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestIterableFunctionNull() { Observable.combineLatest(Arrays.asList(just1), null, 128); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestIterableFunctionReturnsNull() { Observable.combineLatest(Arrays.asList(just1), new Function() { @@ -152,7 +148,6 @@ public Object apply(Object[] v) { }, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestDelayErrorIterableOneIsNull() { Observable.combineLatestDelayError(Arrays.asList(Observable.never(), null), new Function() { @@ -163,13 +158,11 @@ public Object apply(Object[] v) { }, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestDelayErrorIterableFunctionNull() { Observable.combineLatestDelayError(Arrays.asList(just1), null, 128); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void combineLatestDelayErrorIterableFunctionReturnsNull() { Observable.combineLatestDelayError(Arrays.asList(just1), new Function() { @@ -195,7 +188,6 @@ public Iterator> iterator() { }).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void concatIterableOneIsNull() { Observable.concat(Arrays.asList(just1, null)).blockingLast(); @@ -520,7 +512,6 @@ public Iterator> iterator() { }, 128, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeIterableOneIsNull() { Observable.merge(Arrays.asList(just1, null), 128, 128).blockingLast(); @@ -552,7 +543,6 @@ public Iterator> iterator() { }, 128, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeDelayErrorIterableOneIsNull() { Observable.mergeDelayError(Arrays.asList(just1, null), 128, 128).blockingLast(); @@ -683,13 +673,11 @@ public Object apply(Object[] v) { }).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableFunctionNull() { Observable.zip(Arrays.asList(just1, just1), null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableFunctionReturnsNull() { Observable.zip(Arrays.asList(just1, just1), new Function() { @@ -725,13 +713,11 @@ public Object apply(Object[] a) { }, true, 128).blockingLast(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterable2FunctionNull() { Observable.zip(Arrays.asList(just1, just1), null, true, 128); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterable2FunctionReturnsNull() { Observable.zip(Arrays.asList(just1, just1), new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java index 4b888d4ed5..c3110eac3f 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableTest.java @@ -1120,7 +1120,6 @@ public void singleDefaultObservable() { @Test public void zipIterableObject() { - @SuppressWarnings("unchecked") final List> observables = Arrays.asList(Observable.just(1, 2, 3), Observable.just(1, 2, 3)); Observable.zip(observables, new Function() { @Override @@ -1136,7 +1135,6 @@ public Object apply(Object[] o) throws Exception { @Test public void combineLatestObject() { - @SuppressWarnings("unchecked") final List> observables = Arrays.asList(Observable.just(1, 2, 3), Observable.just(1, 2, 3)); Observable.combineLatest(observables, new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java index 26d5af8628..1a94b3c247 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelCollectTest.java @@ -45,7 +45,6 @@ public void accept(List a, Integer b) throws Exception { })); } - @SuppressWarnings("unchecked") @Test public void initialCrash() { Flowable.range(1, 5) @@ -66,7 +65,6 @@ public void accept(List a, Integer b) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void reducerCrash() { Flowable.range(1, 5) @@ -117,7 +115,6 @@ public void accept(List a, Integer b) throws Exception { assertFalse(pp.hasSubscribers()); } - @SuppressWarnings("unchecked") @Test public void error() { Flowable.error(new TestException()) @@ -138,7 +135,6 @@ public void accept(List a, Integer b) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void doubleError() { List errors = TestHelper.trackPluginErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelDoOnNextTryTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelDoOnNextTryTest.java index 47371b3073..0ba01a05fd 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelDoOnNextTryTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelDoOnNextTryTest.java @@ -189,7 +189,6 @@ public void accept(Integer v) throws Exception { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void doOnNextFailHandlerThrows() { TestSubscriberEx ts = Flowable.range(0, 2) @@ -336,7 +335,6 @@ public void accept(Integer v) throws Exception { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void doOnNextFailHandlerThrowsConditional() { TestSubscriberEx ts = Flowable.range(0, 2) diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFilterTryTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFilterTryTest.java index f2f7377942..084566a0a5 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFilterTryTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFilterTryTest.java @@ -192,7 +192,6 @@ public boolean test(Integer v) throws Exception { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void filterFailHandlerThrows() { TestSubscriberEx ts = Flowable.range(0, 2) @@ -327,7 +326,6 @@ public boolean test(Integer v) throws Exception { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void filterFailHandlerThrowsConditional() { TestSubscriberEx ts = Flowable.range(0, 2) diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java index 1e048a1dcc..93ffe8de7d 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java @@ -236,7 +236,6 @@ public Long apply(Long a, Long b) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void toSortedList() { TestSubscriber> ts = new TestSubscriber<>(); diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java index 2df4dd1462..b036b2e17f 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java @@ -167,7 +167,6 @@ public Integer apply(Integer v) throws Exception { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void mapFailHandlerThrows() { TestSubscriberEx ts = Flowable.range(0, 2) @@ -302,7 +301,6 @@ public Integer apply(Integer v) throws Exception { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void mapFailHandlerThrowsConditional() { TestSubscriberEx ts = Flowable.range(0, 2) diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java index aac8c0af62..5646b176b1 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceTest.java @@ -46,7 +46,6 @@ public List apply(List a, Integer b) throws Exception { })); } - @SuppressWarnings("unchecked") @Test public void initialCrash() { Flowable.range(1, 5) @@ -68,7 +67,6 @@ public List apply(List a, Integer b) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void reducerCrash() { Flowable.range(1, 5) @@ -121,7 +119,6 @@ public List apply(List a, Integer b) throws Exception { assertFalse(pp.hasSubscribers()); } - @SuppressWarnings("unchecked") @Test public void error() { Flowable.error(new TestException()) @@ -143,7 +140,6 @@ public List apply(List a, Integer b) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void doubleError() { List errors = TestHelper.trackPluginErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java b/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java index 7da2cdf95f..98ba085b19 100644 --- a/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java @@ -49,7 +49,6 @@ public Iterator> iterator() { }).test().assertError(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void ambIterableOneIsNull() { Single.amb(Arrays.asList(null, just1)) @@ -85,7 +84,6 @@ public Iterator> iterator() { }).blockingSubscribe(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void concatIterableOneIsNull() { Single.concat(Arrays.asList(just1, null)).blockingSubscribe(); @@ -249,7 +247,6 @@ public Iterator> iterator() { }).blockingSubscribe(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeIterableOneIsNull() { Single.merge(Arrays.asList(null, just1)).blockingSubscribe(); @@ -384,7 +381,6 @@ public Object apply(Object[] v) { }).blockingGet(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableOneIsNull() { Single.zip(Arrays.asList(null, just1), new Function() { @@ -395,13 +391,11 @@ public Object apply(Object[] v) { }).blockingGet(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableOneFunctionNull() { Single.zip(Arrays.asList(just1, just1), null).blockingGet(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableOneFunctionReturnsNull() { Single.zip(Arrays.asList(just1, just1), new Function() { @@ -529,7 +523,6 @@ public Object apply(Object[] v) { }, (Single[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipIterableTwoIsNull() { Single.zip(Arrays.asList(just1, null), new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/single/SingleTest.java b/src/test/java/io/reactivex/rxjava3/single/SingleTest.java index 92974c0ef2..1f7cba7e2d 100644 --- a/src/test/java/io/reactivex/rxjava3/single/SingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/single/SingleTest.java @@ -510,7 +510,6 @@ public void toFutureThrows() throws Exception { @Test(expected = UnsupportedOperationException.class) public void toFlowableIterableRemove() { - @SuppressWarnings("unchecked") Iterable> f = SingleInternalHelper.iterableToFlowable(Arrays.asList(Single.just(1))); Iterator> iterator = f.iterator(); @@ -520,7 +519,6 @@ public void toFlowableIterableRemove() { @Test public void zipIterableObject() { - @SuppressWarnings("unchecked") final List> singles = Arrays.asList(Single.just(1), Single.just(4)); Single.zip(singles, new Function() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/tck/AmbArrayTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/AmbArrayTckTest.java index 491eccb85b..88b24a00ab 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/AmbArrayTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/AmbArrayTckTest.java @@ -21,7 +21,6 @@ @Test public class AmbArrayTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/tck/AmbTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/AmbTckTest.java index 2bdcf2c89f..a0d4a63764 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/AmbTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/AmbTckTest.java @@ -23,7 +23,6 @@ @Test public class AmbTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableDelayErrorTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableDelayErrorTckTest.java index b309de7837..5581b9551e 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableDelayErrorTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableDelayErrorTckTest.java @@ -24,7 +24,6 @@ @Test public class CombineLatestIterableDelayErrorTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableTckTest.java index ed59b0aeb7..744d909949 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/CombineLatestIterableTckTest.java @@ -24,7 +24,6 @@ @Test public class CombineLatestIterableTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/tck/ConcatArrayEagerTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/ConcatArrayEagerTckTest.java index 7cf2fb3dc1..9885f3b939 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/ConcatArrayEagerTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/ConcatArrayEagerTckTest.java @@ -23,7 +23,6 @@ @Test public class ConcatArrayEagerTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/tck/ConcatIterableEagerTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/ConcatIterableEagerTckTest.java index 6ab7f2a7ba..e365418449 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/ConcatIterableEagerTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/ConcatIterableEagerTckTest.java @@ -21,7 +21,6 @@ @Test public class ConcatIterableEagerTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/tck/MergeIterableTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/MergeIterableTckTest.java index f597b70905..79d781d848 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/MergeIterableTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/MergeIterableTckTest.java @@ -23,7 +23,6 @@ @Test public class MergeIterableTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java b/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java index 77bdfef211..c65a0bd58f 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java +++ b/src/test/java/io/reactivex/rxjava3/tck/RefCountProcessor.java @@ -46,7 +46,7 @@ RefCountProcessor(FlowableProcessor actual) { this.actual = actual; this.upstream = new AtomicReference<>(); - this.subscribers = new AtomicReference[]>(EMPTY); + this.subscribers = new AtomicReference<>(EMPTY); } @Override diff --git a/src/test/java/io/reactivex/rxjava3/tck/ZipIterableTckTest.java b/src/test/java/io/reactivex/rxjava3/tck/ZipIterableTckTest.java index ec4583ff07..8ef456e609 100644 --- a/src/test/java/io/reactivex/rxjava3/tck/ZipIterableTckTest.java +++ b/src/test/java/io/reactivex/rxjava3/tck/ZipIterableTckTest.java @@ -24,7 +24,6 @@ @Test public class ZipIterableTckTest extends BaseTck { - @SuppressWarnings("unchecked") @Override public Publisher createPublisher(long elements) { return diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java b/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java index 2c68b0151a..8eb1d9f906 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/BaseTestConsumerEx.java @@ -176,6 +176,7 @@ public final U assertErrorMessage(String message) { * @param values the expected values, asserted in order * @return this */ + @SafeVarargs public final U assertFailure(Predicate errorPredicate, T... values) { return assertSubscribed() .assertValues(values) @@ -192,6 +193,7 @@ public final U assertFailure(Predicate errorPredicate, T... values) { * @param values the expected values, asserted in order * @return this */ + @SafeVarargs public final U assertFailureAndMessage(Class error, String message, T... values) { return assertSubscribed() From ecdf2570848847ff97bfb14c6ecde9660f323e33 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Mon, 23 Dec 2019 13:29:03 +0100 Subject: [PATCH 021/665] Bump build-info-extractor-gradle from 4.11.0 to 4.12.0 (#6792) Bumps build-info-extractor-gradle from 4.11.0 to 4.12.0. Signed-off-by: dependabot-preview[bot] --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index f0406bd3f7..2ae17aa25a 100644 --- a/build.gradle +++ b/build.gradle @@ -14,7 +14,7 @@ buildscript { ext.animalSnifferVersion = "1.5.0" ext.licenseVersion = "0.15.0" ext.bintrayVersion = "1.8.4" - ext.jfrogExtractorVersion = "4.11.0" + ext.jfrogExtractorVersion = "4.12.0" ext.bndVersion = "4.3.1" ext.checkstyleVersion = "8.26" From b57169babb184d0adc2fa28cf0425c4f0c1f8555 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Tue, 24 Dec 2019 10:18:06 +0100 Subject: [PATCH 022/665] 3.x: Add NonNull & SafeVarargs annotations + validator (#6791) --- .../reactivex/rxjava3/core/Completable.java | 255 +-- .../io/reactivex/rxjava3/core/Flowable.java | 1616 ++++++++++------- .../java/io/reactivex/rxjava3/core/Maybe.java | 564 +++--- .../reactivex/rxjava3/core/Notification.java | 2 +- .../io/reactivex/rxjava3/core/Observable.java | 309 ++-- .../io/reactivex/rxjava3/core/Single.java | 487 ++--- .../rxjava3/flowable/FlowableNullTests.java | 2 - .../flowable/FlowableConcatMapEagerTest.java | 2 - .../flowable/FlowableMergeDelayErrorTest.java | 1 - .../operators/flowable/FlowableMergeTest.java | 1 - .../flowable/FlowablePublishFunctionTest.java | 3 - .../operators/maybe/MaybeAmbTest.java | 5 - .../operators/maybe/MaybeConcatArrayTest.java | 9 - .../operators/maybe/MaybeMergeArrayTest.java | 7 - .../operators/maybe/MaybeZipArrayTest.java | 2 - .../operators/maybe/MaybeZipIterableTest.java | 1 - .../mixed/ObservableConcatMapMaybeTest.java | 1 - .../mixed/ObservableConcatMapSingleTest.java | 1 - .../observable/ObservableAmbTest.java | 15 - .../ObservableConcatMapEagerTest.java | 11 - .../observable/ObservableConcatMapTest.java | 2 - .../observable/ObservableZipTest.java | 1 - .../operators/single/SingleAmbTest.java | 9 - .../operators/single/SingleConcatTest.java | 2 - .../operators/single/SingleZipArrayTest.java | 3 - .../io/reactivex/rxjava3/maybe/MaybeTest.java | 24 - .../observable/ObservableNullTests.java | 1 - .../rxjava3/single/SingleNullTests.java | 4 - .../validators/SourceAnnotationCheck.java | 257 +++ 29 files changed, 2066 insertions(+), 1531 deletions(-) create mode 100644 src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java diff --git a/src/main/java/io/reactivex/rxjava3/core/Completable.java b/src/main/java/io/reactivex/rxjava3/core/Completable.java index d9991a913e..46ec666337 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Completable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Completable.java @@ -121,7 +121,8 @@ public abstract class Completable implements CompletableSource { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable ambArray(final CompletableSource... sources) { + @SafeVarargs + public static Completable ambArray(@NonNull CompletableSource... sources) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return complete(); @@ -150,7 +151,7 @@ public static Completable ambArray(final CompletableSource... sources) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable amb(final Iterable sources) { + public static Completable amb(@NonNull Iterable sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableAmb(null, sources)); @@ -188,7 +189,8 @@ public static Completable complete() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable concatArray(CompletableSource... sources) { + @SafeVarargs + public static Completable concatArray(@NonNull CompletableSource... sources) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return complete(); @@ -214,7 +216,7 @@ public static Completable concatArray(CompletableSource... sources) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable concat(Iterable sources) { + public static Completable concat(@NonNull Iterable sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableConcatIterable(sources)); @@ -238,7 +240,8 @@ public static Completable concat(Iterable sources) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) - public static Completable concat(Publisher sources) { + @NonNull + public static Completable concat(@NonNull Publisher sources) { return concat(sources, 2); } @@ -262,7 +265,7 @@ public static Completable concat(Publisher sources) @NonNull @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) - public static Completable concat(Publisher sources, int prefetch) { + public static Completable concat(@NonNull Publisher sources, int prefetch) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new CompletableConcat(sources, prefetch)); @@ -312,7 +315,7 @@ public static Completable concat(Publisher sources, @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable create(CompletableOnSubscribe source) { + public static Completable create(@NonNull CompletableOnSubscribe source) { Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new CompletableCreate(source)); } @@ -335,7 +338,7 @@ public static Completable create(CompletableOnSubscribe source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable unsafeCreate(CompletableSource source) { + public static Completable unsafeCreate(@NonNull CompletableSource source) { Objects.requireNonNull(source, "source is null"); if (source instanceof Completable) { throw new IllegalArgumentException("Use of unsafeCreate(Completable)!"); @@ -357,7 +360,7 @@ public static Completable unsafeCreate(CompletableSource source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable defer(final Supplier completableSupplier) { + public static Completable defer(@NonNull Supplier completableSupplier) { Objects.requireNonNull(completableSupplier, "completableSupplier"); return RxJavaPlugins.onAssembly(new CompletableDefer(completableSupplier)); } @@ -381,7 +384,7 @@ public static Completable defer(final Supplier comp @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable error(final Supplier errorSupplier) { + public static Completable error(@NonNull Supplier errorSupplier) { Objects.requireNonNull(errorSupplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new CompletableErrorSupplier(errorSupplier)); } @@ -401,7 +404,7 @@ public static Completable error(final Supplier errorSupplie @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable error(final Throwable error) { + public static Completable error(@NonNull Throwable error) { Objects.requireNonNull(error, "error is null"); return RxJavaPlugins.onAssembly(new CompletableError(error)); } @@ -429,7 +432,7 @@ public static Completable error(final Throwable error) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromAction(final Action run) { + public static Completable fromAction(@NonNull Action run) { Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new CompletableFromAction(run)); } @@ -458,7 +461,7 @@ public static Completable fromAction(final Action run) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromCallable(final Callable callable) { + public static Completable fromCallable(@NonNull Callable callable) { Objects.requireNonNull(callable, "callable is null"); return RxJavaPlugins.onAssembly(new CompletableFromCallable(callable)); } @@ -479,7 +482,7 @@ public static Completable fromCallable(final Callable callable) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromFuture(final Future future) { + public static Completable fromFuture(@NonNull Future future) { Objects.requireNonNull(future, "future is null"); return fromAction(Functions.futureAction(future)); } @@ -504,9 +507,9 @@ public static Completable fromFuture(final Future future) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromMaybe(final MaybeSource maybe) { + public static Completable fromMaybe(@NonNull MaybeSource maybe) { Objects.requireNonNull(maybe, "maybe is null"); - return RxJavaPlugins.onAssembly(new MaybeIgnoreElementCompletable(maybe)); + return RxJavaPlugins.onAssembly(new MaybeIgnoreElementCompletable<>(maybe)); } /** @@ -532,7 +535,7 @@ public static Completable fromMaybe(final MaybeSource maybe) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromRunnable(final Runnable run) { + public static Completable fromRunnable(@NonNull Runnable run) { Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new CompletableFromRunnable(run)); } @@ -554,9 +557,9 @@ public static Completable fromRunnable(final Runnable run) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromObservable(final ObservableSource observable) { + public static Completable fromObservable(@NonNull ObservableSource observable) { Objects.requireNonNull(observable, "observable is null"); - return RxJavaPlugins.onAssembly(new CompletableFromObservable(observable)); + return RxJavaPlugins.onAssembly(new CompletableFromObservable<>(observable)); } /** @@ -592,9 +595,9 @@ public static Completable fromObservable(final ObservableSource observabl @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromPublisher(final Publisher publisher) { + public static Completable fromPublisher(@NonNull Publisher publisher) { Objects.requireNonNull(publisher, "publisher is null"); - return RxJavaPlugins.onAssembly(new CompletableFromPublisher(publisher)); + return RxJavaPlugins.onAssembly(new CompletableFromPublisher<>(publisher)); } /** @@ -614,9 +617,9 @@ public static Completable fromPublisher(final Publisher publisher) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromSingle(final SingleSource single) { + public static Completable fromSingle(@NonNull SingleSource single) { Objects.requireNonNull(single, "single is null"); - return RxJavaPlugins.onAssembly(new CompletableFromSingle(single)); + return RxJavaPlugins.onAssembly(new CompletableFromSingle<>(single)); } /** @@ -644,7 +647,7 @@ public static Completable fromSingle(final SingleSource single) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable fromSupplier(final Supplier supplier) { + public static Completable fromSupplier(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new CompletableFromSupplier(supplier)); } @@ -679,7 +682,8 @@ public static Completable fromSupplier(final Supplier supplier) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable mergeArray(CompletableSource... sources) { + @SafeVarargs + public static Completable mergeArray(@NonNull CompletableSource... sources) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return complete(); @@ -720,7 +724,7 @@ public static Completable mergeArray(CompletableSource... sources) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable merge(Iterable sources) { + public static Completable merge(@NonNull Iterable sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableMergeIterable(sources)); } @@ -758,7 +762,8 @@ public static Completable merge(Iterable sources) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) - public static Completable merge(Publisher sources) { + @NonNull + public static Completable merge(@NonNull Publisher sources) { return merge0(sources, Integer.MAX_VALUE, false); } @@ -797,7 +802,8 @@ public static Completable merge(Publisher sources) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) - public static Completable merge(Publisher sources, int maxConcurrency) { + @NonNull + public static Completable merge(@NonNull Publisher sources, int maxConcurrency) { return merge0(sources, maxConcurrency, false); } @@ -823,7 +829,7 @@ public static Completable merge(Publisher sources, @NonNull @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) - private static Completable merge0(Publisher sources, int maxConcurrency, boolean delayErrors) { + private static Completable merge0(@NonNull Publisher sources, int maxConcurrency, boolean delayErrors) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new CompletableMerge(sources, maxConcurrency, delayErrors)); @@ -846,7 +852,8 @@ private static Completable merge0(Publisher sources @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable mergeArrayDelayError(CompletableSource... sources) { + @SafeVarargs + public static Completable mergeArrayDelayError(@NonNull CompletableSource... sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableMergeDelayErrorArray(sources)); } @@ -868,7 +875,7 @@ public static Completable mergeArrayDelayError(CompletableSource... sources) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable mergeDelayError(Iterable sources) { + public static Completable mergeDelayError(@NonNull Iterable sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new CompletableMergeDelayErrorIterable(sources)); } @@ -893,7 +900,8 @@ public static Completable mergeDelayError(Iterable @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) - public static Completable mergeDelayError(Publisher sources) { + @NonNull + public static Completable mergeDelayError(@NonNull Publisher sources) { return merge0(sources, Integer.MAX_VALUE, true); } @@ -919,7 +927,8 @@ public static Completable mergeDelayError(Publisher @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) - public static Completable mergeDelayError(Publisher sources, int maxConcurrency) { + @NonNull + public static Completable mergeDelayError(@NonNull Publisher sources, int maxConcurrency) { return merge0(sources, maxConcurrency, true); } @@ -935,6 +944,7 @@ public static Completable mergeDelayError(Publisher */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Completable never() { return RxJavaPlugins.onAssembly(CompletableNever.INSTANCE); } @@ -953,7 +963,8 @@ public static Completable never() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Completable timer(long delay, TimeUnit unit) { + @NonNull + public static Completable timer(long delay, @NonNull TimeUnit unit) { return timer(delay, unit, Schedulers.computation()); } @@ -974,7 +985,7 @@ public static Completable timer(long delay, TimeUnit unit) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Completable timer(final long delay, final TimeUnit unit, final Scheduler scheduler) { + public static Completable timer(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableTimer(delay, unit, scheduler)); @@ -1010,9 +1021,10 @@ private static NullPointerException toNpe(Throwable ex) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Completable using(Supplier resourceSupplier, - Function completableFunction, - Consumer disposer) { + @NonNull + public static Completable using(@NonNull Supplier resourceSupplier, + @NonNull Function completableFunction, + @NonNull Consumer disposer) { return using(resourceSupplier, completableFunction, disposer, true); } @@ -1045,15 +1057,15 @@ public static Completable using(Supplier resourceSupplier, @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Completable using( - final Supplier resourceSupplier, - final Function completableFunction, - final Consumer disposer, - final boolean eager) { + @NonNull Supplier resourceSupplier, + @NonNull Function completableFunction, + @NonNull Consumer disposer, + boolean eager) { Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); Objects.requireNonNull(completableFunction, "completableFunction is null"); Objects.requireNonNull(disposer, "disposer is null"); - return RxJavaPlugins.onAssembly(new CompletableUsing(resourceSupplier, completableFunction, disposer, eager)); + return RxJavaPlugins.onAssembly(new CompletableUsing<>(resourceSupplier, completableFunction, disposer, eager)); } /** @@ -1072,7 +1084,7 @@ public static Completable using( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Completable wrap(CompletableSource source) { + public static Completable wrap(@NonNull CompletableSource source) { Objects.requireNonNull(source, "source is null"); if (source instanceof Completable) { return RxJavaPlugins.onAssembly((Completable)source); @@ -1097,7 +1109,7 @@ public static Completable wrap(CompletableSource source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable ambWith(CompletableSource other) { + public final Completable ambWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -1121,9 +1133,9 @@ public final Completable ambWith(CompletableSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Observable andThen(ObservableSource next) { + public final <@NonNull T> Observable andThen(@NonNull ObservableSource next) { Objects.requireNonNull(next, "next is null"); - return RxJavaPlugins.onAssembly(new CompletableAndThenObservable(this, next)); + return RxJavaPlugins.onAssembly(new CompletableAndThenObservable<>(this, next)); } /** @@ -1149,9 +1161,9 @@ public final Observable andThen(ObservableSource next) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable andThen(Publisher next) { + public final <@NonNull T> Flowable andThen(@NonNull Publisher next) { Objects.requireNonNull(next, "next is null"); - return RxJavaPlugins.onAssembly(new CompletableAndThenPublisher(this, next)); + return RxJavaPlugins.onAssembly(new CompletableAndThenPublisher<>(this, next)); } /** @@ -1173,9 +1185,9 @@ public final Flowable andThen(Publisher next) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single andThen(SingleSource next) { + public final <@NonNull T> Single andThen(@NonNull SingleSource next) { Objects.requireNonNull(next, "next is null"); - return RxJavaPlugins.onAssembly(new SingleDelayWithCompletable(next, this)); + return RxJavaPlugins.onAssembly(new SingleDelayWithCompletable<>(next, this)); } /** @@ -1197,9 +1209,9 @@ public final Single andThen(SingleSource next) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe andThen(MaybeSource next) { + public final <@NonNull T> Maybe andThen(@NonNull MaybeSource next) { Objects.requireNonNull(next, "next is null"); - return RxJavaPlugins.onAssembly(new MaybeDelayWithCompletable(next, this)); + return RxJavaPlugins.onAssembly(new MaybeDelayWithCompletable<>(next, this)); } /** @@ -1219,7 +1231,8 @@ public final Maybe andThen(MaybeSource next) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable andThen(CompletableSource next) { + @NonNull + public final Completable andThen(@NonNull CompletableSource next) { Objects.requireNonNull(next, "next is null"); return RxJavaPlugins.onAssembly(new CompletableAndThenCompletable(this, next)); } @@ -1241,7 +1254,7 @@ public final Completable andThen(CompletableSource next) { */ @SchedulerSupport(SchedulerSupport.NONE) public final void blockingAwait() { - BlockingMultiObserver observer = new BlockingMultiObserver(); + BlockingMultiObserver observer = new BlockingMultiObserver<>(); subscribe(observer); observer.blockingGet(); } @@ -1266,11 +1279,10 @@ public final void blockingAwait() { * @throws RuntimeException wrapping an InterruptedException if the current thread is interrupted */ @CheckReturnValue - @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final boolean blockingAwait(long timeout, TimeUnit unit) { + public final boolean blockingAwait(long timeout, @NonNull TimeUnit unit) { Objects.requireNonNull(unit, "unit is null"); - BlockingMultiObserver observer = new BlockingMultiObserver(); + BlockingMultiObserver observer = new BlockingMultiObserver<>(); subscribe(observer); return observer.blockingAwait(timeout, unit); } @@ -1294,6 +1306,7 @@ public final boolean blockingAwait(long timeout, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable cache() { return RxJavaPlugins.onAssembly(new CompletableCache(this)); } @@ -1313,7 +1326,8 @@ public final Completable cache() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable compose(CompletableTransformer transformer) { + @NonNull + public final Completable compose(@NonNull CompletableTransformer transformer) { return wrap(Objects.requireNonNull(transformer, "transformer is null").apply(this)); } @@ -1336,7 +1350,7 @@ public final Completable compose(CompletableTransformer transformer) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable concatWith(CompletableSource other) { + public final Completable concatWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new CompletableAndThenCompletable(this, other)); } @@ -1356,7 +1370,8 @@ public final Completable concatWith(CompletableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Completable delay(long delay, TimeUnit unit) { + @NonNull + public final Completable delay(long delay, @NonNull TimeUnit unit) { return delay(delay, unit, Schedulers.computation(), false); } @@ -1377,7 +1392,8 @@ public final Completable delay(long delay, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Completable delay(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Completable delay(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delay(delay, unit, scheduler, false); } @@ -1400,7 +1416,7 @@ public final Completable delay(long delay, TimeUnit unit, Scheduler scheduler) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Completable delay(final long delay, final TimeUnit unit, final Scheduler scheduler, final boolean delayError) { + public final Completable delay(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableDelay(this, delay, unit, scheduler, delayError)); @@ -1424,7 +1440,8 @@ public final Completable delay(final long delay, final TimeUnit unit, final Sche */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Completable delaySubscription(long delay, TimeUnit unit) { + @NonNull + public final Completable delaySubscription(long delay, @NonNull TimeUnit unit) { return delaySubscription(delay, unit, Schedulers.computation()); } @@ -1448,7 +1465,8 @@ public final Completable delaySubscription(long delay, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Completable delaySubscription(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Completable delaySubscription(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return Completable.timer(delay, unit, scheduler).andThen(this); } @@ -1467,7 +1485,8 @@ public final Completable delaySubscription(long delay, TimeUnit unit, Scheduler */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doOnComplete(Action onComplete) { + @NonNull + public final Completable doOnComplete(@NonNull Action onComplete) { return doOnLifecycle(Functions.emptyConsumer(), Functions.emptyConsumer(), onComplete, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); @@ -1488,7 +1507,8 @@ public final Completable doOnComplete(Action onComplete) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doOnDispose(Action onDispose) { + @NonNull + public final Completable doOnDispose(@NonNull Action onDispose) { return doOnLifecycle(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION, onDispose); @@ -1509,7 +1529,8 @@ public final Completable doOnDispose(Action onDispose) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doOnError(Consumer onError) { + @NonNull + public final Completable doOnError(@NonNull Consumer onError) { return doOnLifecycle(Functions.emptyConsumer(), onError, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); @@ -1531,7 +1552,7 @@ public final Completable doOnError(Consumer onError) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doOnEvent(final Consumer onEvent) { + public final Completable doOnEvent(@NonNull Consumer onEvent) { Objects.requireNonNull(onEvent, "onEvent is null"); return RxJavaPlugins.onAssembly(new CompletableDoOnEvent(this, onEvent)); } @@ -1584,7 +1605,8 @@ private Completable doOnLifecycle( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doOnSubscribe(Consumer onSubscribe) { + @NonNull + public final Completable doOnSubscribe(@NonNull Consumer onSubscribe) { return doOnLifecycle(onSubscribe, Functions.emptyConsumer(), Functions.EMPTY_ACTION, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); @@ -1605,7 +1627,8 @@ public final Completable doOnSubscribe(Consumer onSubscribe) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doOnTerminate(final Action onTerminate) { + @NonNull + public final Completable doOnTerminate(@NonNull Action onTerminate) { return doOnLifecycle(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, onTerminate, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); @@ -1626,7 +1649,8 @@ public final Completable doOnTerminate(final Action onTerminate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doAfterTerminate(final Action onAfterTerminate) { + @NonNull + public final Completable doAfterTerminate(@NonNull Action onAfterTerminate) { return doOnLifecycle( Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -1658,7 +1682,7 @@ public final Completable doAfterTerminate(final Action onAfterTerminate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable doFinally(Action onFinally) { + public final Completable doFinally(@NonNull Action onFinally) { Objects.requireNonNull(onFinally, "onFinally is null"); return RxJavaPlugins.onAssembly(new CompletableDoFinally(this, onFinally)); } @@ -1796,7 +1820,7 @@ public final Completable doFinally(Action onFinally) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable lift(final CompletableOperator onLift) { + public final Completable lift(@NonNull CompletableOperator onLift) { Objects.requireNonNull(onLift, "onLift is null"); return RxJavaPlugins.onAssembly(new CompletableLift(this, onLift)); } @@ -1818,6 +1842,7 @@ public final Completable lift(final CompletableOperator onLift) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> materialize() { return RxJavaPlugins.onAssembly(new CompletableMaterialize(this)); } @@ -1838,7 +1863,7 @@ public final Single> materialize() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable mergeWith(CompletableSource other) { + public final Completable mergeWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); return mergeArray(this, other); } @@ -1858,7 +1883,7 @@ public final Completable mergeWith(CompletableSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Completable observeOn(final Scheduler scheduler) { + public final Completable observeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableObserveOn(this, scheduler)); } @@ -1876,6 +1901,7 @@ public final Completable observeOn(final Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable onErrorComplete() { return onErrorComplete(Functions.alwaysTrue()); } @@ -1896,7 +1922,7 @@ public final Completable onErrorComplete() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable onErrorComplete(final Predicate predicate) { + public final Completable onErrorComplete(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new CompletableOnErrorComplete(this, predicate)); @@ -1919,7 +1945,7 @@ public final Completable onErrorComplete(final Predicate pred @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable onErrorResumeNext(final Function errorMapper) { + public final Completable onErrorResumeNext(@NonNull Function errorMapper) { Objects.requireNonNull(errorMapper, "errorMapper is null"); return RxJavaPlugins.onAssembly(new CompletableResumeNext(this, errorMapper)); } @@ -1940,6 +1966,7 @@ public final Completable onErrorResumeNext(final Function, ? extends Publisher> handler) { + @NonNull + public final Completable repeatWhen(@NonNull Function, ? extends Publisher> handler) { return fromPublisher(toFlowable().repeatWhen(handler)); } @@ -2030,6 +2061,7 @@ public final Completable repeatWhen(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable retry() { return fromPublisher(toFlowable().retry()); } @@ -2049,7 +2081,8 @@ public final Completable retry() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable retry(BiPredicate predicate) { + @NonNull + public final Completable retry(@NonNull BiPredicate predicate) { return fromPublisher(toFlowable().retry(predicate)); } @@ -2068,6 +2101,7 @@ public final Completable retry(BiPredicate p */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable retry(long times) { return fromPublisher(toFlowable().retry(times)); } @@ -2092,7 +2126,8 @@ public final Completable retry(long times) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable retry(long times, Predicate predicate) { + @NonNull + public final Completable retry(long times, @NonNull Predicate predicate) { return fromPublisher(toFlowable().retry(times, predicate)); } @@ -2112,7 +2147,8 @@ public final Completable retry(long times, Predicate predicat */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable retry(Predicate predicate) { + @NonNull + public final Completable retry(@NonNull Predicate predicate) { return fromPublisher(toFlowable().retry(predicate)); } @@ -2158,7 +2194,8 @@ public final Completable retry(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Completable retryWhen(Function, ? extends Publisher> handler) { + @NonNull + public final Completable retryWhen(@NonNull Function, ? extends Publisher> handler) { return fromPublisher(toFlowable().retryWhen(handler)); } @@ -2178,7 +2215,7 @@ public final Completable retryWhen(Function, ? exten @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Completable startWith(CompletableSource other) { + public final Completable startWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); return concatArray(other, this); } @@ -2200,9 +2237,9 @@ public final Completable startWith(CompletableSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Observable startWith(Observable other) { + public final Observable startWith(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); - return other.concatWith(this.toObservable()); + return Observable.wrap(other).concatWith(this.toObservable()); } /** * Returns a Flowable which first delivers the events @@ -2225,7 +2262,7 @@ public final Observable startWith(Observable other) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable startWith(Publisher other) { + public final Flowable startWith(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return this.toFlowable().startWith(other); } @@ -2246,6 +2283,7 @@ public final Flowable startWith(Publisher other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable hide() { return RxJavaPlugins.onAssembly(new CompletableHide(this)); } @@ -2262,6 +2300,7 @@ public final Completable hide() { * @return the Disposable that allows disposing the subscription */ @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Disposable subscribe() { EmptyCompletableObserver observer = new EmptyCompletableObserver(); subscribe(observer); @@ -2270,7 +2309,7 @@ public final Disposable subscribe() { @SchedulerSupport(SchedulerSupport.NONE) @Override - public final void subscribe(CompletableObserver observer) { + public final void subscribe(@NonNull CompletableObserver observer) { Objects.requireNonNull(observer, "observer is null"); try { @@ -2326,7 +2365,8 @@ public final void subscribe(CompletableObserver observer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final E subscribeWith(E observer) { + @NonNull + public final <@NonNull E extends CompletableObserver> E subscribeWith(E observer) { subscribe(observer); return observer; } @@ -2347,7 +2387,7 @@ public final E subscribeWith(E observer) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(final Action onComplete, final Consumer onError) { + public final Disposable subscribe(@NonNull Action onComplete, @NonNull Consumer onError) { Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); @@ -2375,7 +2415,7 @@ public final Disposable subscribe(final Action onComplete, final Consumer R to(@NonNull CompletableConverter converter) { @SuppressWarnings("unchecked") @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable toFlowable() { if (this instanceof FuseToFlowable) { return ((FuseToFlowable)this).fuseToFlowable(); @@ -2613,6 +2656,7 @@ public final Flowable toFlowable() { @CheckReturnValue @SuppressWarnings("unchecked") @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe toMaybe() { if (this instanceof FuseToMaybe) { return ((FuseToMaybe)this).fuseToMaybe(); @@ -2635,6 +2679,7 @@ public final Maybe toMaybe() { @CheckReturnValue @SuppressWarnings("unchecked") @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable toObservable() { if (this instanceof FuseToObservable) { return ((FuseToObservable)this).fuseToObservable(); @@ -2659,7 +2704,7 @@ public final Observable toObservable() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single toSingle(final Supplier completionValueSupplier) { + public final <@NonNull T> Single toSingle(@NonNull Supplier completionValueSupplier) { Objects.requireNonNull(completionValueSupplier, "completionValueSupplier is null"); return RxJavaPlugins.onAssembly(new CompletableToSingle(this, completionValueSupplier, null)); } @@ -2681,9 +2726,9 @@ public final Single toSingle(final Supplier completionValueS @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single toSingleDefault(final T completionValue) { + public final <@NonNull T> Single toSingleDefault(T completionValue) { Objects.requireNonNull(completionValue, "completionValue is null"); - return RxJavaPlugins.onAssembly(new CompletableToSingle(this, null, completionValue)); + return RxJavaPlugins.onAssembly(new CompletableToSingle<>(this, null, completionValue)); } /** @@ -2702,7 +2747,7 @@ public final Single toSingleDefault(final T completionValue) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Completable unsubscribeOn(final Scheduler scheduler) { + public final Completable unsubscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new CompletableDisposeOn(this, scheduler)); } @@ -2724,8 +2769,9 @@ public final Completable unsubscribeOn(final Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestObserver test() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); subscribe(to); return to; } @@ -2745,8 +2791,9 @@ public final TestObserver test() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestObserver test(boolean dispose) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); if (dispose) { to.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 0b2e9197cb..c5bdb06b2c 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -185,7 +185,7 @@ public abstract class Flowable implements Publisher { @SchedulerSupport(SchedulerSupport.NONE) public static Flowable amb(Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); - return RxJavaPlugins.onAssembly(new FlowableAmb(null, sources)); + return RxJavaPlugins.onAssembly(new FlowableAmb<>(null, sources)); } /** @@ -223,7 +223,7 @@ public static Flowable ambArray(Publisher... sources) { if (len == 1) { return fromPublisher(sources[0]); } - return RxJavaPlugins.onAssembly(new FlowableAmb(sources, null)); + return RxJavaPlugins.onAssembly(new FlowableAmb<>(sources, null)); } /** @@ -276,7 +276,8 @@ public static int bufferSize() { @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) - public static Flowable combineLatestArray(Publisher[] sources, Function combiner) { + @NonNull + public static Flowable combineLatestArray(@NonNull Publisher[] sources, @NonNull Function combiner) { return combineLatestArray(sources, combiner, bufferSize()); } @@ -323,7 +324,7 @@ public static Flowable combineLatestArray(Publisher[] sou @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) - public static Flowable combineLatestArray(Publisher[] sources, Function combiner, int bufferSize) { + public static Flowable combineLatestArray(@NonNull Publisher[] sources, @NonNull Function combiner, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return empty(); @@ -373,8 +374,9 @@ public static Flowable combineLatestArray(Publisher[] sou @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) - public static Flowable combineLatest(Iterable> sources, - Function combiner) { + @NonNull + public static Flowable combineLatest(@NonNull Iterable> sources, + @NonNull Function combiner) { return combineLatest(sources, combiner, bufferSize()); } @@ -421,8 +423,8 @@ public static Flowable combineLatest(Iterable Flowable combineLatest(Iterable> sources, - Function combiner, int bufferSize) { + public static Flowable combineLatest(@NonNull Iterable> sources, + @NonNull Function combiner, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -469,8 +471,9 @@ public static Flowable combineLatest(Iterable Flowable combineLatestDelayError(Publisher[] sources, - Function combiner) { + @NonNull + public static Flowable combineLatestDelayError(@NonNull Publisher[] sources, + @NonNull Function combiner) { return combineLatestDelayError(sources, combiner, bufferSize()); } @@ -518,8 +521,8 @@ public static Flowable combineLatestDelayError(Publisher[ @CheckReturnValue @NonNull @BackpressureSupport(BackpressureKind.FULL) - public static Flowable combineLatestDelayError(Publisher[] sources, - Function combiner, int bufferSize) { + public static Flowable combineLatestDelayError(@NonNull Publisher[] sources, + @NonNull Function combiner, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -570,8 +573,9 @@ public static Flowable combineLatestDelayError(Publisher[ @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) - public static Flowable combineLatestDelayError(Iterable> sources, - Function combiner) { + @NonNull + public static Flowable combineLatestDelayError(@NonNull Iterable> sources, + @NonNull Function combiner) { return combineLatestDelayError(sources, combiner, bufferSize()); } @@ -618,8 +622,9 @@ public static Flowable combineLatestDelayError(Iterable Flowable combineLatestDelayError(Iterable> sources, - Function combiner, int bufferSize) { + @NonNull + public static Flowable combineLatestDelayError(@NonNull Iterable> sources, + @NonNull Function combiner, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -662,9 +667,10 @@ public static Flowable combineLatestDelayError(Iterable Flowable combineLatest( - Publisher source1, Publisher source2, - BiFunction combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull BiFunction combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return combineLatestArray(new Publisher[] { source1, source2 }, Functions.toFunction(combiner), bufferSize()); @@ -711,9 +717,9 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable combineLatest( - Publisher source1, Publisher source2, - Publisher source3, - Function3 combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, + @NonNull Function3 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -764,9 +770,9 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable combineLatest( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4, - Function4 combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4, + @NonNull Function4 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -821,10 +827,10 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable combineLatest( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4, - Publisher source5, - Function5 combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4, + @NonNull Publisher source5, + @NonNull Function5 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -883,10 +889,10 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable combineLatest( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4, - Publisher source5, Publisher source6, - Function6 combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4, + @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Function6 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -949,11 +955,11 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable combineLatest( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4, - Publisher source5, Publisher source6, - Publisher source7, - Function7 combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4, + @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Publisher source7, + @NonNull Function7 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1020,11 +1026,11 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable combineLatest( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4, - Publisher source5, Publisher source6, - Publisher source7, Publisher source8, - Function8 combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4, + @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Publisher source7, @NonNull Publisher source8, + @NonNull Function8 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1095,12 +1101,12 @@ public static Flowable combineLatest( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable combineLatest( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4, - Publisher source5, Publisher source6, - Publisher source7, Publisher source8, - Publisher source9, - Function9 combiner) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4, + @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Publisher source7, @NonNull Publisher source8, + @NonNull Publisher source9, + @NonNull Function9 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1136,7 +1142,7 @@ public static Flowable combineLatest( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concat(Iterable> sources) { + public static Flowable concat(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); // unlike general sources, fromIterable can only throw on a boundary because it is consumed only there return fromIterable(sources).concatMapDelayError((Function)Functions.identity(), false, 2); @@ -1167,7 +1173,8 @@ public static Flowable concat(Iterable> @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concat(Publisher> sources) { + @NonNull + public static Flowable concat(@NonNull Publisher> sources) { return concat(sources, bufferSize()); } @@ -1199,7 +1206,8 @@ public static Flowable concat(Publisher> @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concat(Publisher> sources, int prefetch) { + @NonNull + public static Flowable concat(@NonNull Publisher> sources, int prefetch) { return fromPublisher(sources).concatMap((Function)Functions.identity(), prefetch); } @@ -1231,7 +1239,7 @@ public static Flowable concat(Publisher> @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concat(Publisher source1, Publisher source2) { + public static Flowable concat(@NonNull Publisher source1, @NonNull Publisher source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return concatArray(source1, source2); @@ -1268,8 +1276,8 @@ public static Flowable concat(Publisher source1, Publisher Flowable concat( - Publisher source1, Publisher source2, - Publisher source3) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1309,8 +1317,8 @@ public static Flowable concat( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable concat( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1342,14 +1350,15 @@ public static Flowable concat( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs - public static Flowable concatArray(Publisher... sources) { + @NonNull + public static Flowable concatArray(@NonNull Publisher... sources) { if (sources.length == 0) { return empty(); } else if (sources.length == 1) { return fromPublisher(sources[0]); } - return RxJavaPlugins.onAssembly(new FlowableConcatArray(sources, false)); + return RxJavaPlugins.onAssembly(new FlowableConcatArray<>(sources, false)); } /** @@ -1375,14 +1384,15 @@ public static Flowable concatArray(Publisher... sources) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs - public static Flowable concatArrayDelayError(Publisher... sources) { + @NonNull + public static Flowable concatArrayDelayError(@NonNull Publisher... sources) { if (sources.length == 0) { return empty(); } else if (sources.length == 1) { return fromPublisher(sources[0]); } - return RxJavaPlugins.onAssembly(new FlowableConcatArray(sources, true)); + return RxJavaPlugins.onAssembly(new FlowableConcatArray<>(sources, true)); } /** @@ -1411,7 +1421,8 @@ public static Flowable concatArrayDelayError(Publisher... so @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs - public static Flowable concatArrayEager(Publisher... sources) { + @NonNull + public static Flowable concatArrayEager(@NonNull Publisher... sources) { return concatArrayEager(bufferSize(), bufferSize(), sources); } @@ -1445,7 +1456,8 @@ public static Flowable concatArrayEager(Publisher... sources @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "rawtypes", "unchecked" }) - public static Flowable concatArrayEager(int maxConcurrency, int prefetch, Publisher... sources) { + @SafeVarargs + public static Flowable concatArrayEager(int maxConcurrency, int prefetch, @NonNull Publisher... sources) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); @@ -1479,7 +1491,8 @@ public static Flowable concatArrayEager(int maxConcurrency, int prefetch, @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) @SafeVarargs - public static Flowable concatArrayEagerDelayError(Publisher... sources) { + @NonNull + public static Flowable concatArrayEagerDelayError(@NonNull Publisher... sources) { return concatArrayEagerDelayError(bufferSize(), bufferSize(), sources); } @@ -1513,7 +1526,9 @@ public static Flowable concatArrayEagerDelayError(Publisher. @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) - public static Flowable concatArrayEagerDelayError(int maxConcurrency, int prefetch, Publisher... sources) { + @SafeVarargs + @NonNull + public static Flowable concatArrayEagerDelayError(int maxConcurrency, int prefetch, @NonNull Publisher... sources) { return fromArray(sources).concatMapEagerDelayError((Function)Functions.identity(), true, maxConcurrency, prefetch); } @@ -1540,7 +1555,7 @@ public static Flowable concatArrayEagerDelayError(int maxConcurrency, int @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concatDelayError(Iterable> sources) { + public static Flowable concatDelayError(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); return fromIterable(sources).concatMapDelayError((Function)Functions.identity()); } @@ -1563,7 +1578,8 @@ public static Flowable concatDelayError(Iterable Flowable concatDelayError(Publisher> sources) { + @NonNull + public static Flowable concatDelayError(@NonNull Publisher> sources) { return concatDelayError(sources, bufferSize(), true); } @@ -1589,7 +1605,8 @@ public static Flowable concatDelayError(Publisher Flowable concatDelayError(Publisher> sources, int prefetch, boolean tillTheEnd) { + @NonNull + public static Flowable concatDelayError(@NonNull Publisher> sources, int prefetch, boolean tillTheEnd) { return fromPublisher(sources).concatMapDelayError((Function)Functions.identity(), tillTheEnd, prefetch); } @@ -1615,7 +1632,8 @@ public static Flowable concatDelayError(Publisher Flowable concatEager(Publisher> sources) { + @NonNull + public static Flowable concatEager(@NonNull Publisher> sources) { return concatEager(sources, bufferSize(), bufferSize()); } @@ -1646,7 +1664,7 @@ public static Flowable concatEager(Publisher Flowable concatEager(Publisher> sources, int maxConcurrency, int prefetch) { + public static Flowable concatEager(@NonNull Publisher> sources, int maxConcurrency, int prefetch) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); @@ -1675,7 +1693,8 @@ public static Flowable concatEager(Publisher Flowable concatEager(Iterable> sources) { + @NonNull + public static Flowable concatEager(@NonNull Iterable> sources) { return concatEager(sources, bufferSize(), bufferSize()); } @@ -1706,7 +1725,7 @@ public static Flowable concatEager(Iterable Flowable concatEager(Iterable> sources, int maxConcurrency, int prefetch) { + public static Flowable concatEager(@NonNull Iterable> sources, int maxConcurrency, int prefetch) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); @@ -1769,10 +1788,10 @@ public static Flowable concatEager(Iterable Flowable create(FlowableOnSubscribe source, BackpressureStrategy mode) { + public static Flowable create(@NonNull FlowableOnSubscribe source, @NonNull BackpressureStrategy mode) { Objects.requireNonNull(source, "source is null"); Objects.requireNonNull(mode, "mode is null"); - return RxJavaPlugins.onAssembly(new FlowableCreate(source, mode)); + return RxJavaPlugins.onAssembly(new FlowableCreate<>(source, mode)); } /** @@ -1806,9 +1825,9 @@ public static Flowable create(FlowableOnSubscribe source, Backpressure @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable defer(Supplier> supplier) { + public static Flowable defer(@NonNull Supplier> supplier) { Objects.requireNonNull(supplier, "supplier is null"); - return RxJavaPlugins.onAssembly(new FlowableDefer(supplier)); + return RxJavaPlugins.onAssembly(new FlowableDefer<>(supplier)); } /** @@ -1833,6 +1852,7 @@ public static Flowable defer(Supplier> s @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") + @NonNull public static Flowable empty() { return RxJavaPlugins.onAssembly((Flowable) FlowableEmpty.INSTANCE); } @@ -1861,7 +1881,7 @@ public static Flowable empty() { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable error(Supplier supplier) { + public static Flowable error(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new FlowableError(supplier)); } @@ -1890,7 +1910,7 @@ public static Flowable error(Supplier supplier) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable error(final Throwable throwable) { + public static Flowable error(@NonNull Throwable throwable) { Objects.requireNonNull(throwable, "throwable is null"); return error(Functions.justSupplier(throwable)); } @@ -1919,7 +1939,7 @@ public static Flowable error(final Throwable throwable) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs - public static Flowable fromArray(T... items) { + public static <@NonNull T> Flowable fromArray(@NonNull T... items) { Objects.requireNonNull(items, "items is null"); if (items.length == 0) { return empty(); @@ -1927,7 +1947,7 @@ public static Flowable fromArray(T... items) { if (items.length == 1) { return just(items[0]); } - return RxJavaPlugins.onAssembly(new FlowableFromArray(items)); + return RxJavaPlugins.onAssembly(new FlowableFromArray<>(items)); } /** @@ -1966,7 +1986,7 @@ public static Flowable fromArray(T... items) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable fromCallable(Callable supplier) { + public static <@NonNull T> Flowable fromCallable(@NonNull Callable supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new FlowableFromCallable(supplier)); } @@ -2008,7 +2028,7 @@ public static Flowable fromCallable(Callable supplier) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable fromFuture(Future future) { + public static <@NonNull T> Flowable fromFuture(@NonNull Future future) { Objects.requireNonNull(future, "future is null"); return RxJavaPlugins.onAssembly(new FlowableFromFuture(future, 0L, null)); } @@ -2054,7 +2074,7 @@ public static Flowable fromFuture(Future future) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable fromFuture(Future future, long timeout, TimeUnit unit) { + public static <@NonNull T> Flowable fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) { Objects.requireNonNull(future, "future is null"); Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new FlowableFromFuture(future, timeout, unit)); @@ -2105,7 +2125,7 @@ public static Flowable fromFuture(Future future, long timeou @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Flowable fromFuture(Future future, long timeout, TimeUnit unit, Scheduler scheduler) { + public static <@NonNull T> Flowable fromFuture(Future future, long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); return fromFuture((Future)future, timeout, unit).subscribeOn(scheduler); } @@ -2144,7 +2164,7 @@ public static Flowable fromFuture(Future future, long timeou @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Flowable fromFuture(Future future, Scheduler scheduler) { + public static <@NonNull T> Flowable fromFuture(Future future, @NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); return fromFuture((Future)future).subscribeOn(scheduler); } @@ -2174,7 +2194,7 @@ public static Flowable fromFuture(Future future, Scheduler s @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable fromIterable(Iterable source) { + public static <@NonNull T> Flowable fromIterable(@NonNull Iterable source) { Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new FlowableFromIterable(source)); } @@ -2211,7 +2231,7 @@ public static Flowable fromIterable(Iterable source) { @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") - public static Flowable fromPublisher(final Publisher source) { + public static Flowable fromPublisher(@NonNull Publisher source) { if (source instanceof Flowable) { return RxJavaPlugins.onAssembly((Flowable)source); } @@ -2256,7 +2276,7 @@ public static Flowable fromPublisher(final Publisher source) @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable fromSupplier(Supplier supplier) { + public static <@NonNull T> Flowable fromSupplier(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new FlowableFromSupplier(supplier)); } @@ -2286,7 +2306,7 @@ public static Flowable fromSupplier(Supplier supplier) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable generate(final Consumer> generator) { + public static Flowable generate(@NonNull Consumer<@NonNull Emitter> generator) { Objects.requireNonNull(generator, "generator is null"); return generate(Functions.nullSupplier(), FlowableInternalHelper.simpleGenerator(generator), @@ -2320,7 +2340,7 @@ public static Flowable generate(final Consumer> generator) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable generate(Supplier initialState, final BiConsumer> generator) { + public static Flowable generate(@NonNull Supplier initialState, @NonNull BiConsumer> generator) { Objects.requireNonNull(generator, "generator is null"); return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator), Functions.emptyConsumer()); @@ -2355,8 +2375,8 @@ public static Flowable generate(Supplier initialState, final BiCons @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable generate(Supplier initialState, final BiConsumer> generator, - Consumer disposeState) { + public static Flowable generate(@NonNull Supplier initialState, @NonNull BiConsumer> generator, + @NonNull Consumer disposeState) { Objects.requireNonNull(generator, "generator is null"); return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator), disposeState); } @@ -2388,7 +2408,8 @@ public static Flowable generate(Supplier initialState, final BiCons @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable generate(Supplier initialState, BiFunction, S> generator) { + @NonNull + public static Flowable generate(@NonNull Supplier initialState, @NonNull BiFunction, S> generator) { return generate(initialState, generator, Functions.emptyConsumer()); } @@ -2422,11 +2443,11 @@ public static Flowable generate(Supplier initialState, BiFunction Flowable generate(Supplier initialState, BiFunction, S> generator, Consumer disposeState) { + public static Flowable generate(@NonNull Supplier initialState, @NonNull BiFunction, S> generator, @NonNull Consumer disposeState) { Objects.requireNonNull(initialState, "initialState is null"); Objects.requireNonNull(generator, "generator is null"); Objects.requireNonNull(disposeState, "disposeState is null"); - return RxJavaPlugins.onAssembly(new FlowableGenerate(initialState, generator, disposeState)); + return RxJavaPlugins.onAssembly(new FlowableGenerate<>(initialState, generator, disposeState)); } /** @@ -2457,7 +2478,8 @@ public static Flowable generate(Supplier initialState, BiFunction interval(long initialDelay, long period, TimeUnit unit) { + @NonNull + public static Flowable interval(long initialDelay, long period, @NonNull TimeUnit unit) { return interval(initialDelay, period, unit, Schedulers.computation()); } @@ -2492,7 +2514,7 @@ public static Flowable interval(long initialDelay, long period, TimeUnit u @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Flowable interval(long initialDelay, long period, TimeUnit unit, Scheduler scheduler) { + public static Flowable interval(long initialDelay, long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new FlowableInterval(Math.max(0L, initialDelay), Math.max(0L, period), unit, scheduler)); @@ -2520,7 +2542,8 @@ public static Flowable interval(long initialDelay, long period, TimeUnit u @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Flowable interval(long period, TimeUnit unit) { + @NonNull + public static Flowable interval(long period, @NonNull TimeUnit unit) { return interval(period, period, unit, Schedulers.computation()); } @@ -2550,7 +2573,8 @@ public static Flowable interval(long period, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Flowable interval(long period, TimeUnit unit, Scheduler scheduler) { + @NonNull + public static Flowable interval(long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return interval(period, period, unit, scheduler); } @@ -2574,7 +2598,8 @@ public static Flowable interval(long period, TimeUnit unit, Scheduler sche @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Flowable intervalRange(long start, long count, long initialDelay, long period, TimeUnit unit) { + @NonNull + public static Flowable intervalRange(long start, long count, long initialDelay, long period, @NonNull TimeUnit unit) { return intervalRange(start, count, initialDelay, period, unit, Schedulers.computation()); } @@ -2600,7 +2625,7 @@ public static Flowable intervalRange(long start, long count, long initialD @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Flowable intervalRange(long start, long count, long initialDelay, long period, TimeUnit unit, Scheduler scheduler) { + public static Flowable intervalRange(long start, long count, long initialDelay, long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { if (count < 0L) { throw new IllegalArgumentException("count >= 0 required but it was " + count); } @@ -2652,9 +2677,9 @@ public static Flowable intervalRange(long start, long count, long initialD @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item) { + public static <@NonNull T> Flowable just(T item) { Objects.requireNonNull(item, "item is null"); - return RxJavaPlugins.onAssembly(new FlowableJust(item)); + return RxJavaPlugins.onAssembly(new FlowableJust<>(item)); } /** @@ -2681,7 +2706,7 @@ public static Flowable just(T item) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2) { + public static <@NonNull T> Flowable just(T item1, T item2) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); @@ -2714,7 +2739,7 @@ public static Flowable just(T item1, T item2) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2750,7 +2775,7 @@ public static Flowable just(T item1, T item2, T item3) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3, T item4) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3, T item4) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2789,7 +2814,7 @@ public static Flowable just(T item1, T item2, T item3, T item4) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3, T item4, T item5) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3, T item4, T item5) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2831,7 +2856,7 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5) @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3, T item4, T item5, T item6) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2876,7 +2901,7 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2924,7 +2949,7 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2975,7 +3000,7 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -3029,7 +3054,7 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9, T item10) { + public static <@NonNull T> Flowable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9, T item10) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -3091,7 +3116,8 @@ public static Flowable just(T item1, T item2, T item3, T item4, T item5, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Iterable> sources, int maxConcurrency, int bufferSize) { + @NonNull + public static Flowable merge(@NonNull Iterable> sources, int maxConcurrency, int bufferSize) { return fromIterable(sources).flatMap((Function)Functions.identity(), false, maxConcurrency, bufferSize); } @@ -3142,7 +3168,9 @@ public static Flowable merge(Iterable> s @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeArray(int maxConcurrency, int bufferSize, Publisher... sources) { + @SafeVarargs + @NonNull + public static Flowable mergeArray(int maxConcurrency, int bufferSize, @NonNull Publisher... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), false, maxConcurrency, bufferSize); } @@ -3186,7 +3214,8 @@ public static Flowable mergeArray(int maxConcurrency, int bufferSize, Pub @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Iterable> sources) { + @NonNull + public static Flowable merge(@NonNull Iterable> sources) { return fromIterable(sources).flatMap((Function)Functions.identity()); } @@ -3235,7 +3264,8 @@ public static Flowable merge(Iterable> s @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Iterable> sources, int maxConcurrency) { + @NonNull + public static Flowable merge(@NonNull Iterable> sources, int maxConcurrency) { return fromIterable(sources).flatMap((Function)Functions.identity(), maxConcurrency); } @@ -3280,7 +3310,8 @@ public static Flowable merge(Iterable> s @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Publisher> sources) { + @NonNull + public static Flowable merge(@NonNull Publisher> sources) { return merge(sources, bufferSize()); } @@ -3331,7 +3362,8 @@ public static Flowable merge(Publisher> @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Publisher> sources, int maxConcurrency) { + @NonNull + public static Flowable merge(@NonNull Publisher> sources, int maxConcurrency) { return fromPublisher(sources).flatMap((Function)Functions.identity(), maxConcurrency); } @@ -3374,7 +3406,9 @@ public static Flowable merge(Publisher> @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeArray(Publisher... sources) { + @SafeVarargs + @NonNull + public static Flowable mergeArray(@NonNull Publisher... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), sources.length); } @@ -3420,7 +3454,7 @@ public static Flowable mergeArray(Publisher... sources) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Publisher source1, Publisher source2) { + public static Flowable merge(@NonNull Publisher source1, @NonNull Publisher source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), false, 2); @@ -3470,7 +3504,7 @@ public static Flowable merge(Publisher source1, Publisher Flowable merge(Publisher source1, Publisher source2, Publisher source3) { + public static Flowable merge(@NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3524,8 +3558,8 @@ public static Flowable merge(Publisher source1, Publisher Flowable merge( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3565,7 +3599,8 @@ public static Flowable merge( @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(Iterable> sources) { + @NonNull + public static Flowable mergeDelayError(@NonNull Iterable> sources) { return fromIterable(sources).flatMap((Function)Functions.identity(), true); } @@ -3605,7 +3640,8 @@ public static Flowable mergeDelayError(Iterable Flowable mergeDelayError(Iterable> sources, int maxConcurrency, int bufferSize) { + @NonNull + public static Flowable mergeDelayError(@NonNull Iterable> sources, int maxConcurrency, int bufferSize) { return fromIterable(sources).flatMap((Function)Functions.identity(), true, maxConcurrency, bufferSize); } @@ -3645,7 +3681,9 @@ public static Flowable mergeDelayError(Iterable Flowable mergeArrayDelayError(int maxConcurrency, int bufferSize, Publisher... sources) { + @SafeVarargs + @NonNull + public static Flowable mergeArrayDelayError(int maxConcurrency, int bufferSize, @NonNull Publisher... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), true, maxConcurrency, bufferSize); } @@ -3683,7 +3721,8 @@ public static Flowable mergeArrayDelayError(int maxConcurrency, int buffe @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(Iterable> sources, int maxConcurrency) { + @NonNull + public static Flowable mergeDelayError(@NonNull Iterable> sources, int maxConcurrency) { return fromIterable(sources).flatMap((Function)Functions.identity(), true, maxConcurrency); } @@ -3719,7 +3758,8 @@ public static Flowable mergeDelayError(Iterable Flowable mergeDelayError(Publisher> sources) { + @NonNull + public static Flowable mergeDelayError(@NonNull Publisher> sources) { return mergeDelayError(sources, bufferSize()); } @@ -3759,7 +3799,8 @@ public static Flowable mergeDelayError(Publisher Flowable mergeDelayError(Publisher> sources, int maxConcurrency) { + @NonNull + public static Flowable mergeDelayError(@NonNull Publisher> sources, int maxConcurrency) { return fromPublisher(sources).flatMap((Function)Functions.identity(), true, maxConcurrency); } @@ -3795,7 +3836,9 @@ public static Flowable mergeDelayError(Publisher Flowable mergeArrayDelayError(Publisher... sources) { + @SafeVarargs + @NonNull + public static Flowable mergeArrayDelayError(@NonNull Publisher... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), true, sources.length); } @@ -3833,7 +3876,7 @@ public static Flowable mergeArrayDelayError(Publisher... sou @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(Publisher source1, Publisher source2) { + public static Flowable mergeDelayError(@NonNull Publisher source1, @NonNull Publisher source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), true, 2); @@ -3876,7 +3919,7 @@ public static Flowable mergeDelayError(Publisher source1, Pu @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(Publisher source1, Publisher source2, Publisher source3) { + public static Flowable mergeDelayError(@NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3923,8 +3966,8 @@ public static Flowable mergeDelayError(Publisher source1, Pu @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable mergeDelayError( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3954,6 +3997,7 @@ public static Flowable mergeDelayError( @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") + @NonNull public static Flowable never() { return RxJavaPlugins.onAssembly((Flowable) FlowableNever.INSTANCE); } @@ -3982,6 +4026,7 @@ public static Flowable never() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Flowable range(int start, int count) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); @@ -4022,6 +4067,7 @@ public static Flowable range(int start, int count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Flowable rangeLong(long start, long count) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); @@ -4068,7 +4114,8 @@ public static Flowable rangeLong(long start, long count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(Publisher source1, Publisher source2) { + @NonNull + public static Single sequenceEqual(@NonNull Publisher source1, @NonNull Publisher source2) { return sequenceEqual(source1, source2, ObjectHelper.equalsPredicate(), bufferSize()); } @@ -4101,8 +4148,9 @@ public static Single sequenceEqual(Publisher source1, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(Publisher source1, Publisher source2, - BiPredicate isEqual) { + @NonNull + public static Single sequenceEqual(@NonNull Publisher source1, @NonNull Publisher source2, + @NonNull BiPredicate isEqual) { return sequenceEqual(source1, source2, isEqual, bufferSize()); } @@ -4138,13 +4186,13 @@ public static Single sequenceEqual(Publisher source1, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(Publisher source1, Publisher source2, - BiPredicate isEqual, int bufferSize) { + public static Single sequenceEqual(@NonNull Publisher source1, @NonNull Publisher source2, + @NonNull BiPredicate isEqual, int bufferSize) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(isEqual, "isEqual is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new FlowableSequenceEqualSingle(source1, source2, isEqual, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowableSequenceEqualSingle<>(source1, source2, isEqual, bufferSize)); } /** @@ -4174,7 +4222,8 @@ public static Single sequenceEqual(Publisher source1, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(Publisher source1, Publisher source2, int bufferSize) { + @NonNull + public static Single sequenceEqual(@NonNull Publisher source1, @NonNull Publisher source2, int bufferSize) { return sequenceEqual(source1, source2, ObjectHelper.equalsPredicate(), bufferSize); } @@ -4214,7 +4263,8 @@ public static Single sequenceEqual(Publisher source1, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable switchOnNext(Publisher> sources, int bufferSize) { + @NonNull + public static Flowable switchOnNext(@NonNull Publisher> sources, int bufferSize) { return fromPublisher(sources).switchMap((Function)Functions.identity(), bufferSize); } @@ -4252,7 +4302,8 @@ public static Flowable switchOnNext(Publisher Flowable switchOnNext(Publisher> sources) { + @NonNull + public static Flowable switchOnNext(@NonNull Publisher> sources) { return fromPublisher(sources).switchMap((Function)Functions.identity()); } @@ -4291,7 +4342,8 @@ public static Flowable switchOnNext(Publisher Flowable switchOnNextDelayError(Publisher> sources) { + @NonNull + public static Flowable switchOnNextDelayError(@NonNull Publisher> sources) { return switchOnNextDelayError(sources, bufferSize()); } @@ -4332,7 +4384,8 @@ public static Flowable switchOnNextDelayError(Publisher Flowable switchOnNextDelayError(Publisher> sources, int prefetch) { + @NonNull + public static Flowable switchOnNextDelayError(@NonNull Publisher> sources, int prefetch) { return fromPublisher(sources).switchMapDelayError(Functions.>identity(), prefetch); } @@ -4358,7 +4411,8 @@ public static Flowable switchOnNextDelayError(Publisher timer(long delay, TimeUnit unit) { + @NonNull + public static Flowable timer(long delay, @NonNull TimeUnit unit) { return timer(delay, unit, Schedulers.computation()); } @@ -4389,7 +4443,7 @@ public static Flowable timer(long delay, TimeUnit unit) { @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Flowable timer(long delay, TimeUnit unit, Scheduler scheduler) { + public static Flowable timer(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -4418,12 +4472,12 @@ public static Flowable timer(long delay, TimeUnit unit, Scheduler schedule @NonNull @BackpressureSupport(BackpressureKind.NONE) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable unsafeCreate(Publisher onSubscribe) { + public static Flowable unsafeCreate(@NonNull Publisher onSubscribe) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); if (onSubscribe instanceof Flowable) { throw new IllegalArgumentException("unsafeCreate(Flowable) should be upgraded"); } - return RxJavaPlugins.onAssembly(new FlowableFromPublisher(onSubscribe)); + return RxJavaPlugins.onAssembly(new FlowableFromPublisher<>(onSubscribe)); } /** @@ -4452,8 +4506,11 @@ public static Flowable unsafeCreate(Publisher onSubscribe) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable using(Supplier resourceSupplier, - Function> sourceSupplier, Consumer resourceDisposer) { + @NonNull + public static Flowable using( + @NonNull Supplier resourceSupplier, + @NonNull Function> sourceSupplier, + @NonNull Consumer resourceDisposer) { return using(resourceSupplier, sourceSupplier, resourceDisposer, true); } @@ -4494,9 +4551,11 @@ public static Flowable using(Supplier resourceSupplier, @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable using(Supplier resourceSupplier, - Function> sourceSupplier, - Consumer resourceDisposer, boolean eager) { + public static Flowable using( + @NonNull Supplier resourceSupplier, + @NonNull Function> sourceSupplier, + @NonNull Consumer resourceDisposer, + boolean eager) { Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); Objects.requireNonNull(sourceSupplier, "sourceSupplier is null"); Objects.requireNonNull(resourceDisposer, "resourceDisposer is null"); @@ -4551,7 +4610,7 @@ public static Flowable using(Supplier resourceSupplier, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable zip(Iterable> sources, Function zipper) { + public static Flowable zip(@NonNull Iterable> sources, @NonNull Function zipper) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableZip(null, sources, zipper, bufferSize(), false)); @@ -4610,8 +4669,8 @@ public static Flowable zip(Iterable> @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable zip(Iterable> sources, - Function zipper, boolean delayError, + public static Flowable zip(@NonNull Iterable> sources, + @NonNull Function zipper, boolean delayError, int bufferSize) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); @@ -4672,8 +4731,8 @@ public static Flowable zip(Iterable> @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, - BiFunction zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull BiFunction zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2); @@ -4733,8 +4792,8 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, - BiFunction zipper, boolean delayError) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull BiFunction zipper, boolean delayError) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize(), source1, source2); @@ -4795,8 +4854,8 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, - BiFunction zipper, boolean delayError, int bufferSize) { + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull BiFunction zipper, boolean delayError, int bufferSize) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize, source1, source2); @@ -4859,8 +4918,8 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, Publisher source3, - Function3 zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3, + @NonNull Function3 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -4927,9 +4986,9 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, Publisher source3, - Publisher source4, - Function4 zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3, + @NonNull Publisher source4, + @NonNull Function4 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -5000,9 +5059,9 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, Publisher source3, - Publisher source4, Publisher source5, - Function5 zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3, + @NonNull Publisher source4, @NonNull Publisher source5, + @NonNull Function5 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -5076,9 +5135,9 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, Publisher source3, - Publisher source4, Publisher source5, Publisher source6, - Function6 zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3, + @NonNull Publisher source4, @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Function6 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -5156,10 +5215,10 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, Publisher source3, - Publisher source4, Publisher source5, Publisher source6, - Publisher source7, - Function7 zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3, + @NonNull Publisher source4, @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Publisher source7, + @NonNull Function7 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -5241,10 +5300,10 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, Publisher source3, - Publisher source4, Publisher source5, Publisher source6, - Publisher source7, Publisher source8, - Function8 zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3, + @NonNull Publisher source4, @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Publisher source7, @NonNull Publisher source8, + @NonNull Function8 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -5330,10 +5389,10 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public static Flowable zip( - Publisher source1, Publisher source2, Publisher source3, - Publisher source4, Publisher source5, Publisher source6, - Publisher source7, Publisher source8, Publisher source9, - Function9 zipper) { + @NonNull Publisher source1, @NonNull Publisher source2, @NonNull Publisher source3, + @NonNull Publisher source4, @NonNull Publisher source5, @NonNull Publisher source6, + @NonNull Publisher source7, @NonNull Publisher source8, @NonNull Publisher source9, + @NonNull Function9 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -5401,8 +5460,8 @@ public static Flowable zip( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs - public static Flowable zipArray(Function zipper, - boolean delayError, int bufferSize, Publisher... sources) { + public static Flowable zipArray(@NonNull Function zipper, + boolean delayError, int bufferSize, @NonNull Publisher... sources) { if (sources.length == 0) { return empty(); } @@ -5438,9 +5497,9 @@ public static Flowable zipArray(Function all(Predicate predicate) { + public final Single all(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new FlowableAllSingle(this, predicate)); + return RxJavaPlugins.onAssembly(new FlowableAllSingle<>(this, predicate)); } /** @@ -5467,7 +5526,7 @@ public final Single all(Predicate predicate) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable ambWith(Publisher other) { + public final Flowable ambWith(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -5499,9 +5558,9 @@ public final Flowable ambWith(Publisher other) { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single any(Predicate predicate) { + public final Single any(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new FlowableAnySingle(this, predicate)); + return RxJavaPlugins.onAssembly(new FlowableAnySingle<>(this, predicate)); } /** @@ -5527,8 +5586,9 @@ public final Single any(Predicate predicate) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final T blockingFirst() { - BlockingFirstSubscriber s = new BlockingFirstSubscriber(); + BlockingFirstSubscriber s = new BlockingFirstSubscriber<>(); subscribe(s); T v = s.blockingGet(); if (v != null) { @@ -5561,8 +5621,9 @@ public final T blockingFirst() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final T blockingFirst(T defaultItem) { - BlockingFirstSubscriber s = new BlockingFirstSubscriber(); + @NonNull + public final T blockingFirst(@NonNull T defaultItem) { + BlockingFirstSubscriber s = new BlockingFirstSubscriber<>(); subscribe(s); T v = s.blockingGet(); return v != null ? v : defaultItem; @@ -5602,7 +5663,7 @@ public final T blockingFirst(T defaultItem) { */ @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final void blockingForEach(Consumer onNext) { + public final void blockingForEach(@NonNull Consumer onNext) { Iterator it = blockingIterable().iterator(); while (it.hasNext()) { try { @@ -5633,6 +5694,7 @@ public final void blockingForEach(Consumer onNext) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingIterable() { return blockingIterable(bufferSize()); } @@ -5657,9 +5719,10 @@ public final Iterable blockingIterable() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingIterable(int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return new BlockingFlowableIterable(this, bufferSize); + return new BlockingFlowableIterable<>(this, bufferSize); } /** @@ -5687,8 +5750,9 @@ public final Iterable blockingIterable(int bufferSize) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final T blockingLast() { - BlockingLastSubscriber s = new BlockingLastSubscriber(); + BlockingLastSubscriber s = new BlockingLastSubscriber<>(); subscribe(s); T v = s.blockingGet(); if (v != null) { @@ -5723,8 +5787,9 @@ public final T blockingLast() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final T blockingLast(T defaultItem) { - BlockingLastSubscriber s = new BlockingLastSubscriber(); + @NonNull + public final T blockingLast(@NonNull T defaultItem) { + BlockingLastSubscriber s = new BlockingLastSubscriber<>(); subscribe(s); T v = s.blockingGet(); return v != null ? v : defaultItem; @@ -5753,8 +5818,9 @@ public final T blockingLast(T defaultItem) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingLatest() { - return new BlockingFlowableLatest(this); + return new BlockingFlowableLatest<>(this); } /** @@ -5780,8 +5846,9 @@ public final Iterable blockingLatest() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Iterable blockingMostRecent(T initialItem) { - return new BlockingFlowableMostRecent(this, initialItem); + @NonNull + public final Iterable blockingMostRecent(@NonNull T initialItem) { + return new BlockingFlowableMostRecent<>(this, initialItem); } /** @@ -5804,8 +5871,9 @@ public final Iterable blockingMostRecent(T initialItem) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingNext() { - return new BlockingFlowableNext(this); + return new BlockingFlowableNext<>(this); } /** @@ -5831,6 +5899,7 @@ public final Iterable blockingNext() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final T blockingSingle() { return singleOrError().blockingGet(); } @@ -5862,7 +5931,8 @@ public final T blockingSingle() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final T blockingSingle(T defaultItem) { + @NonNull + public final T blockingSingle(@NonNull T defaultItem) { return single(defaultItem).blockingGet(); } @@ -5891,6 +5961,7 @@ public final T blockingSingle(T defaultItem) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Future toFuture() { return subscribeWith(new FutureSubscriber()); } @@ -5945,7 +6016,7 @@ public final void blockingSubscribe() { */ @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final void blockingSubscribe(Consumer onNext) { + public final void blockingSubscribe(@NonNull Consumer onNext) { FlowableBlockingSubscribe.subscribe(this, onNext, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -5977,7 +6048,7 @@ public final void blockingSubscribe(Consumer onNext) { */ @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final void blockingSubscribe(Consumer onNext, int bufferSize) { + public final void blockingSubscribe(@NonNull Consumer onNext, int bufferSize) { FlowableBlockingSubscribe.subscribe(this, onNext, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION, bufferSize); } @@ -6001,7 +6072,7 @@ public final void blockingSubscribe(Consumer onNext, int bufferSize) */ @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final void blockingSubscribe(Consumer onNext, Consumer onError) { + public final void blockingSubscribe(@NonNull Consumer onNext, @NonNull Consumer onError) { FlowableBlockingSubscribe.subscribe(this, onNext, onError, Functions.EMPTY_ACTION); } @@ -6027,7 +6098,7 @@ public final void blockingSubscribe(Consumer onNext, Consumer onNext, Consumer onError, + public final void blockingSubscribe(@NonNull Consumer onNext, @NonNull Consumer onError, int bufferSize) { FlowableBlockingSubscribe.subscribe(this, onNext, onError, Functions.EMPTY_ACTION, bufferSize); } @@ -6052,7 +6123,7 @@ public final void blockingSubscribe(Consumer onNext, Consumer onNext, Consumer onError, Action onComplete) { + public final void blockingSubscribe(@NonNull Consumer onNext, @NonNull Consumer onError, @NonNull Action onComplete) { FlowableBlockingSubscribe.subscribe(this, onNext, onError, onComplete); } @@ -6078,7 +6149,7 @@ public final void blockingSubscribe(Consumer onNext, Consumer onNext, Consumer onError, Action onComplete, + public final void blockingSubscribe(@NonNull Consumer onNext, @NonNull Consumer onError, @NonNull Action onComplete, int bufferSize) { FlowableBlockingSubscribe.subscribe(this, onNext, onError, onComplete, bufferSize); } @@ -6103,7 +6174,7 @@ public final void blockingSubscribe(Consumer onNext, Consumer subscriber) { + public final void blockingSubscribe(@NonNull Subscriber subscriber) { FlowableBlockingSubscribe.subscribe(this, subscriber); } @@ -6133,6 +6204,7 @@ public final void blockingSubscribe(Subscriber subscriber) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> buffer(int count) { return buffer(count, count); } @@ -6167,6 +6239,7 @@ public final Flowable> buffer(int count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> buffer(int count, int skip) { return buffer(count, skip, ArrayListSupplier.asSupplier()); } @@ -6206,11 +6279,11 @@ public final Flowable> buffer(int count, int skip) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final > Flowable buffer(int count, int skip, Supplier bufferSupplier) { + public final > Flowable buffer(int count, int skip, @NonNull Supplier bufferSupplier) { ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(skip, "skip"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); - return RxJavaPlugins.onAssembly(new FlowableBuffer(this, count, skip, bufferSupplier)); + return RxJavaPlugins.onAssembly(new FlowableBuffer<>(this, count, skip, bufferSupplier)); } /** @@ -6243,7 +6316,8 @@ public final > Flowable buffer(int count, int @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final > Flowable buffer(int count, Supplier bufferSupplier) { + @NonNull + public final > Flowable buffer(int count, @NonNull Supplier bufferSupplier) { return buffer(count, count, bufferSupplier); } @@ -6277,7 +6351,8 @@ public final > Flowable buffer(int count, Sup @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable> buffer(long timespan, long timeskip, TimeUnit unit) { + @NonNull + public final Flowable> buffer(long timespan, long timeskip, @NonNull TimeUnit unit) { return buffer(timespan, timeskip, unit, Schedulers.computation(), ArrayListSupplier.asSupplier()); } @@ -6314,7 +6389,8 @@ public final Flowable> buffer(long timespan, long timeskip, TimeUnit uni @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> buffer(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable> buffer(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return buffer(timespan, timeskip, unit, scheduler, ArrayListSupplier.asSupplier()); } @@ -6356,12 +6432,12 @@ public final Flowable> buffer(long timespan, long timeskip, TimeUnit uni @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final > Flowable buffer(long timespan, long timeskip, TimeUnit unit, - Scheduler scheduler, Supplier bufferSupplier) { + public final > Flowable buffer(long timespan, long timeskip, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler, @NonNull Supplier bufferSupplier) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); - return RxJavaPlugins.onAssembly(new FlowableBufferTimed(this, timespan, timeskip, unit, scheduler, bufferSupplier, Integer.MAX_VALUE, false)); + return RxJavaPlugins.onAssembly(new FlowableBufferTimed<>(this, timespan, timeskip, unit, scheduler, bufferSupplier, Integer.MAX_VALUE, false)); } /** @@ -6393,7 +6469,8 @@ public final > Flowable buffer(long timespan, @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable> buffer(long timespan, TimeUnit unit) { + @NonNull + public final Flowable> buffer(long timespan, @NonNull TimeUnit unit) { return buffer(timespan, unit, Schedulers.computation(), Integer.MAX_VALUE); } @@ -6429,7 +6506,8 @@ public final Flowable> buffer(long timespan, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable> buffer(long timespan, TimeUnit unit, int count) { + @NonNull + public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, int count) { return buffer(timespan, unit, Schedulers.computation(), count); } @@ -6468,7 +6546,8 @@ public final Flowable> buffer(long timespan, TimeUnit unit, int count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> buffer(long timespan, TimeUnit unit, Scheduler scheduler, int count) { + @NonNull + public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int count) { return buffer(timespan, unit, scheduler, count, ArrayListSupplier.asSupplier(), false); } @@ -6513,16 +6592,17 @@ public final Flowable> buffer(long timespan, TimeUnit unit, Scheduler sc @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) + @NonNull public final > Flowable buffer( - long timespan, TimeUnit unit, - Scheduler scheduler, int count, - Supplier bufferSupplier, + long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler, int count, + @NonNull Supplier bufferSupplier, boolean restartTimerOnMaxSize) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); ObjectHelper.verifyPositive(count, "count"); - return RxJavaPlugins.onAssembly(new FlowableBufferTimed(this, timespan, timespan, unit, scheduler, bufferSupplier, count, restartTimerOnMaxSize)); + return RxJavaPlugins.onAssembly(new FlowableBufferTimed<>(this, timespan, timespan, unit, scheduler, bufferSupplier, count, restartTimerOnMaxSize)); } /** @@ -6556,7 +6636,8 @@ public final > Flowable buffer( @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> buffer(long timespan, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return buffer(timespan, unit, scheduler, Integer.MAX_VALUE, ArrayListSupplier.asSupplier(), false); } @@ -6590,9 +6671,10 @@ public final Flowable> buffer(long timespan, TimeUnit unit, Scheduler sc @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> buffer( - Flowable openingIndicator, - Function> closingIndicator) { + @NonNull Flowable openingIndicator, + @NonNull Function> closingIndicator) { return buffer(openingIndicator, closingIndicator, ArrayListSupplier.asSupplier()); } @@ -6630,10 +6712,11 @@ public final Flowable> buffer( @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final > Flowable buffer( - Flowable openingIndicator, - Function> closingIndicator, - Supplier bufferSupplier) { + @NonNull Flowable openingIndicator, + @NonNull Function> closingIndicator, + @NonNull Supplier bufferSupplier) { Objects.requireNonNull(openingIndicator, "openingIndicator is null"); Objects.requireNonNull(closingIndicator, "closingIndicator is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); @@ -6670,7 +6753,8 @@ public final > Flowable b @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable> buffer(Publisher boundaryIndicator) { + @NonNull + public final Flowable> buffer(@NonNull Publisher boundaryIndicator) { return buffer(boundaryIndicator, ArrayListSupplier.asSupplier()); } @@ -6706,7 +6790,8 @@ public final Flowable> buffer(Publisher boundaryIndicator) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable> buffer(Publisher boundaryIndicator, final int initialCapacity) { + @NonNull + public final Flowable> buffer(@NonNull Publisher boundaryIndicator, final int initialCapacity) { ObjectHelper.verifyPositive(initialCapacity, "initialCapacity"); return buffer(boundaryIndicator, Functions.createArrayList(initialCapacity)); } @@ -6745,10 +6830,11 @@ public final Flowable> buffer(Publisher boundaryIndicator, final @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final > Flowable buffer(Publisher boundaryIndicator, Supplier bufferSupplier) { + @NonNull + public final > Flowable buffer(@NonNull Publisher boundaryIndicator, @NonNull Supplier bufferSupplier) { Objects.requireNonNull(boundaryIndicator, "boundaryIndicator is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); - return RxJavaPlugins.onAssembly(new FlowableBufferExactBoundary(this, boundaryIndicator, bufferSupplier)); + return RxJavaPlugins.onAssembly(new FlowableBufferExactBoundary<>(this, boundaryIndicator, bufferSupplier)); } /** @@ -6805,6 +6891,7 @@ public final > Flowable buffer(Publisher cache() { return cacheWithInitialCapacity(16); } @@ -6867,9 +6954,10 @@ public final Flowable cache() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable cacheWithInitialCapacity(int initialCapacity) { ObjectHelper.verifyPositive(initialCapacity, "initialCapacity"); - return RxJavaPlugins.onAssembly(new FlowableCache(this, initialCapacity)); + return RxJavaPlugins.onAssembly(new FlowableCache<>(this, initialCapacity)); } /** @@ -6897,7 +6985,7 @@ public final Flowable cacheWithInitialCapacity(int initialCapacity) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable cast(final Class clazz) { + public final Flowable cast(@NonNull Class clazz) { Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -6936,7 +7024,7 @@ public final Flowable cast(final Class clazz) { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single collect(Supplier initialItemSupplier, BiConsumer collector) { + public final Single collect(@NonNull Supplier initialItemSupplier, @NonNull BiConsumer collector) { Objects.requireNonNull(initialItemSupplier, "initialItemSupplier is null"); Objects.requireNonNull(collector, "collector is null"); return RxJavaPlugins.onAssembly(new FlowableCollectSingle(this, initialItemSupplier, collector)); @@ -6975,7 +7063,7 @@ public final Single collect(Supplier initialItemSupplier, Bi @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single collectInto(final U initialItem, BiConsumer collector) { + public final <@NonNull U> Single collectInto(U initialItem, @NonNull BiConsumer collector) { Objects.requireNonNull(initialItem, "initialItem is null"); return collect(Functions.justSupplier(initialItem), collector); } @@ -7006,7 +7094,8 @@ public final Single collectInto(final U initialItem, BiConsumer Flowable compose(FlowableTransformer composer) { + @NonNull + public final Flowable compose(@NonNull FlowableTransformer composer) { return fromPublisher(((FlowableTransformer) Objects.requireNonNull(composer, "composer is null")).apply(this)); } @@ -7042,7 +7131,8 @@ public final Flowable compose(FlowableTransformer @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable concatMap(Function> mapper) { + @NonNull + public final Flowable concatMap(@NonNull Function> mapper) { return concatMap(mapper, 2); } @@ -7082,7 +7172,7 @@ public final Flowable concatMap(Function Flowable concatMap(Function> mapper, int prefetch) { + public final Flowable concatMap(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); if (this instanceof ScalarSupplier) { @@ -7093,7 +7183,7 @@ public final Flowable concatMap(Function(this, mapper, prefetch, ErrorMode.IMMEDIATE)); + return RxJavaPlugins.onAssembly(new FlowableConcatMap<>(this, mapper, prefetch, ErrorMode.IMMEDIATE)); } /** @@ -7135,11 +7225,11 @@ public final Flowable concatMap(Function Flowable concatMap(Function> mapper, int prefetch, Scheduler scheduler) { + public final Flowable concatMap(@NonNull Function> mapper, int prefetch, @NonNull Scheduler scheduler) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); Objects.requireNonNull(scheduler, "scheduler"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapScheduler(this, mapper, prefetch, ErrorMode.IMMEDIATE, scheduler)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapScheduler<>(this, mapper, prefetch, ErrorMode.IMMEDIATE, scheduler)); } /** @@ -7165,7 +7255,8 @@ public final Flowable concatMap(Function mapper) { + @NonNull + public final Completable concatMapCompletable(@NonNull Function mapper) { return concatMapCompletable(mapper, 2); } @@ -7197,10 +7288,10 @@ public final Completable concatMapCompletable(Function mapper, int prefetch) { + public final Completable concatMapCompletable(@NonNull Function mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapCompletable(this, mapper, ErrorMode.IMMEDIATE, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapCompletable<>(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } /** @@ -7227,7 +7318,8 @@ public final Completable concatMapCompletable(Function mapper) { + @NonNull + public final Completable concatMapCompletableDelayError(@NonNull Function mapper) { return concatMapCompletableDelayError(mapper, true, 2); } @@ -7261,7 +7353,8 @@ public final Completable concatMapCompletableDelayError(Function mapper, boolean tillTheEnd) { + @NonNull + public final Completable concatMapCompletableDelayError(@NonNull Function mapper, boolean tillTheEnd) { return concatMapCompletableDelayError(mapper, tillTheEnd, 2); } @@ -7300,10 +7393,10 @@ public final Completable concatMapCompletableDelayError(Function mapper, boolean tillTheEnd, int prefetch) { + public final Completable concatMapCompletableDelayError(@NonNull Function mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapCompletable(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapCompletable<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } /** @@ -7334,7 +7427,8 @@ public final Completable concatMapCompletableDelayError(Function Flowable concatMapDelayError(Function> mapper) { + @NonNull + public final Flowable concatMapDelayError(@NonNull Function> mapper) { return concatMapDelayError(mapper, true, 2); } @@ -7373,7 +7467,7 @@ public final Flowable concatMapDelayError(Function Flowable concatMapDelayError(Function> mapper, + public final Flowable concatMapDelayError(@NonNull Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); @@ -7385,7 +7479,7 @@ public final Flowable concatMapDelayError(Function(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); + return RxJavaPlugins.onAssembly(new FlowableConcatMap<>(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); } /** @@ -7425,12 +7519,12 @@ public final Flowable concatMapDelayError(Function Flowable concatMapDelayError(Function> mapper, - boolean tillTheEnd, int prefetch, Scheduler scheduler) { + public final Flowable concatMapDelayError(@NonNull Function> mapper, + boolean tillTheEnd, int prefetch, @NonNull Scheduler scheduler) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapScheduler(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, scheduler)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapScheduler<>(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, scheduler)); } /** @@ -7456,7 +7550,8 @@ public final Flowable concatMapDelayError(Function Flowable concatMapEager(Function> mapper) { + @NonNull + public final Flowable concatMapEager(@NonNull Function> mapper) { return concatMapEager(mapper, bufferSize(), bufferSize()); } @@ -7486,12 +7581,12 @@ public final Flowable concatMapEager(Function Flowable concatMapEager(Function> mapper, + public final Flowable concatMapEager(@NonNull Function> mapper, int maxConcurrency, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapEager(this, mapper, maxConcurrency, prefetch, ErrorMode.IMMEDIATE)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapEager<>(this, mapper, maxConcurrency, prefetch, ErrorMode.IMMEDIATE)); } /** @@ -7520,7 +7615,8 @@ public final Flowable concatMapEager(Function Flowable concatMapEagerDelayError(Function> mapper, + @NonNull + public final Flowable concatMapEagerDelayError(@NonNull Function> mapper, boolean tillTheEnd) { return concatMapEagerDelayError(mapper, tillTheEnd, bufferSize(), bufferSize()); } @@ -7556,12 +7652,12 @@ public final Flowable concatMapEagerDelayError(Function Flowable concatMapEagerDelayError(Function> mapper, + public final Flowable concatMapEagerDelayError(@NonNull Function> mapper, boolean tillTheEnd, int maxConcurrency, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapEager(this, mapper, maxConcurrency, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapEager<>(this, mapper, maxConcurrency, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); } /** @@ -7589,7 +7685,8 @@ public final Flowable concatMapEagerDelayError(Function Flowable concatMapIterable(Function> mapper) { + @NonNull + public final Flowable concatMapIterable(@NonNull Function> mapper) { return concatMapIterable(mapper, 2); } @@ -7621,10 +7718,10 @@ public final Flowable concatMapIterable(Function Flowable concatMapIterable(final Function> mapper, int prefetch) { + public final Flowable concatMapIterable(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableFlattenIterable(this, mapper, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableFlattenIterable<>(this, mapper, prefetch)); } /** @@ -7654,7 +7751,8 @@ public final Flowable concatMapIterable(final Function Flowable concatMapMaybe(Function> mapper) { + @NonNull + public final Flowable concatMapMaybe(@NonNull Function> mapper) { return concatMapMaybe(mapper, 2); } @@ -7690,10 +7788,10 @@ public final Flowable concatMapMaybe(Function Flowable concatMapMaybe(Function> mapper, int prefetch) { + public final Flowable concatMapMaybe(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapMaybe(this, mapper, ErrorMode.IMMEDIATE, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapMaybe<>(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } /** @@ -7723,7 +7821,8 @@ public final Flowable concatMapMaybe(Function Flowable concatMapMaybeDelayError(Function> mapper) { + @NonNull + public final Flowable concatMapMaybeDelayError(@NonNull Function> mapper) { return concatMapMaybeDelayError(mapper, true, 2); } @@ -7760,7 +7859,8 @@ public final Flowable concatMapMaybeDelayError(Function Flowable concatMapMaybeDelayError(Function> mapper, boolean tillTheEnd) { + @NonNull + public final Flowable concatMapMaybeDelayError(@NonNull Function> mapper, boolean tillTheEnd) { return concatMapMaybeDelayError(mapper, tillTheEnd, 2); } @@ -7801,10 +7901,10 @@ public final Flowable concatMapMaybeDelayError(Function Flowable concatMapMaybeDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { + public final Flowable concatMapMaybeDelayError(@NonNull Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapMaybe(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapMaybe<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } /** @@ -7834,7 +7934,8 @@ public final Flowable concatMapMaybeDelayError(Function Flowable concatMapSingle(Function> mapper) { + @NonNull + public final Flowable concatMapSingle(@NonNull Function> mapper) { return concatMapSingle(mapper, 2); } @@ -7870,10 +7971,10 @@ public final Flowable concatMapSingle(Function Flowable concatMapSingle(Function> mapper, int prefetch) { + public final Flowable concatMapSingle(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapSingle(this, mapper, ErrorMode.IMMEDIATE, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapSingle<>(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } /** @@ -7903,7 +8004,8 @@ public final Flowable concatMapSingle(Function Flowable concatMapSingleDelayError(Function> mapper) { + @NonNull + public final Flowable concatMapSingleDelayError(@NonNull Function> mapper) { return concatMapSingleDelayError(mapper, true, 2); } @@ -7940,7 +8042,8 @@ public final Flowable concatMapSingleDelayError(Function Flowable concatMapSingleDelayError(Function> mapper, boolean tillTheEnd) { + @NonNull + public final Flowable concatMapSingleDelayError(@NonNull Function> mapper, boolean tillTheEnd) { return concatMapSingleDelayError(mapper, tillTheEnd, 2); } @@ -7981,10 +8084,10 @@ public final Flowable concatMapSingleDelayError(Function Flowable concatMapSingleDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { + public final Flowable concatMapSingleDelayError(@NonNull Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowableConcatMapSingle(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableConcatMapSingle<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } /** @@ -8011,7 +8114,7 @@ public final Flowable concatMapSingleDelayError(Function concatWith(Publisher other) { + public final Flowable concatWith(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return concat(this, other); } @@ -8036,9 +8139,10 @@ public final Flowable concatWith(Publisher other) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable concatWith(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableConcatWithSingle(this, other)); + return RxJavaPlugins.onAssembly(new FlowableConcatWithSingle<>(this, other)); } /** @@ -8061,9 +8165,10 @@ public final Flowable concatWith(@NonNull SingleSource other) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable concatWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableConcatWithMaybe(this, other)); + return RxJavaPlugins.onAssembly(new FlowableConcatWithMaybe<>(this, other)); } /** @@ -8088,9 +8193,10 @@ public final Flowable concatWith(@NonNull MaybeSource other) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable concatWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableConcatWithCompletable(this, other)); + return RxJavaPlugins.onAssembly(new FlowableConcatWithCompletable<>(this, other)); } /** @@ -8116,7 +8222,7 @@ public final Flowable concatWith(@NonNull CompletableSource other) { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single contains(final Object item) { + public final Single contains(@NonNull Object item) { Objects.requireNonNull(item, "item is null"); return any(Functions.equalsWith(item)); } @@ -8141,8 +8247,9 @@ public final Single contains(final Object item) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single count() { - return RxJavaPlugins.onAssembly(new FlowableCountSingle(this)); + return RxJavaPlugins.onAssembly(new FlowableCountSingle<>(this)); } /** @@ -8179,9 +8286,9 @@ public final Single count() { @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable debounce(Function> debounceIndicator) { + public final Flowable debounce(@NonNull Function> debounceIndicator) { Objects.requireNonNull(debounceIndicator, "debounceIndicator is null"); - return RxJavaPlugins.onAssembly(new FlowableDebounce(this, debounceIndicator)); + return RxJavaPlugins.onAssembly(new FlowableDebounce<>(this, debounceIndicator)); } /** @@ -8222,7 +8329,8 @@ public final Flowable debounce(Function @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable debounce(long timeout, TimeUnit unit) { + @NonNull + public final Flowable debounce(long timeout, @NonNull TimeUnit unit) { return debounce(timeout, unit, Schedulers.computation()); } @@ -8267,10 +8375,10 @@ public final Flowable debounce(long timeout, TimeUnit unit) { @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable debounce(long timeout, TimeUnit unit, Scheduler scheduler) { + public final Flowable debounce(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableDebounceTimed(this, timeout, unit, scheduler)); + return RxJavaPlugins.onAssembly(new FlowableDebounceTimed<>(this, timeout, unit, scheduler)); } /** @@ -8298,7 +8406,7 @@ public final Flowable debounce(long timeout, TimeUnit unit, Scheduler schedul @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable defaultIfEmpty(T defaultItem) { + public final Flowable defaultIfEmpty(@NonNull T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); return switchIfEmpty(just(defaultItem)); } @@ -8334,7 +8442,7 @@ public final Flowable defaultIfEmpty(T defaultItem) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable delay(final Function> itemDelayIndicator) { + public final Flowable delay(@NonNull Function> itemDelayIndicator) { Objects.requireNonNull(itemDelayIndicator, "itemDelayIndicator is null"); return flatMap(FlowableInternalHelper.itemDelay(itemDelayIndicator)); } @@ -8361,7 +8469,8 @@ public final Flowable delay(final Function delay(long delay, TimeUnit unit) { + @NonNull + public final Flowable delay(long delay, @NonNull TimeUnit unit) { return delay(delay, unit, Schedulers.computation(), false); } @@ -8390,7 +8499,8 @@ public final Flowable delay(long delay, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable delay(long delay, TimeUnit unit, boolean delayError) { + @NonNull + public final Flowable delay(long delay, @NonNull TimeUnit unit, boolean delayError) { return delay(delay, unit, Schedulers.computation(), delayError); } @@ -8418,7 +8528,8 @@ public final Flowable delay(long delay, TimeUnit unit, boolean delayError) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable delay(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable delay(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delay(delay, unit, scheduler, false); } @@ -8450,11 +8561,11 @@ public final Flowable delay(long delay, TimeUnit unit, Scheduler scheduler) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable delay(long delay, TimeUnit unit, Scheduler scheduler, boolean delayError) { + public final Flowable delay(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableDelay(this, Math.max(0L, delay), unit, scheduler, delayError)); + return RxJavaPlugins.onAssembly(new FlowableDelay<>(this, Math.max(0L, delay), unit, scheduler, delayError)); } /** @@ -8492,8 +8603,9 @@ public final Flowable delay(long delay, TimeUnit unit, Scheduler scheduler, b @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable delay(Publisher subscriptionIndicator, - Function> itemDelayIndicator) { + @NonNull + public final Flowable delay(@NonNull Publisher subscriptionIndicator, + @NonNull Function> itemDelayIndicator) { return delaySubscription(subscriptionIndicator).delay(itemDelayIndicator); } @@ -8519,9 +8631,9 @@ public final Flowable delay(Publisher subscriptionIndicator, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable delaySubscription(Publisher subscriptionIndicator) { + public final Flowable delaySubscription(@NonNull Publisher subscriptionIndicator) { Objects.requireNonNull(subscriptionIndicator, "subscriptionIndicator is null"); - return RxJavaPlugins.onAssembly(new FlowableDelaySubscriptionOther(this, subscriptionIndicator)); + return RxJavaPlugins.onAssembly(new FlowableDelaySubscriptionOther<>(this, subscriptionIndicator)); } /** @@ -8545,7 +8657,8 @@ public final Flowable delaySubscription(Publisher subscriptionIndicato @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable delaySubscription(long delay, TimeUnit unit) { + @NonNull + public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit) { return delaySubscription(delay, unit, Schedulers.computation()); } @@ -8574,7 +8687,8 @@ public final Flowable delaySubscription(long delay, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable delaySubscription(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delaySubscription(timer(delay, unit, scheduler)); } @@ -8632,9 +8746,9 @@ public final Flowable delaySubscription(long delay, TimeUnit unit, Scheduler @NonNull @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.PASS_THROUGH) - public final Flowable dematerialize(Function> selector) { + public final Flowable dematerialize(@NonNull Function<@NonNull ? super T, @NonNull Notification> selector) { Objects.requireNonNull(selector, "selector is null"); - return RxJavaPlugins.onAssembly(new FlowableDematerialize(this, selector)); + return RxJavaPlugins.onAssembly(new FlowableDematerialize<>(this, selector)); } /** @@ -8674,6 +8788,7 @@ public final Flowable dematerialize(Function> @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable distinct() { return distinct((Function)Functions.identity(), Functions.createHashSet()); } @@ -8717,7 +8832,8 @@ public final Flowable distinct() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable distinct(Function keySelector) { + @NonNull + public final Flowable distinct(@NonNull Function keySelector) { return distinct(keySelector, Functions.createHashSet()); } @@ -8751,11 +8867,12 @@ public final Flowable distinct(Function keySelector) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable distinct(Function keySelector, - Supplier> collectionSupplier) { + @NonNull + public final Flowable distinct(@NonNull Function keySelector, + @NonNull Supplier> collectionSupplier) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); - return RxJavaPlugins.onAssembly(new FlowableDistinct(this, keySelector, collectionSupplier)); + return RxJavaPlugins.onAssembly(new FlowableDistinct<>(this, keySelector, collectionSupplier)); } /** @@ -8795,6 +8912,7 @@ public final Flowable distinct(Function keySelector, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable distinctUntilChanged() { return distinctUntilChanged(Functions.identity()); } @@ -8841,9 +8959,10 @@ public final Flowable distinctUntilChanged() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable distinctUntilChanged(Function keySelector) { + @NonNull + public final Flowable distinctUntilChanged(@NonNull Function keySelector) { Objects.requireNonNull(keySelector, "keySelector is null"); - return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged(this, keySelector, ObjectHelper.equalsPredicate())); + return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged<>(this, keySelector, ObjectHelper.equalsPredicate())); } /** @@ -8879,9 +8998,10 @@ public final Flowable distinctUntilChanged(Function keySele @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable distinctUntilChanged(BiPredicate comparer) { + @NonNull + public final Flowable distinctUntilChanged(@NonNull BiPredicate comparer) { Objects.requireNonNull(comparer, "comparer is null"); - return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged(this, Functions.identity(), comparer)); + return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged<>(this, Functions.identity(), comparer)); } /** @@ -8909,9 +9029,10 @@ public final Flowable distinctUntilChanged(BiPredicate @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doFinally(Action onFinally) { + @NonNull + public final Flowable doFinally(@NonNull Action onFinally) { Objects.requireNonNull(onFinally, "onFinally is null"); - return RxJavaPlugins.onAssembly(new FlowableDoFinally(this, onFinally)); + return RxJavaPlugins.onAssembly(new FlowableDoFinally<>(this, onFinally)); } /** @@ -8936,9 +9057,10 @@ public final Flowable doFinally(Action onFinally) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doAfterNext(Consumer onAfterNext) { + @NonNull + public final Flowable doAfterNext(@NonNull Consumer onAfterNext) { Objects.requireNonNull(onAfterNext, "onAfterNext is null"); - return RxJavaPlugins.onAssembly(new FlowableDoAfterNext(this, onAfterNext)); + return RxJavaPlugins.onAssembly(new FlowableDoAfterNext<>(this, onAfterNext)); } /** @@ -8964,7 +9086,8 @@ public final Flowable doAfterNext(Consumer onAfterNext) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doAfterTerminate(Action onAfterTerminate) { + @NonNull + public final Flowable doAfterTerminate(@NonNull Action onAfterTerminate) { return doOnEach(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, onAfterTerminate); } @@ -8997,7 +9120,8 @@ public final Flowable doAfterTerminate(Action onAfterTerminate) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnCancel(Action onCancel) { + @NonNull + public final Flowable doOnCancel(@NonNull Action onCancel) { return doOnLifecycle(Functions.emptyConsumer(), Functions.EMPTY_LONG_CONSUMER, onCancel); } @@ -9021,7 +9145,8 @@ public final Flowable doOnCancel(Action onCancel) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnComplete(Action onComplete) { + @NonNull + public final Flowable doOnComplete(@NonNull Action onComplete) { return doOnEach(Functions.emptyConsumer(), Functions.emptyConsumer(), onComplete, Functions.EMPTY_ACTION); } @@ -9046,13 +9171,13 @@ public final Flowable doOnComplete(Action onComplete) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - private Flowable doOnEach(Consumer onNext, Consumer onError, + private Flowable doOnEach(@NonNull Consumer onNext, @NonNull Consumer onError, Action onComplete, Action onAfterTerminate) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); - return RxJavaPlugins.onAssembly(new FlowableDoOnEach(this, onNext, onError, onComplete, onAfterTerminate)); + return RxJavaPlugins.onAssembly(new FlowableDoOnEach<>(this, onNext, onError, onComplete, onAfterTerminate)); } /** @@ -9076,7 +9201,7 @@ private Flowable doOnEach(Consumer onNext, Consumer doOnEach(final Consumer> onNotification) { + public final Flowable doOnEach(@NonNull Consumer<@NonNull ? super Notification> onNotification) { Objects.requireNonNull(onNotification, "onNotification is null"); return doOnEach( Functions.notificationOnNext(onNotification), @@ -9113,7 +9238,7 @@ public final Flowable doOnEach(final Consumer> onNoti @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnEach(final Subscriber subscriber) { + public final Flowable doOnEach(@NonNull Subscriber subscriber) { Objects.requireNonNull(subscriber, "subscriber is null"); return doOnEach( FlowableInternalHelper.subscriberOnNext(subscriber), @@ -9145,7 +9270,8 @@ public final Flowable doOnEach(final Subscriber subscriber) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnError(Consumer onError) { + @NonNull + public final Flowable doOnError(@NonNull Consumer onError) { return doOnEach(Functions.emptyConsumer(), onError, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); } @@ -9176,12 +9302,12 @@ public final Flowable doOnError(Consumer onError) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnLifecycle(final Consumer onSubscribe, - final LongConsumer onRequest, final Action onCancel) { + public final Flowable doOnLifecycle(@NonNull Consumer onSubscribe, + @NonNull LongConsumer onRequest, @NonNull Action onCancel) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); Objects.requireNonNull(onRequest, "onRequest is null"); Objects.requireNonNull(onCancel, "onCancel is null"); - return RxJavaPlugins.onAssembly(new FlowableDoOnLifecycle(this, onSubscribe, onRequest, onCancel)); + return RxJavaPlugins.onAssembly(new FlowableDoOnLifecycle<>(this, onSubscribe, onRequest, onCancel)); } /** @@ -9204,7 +9330,8 @@ public final Flowable doOnLifecycle(final Consumer onSu @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnNext(Consumer onNext) { + @NonNull + public final Flowable doOnNext(@NonNull Consumer onNext) { return doOnEach(onNext, Functions.emptyConsumer(), Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); } @@ -9234,7 +9361,8 @@ public final Flowable doOnNext(Consumer onNext) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnRequest(LongConsumer onRequest) { + @NonNull + public final Flowable doOnRequest(@NonNull LongConsumer onRequest) { return doOnLifecycle(Functions.emptyConsumer(), onRequest, Functions.EMPTY_ACTION); } @@ -9261,7 +9389,8 @@ public final Flowable doOnRequest(LongConsumer onRequest) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnSubscribe(Consumer onSubscribe) { + @NonNull + public final Flowable doOnSubscribe(@NonNull Consumer onSubscribe) { return doOnLifecycle(onSubscribe, Functions.EMPTY_LONG_CONSUMER, Functions.EMPTY_ACTION); } @@ -9290,7 +9419,8 @@ public final Flowable doOnSubscribe(Consumer onSubscrib @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable doOnTerminate(final Action onTerminate) { + @NonNull + public final Flowable doOnTerminate(@NonNull Action onTerminate) { return doOnEach(Functions.emptyConsumer(), Functions.actionConsumer(onTerminate), onTerminate, Functions.EMPTY_ACTION); } @@ -9316,11 +9446,12 @@ public final Flowable doOnTerminate(final Action onTerminate) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe elementAt(long index) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } - return RxJavaPlugins.onAssembly(new FlowableElementAtMaybe(this, index)); + return RxJavaPlugins.onAssembly(new FlowableElementAtMaybe<>(this, index)); } /** @@ -9349,12 +9480,12 @@ public final Maybe elementAt(long index) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Single elementAt(long index, T defaultItem) { + public final Single elementAt(long index, @NonNull T defaultItem) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } Objects.requireNonNull(defaultItem, "defaultItem is null"); - return RxJavaPlugins.onAssembly(new FlowableElementAtSingle(this, index, defaultItem)); + return RxJavaPlugins.onAssembly(new FlowableElementAtSingle<>(this, index, defaultItem)); } /** @@ -9380,11 +9511,12 @@ public final Single elementAt(long index, T defaultItem) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single elementAtOrError(long index) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } - return RxJavaPlugins.onAssembly(new FlowableElementAtSingle(this, index, null)); + return RxJavaPlugins.onAssembly(new FlowableElementAtSingle<>(this, index, null)); } /** @@ -9410,9 +9542,9 @@ public final Single elementAtOrError(long index) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable filter(Predicate predicate) { + public final Flowable filter(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new FlowableFilter(this, predicate)); + return RxJavaPlugins.onAssembly(new FlowableFilter<>(this, predicate)); } /** @@ -9433,6 +9565,7 @@ public final Flowable filter(Predicate predicate) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe firstElement() { return elementAt(0); } @@ -9458,7 +9591,8 @@ public final Maybe firstElement() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Single first(T defaultItem) { + @NonNull + public final Single first(@NonNull T defaultItem) { return elementAt(0, defaultItem); } @@ -9480,6 +9614,7 @@ public final Single first(T defaultItem) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) // take may trigger UNBOUNDED_IN @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single firstOrError() { return elementAtOrError(0); } @@ -9512,7 +9647,8 @@ public final Single firstOrError() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable flatMap(Function> mapper) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper) { return flatMap(mapper, false, bufferSize(), bufferSize()); } @@ -9547,7 +9683,8 @@ public final Flowable flatMap(Function Flowable flatMap(Function> mapper, boolean delayErrors) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper, boolean delayErrors) { return flatMap(mapper, delayErrors, bufferSize(), bufferSize()); } @@ -9583,7 +9720,8 @@ public final Flowable flatMap(Function Flowable flatMap(Function> mapper, int maxConcurrency) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper, int maxConcurrency) { return flatMap(mapper, false, maxConcurrency, bufferSize()); } @@ -9622,7 +9760,8 @@ public final Flowable flatMap(Function Flowable flatMap(Function> mapper, boolean delayErrors, int maxConcurrency) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper, boolean delayErrors, int maxConcurrency) { return flatMap(mapper, delayErrors, maxConcurrency, bufferSize()); } @@ -9664,7 +9803,7 @@ public final Flowable flatMap(Function Flowable flatMap(Function> mapper, + public final Flowable flatMap(@NonNull Function> mapper, boolean delayErrors, int maxConcurrency, int bufferSize) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); @@ -9677,7 +9816,7 @@ public final Flowable flatMap(Function(this, mapper, delayErrors, maxConcurrency, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowableFlatMap<>(this, mapper, delayErrors, maxConcurrency, bufferSize)); } /** @@ -9714,13 +9853,13 @@ public final Flowable flatMap(Function Flowable flatMap( - Function> onNextMapper, - Function> onErrorMapper, - Supplier> onCompleteSupplier) { + @NonNull Function> onNextMapper, + @NonNull Function> onErrorMapper, + @NonNull Supplier> onCompleteSupplier) { Objects.requireNonNull(onNextMapper, "onNextMapper is null"); Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); - return merge(new FlowableMapNotification>(this, onNextMapper, onErrorMapper, onCompleteSupplier)); + return merge(new FlowableMapNotification<>(this, onNextMapper, onErrorMapper, onCompleteSupplier)); } /** @@ -9761,14 +9900,14 @@ public final Flowable flatMap( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable flatMap( - Function> onNextMapper, - Function> onErrorMapper, - Supplier> onCompleteSupplier, + @NonNull Function> onNextMapper, + @NonNull Function> onErrorMapper, + @NonNull Supplier> onCompleteSupplier, int maxConcurrency) { Objects.requireNonNull(onNextMapper, "onNextMapper is null"); Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); - return merge(new FlowableMapNotification>( + return merge(new FlowableMapNotification<>( this, onNextMapper, onErrorMapper, onCompleteSupplier), maxConcurrency); } @@ -9803,8 +9942,9 @@ public final Flowable flatMap( @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable flatMap(Function> mapper, - BiFunction combiner) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner) { return flatMap(mapper, combiner, false, bufferSize(), bufferSize()); } @@ -9842,8 +9982,9 @@ public final Flowable flatMap(Function Flowable flatMap(Function> mapper, - BiFunction combiner, boolean delayErrors) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, boolean delayErrors) { return flatMap(mapper, combiner, delayErrors, bufferSize(), bufferSize()); } @@ -9885,8 +10026,9 @@ public final Flowable flatMap(Function Flowable flatMap(Function> mapper, - BiFunction combiner, boolean delayErrors, int maxConcurrency) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, boolean delayErrors, int maxConcurrency) { return flatMap(mapper, combiner, delayErrors, maxConcurrency, bufferSize()); } @@ -9931,8 +10073,8 @@ public final Flowable flatMap(Function Flowable flatMap(final Function> mapper, - final BiFunction combiner, boolean delayErrors, int maxConcurrency, int bufferSize) { + public final Flowable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, boolean delayErrors, int maxConcurrency, int bufferSize) { Objects.requireNonNull(mapper, "mapper is null"); Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); @@ -9975,8 +10117,9 @@ public final Flowable flatMap(final Function Flowable flatMap(Function> mapper, - BiFunction combiner, int maxConcurrency) { + @NonNull + public final Flowable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, int maxConcurrency) { return flatMap(mapper, combiner, false, maxConcurrency, bufferSize()); } @@ -9995,7 +10138,8 @@ public final Flowable flatMap(Function mapper) { + @NonNull + public final Completable flatMapCompletable(@NonNull Function mapper) { return flatMapCompletable(mapper, false, Integer.MAX_VALUE); } @@ -10020,10 +10164,10 @@ public final Completable flatMapCompletable(Function mapper, boolean delayErrors, int maxConcurrency) { + public final Completable flatMapCompletable(@NonNull Function mapper, boolean delayErrors, int maxConcurrency) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); - return RxJavaPlugins.onAssembly(new FlowableFlatMapCompletableCompletable(this, mapper, delayErrors, maxConcurrency)); + return RxJavaPlugins.onAssembly(new FlowableFlatMapCompletableCompletable<>(this, mapper, delayErrors, maxConcurrency)); } /** @@ -10052,7 +10196,8 @@ public final Completable flatMapCompletable(Function Flowable flatMapIterable(final Function> mapper) { + @NonNull + public final Flowable flatMapIterable(@NonNull Function> mapper) { return flatMapIterable(mapper, bufferSize()); } @@ -10085,10 +10230,10 @@ public final Flowable flatMapIterable(final Function Flowable flatMapIterable(final Function> mapper, int bufferSize) { + public final Flowable flatMapIterable(@NonNull Function> mapper, int bufferSize) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new FlowableFlattenIterable(this, mapper, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowableFlattenIterable<>(this, mapper, bufferSize)); } /** @@ -10122,8 +10267,8 @@ public final Flowable flatMapIterable(final Function Flowable flatMapIterable(final Function> mapper, - final BiFunction resultSelector) { + public final Flowable flatMapIterable(@NonNull Function> mapper, + @NonNull BiFunction resultSelector) { Objects.requireNonNull(mapper, "mapper is null"); Objects.requireNonNull(resultSelector, "resultSelector is null"); return flatMap(FlowableInternalHelper.flatMapIntoIterable(mapper), resultSelector, false, bufferSize(), bufferSize()); @@ -10165,8 +10310,8 @@ public final Flowable flatMapIterable(final Function Flowable flatMapIterable(final Function> mapper, - final BiFunction resultSelector, int prefetch) { + public final Flowable flatMapIterable(@NonNull Function> mapper, + @NonNull BiFunction resultSelector, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); Objects.requireNonNull(resultSelector, "resultSelector is null"); return flatMap(FlowableInternalHelper.flatMapIntoIterable(mapper), resultSelector, false, bufferSize(), prefetch); @@ -10188,7 +10333,8 @@ public final Flowable flatMapIterable(final Function Flowable flatMapMaybe(Function> mapper) { + @NonNull + public final Flowable flatMapMaybe(@NonNull Function> mapper) { return flatMapMaybe(mapper, false, Integer.MAX_VALUE); } @@ -10215,10 +10361,10 @@ public final Flowable flatMapMaybe(Function Flowable flatMapMaybe(Function> mapper, boolean delayErrors, int maxConcurrency) { + public final Flowable flatMapMaybe(@NonNull Function> mapper, boolean delayErrors, int maxConcurrency) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); - return RxJavaPlugins.onAssembly(new FlowableFlatMapMaybe(this, mapper, delayErrors, maxConcurrency)); + return RxJavaPlugins.onAssembly(new FlowableFlatMapMaybe<>(this, mapper, delayErrors, maxConcurrency)); } /** @@ -10237,7 +10383,8 @@ public final Flowable flatMapMaybe(Function Flowable flatMapSingle(Function> mapper) { + @NonNull + public final Flowable flatMapSingle(@NonNull Function> mapper) { return flatMapSingle(mapper, false, Integer.MAX_VALUE); } @@ -10264,10 +10411,10 @@ public final Flowable flatMapSingle(Function Flowable flatMapSingle(Function> mapper, boolean delayErrors, int maxConcurrency) { + public final Flowable flatMapSingle(@NonNull Function> mapper, boolean delayErrors, int maxConcurrency) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); - return RxJavaPlugins.onAssembly(new FlowableFlatMapSingle(this, mapper, delayErrors, maxConcurrency)); + return RxJavaPlugins.onAssembly(new FlowableFlatMapSingle<>(this, mapper, delayErrors, maxConcurrency)); } /** @@ -10293,7 +10440,8 @@ public final Flowable flatMapSingle(Function onNext) { + @NonNull + public final Disposable forEach(@NonNull Consumer onNext) { return subscribe(onNext); } @@ -10323,7 +10471,8 @@ public final Disposable forEach(Consumer onNext) { @CheckReturnValue @BackpressureSupport(BackpressureKind.NONE) @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable forEachWhile(Predicate onNext) { + @NonNull + public final Disposable forEachWhile(@NonNull Predicate onNext) { return forEachWhile(onNext, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -10352,7 +10501,8 @@ public final Disposable forEachWhile(Predicate onNext) { @CheckReturnValue @BackpressureSupport(BackpressureKind.NONE) @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable forEachWhile(Predicate onNext, Consumer onError) { + @NonNull + public final Disposable forEachWhile(@NonNull Predicate onNext, @NonNull Consumer onError) { return forEachWhile(onNext, onError, Functions.EMPTY_ACTION); } @@ -10385,13 +10535,13 @@ public final Disposable forEachWhile(Predicate onNext, Consumer onNext, final Consumer onError, - final Action onComplete) { + public final Disposable forEachWhile(@NonNull Predicate onNext, @NonNull Consumer onError, + @NonNull Action onComplete) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); - ForEachWhileSubscriber s = new ForEachWhileSubscriber(onNext, onError, onComplete); + ForEachWhileSubscriber s = new ForEachWhileSubscriber<>(onNext, onError, onComplete); subscribe(s); return s; } @@ -10450,7 +10600,8 @@ public final Disposable forEachWhile(final Predicate onNext, final Co @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable> groupBy(Function keySelector) { + @NonNull + public final Flowable> groupBy(@NonNull Function keySelector) { return groupBy(keySelector, Functions.identity(), false, bufferSize()); } @@ -10509,7 +10660,8 @@ public final Flowable> groupBy(Function Flowable> groupBy(Function keySelector, boolean delayError) { + @NonNull + public final Flowable> groupBy(@NonNull Function keySelector, boolean delayError) { return groupBy(keySelector, Functions.identity(), delayError, bufferSize()); } @@ -10572,8 +10724,9 @@ public final Flowable> groupBy(Function Flowable> groupBy(Function keySelector, - Function valueSelector) { + @NonNull + public final Flowable> groupBy(@NonNull Function keySelector, + @NonNull Function valueSelector) { return groupBy(keySelector, valueSelector, false, bufferSize()); } @@ -10637,8 +10790,9 @@ public final Flowable> groupBy(Function Flowable> groupBy(Function keySelector, - Function valueSelector, boolean delayError) { + @NonNull + public final Flowable> groupBy(@NonNull Function keySelector, + @NonNull Function valueSelector, boolean delayError) { return groupBy(keySelector, valueSelector, delayError, bufferSize()); } @@ -10704,8 +10858,8 @@ public final Flowable> groupBy(Function Flowable> groupBy(Function keySelector, - Function valueSelector, + public final Flowable> groupBy(@NonNull Function keySelector, + @NonNull Function valueSelector, boolean delayError, int bufferSize) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); @@ -10825,10 +10979,10 @@ public final Flowable> groupBy(Function Flowable> groupBy(Function keySelector, - Function valueSelector, + public final Flowable> groupBy(@NonNull Function keySelector, + @NonNull Function valueSelector, boolean delayError, int bufferSize, - Function, ? extends Map> evictingMapFactory) { + @NonNull Function, ? extends Map> evictingMapFactory) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -10876,10 +11030,10 @@ public final Flowable> groupBy(Function Flowable groupJoin( - Publisher other, - Function> leftEnd, - Function> rightEnd, - BiFunction, ? extends R> resultSelector) { + @NonNull Publisher other, + @NonNull Function> leftEnd, + @NonNull Function> rightEnd, + @NonNull BiFunction, ? extends R> resultSelector) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(leftEnd, "leftEnd is null"); Objects.requireNonNull(rightEnd, "rightEnd is null"); @@ -10907,8 +11061,9 @@ public final Flowable groupJoin( @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable hide() { - return RxJavaPlugins.onAssembly(new FlowableHide(this)); + return RxJavaPlugins.onAssembly(new FlowableHide<>(this)); } /** @@ -10930,8 +11085,9 @@ public final Flowable hide() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable ignoreElements() { - return RxJavaPlugins.onAssembly(new FlowableIgnoreElementsCompletable(this)); + return RxJavaPlugins.onAssembly(new FlowableIgnoreElementsCompletable<>(this)); } /** @@ -10955,6 +11111,7 @@ public final Completable ignoreElements() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single isEmpty() { return all(Functions.alwaysFalse()); } @@ -10998,10 +11155,10 @@ public final Single isEmpty() { @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable join( - Publisher other, - Function> leftEnd, - Function> rightEnd, - BiFunction resultSelector) { + @NonNull Publisher other, + @NonNull Function> leftEnd, + @NonNull Function> rightEnd, + @NonNull BiFunction resultSelector) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(leftEnd, "leftEnd is null"); Objects.requireNonNull(rightEnd, "rightEnd is null"); @@ -11029,8 +11186,9 @@ public final Flowable join( @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe lastElement() { - return RxJavaPlugins.onAssembly(new FlowableLastMaybe(this)); + return RxJavaPlugins.onAssembly(new FlowableLastMaybe<>(this)); } /** @@ -11055,9 +11213,9 @@ public final Maybe lastElement() { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single last(T defaultItem) { + public final Single last(@NonNull T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem"); - return RxJavaPlugins.onAssembly(new FlowableLastSingle(this, defaultItem)); + return RxJavaPlugins.onAssembly(new FlowableLastSingle<>(this, defaultItem)); } /** @@ -11079,8 +11237,9 @@ public final Single last(T defaultItem) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single lastOrError() { - return RxJavaPlugins.onAssembly(new FlowableLastSingle(this, null)); + return RxJavaPlugins.onAssembly(new FlowableLastSingle<>(this, null)); } /** @@ -11233,7 +11392,7 @@ public final Single lastOrError() { @NonNull @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable lift(FlowableOperator lifter) { + public final Flowable lift(@NonNull FlowableOperator lifter) { Objects.requireNonNull(lifter, "lifter is null"); return RxJavaPlugins.onAssembly(new FlowableLift(this, lifter)); } @@ -11289,8 +11448,9 @@ public final Flowable lift(FlowableOperator lifte @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> materialize() { - return RxJavaPlugins.onAssembly(new FlowableMaterialize(this)); + return RxJavaPlugins.onAssembly(new FlowableMaterialize<>(this)); } /** @@ -11317,7 +11477,7 @@ public final Flowable> materialize() { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable mergeWith(Publisher other) { + public final Flowable mergeWith(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return merge(this, other); } @@ -11347,7 +11507,7 @@ public final Flowable mergeWith(Publisher other) { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableMergeWithSingle(this, other)); + return RxJavaPlugins.onAssembly(new FlowableMergeWithSingle<>(this, other)); } /** @@ -11376,7 +11536,7 @@ public final Flowable mergeWith(@NonNull SingleSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableMergeWithMaybe(this, other)); + return RxJavaPlugins.onAssembly(new FlowableMergeWithMaybe<>(this, other)); } /** @@ -11402,7 +11562,7 @@ public final Flowable mergeWith(@NonNull MaybeSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable mergeWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableMergeWithCompletable(this, other)); + return RxJavaPlugins.onAssembly(new FlowableMergeWithCompletable<>(this, other)); } /** @@ -11451,7 +11611,8 @@ public final Flowable mergeWith(@NonNull CompletableSource other) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable observeOn(Scheduler scheduler) { + @NonNull + public final Flowable observeOn(@NonNull Scheduler scheduler) { return observeOn(scheduler, false, bufferSize()); } @@ -11502,7 +11663,8 @@ public final Flowable observeOn(Scheduler scheduler) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable observeOn(Scheduler scheduler, boolean delayError) { + @NonNull + public final Flowable observeOn(@NonNull Scheduler scheduler, boolean delayError) { return observeOn(scheduler, delayError, bufferSize()); } @@ -11555,10 +11717,10 @@ public final Flowable observeOn(Scheduler scheduler, boolean delayError) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable observeOn(Scheduler scheduler, boolean delayError, int bufferSize) { + public final Flowable observeOn(@NonNull Scheduler scheduler, boolean delayError, int bufferSize) { Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new FlowableObserveOn(this, scheduler, delayError, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowableObserveOn<>(this, scheduler, delayError, bufferSize)); } /** @@ -11583,7 +11745,7 @@ public final Flowable observeOn(Scheduler scheduler, boolean delayError, int @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable ofType(final Class clazz) { + public final Flowable ofType(@NonNull Class clazz) { Objects.requireNonNull(clazz, "clazz is null"); return filter(Functions.isInstanceOf(clazz)).cast(clazz); } @@ -11607,6 +11769,7 @@ public final Flowable ofType(final Class clazz) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onBackpressureBuffer() { return onBackpressureBuffer(bufferSize(), false, true); } @@ -11633,6 +11796,7 @@ public final Flowable onBackpressureBuffer() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onBackpressureBuffer(boolean delayError) { return onBackpressureBuffer(bufferSize(), delayError, true); } @@ -11660,6 +11824,7 @@ public final Flowable onBackpressureBuffer(boolean delayError) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onBackpressureBuffer(int capacity) { return onBackpressureBuffer(capacity, false, false); } @@ -11691,6 +11856,7 @@ public final Flowable onBackpressureBuffer(int capacity) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onBackpressureBuffer(int capacity, boolean delayError) { return onBackpressureBuffer(capacity, delayError, false); } @@ -11724,9 +11890,10 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError) @CheckReturnValue @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onBackpressureBuffer(int capacity, boolean delayError, boolean unbounded) { ObjectHelper.verifyPositive(capacity, "capacity"); - return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBuffer(this, capacity, unbounded, delayError, Functions.EMPTY_ACTION)); + return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBuffer<>(this, capacity, unbounded, delayError, Functions.EMPTY_ACTION)); } /** @@ -11761,10 +11928,10 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError, @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable onBackpressureBuffer(int capacity, boolean delayError, boolean unbounded, - Action onOverflow) { + @NonNull Action onOverflow) { Objects.requireNonNull(onOverflow, "onOverflow is null"); ObjectHelper.verifyPositive(capacity, "capacity"); - return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBuffer(this, capacity, unbounded, delayError, onOverflow)); + return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBuffer<>(this, capacity, unbounded, delayError, onOverflow)); } /** @@ -11791,7 +11958,8 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError, @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable onBackpressureBuffer(int capacity, Action onOverflow) { + @NonNull + public final Flowable onBackpressureBuffer(int capacity, @NonNull Action onOverflow) { return onBackpressureBuffer(capacity, false, false, onOverflow); } @@ -11832,10 +12000,10 @@ public final Flowable onBackpressureBuffer(int capacity, Action onOverflow) { @NonNull @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable onBackpressureBuffer(long capacity, Action onOverflow, BackpressureOverflowStrategy overflowStrategy) { + public final Flowable onBackpressureBuffer(long capacity, @NonNull Action onOverflow, @NonNull BackpressureOverflowStrategy overflowStrategy) { Objects.requireNonNull(overflowStrategy, "overflowStrategy is null"); ObjectHelper.verifyPositive(capacity, "capacity"); - return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBufferStrategy(this, capacity, onOverflow, overflowStrategy)); + return RxJavaPlugins.onAssembly(new FlowableOnBackpressureBufferStrategy<>(this, capacity, onOverflow, overflowStrategy)); } /** @@ -11860,8 +12028,9 @@ public final Flowable onBackpressureBuffer(long capacity, Action onOverflow, @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onBackpressureDrop() { - return RxJavaPlugins.onAssembly(new FlowableOnBackpressureDrop(this)); + return RxJavaPlugins.onAssembly(new FlowableOnBackpressureDrop<>(this)); } /** @@ -11889,9 +12058,9 @@ public final Flowable onBackpressureDrop() { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable onBackpressureDrop(Consumer onDrop) { + public final Flowable onBackpressureDrop(@NonNull Consumer onDrop) { Objects.requireNonNull(onDrop, "onDrop is null"); - return RxJavaPlugins.onAssembly(new FlowableOnBackpressureDrop(this, onDrop)); + return RxJavaPlugins.onAssembly(new FlowableOnBackpressureDrop<>(this, onDrop)); } /** @@ -11922,8 +12091,9 @@ public final Flowable onBackpressureDrop(Consumer onDrop) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onBackpressureLatest() { - return RxJavaPlugins.onAssembly(new FlowableOnBackpressureLatest(this)); + return RxJavaPlugins.onAssembly(new FlowableOnBackpressureLatest<>(this)); } /** @@ -11965,9 +12135,9 @@ public final Flowable onBackpressureLatest() { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable onErrorResumeNext(Function> resumeFunction) { + public final Flowable onErrorResumeNext(@NonNull Function> resumeFunction) { Objects.requireNonNull(resumeFunction, "resumeFunction is null"); - return RxJavaPlugins.onAssembly(new FlowableOnErrorNext(this, resumeFunction)); + return RxJavaPlugins.onAssembly(new FlowableOnErrorNext<>(this, resumeFunction)); } /** @@ -12009,7 +12179,7 @@ public final Flowable onErrorResumeNext(Function onErrorResumeWith(final Publisher next) { + public final Flowable onErrorResumeWith(@NonNull Publisher next) { Objects.requireNonNull(next, "next is null"); return onErrorResumeNext(Functions.justFunction(next)); } @@ -12049,9 +12219,9 @@ public final Flowable onErrorResumeWith(final Publisher next) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable onErrorReturn(Function valueSupplier) { + public final Flowable onErrorReturn(@NonNull Function valueSupplier) { Objects.requireNonNull(valueSupplier, "valueSupplier is null"); - return RxJavaPlugins.onAssembly(new FlowableOnErrorReturn(this, valueSupplier)); + return RxJavaPlugins.onAssembly(new FlowableOnErrorReturn<>(this, valueSupplier)); } /** @@ -12089,7 +12259,7 @@ public final Flowable onErrorReturn(Function @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable onErrorReturnItem(final T item) { + public final Flowable onErrorReturnItem(@NonNull T item) { Objects.requireNonNull(item, "item is null"); return onErrorReturn(Functions.justFunction(item)); } @@ -12111,8 +12281,9 @@ public final Flowable onErrorReturnItem(final T item) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable onTerminateDetach() { - return RxJavaPlugins.onAssembly(new FlowableDetach(this)); + return RxJavaPlugins.onAssembly(new FlowableDetach<>(this)); } /** @@ -12140,6 +12311,7 @@ public final Flowable onTerminateDetach() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue + @NonNull public final ParallelFlowable parallel() { return ParallelFlowable.from(this); } @@ -12170,6 +12342,7 @@ public final ParallelFlowable parallel() { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue + @NonNull public final ParallelFlowable parallel(int parallelism) { ObjectHelper.verifyPositive(parallelism, "parallelism"); return ParallelFlowable.from(this, parallelism); @@ -12203,6 +12376,7 @@ public final ParallelFlowable parallel(int parallelism) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue + @NonNull public final ParallelFlowable parallel(int parallelism, int prefetch) { ObjectHelper.verifyPositive(parallelism, "parallelism"); ObjectHelper.verifyPositive(prefetch, "prefetch"); @@ -12231,6 +12405,7 @@ public final ParallelFlowable parallel(int parallelism, int prefetch) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final ConnectableFlowable publish() { return publish(bufferSize()); } @@ -12263,7 +12438,8 @@ public final ConnectableFlowable publish() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable publish(Function, ? extends Publisher> selector) { + @NonNull + public final Flowable publish(@NonNull Function, ? extends Publisher> selector) { return publish(selector, bufferSize()); } @@ -12298,10 +12474,10 @@ public final Flowable publish(Function, ? extends Pub @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable publish(Function, ? extends Publisher> selector, int prefetch) { + public final Flowable publish(@NonNull Function, ? extends Publisher> selector, int prefetch) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new FlowablePublishMulticast(this, selector, prefetch, false)); + return RxJavaPlugins.onAssembly(new FlowablePublishMulticast<>(this, selector, prefetch, false)); } /** @@ -12328,9 +12504,10 @@ public final Flowable publish(Function, ? extends Pub @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final ConnectableFlowable publish(int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new FlowablePublish(this, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowablePublish<>(this, bufferSize)); } /** @@ -12354,6 +12531,7 @@ public final ConnectableFlowable publish(int bufferSize) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable rebatchRequests(int n) { return observeOn(ImmediateThinScheduler.INSTANCE, true, n); } @@ -12393,9 +12571,9 @@ public final Flowable rebatchRequests(int n) { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe reduce(BiFunction reducer) { + public final Maybe reduce(@NonNull BiFunction reducer) { Objects.requireNonNull(reducer, "reducer is null"); - return RxJavaPlugins.onAssembly(new FlowableReduceMaybe(this, reducer)); + return RxJavaPlugins.onAssembly(new FlowableReduceMaybe<>(this, reducer)); } /** @@ -12455,10 +12633,10 @@ public final Maybe reduce(BiFunction reducer) { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single reduce(R seed, BiFunction reducer) { + public final <@NonNull R> Single reduce(R seed, @NonNull BiFunction reducer) { Objects.requireNonNull(seed, "seed is null"); Objects.requireNonNull(reducer, "reducer is null"); - return RxJavaPlugins.onAssembly(new FlowableReduceSeedSingle(this, seed, reducer)); + return RxJavaPlugins.onAssembly(new FlowableReduceSeedSingle<>(this, seed, reducer)); } /** @@ -12500,10 +12678,10 @@ public final Single reduce(R seed, BiFunction reducer) { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single reduceWith(Supplier seedSupplier, BiFunction reducer) { + public final <@NonNull R> Single reduceWith(@NonNull Supplier seedSupplier, @NonNull BiFunction reducer) { Objects.requireNonNull(seedSupplier, "seedSupplier is null"); Objects.requireNonNull(reducer, "reducer is null"); - return RxJavaPlugins.onAssembly(new FlowableReduceWithSingle(this, seedSupplier, reducer)); + return RxJavaPlugins.onAssembly(new FlowableReduceWithSingle<>(this, seedSupplier, reducer)); } /** @@ -12524,6 +12702,7 @@ public final Single reduceWith(Supplier seedSupplier, BiFunction repeat() { return repeat(Long.MAX_VALUE); } @@ -12553,6 +12732,7 @@ public final Flowable repeat() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable repeat(long times) { if (times < 0) { throw new IllegalArgumentException("times >= 0 required but it was " + times); @@ -12560,7 +12740,7 @@ public final Flowable repeat(long times) { if (times == 0) { return empty(); } - return RxJavaPlugins.onAssembly(new FlowableRepeat(this, times)); + return RxJavaPlugins.onAssembly(new FlowableRepeat<>(this, times)); } /** @@ -12588,9 +12768,9 @@ public final Flowable repeat(long times) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable repeatUntil(BooleanSupplier stop) { + public final Flowable repeatUntil(@NonNull BooleanSupplier stop) { Objects.requireNonNull(stop, "stop is null"); - return RxJavaPlugins.onAssembly(new FlowableRepeatUntil(this, stop)); + return RxJavaPlugins.onAssembly(new FlowableRepeatUntil<>(this, stop)); } /** @@ -12619,9 +12799,9 @@ public final Flowable repeatUntil(BooleanSupplier stop) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable repeatWhen(final Function, ? extends Publisher> handler) { + public final Flowable repeatWhen(@NonNull Function, ? extends Publisher> handler) { Objects.requireNonNull(handler, "handler is null"); - return RxJavaPlugins.onAssembly(new FlowableRepeatWhen(this, handler)); + return RxJavaPlugins.onAssembly(new FlowableRepeatWhen<>(this, handler)); } /** @@ -12647,6 +12827,7 @@ public final Flowable repeatWhen(final Function, ? e @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final ConnectableFlowable replay() { return FlowableReplay.createFrom(this); } @@ -12678,7 +12859,7 @@ public final ConnectableFlowable replay() { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable replay(Function, ? extends Publisher> selector) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector) { Objects.requireNonNull(selector, "selector is null"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this), selector); } @@ -12718,7 +12899,7 @@ public final Flowable replay(Function, ? extends Publ @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, final int bufferSize) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, bufferSize, false), selector); @@ -12761,7 +12942,7 @@ public final Flowable replay(Function, ? extends Publ @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize, boolean eagerTruncate) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, final int bufferSize, boolean eagerTruncate) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, bufferSize, eagerTruncate), selector); @@ -12805,7 +12986,8 @@ public final Flowable replay(Function, ? extends Publ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable replay(Function, ? extends Publisher> selector, int bufferSize, long time, TimeUnit unit) { + @NonNull + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, int bufferSize, long time, @NonNull TimeUnit unit) { return replay(selector, bufferSize, time, unit, Schedulers.computation()); } @@ -12853,7 +13035,7 @@ public final Flowable replay(Function, ? extends Publ @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(selector, "selector is null"); Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -12908,7 +13090,7 @@ public final Flowable replay(Function, ? extends Publ @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable replay(Function, ? extends Publisher> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { Objects.requireNonNull(selector, "selector is null"); Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -12949,7 +13131,8 @@ public final Flowable replay(Function, ? extends Publ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable replay(Function, ? extends Publisher> selector, long time, TimeUnit unit) { + @NonNull + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, long time, @NonNull TimeUnit unit) { return replay(selector, time, unit, Schedulers.computation()); } @@ -12989,7 +13172,7 @@ public final Flowable replay(Function, ? extends Publ @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable replay(Function, ? extends Publisher> selector, final long time, final TimeUnit unit, final Scheduler scheduler) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(selector, "selector is null"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -13034,7 +13217,7 @@ public final Flowable replay(Function, ? extends Publ @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable replay(Function, ? extends Publisher> selector, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { Objects.requireNonNull(selector, "selector is null"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -13072,7 +13255,8 @@ public final Flowable replay(Function, ? extends Publ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final ConnectableFlowable replay(final int bufferSize) { + @NonNull + public final ConnectableFlowable replay(int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.create(this, bufferSize, false); } @@ -13109,7 +13293,8 @@ public final ConnectableFlowable replay(final int bufferSize) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final ConnectableFlowable replay(final int bufferSize, boolean eagerTruncate) { + @NonNull + public final ConnectableFlowable replay(int bufferSize, boolean eagerTruncate) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.create(this, bufferSize, eagerTruncate); } @@ -13150,7 +13335,8 @@ public final ConnectableFlowable replay(final int bufferSize, boolean eagerTr @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final ConnectableFlowable replay(int bufferSize, long time, TimeUnit unit) { + @NonNull + public final ConnectableFlowable replay(int bufferSize, long time, @NonNull TimeUnit unit) { return replay(bufferSize, time, unit, Schedulers.computation()); } @@ -13194,7 +13380,8 @@ public final ConnectableFlowable replay(int bufferSize, long time, TimeUnit u @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableFlowable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final ConnectableFlowable replay(int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -13244,7 +13431,8 @@ public final ConnectableFlowable replay(final int bufferSize, final long time @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableFlowable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + @NonNull + public final ConnectableFlowable replay(int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -13282,7 +13470,8 @@ public final ConnectableFlowable replay(final int bufferSize, final long time @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final ConnectableFlowable replay(long time, TimeUnit unit) { + @NonNull + public final ConnectableFlowable replay(long time, @NonNull TimeUnit unit) { return replay(time, unit, Schedulers.computation()); } @@ -13319,7 +13508,8 @@ public final ConnectableFlowable replay(long time, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableFlowable replay(final long time, final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final ConnectableFlowable replay(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.create(this, time, unit, scheduler, false); @@ -13360,7 +13550,8 @@ public final ConnectableFlowable replay(final long time, final TimeUnit unit, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableFlowable replay(final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + @NonNull + public final ConnectableFlowable replay(final long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.create(this, time, unit, scheduler, eagerTruncate); @@ -13393,6 +13584,7 @@ public final ConnectableFlowable replay(final long time, final TimeUnit unit, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable retry() { return retry(Long.MAX_VALUE, Functions.alwaysTrue()); } @@ -13421,10 +13613,10 @@ public final Flowable retry() { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable retry(BiPredicate predicate) { + public final Flowable retry(@NonNull BiPredicate<@NonNull ? super Integer, @NonNull ? super Throwable> predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new FlowableRetryBiPredicate(this, predicate)); + return RxJavaPlugins.onAssembly(new FlowableRetryBiPredicate<>(this, predicate)); } /** @@ -13457,6 +13649,7 @@ public final Flowable retry(BiPredicate p @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable retry(long count) { return retry(count, Functions.alwaysTrue()); } @@ -13479,13 +13672,13 @@ public final Flowable retry(long count) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable retry(long times, Predicate predicate) { + public final Flowable retry(long times, @NonNull Predicate<@NonNull ? super Throwable> predicate) { if (times < 0) { throw new IllegalArgumentException("times >= 0 required but it was " + times); } Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new FlowableRetryPredicate(this, times, predicate)); + return RxJavaPlugins.onAssembly(new FlowableRetryPredicate<>(this, times, predicate)); } /** @@ -13504,7 +13697,8 @@ public final Flowable retry(long times, Predicate predicat @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable retry(Predicate predicate) { + @NonNull + public final Flowable retry(@NonNull Predicate<@NonNull ? super Throwable> predicate) { return retry(Long.MAX_VALUE, predicate); } @@ -13524,7 +13718,7 @@ public final Flowable retry(Predicate predicate) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable retryUntil(final BooleanSupplier stop) { + public final Flowable retryUntil(@NonNull BooleanSupplier stop) { Objects.requireNonNull(stop, "stop is null"); return retry(Long.MAX_VALUE, Functions.predicateReverseFor(stop)); } @@ -13611,10 +13805,10 @@ public final Flowable retryUntil(final BooleanSupplier stop) { @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable retryWhen( - final Function, ? extends Publisher> handler) { + @NonNull Function, ? extends Publisher> handler) { Objects.requireNonNull(handler, "handler is null"); - return RxJavaPlugins.onAssembly(new FlowableRetryWhen(this, handler)); + return RxJavaPlugins.onAssembly(new FlowableRetryWhen<>(this, handler)); } /** @@ -13633,7 +13827,7 @@ public final Flowable retryWhen( */ @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final void safeSubscribe(Subscriber s) { + public final void safeSubscribe(@NonNull Subscriber s) { Objects.requireNonNull(s, "s is null"); if (s instanceof SafeSubscriber) { subscribe((SafeSubscriber)s); @@ -13667,7 +13861,8 @@ public final void safeSubscribe(Subscriber s) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable sample(long period, TimeUnit unit) { + @NonNull + public final Flowable sample(long period, @NonNull TimeUnit unit) { return sample(period, unit, Schedulers.computation()); } @@ -13702,7 +13897,8 @@ public final Flowable sample(long period, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable sample(long period, TimeUnit unit, boolean emitLast) { + @NonNull + public final Flowable sample(long period, @NonNull TimeUnit unit, boolean emitLast) { return sample(period, unit, Schedulers.computation(), emitLast); } @@ -13734,10 +13930,10 @@ public final Flowable sample(long period, TimeUnit unit, boolean emitLast) { @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler) { + public final Flowable sample(long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableSampleTimed(this, period, unit, scheduler, false)); + return RxJavaPlugins.onAssembly(new FlowableSampleTimed<>(this, period, unit, scheduler, false)); } /** @@ -13775,10 +13971,10 @@ public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler) @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler, boolean emitLast) { + public final Flowable sample(long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean emitLast) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableSampleTimed(this, period, unit, scheduler, emitLast)); + return RxJavaPlugins.onAssembly(new FlowableSampleTimed<>(this, period, unit, scheduler, emitLast)); } /** @@ -13807,9 +14003,9 @@ public final Flowable sample(long period, TimeUnit unit, Scheduler scheduler, @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable sample(Publisher sampler) { + public final Flowable sample(@NonNull Publisher sampler) { Objects.requireNonNull(sampler, "sampler is null"); - return RxJavaPlugins.onAssembly(new FlowableSamplePublisher(this, sampler, false)); + return RxJavaPlugins.onAssembly(new FlowableSamplePublisher<>(this, sampler, false)); } /** @@ -13845,9 +14041,9 @@ public final Flowable sample(Publisher sampler) { @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable sample(Publisher sampler, boolean emitLast) { + public final Flowable sample(@NonNull Publisher sampler, boolean emitLast) { Objects.requireNonNull(sampler, "sampler is null"); - return RxJavaPlugins.onAssembly(new FlowableSamplePublisher(this, sampler, emitLast)); + return RxJavaPlugins.onAssembly(new FlowableSamplePublisher<>(this, sampler, emitLast)); } /** @@ -13878,9 +14074,9 @@ public final Flowable sample(Publisher sampler, boolean emitLast) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable scan(BiFunction accumulator) { + public final Flowable scan(@NonNull BiFunction accumulator) { Objects.requireNonNull(accumulator, "accumulator is null"); - return RxJavaPlugins.onAssembly(new FlowableScan(this, accumulator)); + return RxJavaPlugins.onAssembly(new FlowableScan<>(this, accumulator)); } /** @@ -13932,7 +14128,7 @@ public final Flowable scan(BiFunction accumulator) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable scan(final R initialValue, BiFunction accumulator) { + public final <@NonNull R> Flowable scan(R initialValue, @NonNull BiFunction accumulator) { Objects.requireNonNull(initialValue, "initialValue is null"); return scanWith(Functions.justSupplier(initialValue), accumulator); } @@ -13972,10 +14168,10 @@ public final Flowable scan(final R initialValue, BiFunction Flowable scanWith(Supplier seedSupplier, BiFunction accumulator) { + public final <@NonNull R> Flowable scanWith(@NonNull Supplier seedSupplier, @NonNull BiFunction accumulator) { Objects.requireNonNull(seedSupplier, "seedSupplier is null"); Objects.requireNonNull(accumulator, "accumulator is null"); - return RxJavaPlugins.onAssembly(new FlowableScanSeed(this, seedSupplier, accumulator)); + return RxJavaPlugins.onAssembly(new FlowableScanSeed<>(this, seedSupplier, accumulator)); } /** @@ -14004,8 +14200,9 @@ public final Flowable scanWith(Supplier seedSupplier, BiFunction serialize() { - return RxJavaPlugins.onAssembly(new FlowableSerialized(this)); + return RxJavaPlugins.onAssembly(new FlowableSerialized<>(this)); } /** @@ -14032,6 +14229,7 @@ public final Flowable serialize() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable share() { return publish().refCount(); } @@ -14056,8 +14254,9 @@ public final Flowable share() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe singleElement() { - return RxJavaPlugins.onAssembly(new FlowableSingleMaybe(this)); + return RxJavaPlugins.onAssembly(new FlowableSingleMaybe<>(this)); } /** @@ -14084,9 +14283,9 @@ public final Maybe singleElement() { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single single(T defaultItem) { + public final Single single(@NonNull T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); - return RxJavaPlugins.onAssembly(new FlowableSingleSingle(this, defaultItem)); + return RxJavaPlugins.onAssembly(new FlowableSingleSingle<>(this, defaultItem)); } /** @@ -14110,8 +14309,9 @@ public final Single single(T defaultItem) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single singleOrError() { - return RxJavaPlugins.onAssembly(new FlowableSingleSingle(this, null)); + return RxJavaPlugins.onAssembly(new FlowableSingleSingle<>(this, null)); } /** @@ -14136,11 +14336,12 @@ public final Single singleOrError() { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable skip(long count) { if (count <= 0L) { return RxJavaPlugins.onAssembly(this); } - return RxJavaPlugins.onAssembly(new FlowableSkip(this, count)); + return RxJavaPlugins.onAssembly(new FlowableSkip<>(this, count)); } /** @@ -14168,7 +14369,8 @@ public final Flowable skip(long count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable skip(long time, TimeUnit unit) { + @NonNull + public final Flowable skip(long time, @NonNull TimeUnit unit) { return skipUntil(timer(time, unit)); } @@ -14198,7 +14400,8 @@ public final Flowable skip(long time, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable skip(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable skip(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return skipUntil(timer(time, unit, scheduler)); } @@ -14230,6 +14433,7 @@ public final Flowable skip(long time, TimeUnit unit, Scheduler scheduler) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable skipLast(int count) { if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); @@ -14237,7 +14441,7 @@ public final Flowable skipLast(int count) { if (count == 0) { return RxJavaPlugins.onAssembly(this); } - return RxJavaPlugins.onAssembly(new FlowableSkipLast(this, count)); + return RxJavaPlugins.onAssembly(new FlowableSkipLast<>(this, count)); } /** @@ -14267,7 +14471,8 @@ public final Flowable skipLast(int count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable skipLast(long time, TimeUnit unit) { + @NonNull + public final Flowable skipLast(long time, @NonNull TimeUnit unit) { return skipLast(time, unit, Schedulers.computation(), false, bufferSize()); } @@ -14301,7 +14506,8 @@ public final Flowable skipLast(long time, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable skipLast(long time, TimeUnit unit, boolean delayError) { + @NonNull + public final Flowable skipLast(long time, @NonNull TimeUnit unit, boolean delayError) { return skipLast(time, unit, Schedulers.computation(), delayError, bufferSize()); } @@ -14333,7 +14539,8 @@ public final Flowable skipLast(long time, TimeUnit unit, boolean delayError) @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable skipLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return skipLast(time, unit, scheduler, false, bufferSize()); } @@ -14368,7 +14575,8 @@ public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler) @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError) { + @NonNull + public final Flowable skipLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { return skipLast(time, unit, scheduler, delayError, bufferSize()); } @@ -14406,13 +14614,13 @@ public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler, @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { + public final Flowable skipLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError, int bufferSize) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); // the internal buffer holds pairs of (timestamp, value) so double the default buffer size int s = bufferSize << 1; - return RxJavaPlugins.onAssembly(new FlowableSkipLastTimed(this, time, unit, scheduler, s, delayError)); + return RxJavaPlugins.onAssembly(new FlowableSkipLastTimed<>(this, time, unit, scheduler, s, delayError)); } /** @@ -14440,9 +14648,9 @@ public final Flowable skipLast(long time, TimeUnit unit, Scheduler scheduler, @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable skipUntil(Publisher other) { + public final Flowable skipUntil(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableSkipUntil(this, other)); + return RxJavaPlugins.onAssembly(new FlowableSkipUntil<>(this, other)); } /** @@ -14468,9 +14676,9 @@ public final Flowable skipUntil(Publisher other) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable skipWhile(Predicate predicate) { + public final Flowable skipWhile(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new FlowableSkipWhile(this, predicate)); + return RxJavaPlugins.onAssembly(new FlowableSkipWhile<>(this, predicate)); } /** * Returns a Flowable that emits the events emitted by source Publisher, in a @@ -14496,6 +14704,7 @@ public final Flowable skipWhile(Predicate predicate) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable sorted() { return toList().toFlowable().map(Functions.listSorter(Functions.naturalComparator())).flatMapIterable(Functions.>identity()); } @@ -14524,7 +14733,7 @@ public final Flowable sorted() { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable sorted(Comparator sortFunction) { + public final Flowable sorted(@NonNull Comparator<@NonNull ? super T> sortFunction) { Objects.requireNonNull(sortFunction, "sortFunction"); return toList().toFlowable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.>identity()); } @@ -14555,7 +14764,8 @@ public final Flowable sorted(Comparator sortFunction) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable startWithIterable(Iterable items) { + @NonNull + public final Flowable startWithIterable(@NonNull Iterable items) { return concatArray(fromIterable(items), this); } @@ -14583,7 +14793,7 @@ public final Flowable startWithIterable(Iterable items) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable startWith(Publisher other) { + public final Flowable startWith(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return concatArray(other, this); } @@ -14615,7 +14825,7 @@ public final Flowable startWith(Publisher other) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable startWithItem(T item) { + public final Flowable startWithItem(@NonNull T item) { Objects.requireNonNull(item, "item is null"); return concatArray(just(item), this); } @@ -14642,11 +14852,12 @@ public final Flowable startWithItem(T item) { * @see #startWithItem(Object) * @see #startWithIterable(Iterable) */ - @SuppressWarnings("unchecked") @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable startWithArray(T... items) { + @SafeVarargs + @NonNull + public final Flowable startWithArray(@NonNull T... items) { Flowable fromArray = fromArray(items); if (fromArray == empty()) { return RxJavaPlugins.onAssembly(this); @@ -14674,6 +14885,7 @@ public final Flowable startWithArray(T... items) { */ @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Disposable subscribe() { return subscribe(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -14703,7 +14915,8 @@ public final Disposable subscribe() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(Consumer onNext) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onNext) { return subscribe(onNext, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -14733,7 +14946,8 @@ public final Disposable subscribe(Consumer onNext) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(Consumer onNext, Consumer onError) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onNext, @NonNull Consumer onError) { return subscribe(onNext, onError, Functions.EMPTY_ACTION); } @@ -14767,13 +14981,14 @@ public final Disposable subscribe(Consumer onNext, Consumer onNext, Consumer onError, - Action onComplete) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onNext, @NonNull Consumer onError, + @NonNull Action onComplete) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); - LambdaSubscriber ls = new LambdaSubscriber(onNext, onError, onComplete, FlowableInternalHelper.RequestMax.INSTANCE); + LambdaSubscriber ls = new LambdaSubscriber<>(onNext, onError, onComplete, FlowableInternalHelper.RequestMax.INSTANCE); subscribe(ls); @@ -14783,7 +14998,7 @@ public final Disposable subscribe(Consumer onNext, Consumer s) { + public final void subscribe(@NonNull Subscriber s) { if (s instanceof FlowableSubscriber) { subscribe((FlowableSubscriber)s); } else { @@ -14831,7 +15046,7 @@ public final void subscribe(Subscriber s) { */ @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) - public final void subscribe(FlowableSubscriber s) { + public final void subscribe(@NonNull FlowableSubscriber s) { Objects.requireNonNull(s, "s is null"); try { Subscriber z = RxJavaPlugins.onSubscribe(this, s); @@ -14861,7 +15076,7 @@ public final void subscribe(FlowableSubscriber s) { * applied by {@link #subscribe(Subscriber)} before this method gets called. * @param s the incoming Subscriber, never null */ - protected abstract void subscribeActual(Subscriber s); + protected abstract void subscribeActual(@NonNull Subscriber s); /** * Subscribes a given Subscriber (subclass) to this Flowable and returns the given @@ -14893,7 +15108,8 @@ public final void subscribe(FlowableSubscriber s) { @CheckReturnValue @BackpressureSupport(BackpressureKind.SPECIAL) @SchedulerSupport(SchedulerSupport.NONE) - public final > E subscribeWith(E subscriber) { + @NonNull + public final <@NonNull E extends Subscriber> E subscribeWith(E subscriber) { subscribe(subscriber); return subscriber; } @@ -14967,7 +15183,7 @@ public final Flowable subscribeOn(@NonNull Scheduler scheduler) { @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable subscribeOn(@NonNull Scheduler scheduler, boolean requestOn) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableSubscribeOn(this, scheduler, requestOn)); + return RxJavaPlugins.onAssembly(new FlowableSubscribeOn<>(this, scheduler, requestOn)); } /** @@ -14996,9 +15212,9 @@ public final Flowable subscribeOn(@NonNull Scheduler scheduler, boolean reque @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable switchIfEmpty(Publisher other) { + public final Flowable switchIfEmpty(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableSwitchIfEmpty(this, other)); + return RxJavaPlugins.onAssembly(new FlowableSwitchIfEmpty<>(this, other)); } /** @@ -15031,7 +15247,8 @@ public final Flowable switchIfEmpty(Publisher other) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable switchMap(Function> mapper) { + @NonNull + public final Flowable switchMap(@NonNull Function> mapper) { return switchMap(mapper, bufferSize()); } @@ -15067,7 +15284,8 @@ public final Flowable switchMap(Function Flowable switchMap(Function> mapper, int bufferSize) { + @NonNull + public final Flowable switchMap(@NonNull Function> mapper, int bufferSize) { return switchMap0(mapper, bufferSize, false); } @@ -15114,7 +15332,7 @@ public final Flowable switchMap(Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new FlowableSwitchMapCompletable(this, mapper, false)); + return RxJavaPlugins.onAssembly(new FlowableSwitchMapCompletable<>(this, mapper, false)); } /** @@ -15161,7 +15379,7 @@ public final Completable switchMapCompletable(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new FlowableSwitchMapCompletable(this, mapper, true)); + return RxJavaPlugins.onAssembly(new FlowableSwitchMapCompletable<>(this, mapper, true)); } /** @@ -15196,7 +15414,8 @@ public final Completable switchMapCompletableDelayError(@NonNull Function Flowable switchMapDelayError(Function> mapper) { + @NonNull + public final Flowable switchMapDelayError(@NonNull Function> mapper) { return switchMapDelayError(mapper, bufferSize()); } @@ -15234,7 +15453,8 @@ public final Flowable switchMapDelayError(Function Flowable switchMapDelayError(Function> mapper, int bufferSize) { + @NonNull + public final Flowable switchMapDelayError(@NonNull Function> mapper, int bufferSize) { return switchMap0(mapper, bufferSize, true); } @@ -15249,7 +15469,7 @@ Flowable switchMap0(Function> } return FlowableScalarXMap.scalarXMap(v, mapper); } - return RxJavaPlugins.onAssembly(new FlowableSwitchMap(this, mapper, bufferSize, delayError)); + return RxJavaPlugins.onAssembly(new FlowableSwitchMap<>(this, mapper, bufferSize, delayError)); } /** @@ -15290,7 +15510,7 @@ Flowable switchMap0(Function> @SchedulerSupport(SchedulerSupport.NONE) public final Flowable switchMapMaybe(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new FlowableSwitchMapMaybe(this, mapper, false)); + return RxJavaPlugins.onAssembly(new FlowableSwitchMapMaybe<>(this, mapper, false)); } /** @@ -15321,7 +15541,7 @@ public final Flowable switchMapMaybe(@NonNull Function Flowable switchMapMaybeDelayError(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new FlowableSwitchMapMaybe(this, mapper, true)); + return RxJavaPlugins.onAssembly(new FlowableSwitchMapMaybe<>(this, mapper, true)); } /** @@ -15362,7 +15582,7 @@ public final Flowable switchMapMaybeDelayError(@NonNull Function Flowable switchMapSingle(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new FlowableSwitchMapSingle(this, mapper, false)); + return RxJavaPlugins.onAssembly(new FlowableSwitchMapSingle<>(this, mapper, false)); } /** @@ -15393,7 +15613,7 @@ public final Flowable switchMapSingle(@NonNull Function Flowable switchMapSingleDelayError(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new FlowableSwitchMapSingle(this, mapper, true)); + return RxJavaPlugins.onAssembly(new FlowableSwitchMapSingle<>(this, mapper, true)); } /** @@ -15439,11 +15659,12 @@ public final Flowable switchMapSingleDelayError(@NonNull Function take(long count) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); } - return RxJavaPlugins.onAssembly(new FlowableTake(this, count)); + return RxJavaPlugins.onAssembly(new FlowableTake<>(this, count)); } /** @@ -15472,7 +15693,8 @@ public final Flowable take(long count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable take(long time, TimeUnit unit) { + @NonNull + public final Flowable take(long time, @NonNull TimeUnit unit) { return takeUntil(timer(time, unit)); } @@ -15505,7 +15727,8 @@ public final Flowable take(long time, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable take(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable take(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return takeUntil(timer(time, unit, scheduler)); } @@ -15533,17 +15756,18 @@ public final Flowable take(long time, TimeUnit unit, Scheduler scheduler) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable takeLast(int count) { if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); } else if (count == 0) { - return RxJavaPlugins.onAssembly(new FlowableIgnoreElements(this)); + return RxJavaPlugins.onAssembly(new FlowableIgnoreElements<>(this)); } else if (count == 1) { - return RxJavaPlugins.onAssembly(new FlowableTakeLastOne(this)); + return RxJavaPlugins.onAssembly(new FlowableTakeLastOne<>(this)); } - return RxJavaPlugins.onAssembly(new FlowableTakeLast(this, count)); + return RxJavaPlugins.onAssembly(new FlowableTakeLast<>(this, count)); } /** @@ -15573,7 +15797,8 @@ public final Flowable takeLast(int count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable takeLast(long count, long time, TimeUnit unit) { + @NonNull + public final Flowable takeLast(long count, long time, @NonNull TimeUnit unit) { return takeLast(count, time, unit, Schedulers.computation(), false, bufferSize()); } @@ -15609,7 +15834,8 @@ public final Flowable takeLast(long count, long time, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable takeLast(long count, long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable takeLast(long count, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return takeLast(count, time, unit, scheduler, false, bufferSize()); } @@ -15651,14 +15877,14 @@ public final Flowable takeLast(long count, long time, TimeUnit unit, Schedule @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable takeLast(long count, long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { + public final Flowable takeLast(long count, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError, int bufferSize) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); } - return RxJavaPlugins.onAssembly(new FlowableTakeLastTimed(this, count, time, unit, scheduler, bufferSize, delayError)); + return RxJavaPlugins.onAssembly(new FlowableTakeLastTimed<>(this, count, time, unit, scheduler, bufferSize, delayError)); } /** @@ -15687,7 +15913,8 @@ public final Flowable takeLast(long count, long time, TimeUnit unit, Schedule @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable takeLast(long time, TimeUnit unit) { + @NonNull + public final Flowable takeLast(long time, @NonNull TimeUnit unit) { return takeLast(time, unit, Schedulers.computation(), false, bufferSize()); } @@ -15720,7 +15947,8 @@ public final Flowable takeLast(long time, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable takeLast(long time, TimeUnit unit, boolean delayError) { + @NonNull + public final Flowable takeLast(long time, @NonNull TimeUnit unit, boolean delayError) { return takeLast(time, unit, Schedulers.computation(), delayError, bufferSize()); } @@ -15754,7 +15982,8 @@ public final Flowable takeLast(long time, TimeUnit unit, boolean delayError) @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable takeLast(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable takeLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return takeLast(time, unit, scheduler, false, bufferSize()); } @@ -15791,7 +16020,8 @@ public final Flowable takeLast(long time, TimeUnit unit, Scheduler scheduler) @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable takeLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError) { + @NonNull + public final Flowable takeLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { return takeLast(time, unit, scheduler, delayError, bufferSize()); } @@ -15830,7 +16060,8 @@ public final Flowable takeLast(long time, TimeUnit unit, Scheduler scheduler, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable takeLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { + @NonNull + public final Flowable takeLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError, int bufferSize) { return takeLast(Long.MAX_VALUE, time, unit, scheduler, delayError, bufferSize); } @@ -15863,9 +16094,9 @@ public final Flowable takeLast(long time, TimeUnit unit, Scheduler scheduler, @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable takeUntil(Predicate stopPredicate) { + public final Flowable takeUntil(@NonNull Predicate stopPredicate) { Objects.requireNonNull(stopPredicate, "stopPredicate is null"); - return RxJavaPlugins.onAssembly(new FlowableTakeUntilPredicate(this, stopPredicate)); + return RxJavaPlugins.onAssembly(new FlowableTakeUntilPredicate<>(this, stopPredicate)); } /** @@ -15893,9 +16124,9 @@ public final Flowable takeUntil(Predicate stopPredicate) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable takeUntil(Publisher other) { + public final Flowable takeUntil(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new FlowableTakeUntil(this, other)); + return RxJavaPlugins.onAssembly(new FlowableTakeUntil<>(this, other)); } /** @@ -15922,9 +16153,9 @@ public final Flowable takeUntil(Publisher other) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable takeWhile(Predicate predicate) { + public final Flowable takeWhile(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new FlowableTakeWhile(this, predicate)); + return RxJavaPlugins.onAssembly(new FlowableTakeWhile<>(this, predicate)); } /** @@ -15953,7 +16184,8 @@ public final Flowable takeWhile(Predicate predicate) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable throttleFirst(long windowDuration, TimeUnit unit) { + @NonNull + public final Flowable throttleFirst(long windowDuration, @NonNull TimeUnit unit) { return throttleFirst(windowDuration, unit, Schedulers.computation()); } @@ -15987,10 +16219,10 @@ public final Flowable throttleFirst(long windowDuration, TimeUnit unit) { @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable throttleFirst(long skipDuration, TimeUnit unit, Scheduler scheduler) { + public final Flowable throttleFirst(long skipDuration, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableThrottleFirstTimed(this, skipDuration, unit, scheduler)); + return RxJavaPlugins.onAssembly(new FlowableThrottleFirstTimed<>(this, skipDuration, unit, scheduler)); } /** @@ -16021,7 +16253,8 @@ public final Flowable throttleFirst(long skipDuration, TimeUnit unit, Schedul @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable throttleLast(long intervalDuration, TimeUnit unit) { + @NonNull + public final Flowable throttleLast(long intervalDuration, @NonNull TimeUnit unit) { return sample(intervalDuration, unit); } @@ -16056,7 +16289,8 @@ public final Flowable throttleLast(long intervalDuration, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable throttleLast(long intervalDuration, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable throttleLast(long intervalDuration, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return sample(intervalDuration, unit, scheduler); } @@ -16093,7 +16327,8 @@ public final Flowable throttleLast(long intervalDuration, TimeUnit unit, Sche @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable throttleLatest(long timeout, TimeUnit unit) { + @NonNull + public final Flowable throttleLatest(long timeout, @NonNull TimeUnit unit) { return throttleLatest(timeout, unit, Schedulers.computation(), false); } @@ -16130,7 +16365,8 @@ public final Flowable throttleLatest(long timeout, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable throttleLatest(long timeout, TimeUnit unit, boolean emitLast) { + @NonNull + public final Flowable throttleLatest(long timeout, @NonNull TimeUnit unit, boolean emitLast) { return throttleLatest(timeout, unit, Schedulers.computation(), emitLast); } @@ -16168,7 +16404,8 @@ public final Flowable throttleLatest(long timeout, TimeUnit unit, boolean emi @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable throttleLatest(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable throttleLatest(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return throttleLatest(timeout, unit, scheduler, false); } @@ -16207,10 +16444,10 @@ public final Flowable throttleLatest(long timeout, TimeUnit unit, Scheduler s @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable throttleLatest(long timeout, TimeUnit unit, Scheduler scheduler, boolean emitLast) { + public final Flowable throttleLatest(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean emitLast) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableThrottleLatest(this, timeout, unit, scheduler, emitLast)); + return RxJavaPlugins.onAssembly(new FlowableThrottleLatest<>(this, timeout, unit, scheduler, emitLast)); } /** @@ -16244,7 +16481,8 @@ public final Flowable throttleLatest(long timeout, TimeUnit unit, Scheduler s @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable throttleWithTimeout(long timeout, TimeUnit unit) { + @NonNull + public final Flowable throttleWithTimeout(long timeout, @NonNull TimeUnit unit) { return debounce(timeout, unit); } @@ -16282,7 +16520,8 @@ public final Flowable throttleWithTimeout(long timeout, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable throttleWithTimeout(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable throttleWithTimeout(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return debounce(timeout, unit, scheduler); } @@ -16306,6 +16545,7 @@ public final Flowable throttleWithTimeout(long timeout, TimeUnit unit, Schedu @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> timeInterval() { return timeInterval(TimeUnit.MILLISECONDS, Schedulers.computation()); } @@ -16332,7 +16572,8 @@ public final Flowable> timeInterval() { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Flowable> timeInterval(Scheduler scheduler) { + @NonNull + public final Flowable> timeInterval(@NonNull Scheduler scheduler) { return timeInterval(TimeUnit.MILLISECONDS, scheduler); } @@ -16357,7 +16598,8 @@ public final Flowable> timeInterval(Scheduler scheduler) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable> timeInterval(TimeUnit unit) { + @NonNull + public final Flowable> timeInterval(@NonNull TimeUnit unit) { return timeInterval(unit, Schedulers.computation()); } @@ -16384,10 +16626,11 @@ public final Flowable> timeInterval(TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Flowable> timeInterval(TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable> timeInterval(@NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableTimeInterval(this, unit, scheduler)); + return RxJavaPlugins.onAssembly(new FlowableTimeInterval<>(this, unit, scheduler)); } /** @@ -16422,7 +16665,8 @@ public final Flowable> timeInterval(TimeUnit unit, Scheduler scheduler) @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable timeout(Function> itemTimeoutIndicator) { + @NonNull + public final Flowable timeout(@NonNull Function> itemTimeoutIndicator) { return timeout0(null, itemTimeoutIndicator, null); } @@ -16461,7 +16705,7 @@ public final Flowable timeout(Function> @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable timeout(Function> itemTimeoutIndicator, Flowable other) { + public final Flowable timeout(@NonNull Function> itemTimeoutIndicator, @NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return timeout0(null, itemTimeoutIndicator, other); } @@ -16491,7 +16735,8 @@ public final Flowable timeout(Function> @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable timeout(long timeout, TimeUnit timeUnit) { + @NonNull + public final Flowable timeout(long timeout, @NonNull TimeUnit timeUnit) { return timeout0(timeout, timeUnit, null, Schedulers.computation()); } @@ -16524,7 +16769,7 @@ public final Flowable timeout(long timeout, TimeUnit timeUnit) { @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable timeout(long timeout, TimeUnit timeUnit, Publisher other) { + public final Flowable timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, Schedulers.computation()); } @@ -16562,7 +16807,7 @@ public final Flowable timeout(long timeout, TimeUnit timeUnit, Publisher timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler, Publisher other) { + public final Flowable timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull Scheduler scheduler, @NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, scheduler); } @@ -16595,7 +16840,8 @@ public final Flowable timeout(long timeout, TimeUnit timeUnit, Scheduler sche @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler) { + @NonNull + public final Flowable timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull Scheduler scheduler) { return timeout0(timeout, timeUnit, null, scheduler); } @@ -16634,8 +16880,8 @@ public final Flowable timeout(long timeout, TimeUnit timeUnit, Scheduler sche @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable timeout(Publisher firstTimeoutIndicator, - Function> itemTimeoutIndicator) { + public final Flowable timeout(@NonNull Publisher firstTimeoutIndicator, + @NonNull Function> itemTimeoutIndicator) { Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, null); } @@ -16681,9 +16927,9 @@ public final Flowable timeout(Publisher firstTimeoutIndicator, @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable timeout( - Publisher firstTimeoutIndicator, - Function> itemTimeoutIndicator, - Publisher other) { + @NonNull Publisher firstTimeoutIndicator, + @NonNull Function> itemTimeoutIndicator, + @NonNull Publisher other) { Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutSelector is null"); Objects.requireNonNull(other, "other is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, other); @@ -16693,7 +16939,7 @@ private Flowable timeout0(long timeout, TimeUnit timeUnit, Publisher(this, timeout, timeUnit, scheduler, other)); + return RxJavaPlugins.onAssembly(new FlowableTimeoutTimed<>(this, timeout, timeUnit, scheduler, other)); } private Flowable timeout0( @@ -16701,7 +16947,7 @@ private Flowable timeout0( Function> itemTimeoutIndicator, Publisher other) { Objects.requireNonNull(itemTimeoutIndicator, "itemTimeoutIndicator is null"); - return RxJavaPlugins.onAssembly(new FlowableTimeout(this, firstTimeoutIndicator, itemTimeoutIndicator, other)); + return RxJavaPlugins.onAssembly(new FlowableTimeout<>(this, firstTimeoutIndicator, itemTimeoutIndicator, other)); } /** @@ -16724,6 +16970,7 @@ private Flowable timeout0( @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> timestamp() { return timestamp(TimeUnit.MILLISECONDS, Schedulers.computation()); } @@ -16751,7 +16998,8 @@ public final Flowable> timestamp() { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Flowable> timestamp(Scheduler scheduler) { + @NonNull + public final Flowable> timestamp(@NonNull Scheduler scheduler) { return timestamp(TimeUnit.MILLISECONDS, scheduler); } @@ -16776,7 +17024,8 @@ public final Flowable> timestamp(Scheduler scheduler) { @CheckReturnValue @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable> timestamp(TimeUnit unit) { + @NonNull + public final Flowable> timestamp(@NonNull TimeUnit unit) { return timestamp(unit, Schedulers.computation()); } @@ -16805,7 +17054,7 @@ public final Flowable> timestamp(TimeUnit unit) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Flowable> timestamp(final TimeUnit unit, final Scheduler scheduler) { + public final Flowable> timestamp(@NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return map(Functions.timestampWith(unit, scheduler)); @@ -16865,6 +17114,7 @@ public final R to(@NonNull FlowableConverter converter) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> toList() { return RxJavaPlugins.onAssembly(new FlowableToListSingle>(this)); } @@ -16901,9 +17151,10 @@ public final Single> toList() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> toList(final int capacityHint) { ObjectHelper.verifyPositive(capacityHint, "capacityHint"); - return RxJavaPlugins.onAssembly(new FlowableToListSingle>(this, Functions.createArrayList(capacityHint))); + return RxJavaPlugins.onAssembly(new FlowableToListSingle<>(this, Functions.createArrayList(capacityHint))); } /** @@ -16939,9 +17190,10 @@ public final Single> toList(final int capacityHint) { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final > Single toList(Supplier collectionSupplier) { + @NonNull + public final > Single toList(@NonNull Supplier collectionSupplier) { Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); - return RxJavaPlugins.onAssembly(new FlowableToListSingle(this, collectionSupplier)); + return RxJavaPlugins.onAssembly(new FlowableToListSingle<>(this, collectionSupplier)); } /** @@ -16974,7 +17226,7 @@ public final > Single toList(Supplier coll @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toMap(final Function keySelector) { + public final Single> toMap(@NonNull Function keySelector) { Objects.requireNonNull(keySelector, "keySelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); } @@ -17013,7 +17265,7 @@ public final Single> toMap(final Function @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toMap(final Function keySelector, final Function valueSelector) { + public final Single> toMap(@NonNull Function keySelector, @NonNull Function valueSelector) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); @@ -17052,9 +17304,9 @@ public final Single> toMap(final Function Single> toMap(final Function keySelector, - final Function valueSelector, - final Supplier> mapSupplier) { + public final Single> toMap(@NonNull Function keySelector, + @NonNull Function valueSelector, + @NonNull Supplier> mapSupplier) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); return collect(mapSupplier, Functions.toMapKeyValueSelector(keySelector, valueSelector)); @@ -17086,7 +17338,8 @@ public final Single> toMap(final Function Single>> toMultimap(Function keySelector) { + @NonNull + public final Single>> toMultimap(@NonNull Function keySelector) { Function valueSelector = Functions.identity(); Supplier>> mapSupplier = HashMapSupplier.asSupplier(); Function> collectionFactory = ArrayListSupplier.asFunction(); @@ -17124,7 +17377,8 @@ public final Single>> toMultimap(Function Single>> toMultimap(Function keySelector, Function valueSelector) { + @NonNull + public final Single>> toMultimap(@NonNull Function keySelector, @NonNull Function valueSelector) { Supplier>> mapSupplier = HashMapSupplier.asSupplier(); Function> collectionFactory = ArrayListSupplier.asFunction(); return toMultimap(keySelector, valueSelector, mapSupplier, collectionFactory); @@ -17167,10 +17421,10 @@ public final Single>> toMultimap(Function Single>> toMultimap( - final Function keySelector, - final Function valueSelector, - final Supplier>> mapSupplier, - final Function> collectionFactory) { + @NonNull Function keySelector, + @NonNull Function valueSelector, + @NonNull Supplier>> mapSupplier, + @NonNull Function> collectionFactory) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); Objects.requireNonNull(mapSupplier, "mapSupplier is null"); @@ -17211,10 +17465,11 @@ public final Single>> toMultimap( @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single>> toMultimap( - Function keySelector, - Function valueSelector, - Supplier>> mapSupplier + @NonNull Function keySelector, + @NonNull Function valueSelector, + @NonNull Supplier>> mapSupplier ) { return toMultimap(keySelector, valueSelector, mapSupplier, ArrayListSupplier.asFunction()); } @@ -17234,8 +17489,9 @@ public final Single>> toMultimap( @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable toObservable() { - return RxJavaPlugins.onAssembly(new ObservableFromPublisher(this)); + return RxJavaPlugins.onAssembly(new ObservableFromPublisher<>(this)); } /** @@ -17266,6 +17522,7 @@ public final Observable toObservable() { @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> toSortedList() { return toSortedList(Functions.naturalComparator()); } @@ -17298,7 +17555,7 @@ public final Single> toSortedList() { @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toSortedList(final Comparator comparator) { + public final Single> toSortedList(@NonNull Comparator comparator) { Objects.requireNonNull(comparator, "comparator is null"); return toList().map(Functions.listSorter(comparator)); } @@ -17334,7 +17591,7 @@ public final Single> toSortedList(final Comparator comparator @NonNull @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toSortedList(final Comparator comparator, int capacityHint) { + public final Single> toSortedList(@NonNull Comparator comparator, int capacityHint) { Objects.requireNonNull(comparator, "comparator is null"); return toList(capacityHint).map(Functions.listSorter(comparator)); } @@ -17371,6 +17628,7 @@ public final Single> toSortedList(final Comparator comparator @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> toSortedList(int capacityHint) { return toSortedList(Functions.naturalComparator(), capacityHint); } @@ -17396,9 +17654,9 @@ public final Single> toSortedList(int capacityHint) { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable unsubscribeOn(Scheduler scheduler) { + public final Flowable unsubscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableUnsubscribeOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new FlowableUnsubscribeOn<>(this, scheduler)); } /** @@ -17430,6 +17688,7 @@ public final Flowable unsubscribeOn(Scheduler scheduler) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> window(long count) { return window(count, count, bufferSize()); } @@ -17467,6 +17726,7 @@ public final Flowable> window(long count) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> window(long count, long skip) { return window(count, skip, bufferSize()); } @@ -17506,11 +17766,12 @@ public final Flowable> window(long count, long skip) { @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> window(long count, long skip, int bufferSize) { ObjectHelper.verifyPositive(skip, "skip"); ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new FlowableWindow(this, count, skip, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowableWindow<>(this, count, skip, bufferSize)); } /** @@ -17549,7 +17810,8 @@ public final Flowable> window(long count, long skip, int bufferSize) @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable> window(long timespan, long timeskip, TimeUnit unit) { + @NonNull + public final Flowable> window(long timespan, long timeskip, @NonNull TimeUnit unit) { return window(timespan, timeskip, unit, Schedulers.computation(), bufferSize()); } @@ -17591,7 +17853,8 @@ public final Flowable> window(long timespan, long timeskip, TimeUnit @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> window(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable> window(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return window(timespan, timeskip, unit, scheduler, bufferSize()); } @@ -17636,13 +17899,13 @@ public final Flowable> window(long timespan, long timeskip, TimeUnit @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> window(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler, int bufferSize) { + public final Flowable> window(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); ObjectHelper.verifyPositive(timespan, "timespan"); ObjectHelper.verifyPositive(timeskip, "timeskip"); Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(unit, "unit is null"); - return RxJavaPlugins.onAssembly(new FlowableWindowTimed(this, timespan, timeskip, unit, scheduler, Long.MAX_VALUE, bufferSize, false)); + return RxJavaPlugins.onAssembly(new FlowableWindowTimed<>(this, timespan, timeskip, unit, scheduler, Long.MAX_VALUE, bufferSize, false)); } /** @@ -17679,7 +17942,8 @@ public final Flowable> window(long timespan, long timeskip, TimeUnit @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable> window(long timespan, TimeUnit unit) { + @NonNull + public final Flowable> window(long timespan, @NonNull TimeUnit unit) { return window(timespan, unit, Schedulers.computation(), Long.MAX_VALUE, false); } @@ -17721,7 +17985,8 @@ public final Flowable> window(long timespan, TimeUnit unit) { @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable> window(long timespan, TimeUnit unit, + @NonNull + public final Flowable> window(long timespan, @NonNull TimeUnit unit, long count) { return window(timespan, unit, Schedulers.computation(), count, false); } @@ -17766,7 +18031,8 @@ public final Flowable> window(long timespan, TimeUnit unit, @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Flowable> window(long timespan, TimeUnit unit, + @NonNull + public final Flowable> window(long timespan, @NonNull TimeUnit unit, long count, boolean restart) { return window(timespan, unit, Schedulers.computation(), count, restart); } @@ -17808,8 +18074,9 @@ public final Flowable> window(long timespan, TimeUnit unit, @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> window(long timespan, TimeUnit unit, - Scheduler scheduler) { + @NonNull + public final Flowable> window(long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler) { return window(timespan, unit, scheduler, Long.MAX_VALUE, false); } @@ -17853,8 +18120,9 @@ public final Flowable> window(long timespan, TimeUnit unit, @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> window(long timespan, TimeUnit unit, - Scheduler scheduler, long count) { + @NonNull + public final Flowable> window(long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler, long count) { return window(timespan, unit, scheduler, count, false); } @@ -17900,8 +18168,9 @@ public final Flowable> window(long timespan, TimeUnit unit, @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Flowable> window(long timespan, TimeUnit unit, - Scheduler scheduler, long count, boolean restart) { + @NonNull + public final Flowable> window(long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler, long count, boolean restart) { return window(timespan, unit, scheduler, count, restart, bufferSize()); } @@ -17951,13 +18220,13 @@ public final Flowable> window(long timespan, TimeUnit unit, @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.CUSTOM) public final Flowable> window( - long timespan, TimeUnit unit, Scheduler scheduler, + long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, long count, boolean restart, int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(count, "count"); - return RxJavaPlugins.onAssembly(new FlowableWindowTimed(this, timespan, timespan, unit, scheduler, count, bufferSize, restart)); + return RxJavaPlugins.onAssembly(new FlowableWindowTimed<>(this, timespan, timespan, unit, scheduler, count, bufferSize, restart)); } /** @@ -17991,7 +18260,8 @@ public final Flowable> window( @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable> window(Publisher boundaryIndicator) { + @NonNull + public final Flowable> window(@NonNull Publisher boundaryIndicator) { return window(boundaryIndicator, bufferSize()); } @@ -18029,10 +18299,10 @@ public final Flowable> window(Publisher boundaryIndicator) { @NonNull @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable> window(Publisher boundaryIndicator, int bufferSize) { + public final Flowable> window(@NonNull Publisher boundaryIndicator, int bufferSize) { Objects.requireNonNull(boundaryIndicator, "boundaryIndicator is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new FlowableWindowBoundary(this, boundaryIndicator, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowableWindowBoundary<>(this, boundaryIndicator, bufferSize)); } /** @@ -18071,9 +18341,10 @@ public final Flowable> window(Publisher boundaryIndicator, in @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable> window( - Publisher openingIndicator, - Function> closingIndicator) { + @NonNull Publisher openingIndicator, + @NonNull Function> closingIndicator) { return window(openingIndicator, closingIndicator, bufferSize()); } @@ -18117,12 +18388,12 @@ public final Flowable> window( @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) public final Flowable> window( - Publisher openingIndicator, - Function> closingIndicator, int bufferSize) { + @NonNull Publisher openingIndicator, + @NonNull Function> closingIndicator, int bufferSize) { Objects.requireNonNull(openingIndicator, "openingIndicator is null"); Objects.requireNonNull(closingIndicator, "closingIndicator is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new FlowableWindowBoundarySelector(this, openingIndicator, closingIndicator, bufferSize)); + return RxJavaPlugins.onAssembly(new FlowableWindowBoundarySelector<>(this, openingIndicator, closingIndicator, bufferSize)); } /** @@ -18157,8 +18428,8 @@ public final Flowable> window( @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable withLatestFrom(Publisher other, - BiFunction combiner) { + public final Flowable withLatestFrom(@NonNull Publisher other, + @NonNull BiFunction combiner) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(combiner, "combiner is null"); @@ -18195,8 +18466,8 @@ public final Flowable withLatestFrom(Publisher other, @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable withLatestFrom(Publisher source1, Publisher source2, - Function3 combiner) { + public final Flowable withLatestFrom(@NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Function3 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Function f = Functions.toFunction(combiner); @@ -18235,10 +18506,10 @@ public final Flowable withLatestFrom(Publisher source1, Publi @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable withLatestFrom( - Publisher source1, Publisher source2, - Publisher source3, - Function4 combiner) { + public final Flowable withLatestFrom( + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, + @NonNull Function4 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -18280,10 +18551,10 @@ public final Flowable withLatestFrom( @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable withLatestFrom( - Publisher source1, Publisher source2, - Publisher source3, Publisher source4, - Function5 combiner) { + public final Flowable withLatestFrom( + @NonNull Publisher source1, @NonNull Publisher source2, + @NonNull Publisher source3, @NonNull Publisher source4, + @NonNull Function5 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -18319,10 +18590,10 @@ public final Flowable withLatestFrom( @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable withLatestFrom(Publisher[] others, Function combiner) { + public final <@NonNull R> Flowable withLatestFrom(@NonNull Publisher[] others, @NonNull Function combiner) { Objects.requireNonNull(others, "others is null"); Objects.requireNonNull(combiner, "combiner is null"); - return RxJavaPlugins.onAssembly(new FlowableWithLatestFromMany(this, others, combiner)); + return RxJavaPlugins.onAssembly(new FlowableWithLatestFromMany<>(this, others, combiner)); } /** @@ -18352,10 +18623,10 @@ public final Flowable withLatestFrom(Publisher[] others, Function Flowable withLatestFrom(Iterable> others, Function combiner) { + public final Flowable withLatestFrom(@NonNull Iterable> others, @NonNull Function combiner) { Objects.requireNonNull(others, "others is null"); Objects.requireNonNull(combiner, "combiner is null"); - return RxJavaPlugins.onAssembly(new FlowableWithLatestFromMany(this, others, combiner)); + return RxJavaPlugins.onAssembly(new FlowableWithLatestFromMany<>(this, others, combiner)); } /** @@ -18392,7 +18663,7 @@ public final Flowable withLatestFrom(Iterable> oth @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable zipWith(Iterable other, BiFunction zipper) { + public final Flowable zipWith(@NonNull Iterable other, @NonNull BiFunction zipper) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(zipper, "zipper is null"); return RxJavaPlugins.onAssembly(new FlowableZipIterable(this, other, zipper)); @@ -18441,7 +18712,7 @@ public final Flowable zipWith(Iterable other, BiFunction Flowable zipWith(Publisher other, BiFunction zipper) { + public final Flowable zipWith(@NonNull Publisher other, @NonNull BiFunction zipper) { Objects.requireNonNull(other, "other is null"); return zip(this, other, zipper); } @@ -18491,8 +18762,9 @@ public final Flowable zipWith(Publisher other, BiFunction @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable zipWith(Publisher other, - BiFunction zipper, boolean delayError) { + @NonNull + public final Flowable zipWith(@NonNull Publisher other, + @NonNull BiFunction zipper, boolean delayError) { return zip(this, other, zipper, delayError); } @@ -18543,8 +18815,9 @@ public final Flowable zipWith(Publisher other, @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable zipWith(Publisher other, - BiFunction zipper, boolean delayError, int bufferSize) { + @NonNull + public final Flowable zipWith(@NonNull Publisher other, + @NonNull BiFunction zipper, boolean delayError, int bufferSize) { return zip(this, other, zipper, delayError, bufferSize); } @@ -18566,8 +18839,9 @@ public final Flowable zipWith(Publisher other, @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestSubscriber test() { // NoPMD - TestSubscriber ts = new TestSubscriber(); + TestSubscriber ts = new TestSubscriber<>(); subscribe(ts); return ts; } @@ -18588,8 +18862,9 @@ public final TestSubscriber test() { // NoPMD @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestSubscriber test(long initialRequest) { // NoPMD - TestSubscriber ts = new TestSubscriber(initialRequest); + TestSubscriber ts = new TestSubscriber<>(initialRequest); subscribe(ts); return ts; } @@ -18612,8 +18887,9 @@ public final TestSubscriber test(long initialRequest) { // NoPMD @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestSubscriber test(long initialRequest, boolean cancel) { // NoPMD - TestSubscriber ts = new TestSubscriber(initialRequest); + TestSubscriber ts = new TestSubscriber<>(initialRequest); if (cancel) { ts.cancel(); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Maybe.java b/src/main/java/io/reactivex/rxjava3/core/Maybe.java index 35484ebef4..bb0a6dcbdb 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Maybe.java +++ b/src/main/java/io/reactivex/rxjava3/core/Maybe.java @@ -129,7 +129,7 @@ public abstract class Maybe implements MaybeSource { @SchedulerSupport(SchedulerSupport.NONE) public static Maybe amb(final Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); - return RxJavaPlugins.onAssembly(new MaybeAmb(null, sources)); + return RxJavaPlugins.onAssembly(new MaybeAmb<>(null, sources)); } /** @@ -148,15 +148,18 @@ public static Maybe amb(final Iterable */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") - public static Maybe ambArray(final MaybeSource... sources) { + @NonNull + @SafeVarargs + public static Maybe ambArray(@NonNull MaybeSource... sources) { if (sources.length == 0) { return empty(); } if (sources.length == 1) { - return wrap((MaybeSource)sources[0]); + @SuppressWarnings("unchecked") + MaybeSource source = (MaybeSource)sources[0]; + return wrap(source); } - return RxJavaPlugins.onAssembly(new MaybeAmb(sources, null)); + return RxJavaPlugins.onAssembly(new MaybeAmb<>(sources, null)); } /** @@ -178,9 +181,9 @@ public static Maybe ambArray(final MaybeSource... sources) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concat(Iterable> sources) { + public static Flowable concat(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); - return RxJavaPlugins.onAssembly(new MaybeConcatIterable(sources)); + return RxJavaPlugins.onAssembly(new MaybeConcatIterable<>(sources)); } /** @@ -206,8 +209,7 @@ public static Flowable concat(Iterable @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") - public static Flowable concat(MaybeSource source1, MaybeSource source2) { + public static Flowable concat(@NonNull MaybeSource source1, @NonNull MaybeSource source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return concatArray(source1, source2); @@ -238,9 +240,8 @@ public static Flowable concat(MaybeSource source1, MaybeSour @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable concat( - MaybeSource source1, MaybeSource source2, MaybeSource source3) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -274,9 +275,8 @@ public static Flowable concat( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable concat( - MaybeSource source1, MaybeSource source2, MaybeSource source3, MaybeSource source4) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, @NonNull MaybeSource source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -304,7 +304,8 @@ public static Flowable concat( @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concat(Publisher> sources) { + @NonNull + public static Flowable concat(@NonNull Publisher> sources) { return concat(sources, 2); } @@ -331,7 +332,7 @@ public static Flowable concat(Publisher Flowable concat(Publisher> sources, int prefetch) { + public static Flowable concat(@NonNull Publisher> sources, int prefetch) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapPublisher(sources, MaybeToPublisher.instance(), prefetch, ErrorMode.IMMEDIATE)); @@ -355,16 +356,18 @@ public static Flowable concat(Publisher Flowable concatArray(MaybeSource... sources) { + @SafeVarargs + public static Flowable concatArray(@NonNull MaybeSource... sources) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return Flowable.empty(); } if (sources.length == 1) { - return RxJavaPlugins.onAssembly(new MaybeToFlowable((MaybeSource)sources[0])); + @SuppressWarnings("unchecked") + MaybeSource source = (MaybeSource)sources[0]; + return RxJavaPlugins.onAssembly(new MaybeToFlowable<>(source)); } - return RxJavaPlugins.onAssembly(new MaybeConcatArray(sources)); + return RxJavaPlugins.onAssembly(new MaybeConcatArray<>(sources)); } /** @@ -383,18 +386,21 @@ public static Flowable concatArray(MaybeSource... sources) { * @return the new Flowable instance * @throws NullPointerException if sources is null */ - @SuppressWarnings("unchecked") @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concatArrayDelayError(MaybeSource... sources) { + @SafeVarargs + @NonNull + public static Flowable concatArrayDelayError(@NonNull MaybeSource... sources) { if (sources.length == 0) { return Flowable.empty(); } else if (sources.length == 1) { - return RxJavaPlugins.onAssembly(new MaybeToFlowable((MaybeSource)sources[0])); + @SuppressWarnings("unchecked") + MaybeSource source = (MaybeSource)sources[0]; + return RxJavaPlugins.onAssembly(new MaybeToFlowable<>(source)); } - return RxJavaPlugins.onAssembly(new MaybeConcatArrayDelayError(sources)); + return RxJavaPlugins.onAssembly(new MaybeConcatArrayDelayError<>(sources)); } /** @@ -419,7 +425,9 @@ public static Flowable concatArrayDelayError(MaybeSource... @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concatArrayEager(MaybeSource... sources) { + @NonNull + @SafeVarargs + public static Flowable concatArrayEager(@NonNull MaybeSource... sources) { return Flowable.fromArray(sources).concatMapEager((Function)MaybeToPublisher.instance()); } @@ -469,7 +477,8 @@ public static Flowable concatDelayError(Iterable Flowable concatDelayError(Publisher> sources) { + @NonNull + public static Flowable concatDelayError(@NonNull Publisher> sources) { return Flowable.fromPublisher(sources).concatMapDelayError((Function)MaybeToPublisher.instance()); } @@ -495,7 +504,8 @@ public static Flowable concatDelayError(Publisher Flowable concatEager(Iterable> sources) { + @NonNull + public static Flowable concatEager(@NonNull Iterable> sources) { return Flowable.fromIterable(sources).concatMapEager((Function)MaybeToPublisher.instance()); } @@ -523,7 +533,8 @@ public static Flowable concatEager(Iterable Flowable concatEager(Publisher> sources) { + @NonNull + public static Flowable concatEager(@NonNull Publisher> sources) { return Flowable.fromPublisher(sources).concatMapEager((Function)MaybeToPublisher.instance()); } @@ -574,9 +585,9 @@ public static Flowable concatEager(Publisher Maybe create(MaybeOnSubscribe onSubscribe) { + public static Maybe create(@NonNull MaybeOnSubscribe onSubscribe) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); - return RxJavaPlugins.onAssembly(new MaybeCreate(onSubscribe)); + return RxJavaPlugins.onAssembly(new MaybeCreate<>(onSubscribe)); } /** @@ -594,9 +605,9 @@ public static Maybe create(MaybeOnSubscribe onSubscribe) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe defer(final Supplier> maybeSupplier) { + public static Maybe defer(@NonNull Supplier> maybeSupplier) { Objects.requireNonNull(maybeSupplier, "maybeSupplier is null"); - return RxJavaPlugins.onAssembly(new MaybeDefer(maybeSupplier)); + return RxJavaPlugins.onAssembly(new MaybeDefer<>(maybeSupplier)); } /** @@ -614,6 +625,7 @@ public static Maybe defer(final Supplier Maybe empty() { return RxJavaPlugins.onAssembly((Maybe)MaybeEmpty.INSTANCE); } @@ -639,7 +651,7 @@ public static Maybe empty() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe error(Throwable exception) { + public static Maybe error(@NonNull Throwable exception) { Objects.requireNonNull(exception, "exception is null"); return RxJavaPlugins.onAssembly(new MaybeError(exception)); } @@ -665,7 +677,7 @@ public static Maybe error(Throwable exception) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe error(Supplier supplier) { + public static Maybe error(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new MaybeErrorCallable(supplier)); } @@ -692,7 +704,7 @@ public static Maybe error(Supplier supplier) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromAction(final Action run) { + public static Maybe fromAction(@NonNull Action run) { Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new MaybeFromAction(run)); } @@ -712,7 +724,7 @@ public static Maybe fromAction(final Action run) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromCompletable(CompletableSource completableSource) { + public static Maybe fromCompletable(@NonNull CompletableSource completableSource) { Objects.requireNonNull(completableSource, "completableSource is null"); return RxJavaPlugins.onAssembly(new MaybeFromCompletable(completableSource)); } @@ -732,9 +744,9 @@ public static Maybe fromCompletable(CompletableSource completableSource) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromSingle(SingleSource singleSource) { + public static Maybe fromSingle(@NonNull SingleSource singleSource) { Objects.requireNonNull(singleSource, "singleSource is null"); - return RxJavaPlugins.onAssembly(new MaybeFromSingle(singleSource)); + return RxJavaPlugins.onAssembly(new MaybeFromSingle<>(singleSource)); } /** @@ -776,7 +788,7 @@ public static Maybe fromSingle(SingleSource singleSource) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromCallable(@NonNull final Callable callable) { + public static <@NonNull T> Maybe fromCallable(@NonNull final Callable callable) { Objects.requireNonNull(callable, "callable is null"); return RxJavaPlugins.onAssembly(new MaybeFromCallable(callable)); } @@ -810,7 +822,7 @@ public static Maybe fromCallable(@NonNull final Callable cal @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromFuture(Future future) { + public static <@NonNull T> Maybe fromFuture(@NonNull Future future) { Objects.requireNonNull(future, "future is null"); return RxJavaPlugins.onAssembly(new MaybeFromFuture(future, 0L, null)); } @@ -848,7 +860,7 @@ public static Maybe fromFuture(Future future) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromFuture(Future future, long timeout, TimeUnit unit) { + public static <@NonNull T> Maybe fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) { Objects.requireNonNull(future, "future is null"); Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new MaybeFromFuture(future, timeout, unit)); @@ -869,7 +881,7 @@ public static Maybe fromFuture(Future future, long timeout, @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromRunnable(final Runnable run) { + public static Maybe fromRunnable(@NonNull Runnable run) { Objects.requireNonNull(run, "run is null"); return RxJavaPlugins.onAssembly(new MaybeFromRunnable(run)); } @@ -916,7 +928,7 @@ public static Maybe fromRunnable(final Runnable run) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe fromSupplier(@NonNull final Supplier supplier) { + public static <@NonNull T> Maybe fromSupplier(@NonNull final Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new MaybeFromSupplier(supplier)); } @@ -943,9 +955,9 @@ public static Maybe fromSupplier(@NonNull final Supplier sup @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe just(T item) { + public static <@NonNull T> Maybe just(T item) { Objects.requireNonNull(item, "item is null"); - return RxJavaPlugins.onAssembly(new MaybeJust(item)); + return RxJavaPlugins.onAssembly(new MaybeJust<>(item)); } /** @@ -978,7 +990,8 @@ public static Maybe just(T item) { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Iterable> sources) { + @NonNull + public static Flowable merge(@NonNull Iterable> sources) { return merge(Flowable.fromIterable(sources)); } @@ -1012,7 +1025,8 @@ public static Flowable merge(Iterable> @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable merge(Publisher> sources) { + @NonNull + public static Flowable merge(@NonNull Publisher> sources) { return merge(sources, Integer.MAX_VALUE); } @@ -1049,7 +1063,7 @@ public static Flowable merge(Publisher @NonNull @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "unchecked", "rawtypes" }) - public static Flowable merge(Publisher> sources, int maxConcurrency) { + public static Flowable merge(@NonNull Publisher> sources, int maxConcurrency) { Objects.requireNonNull(sources, "source is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, MaybeToPublisher.instance(), false, maxConcurrency, 1)); @@ -1082,7 +1096,7 @@ public static Flowable merge(Publisher @NonNull @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "unchecked", "rawtypes" }) - public static Maybe merge(MaybeSource> source) { + public static Maybe merge(@NonNull MaybeSource> source) { Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new MaybeFlatten(source, Functions.identity())); } @@ -1127,9 +1141,8 @@ public static Maybe merge(MaybeSource> @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable merge( - MaybeSource source1, MaybeSource source2 + @NonNull MaybeSource source1, @NonNull MaybeSource source2 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1178,10 +1191,9 @@ public static Flowable merge( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable merge( - MaybeSource source1, MaybeSource source2, - MaybeSource source3 + @NonNull MaybeSource source1, @NonNull MaybeSource source2, + @NonNull MaybeSource source3 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1233,10 +1245,9 @@ public static Flowable merge( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable merge( - MaybeSource source1, MaybeSource source2, - MaybeSource source3, MaybeSource source4 + @NonNull MaybeSource source1, @NonNull MaybeSource source2, + @NonNull MaybeSource source3, @NonNull MaybeSource source4 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1276,16 +1287,18 @@ public static Flowable merge( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") + @SafeVarargs public static Flowable mergeArray(MaybeSource... sources) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return Flowable.empty(); } if (sources.length == 1) { - return RxJavaPlugins.onAssembly(new MaybeToFlowable((MaybeSource)sources[0])); + @SuppressWarnings("unchecked") + MaybeSource source = (MaybeSource)sources[0]; + return RxJavaPlugins.onAssembly(new MaybeToFlowable<>(source)); } - return RxJavaPlugins.onAssembly(new MaybeMergeArray(sources)); + return RxJavaPlugins.onAssembly(new MaybeMergeArray<>(sources)); } /** @@ -1319,7 +1332,9 @@ public static Flowable mergeArray(MaybeSource... sources) { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeArrayDelayError(MaybeSource... sources) { + @SafeVarargs + @NonNull + public static Flowable mergeArrayDelayError(@NonNull MaybeSource... sources) { if (sources.length == 0) { return Flowable.empty(); } @@ -1357,7 +1372,8 @@ public static Flowable mergeArrayDelayError(MaybeSource... s @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(Iterable> sources) { + @NonNull + public static Flowable mergeDelayError(@NonNull Iterable> sources) { return Flowable.fromIterable(sources).flatMap((Function)MaybeToPublisher.instance(), true); } @@ -1392,7 +1408,8 @@ public static Flowable mergeDelayError(Iterable Flowable mergeDelayError(Publisher> sources) { + @NonNull + public static Flowable mergeDelayError(@NonNull Publisher> sources) { return mergeDelayError(sources, Integer.MAX_VALUE); } @@ -1431,7 +1448,7 @@ public static Flowable mergeDelayError(Publisher Flowable mergeDelayError(Publisher> sources, int maxConcurrency) { + public static Flowable mergeDelayError(@NonNull Publisher> sources, int maxConcurrency) { Objects.requireNonNull(sources, "source is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, MaybeToPublisher.instance(), true, maxConcurrency, 1)); @@ -1465,12 +1482,11 @@ public static Flowable mergeDelayError(PublisherReactiveX operators documentation: Merge */ - @SuppressWarnings({ "unchecked" }) @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(MaybeSource source1, MaybeSource source2) { + public static Flowable mergeDelayError(@NonNull MaybeSource source1, @NonNull MaybeSource source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return mergeArrayDelayError(source1, source2); @@ -1507,13 +1523,12 @@ public static Flowable mergeDelayError(MaybeSource source1, * @return a Flowable that emits all of the items that are emitted by the source MaybeSources * @see ReactiveX operators documentation: Merge */ - @SuppressWarnings({ "unchecked" }) @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(MaybeSource source1, - MaybeSource source2, MaybeSource source3) { + public static Flowable mergeDelayError(@NonNull MaybeSource source1, + @NonNull MaybeSource source2, @NonNull MaybeSource source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1553,14 +1568,13 @@ public static Flowable mergeDelayError(MaybeSource source1, * @return a Flowable that emits all of the items that are emitted by the source MaybeSources * @see ReactiveX operators documentation: Merge */ - @SuppressWarnings({ "unchecked" }) @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Flowable mergeDelayError( - MaybeSource source1, MaybeSource source2, - MaybeSource source3, MaybeSource source4) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, + @NonNull MaybeSource source3, @NonNull MaybeSource source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1587,6 +1601,7 @@ public static Flowable mergeDelayError( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") + @NonNull public static Maybe never() { return RxJavaPlugins.onAssembly((Maybe)MaybeNever.INSTANCE); } @@ -1612,7 +1627,8 @@ public static Maybe never() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(MaybeSource source1, MaybeSource source2) { + @NonNull + public static Single sequenceEqual(@NonNull MaybeSource source1, @NonNull MaybeSource source2) { return sequenceEqual(source1, source2, ObjectHelper.equalsPredicate()); } @@ -1642,12 +1658,12 @@ public static Single sequenceEqual(MaybeSource source1 @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(MaybeSource source1, MaybeSource source2, - BiPredicate isEqual) { + public static Single sequenceEqual(@NonNull MaybeSource source1, @NonNull MaybeSource source2, + @NonNull BiPredicate isEqual) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(isEqual, "isEqual is null"); - return RxJavaPlugins.onAssembly(new MaybeEqualSingle(source1, source2, isEqual)); + return RxJavaPlugins.onAssembly(new MaybeEqualSingle<>(source1, source2, isEqual)); } /** @@ -1668,7 +1684,8 @@ public static Single sequenceEqual(MaybeSource source1 */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Maybe timer(long delay, TimeUnit unit) { + @NonNull + public static Maybe timer(long delay, @NonNull TimeUnit unit) { return timer(delay, unit, Schedulers.computation()); } @@ -1693,7 +1710,7 @@ public static Maybe timer(long delay, TimeUnit unit) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Maybe timer(long delay, TimeUnit unit, Scheduler scheduler) { + public static Maybe timer(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -1714,12 +1731,12 @@ public static Maybe timer(long delay, TimeUnit unit, Scheduler scheduler) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe unsafeCreate(MaybeSource onSubscribe) { + public static Maybe unsafeCreate(@NonNull MaybeSource onSubscribe) { if (onSubscribe instanceof Maybe) { throw new IllegalArgumentException("unsafeCreate(Maybe) should be upgraded"); } Objects.requireNonNull(onSubscribe, "onSubscribe is null"); - return RxJavaPlugins.onAssembly(new MaybeUnsafeCreate(onSubscribe)); + return RxJavaPlugins.onAssembly(new MaybeUnsafeCreate<>(onSubscribe)); } /** @@ -1745,9 +1762,10 @@ public static Maybe unsafeCreate(MaybeSource onSubscribe) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe using(Supplier resourceSupplier, - Function> sourceSupplier, - Consumer resourceDisposer) { + @NonNull + public static Maybe using(@NonNull Supplier resourceSupplier, + @NonNull Function> sourceSupplier, + @NonNull Consumer resourceDisposer) { return using(resourceSupplier, sourceSupplier, resourceDisposer, true); } @@ -1783,9 +1801,9 @@ public static Maybe using(Supplier resourceSupplier, @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe using(Supplier resourceSupplier, - Function> sourceSupplier, - Consumer resourceDisposer, boolean eager) { + public static Maybe using(@NonNull Supplier resourceSupplier, + @NonNull Function> sourceSupplier, + @NonNull Consumer resourceDisposer, boolean eager) { Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); Objects.requireNonNull(sourceSupplier, "sourceSupplier is null"); Objects.requireNonNull(resourceDisposer, "disposer is null"); @@ -1806,12 +1824,12 @@ public static Maybe using(Supplier resourceSupplier, @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe wrap(MaybeSource source) { + public static Maybe wrap(@NonNull MaybeSource source) { if (source instanceof Maybe) { return RxJavaPlugins.onAssembly((Maybe)source); } Objects.requireNonNull(source, "onSubscribe is null"); - return RxJavaPlugins.onAssembly(new MaybeUnsafeCreate(source)); + return RxJavaPlugins.onAssembly(new MaybeUnsafeCreate<>(source)); } /** @@ -1844,7 +1862,7 @@ public static Maybe wrap(MaybeSource source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe zip(Iterable> sources, Function zipper) { + public static Maybe zip(@NonNull Iterable> sources, @NonNull Function zipper) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new MaybeZipIterable(sources, zipper)); @@ -1876,13 +1894,12 @@ public static Maybe zip(Iterable> s * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, - BiFunction zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, + @NonNull BiFunction zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return zipArray(Functions.toFunction(zipper), source1, source2); @@ -1917,13 +1934,12 @@ public static Maybe zip( * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, MaybeSource source3, - Function3 zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, + @NonNull Function3 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1962,14 +1978,13 @@ public static Maybe zip( * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, MaybeSource source3, - MaybeSource source4, - Function4 zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, + @NonNull MaybeSource source4, + @NonNull Function4 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -2012,14 +2027,13 @@ public static Maybe zip( * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, MaybeSource source3, - MaybeSource source4, MaybeSource source5, - Function5 zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, + @NonNull MaybeSource source4, @NonNull MaybeSource source5, + @NonNull Function5 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -2066,14 +2080,13 @@ public static Maybe zip( * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, MaybeSource source3, - MaybeSource source4, MaybeSource source5, MaybeSource source6, - Function6 zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, + @NonNull MaybeSource source4, @NonNull MaybeSource source5, @NonNull MaybeSource source6, + @NonNull Function6 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -2124,15 +2137,14 @@ public static Maybe zip( * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, MaybeSource source3, - MaybeSource source4, MaybeSource source5, MaybeSource source6, - MaybeSource source7, - Function7 zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, + @NonNull MaybeSource source4, @NonNull MaybeSource source5, @NonNull MaybeSource source6, + @NonNull MaybeSource source7, + @NonNull Function7 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -2187,15 +2199,14 @@ public static Maybe zip( * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, MaybeSource source3, - MaybeSource source4, MaybeSource source5, MaybeSource source6, - MaybeSource source7, MaybeSource source8, - Function8 zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, + @NonNull MaybeSource source4, @NonNull MaybeSource source5, @NonNull MaybeSource source6, + @NonNull MaybeSource source7, @NonNull MaybeSource source8, + @NonNull Function8 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -2253,15 +2264,14 @@ public static Maybe zip( * @return a Maybe that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe zip( - MaybeSource source1, MaybeSource source2, MaybeSource source3, - MaybeSource source4, MaybeSource source5, MaybeSource source6, - MaybeSource source7, MaybeSource source8, MaybeSource source9, - Function9 zipper) { + @NonNull MaybeSource source1, @NonNull MaybeSource source2, @NonNull MaybeSource source3, + @NonNull MaybeSource source4, @NonNull MaybeSource source5, @NonNull MaybeSource source6, + @NonNull MaybeSource source7, @NonNull MaybeSource source8, @NonNull MaybeSource source9, + @NonNull Function9 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -2305,8 +2315,9 @@ public static Maybe zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe zipArray(Function zipper, - MaybeSource... sources) { + @SafeVarargs + public static Maybe zipArray(@NonNull Function zipper, + @NonNull MaybeSource... sources) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return empty(); @@ -2335,11 +2346,10 @@ public static Maybe zipArray(Function z * signalled * @see ReactiveX operators documentation: Amb */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe ambWith(MaybeSource other) { + public final Maybe ambWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -2359,8 +2369,9 @@ public final Maybe ambWith(MaybeSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @Nullable public final T blockingGet() { - BlockingMultiObserver observer = new BlockingMultiObserver(); + BlockingMultiObserver observer = new BlockingMultiObserver<>(); subscribe(observer); return observer.blockingGet(); } @@ -2381,9 +2392,10 @@ public final T blockingGet() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final T blockingGet(T defaultValue) { + @Nullable + public final T blockingGet(@Nullable T defaultValue) { Objects.requireNonNull(defaultValue, "defaultValue is null"); - BlockingMultiObserver observer = new BlockingMultiObserver(); + BlockingMultiObserver observer = new BlockingMultiObserver<>(); subscribe(observer); return observer.blockingGet(defaultValue); } @@ -2409,8 +2421,9 @@ public final T blockingGet(T defaultValue) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe cache() { - return RxJavaPlugins.onAssembly(new MaybeCache(this)); + return RxJavaPlugins.onAssembly(new MaybeCache<>(this)); } /** @@ -2429,7 +2442,7 @@ public final Maybe cache() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe cast(final Class clazz) { + public final Maybe cast(@NonNull Class clazz) { Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -2455,7 +2468,8 @@ public final Maybe cast(final Class clazz) { @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe compose(MaybeTransformer transformer) { + @NonNull + public final Maybe compose(@NonNull MaybeTransformer transformer) { return wrap(((MaybeTransformer) Objects.requireNonNull(transformer, "transformer is null")).apply(this)); } @@ -2478,9 +2492,9 @@ public final Maybe compose(MaybeTransformer trans @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe concatMap(Function> mapper) { + public final Maybe concatMap(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatten(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatten<>(this, mapper)); } /** @@ -2505,7 +2519,7 @@ public final Maybe concatMap(Function concatWith(MaybeSource other) { + public final Flowable concatWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); return concat(this, other); } @@ -2529,9 +2543,9 @@ public final Flowable concatWith(MaybeSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single contains(final Object item) { + public final Single contains(@NonNull Object item) { Objects.requireNonNull(item, "item is null"); - return RxJavaPlugins.onAssembly(new MaybeContains(this, item)); + return RxJavaPlugins.onAssembly(new MaybeContains<>(this, item)); } /** @@ -2550,8 +2564,9 @@ public final Single contains(final Object item) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single count() { - return RxJavaPlugins.onAssembly(new MaybeCount(this)); + return RxJavaPlugins.onAssembly(new MaybeCount<>(this)); } /** @@ -2573,9 +2588,9 @@ public final Single count() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single defaultIfEmpty(T defaultItem) { + public final Single defaultIfEmpty(@NonNull T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); - return RxJavaPlugins.onAssembly(new MaybeToSingle(this, defaultItem)); + return RxJavaPlugins.onAssembly(new MaybeToSingle<>(this, defaultItem)); } /** @@ -2598,7 +2613,8 @@ public final Single defaultIfEmpty(T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Maybe delay(long delay, TimeUnit unit) { + @NonNull + public final Maybe delay(long delay, @NonNull TimeUnit unit) { return delay(delay, unit, Schedulers.computation()); } @@ -2624,10 +2640,10 @@ public final Maybe delay(long delay, TimeUnit unit) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Maybe delay(long delay, TimeUnit unit, Scheduler scheduler) { + public final Maybe delay(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new MaybeDelay(this, Math.max(0L, delay), unit, scheduler)); + return RxJavaPlugins.onAssembly(new MaybeDelay<>(this, Math.max(0L, delay), unit, scheduler)); } /** @@ -2644,8 +2660,6 @@ public final Maybe delay(long delay, TimeUnit unit, Scheduler scheduler) { * * @param * the subscription delay value type (ignored) - * @param - * the item delay value type (ignored) * @param delayIndicator * the Publisher that gets subscribed to when this Maybe signals an event and that * signal is emitted when the Publisher signals an item or completes @@ -2656,9 +2670,9 @@ public final Maybe delay(long delay, TimeUnit unit, Scheduler scheduler) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) - public final Maybe delay(Publisher delayIndicator) { + public final Maybe delay(@NonNull Publisher delayIndicator) { Objects.requireNonNull(delayIndicator, "delayIndicator is null"); - return RxJavaPlugins.onAssembly(new MaybeDelayOtherPublisher(this, delayIndicator)); + return RxJavaPlugins.onAssembly(new MaybeDelayOtherPublisher<>(this, delayIndicator)); } /** @@ -2683,9 +2697,9 @@ public final Maybe delay(Publisher delayIndicator) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe delaySubscription(Publisher subscriptionIndicator) { + public final Maybe delaySubscription(@NonNull Publisher subscriptionIndicator) { Objects.requireNonNull(subscriptionIndicator, "subscriptionIndicator is null"); - return RxJavaPlugins.onAssembly(new MaybeDelaySubscriptionOtherPublisher(this, subscriptionIndicator)); + return RxJavaPlugins.onAssembly(new MaybeDelaySubscriptionOtherPublisher<>(this, subscriptionIndicator)); } /** @@ -2707,7 +2721,8 @@ public final Maybe delaySubscription(Publisher subscriptionIndicator) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Maybe delaySubscription(long delay, TimeUnit unit) { + @NonNull + public final Maybe delaySubscription(long delay, @NonNull TimeUnit unit) { return delaySubscription(delay, unit, Schedulers.computation()); } @@ -2733,7 +2748,8 @@ public final Maybe delaySubscription(long delay, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Maybe delaySubscription(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Maybe delaySubscription(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delaySubscription(Flowable.timer(delay, unit, scheduler)); } @@ -2753,9 +2769,9 @@ public final Maybe delaySubscription(long delay, TimeUnit unit, Scheduler sch @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doAfterSuccess(Consumer onAfterSuccess) { + public final Maybe doAfterSuccess(@NonNull Consumer onAfterSuccess) { Objects.requireNonNull(onAfterSuccess, "onAfterSuccess is null"); - return RxJavaPlugins.onAssembly(new MaybeDoAfterSuccess(this, onAfterSuccess)); + return RxJavaPlugins.onAssembly(new MaybeDoAfterSuccess<>(this, onAfterSuccess)); } /** @@ -2778,8 +2794,8 @@ public final Maybe doAfterSuccess(Consumer onAfterSuccess) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doAfterTerminate(Action onAfterTerminate) { - return RxJavaPlugins.onAssembly(new MaybePeek(this, + public final Maybe doAfterTerminate(@NonNull Action onAfterTerminate) { + return RxJavaPlugins.onAssembly(new MaybePeek<>(this, Functions.emptyConsumer(), // onSubscribe Functions.emptyConsumer(), // onSuccess Functions.emptyConsumer(), // onError @@ -2808,9 +2824,9 @@ public final Maybe doAfterTerminate(Action onAfterTerminate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doFinally(Action onFinally) { + public final Maybe doFinally(@NonNull Action onFinally) { Objects.requireNonNull(onFinally, "onFinally is null"); - return RxJavaPlugins.onAssembly(new MaybeDoFinally(this, onFinally)); + return RxJavaPlugins.onAssembly(new MaybeDoFinally<>(this, onFinally)); } /** @@ -2827,8 +2843,8 @@ public final Maybe doFinally(Action onFinally) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doOnDispose(Action onDispose) { - return RxJavaPlugins.onAssembly(new MaybePeek(this, + public final Maybe doOnDispose(@NonNull Action onDispose) { + return RxJavaPlugins.onAssembly(new MaybePeek<>(this, Functions.emptyConsumer(), // onSubscribe Functions.emptyConsumer(), // onSuccess Functions.emptyConsumer(), // onError @@ -2855,8 +2871,8 @@ public final Maybe doOnDispose(Action onDispose) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doOnComplete(Action onComplete) { - return RxJavaPlugins.onAssembly(new MaybePeek(this, + public final Maybe doOnComplete(@NonNull Action onComplete) { + return RxJavaPlugins.onAssembly(new MaybePeek<>(this, Functions.emptyConsumer(), // onSubscribe Functions.emptyConsumer(), // onSuccess Functions.emptyConsumer(), // onError @@ -2881,8 +2897,8 @@ public final Maybe doOnComplete(Action onComplete) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doOnError(Consumer onError) { - return RxJavaPlugins.onAssembly(new MaybePeek(this, + public final Maybe doOnError(@NonNull Consumer onError) { + return RxJavaPlugins.onAssembly(new MaybePeek<>(this, Functions.emptyConsumer(), // onSubscribe Functions.emptyConsumer(), // onSuccess Objects.requireNonNull(onError, "onError is null"), @@ -2910,9 +2926,10 @@ public final Maybe doOnError(Consumer onError) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doOnEvent(BiConsumer onEvent) { + @NonNull + public final Maybe doOnEvent(@NonNull BiConsumer onEvent) { Objects.requireNonNull(onEvent, "onEvent is null"); - return RxJavaPlugins.onAssembly(new MaybeDoOnEvent(this, onEvent)); + return RxJavaPlugins.onAssembly(new MaybeDoOnEvent<>(this, onEvent)); } /** @@ -2928,8 +2945,8 @@ public final Maybe doOnEvent(BiConsumer onEvent @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doOnSubscribe(Consumer onSubscribe) { - return RxJavaPlugins.onAssembly(new MaybePeek(this, + public final Maybe doOnSubscribe(@NonNull Consumer onSubscribe) { + return RxJavaPlugins.onAssembly(new MaybePeek<>(this, Objects.requireNonNull(onSubscribe, "onSubscribe is null"), Functions.emptyConsumer(), // onSuccess Functions.emptyConsumer(), // onError @@ -2961,9 +2978,9 @@ public final Maybe doOnSubscribe(Consumer onSubscribe) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doOnTerminate(final Action onTerminate) { + public final Maybe doOnTerminate(@NonNull Action onTerminate) { Objects.requireNonNull(onTerminate, "onTerminate is null"); - return RxJavaPlugins.onAssembly(new MaybeDoOnTerminate(this, onTerminate)); + return RxJavaPlugins.onAssembly(new MaybeDoOnTerminate<>(this, onTerminate)); } /** @@ -2981,8 +2998,8 @@ public final Maybe doOnTerminate(final Action onTerminate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe doOnSuccess(Consumer onSuccess) { - return RxJavaPlugins.onAssembly(new MaybePeek(this, + public final Maybe doOnSuccess(@NonNull Consumer onSuccess) { + return RxJavaPlugins.onAssembly(new MaybePeek<>(this, Functions.emptyConsumer(), // onSubscribe Objects.requireNonNull(onSuccess, "onSuccess is null"), Functions.emptyConsumer(), // onError @@ -3012,9 +3029,9 @@ public final Maybe doOnSuccess(Consumer onSuccess) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe filter(Predicate predicate) { + public final Maybe filter(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new MaybeFilter(this, predicate)); + return RxJavaPlugins.onAssembly(new MaybeFilter<>(this, predicate)); } /** @@ -3037,9 +3054,9 @@ public final Maybe filter(Predicate predicate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe flatMap(Function> mapper) { + public final Maybe flatMap(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatten(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatten<>(this, mapper)); } /** @@ -3067,13 +3084,13 @@ public final Maybe flatMap(Function Maybe flatMap( - Function> onSuccessMapper, - Function> onErrorMapper, - Supplier> onCompleteSupplier) { + @NonNull Function> onSuccessMapper, + @NonNull Function> onErrorMapper, + @NonNull Supplier> onCompleteSupplier) { Objects.requireNonNull(onSuccessMapper, "onSuccessMapper is null"); Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapNotification(this, onSuccessMapper, onErrorMapper, onCompleteSupplier)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapNotification<>(this, onSuccessMapper, onErrorMapper, onCompleteSupplier)); } /** @@ -3101,11 +3118,11 @@ public final Maybe flatMap( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe flatMap(Function> mapper, - BiFunction resultSelector) { + public final Maybe flatMap(@NonNull Function> mapper, + @NonNull BiFunction resultSelector) { Objects.requireNonNull(mapper, "mapper is null"); Objects.requireNonNull(resultSelector, "resultSelector is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapBiSelector(this, mapper, resultSelector)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapBiSelector<>(this, mapper, resultSelector)); } /** @@ -3132,9 +3149,9 @@ public final Maybe flatMap(Function Flowable flattenAsFlowable(final Function> mapper) { + public final Flowable flattenAsFlowable(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapIterableFlowable(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapIterableFlowable<>(this, mapper)); } /** @@ -3158,9 +3175,9 @@ public final Flowable flattenAsFlowable(final Function Observable flattenAsObservable(final Function> mapper) { + public final Observable flattenAsObservable(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapIterableObservable(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapIterableObservable<>(this, mapper)); } /** @@ -3182,9 +3199,9 @@ public final Observable flattenAsObservable(final Function Observable flatMapObservable(Function> mapper) { + public final Observable flatMapObservable(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapObservable(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapObservable<>(this, mapper)); } /** @@ -3210,9 +3227,9 @@ public final Observable flatMapObservable(Function Flowable flatMapPublisher(Function> mapper) { + public final Flowable flatMapPublisher(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapPublisher(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapPublisher<>(this, mapper)); } /** @@ -3236,9 +3253,9 @@ public final Flowable flatMapPublisher(Function Single flatMapSingle(final Function> mapper) { + public final Single flatMapSingle(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapSingle(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapSingle<>(this, mapper)); } /** @@ -3264,9 +3281,9 @@ public final Single flatMapSingle(final Function Maybe flatMapSingleElement(final Function> mapper) { + public final Maybe flatMapSingleElement(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapSingleElement(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapSingleElement<>(this, mapper)); } /** @@ -3288,9 +3305,9 @@ public final Maybe flatMapSingleElement(final Function mapper) { + public final Completable flatMapCompletable(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new MaybeFlatMapCompletable(this, mapper)); + return RxJavaPlugins.onAssembly(new MaybeFlatMapCompletable<>(this, mapper)); } /** @@ -3307,8 +3324,9 @@ public final Completable flatMapCompletable(final Function hide() { - return RxJavaPlugins.onAssembly(new MaybeHide(this)); + return RxJavaPlugins.onAssembly(new MaybeHide<>(this)); } /** @@ -3326,8 +3344,9 @@ public final Maybe hide() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable ignoreElement() { - return RxJavaPlugins.onAssembly(new MaybeIgnoreElementCompletable(this)); + return RxJavaPlugins.onAssembly(new MaybeIgnoreElementCompletable<>(this)); } /** @@ -3344,8 +3363,9 @@ public final Completable ignoreElement() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single isEmpty() { - return RxJavaPlugins.onAssembly(new MaybeIsEmptySingle(this)); + return RxJavaPlugins.onAssembly(new MaybeIsEmptySingle<>(this)); } /** @@ -3500,7 +3520,7 @@ public final Single isEmpty() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe lift(final MaybeOperator lift) { + public final Maybe lift(@NonNull MaybeOperator lift) { Objects.requireNonNull(lift, "lift is null"); return RxJavaPlugins.onAssembly(new MaybeLift(this, lift)); } @@ -3524,7 +3544,7 @@ public final Maybe lift(final MaybeOperator lift) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe map(Function mapper) { + public final Maybe map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new MaybeMap(this, mapper)); } @@ -3545,8 +3565,9 @@ public final Maybe map(Function mapper) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> materialize() { - return RxJavaPlugins.onAssembly(new MaybeMaterialize(this)); + return RxJavaPlugins.onAssembly(new MaybeMaterialize<>(this)); } /** @@ -3572,7 +3593,7 @@ public final Single> materialize() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable mergeWith(MaybeSource other) { + public final Flowable mergeWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); return merge(this, other); } @@ -3598,9 +3619,9 @@ public final Flowable mergeWith(MaybeSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Maybe observeOn(final Scheduler scheduler) { + public final Maybe observeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new MaybeObserveOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new MaybeObserveOn<>(this, scheduler)); } /** @@ -3622,7 +3643,7 @@ public final Maybe observeOn(final Scheduler scheduler) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe ofType(final Class clazz) { + public final Maybe ofType(@NonNull Class clazz) { Objects.requireNonNull(clazz, "clazz is null"); return filter(Functions.isInstanceOf(clazz)).cast(clazz); } @@ -3663,11 +3684,12 @@ public final R to(@NonNull MaybeConverter converter) { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable toFlowable() { if (this instanceof FuseToFlowable) { return ((FuseToFlowable)this).fuseToFlowable(); } - return RxJavaPlugins.onAssembly(new MaybeToFlowable(this)); + return RxJavaPlugins.onAssembly(new MaybeToFlowable<>(this)); } /** @@ -3682,11 +3704,12 @@ public final Flowable toFlowable() { @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable toObservable() { if (this instanceof FuseToObservable) { return ((FuseToObservable)this).fuseToObservable(); } - return RxJavaPlugins.onAssembly(new MaybeToObservable(this)); + return RxJavaPlugins.onAssembly(new MaybeToObservable<>(this)); } /** @@ -3700,8 +3723,9 @@ public final Observable toObservable() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single toSingle() { - return RxJavaPlugins.onAssembly(new MaybeToSingle(this, null)); + return RxJavaPlugins.onAssembly(new MaybeToSingle<>(this, null)); } /** @@ -3715,6 +3739,7 @@ public final Single toSingle() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe onErrorComplete() { return onErrorComplete(Functions.alwaysTrue()); } @@ -3733,10 +3758,10 @@ public final Maybe onErrorComplete() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe onErrorComplete(final Predicate predicate) { + public final Maybe onErrorComplete(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new MaybeOnErrorComplete(this, predicate)); + return RxJavaPlugins.onAssembly(new MaybeOnErrorComplete<>(this, predicate)); } /** @@ -3761,7 +3786,7 @@ public final Maybe onErrorComplete(final Predicate predica @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe onErrorResumeWith(final MaybeSource next) { + public final Maybe onErrorResumeWith(@NonNull MaybeSource next) { Objects.requireNonNull(next, "next is null"); return onErrorResumeNext(Functions.justFunction(next)); } @@ -3788,9 +3813,9 @@ public final Maybe onErrorResumeWith(final MaybeSource next) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe onErrorResumeNext(Function> resumeFunction) { + public final Maybe onErrorResumeNext(@NonNull Function> resumeFunction) { Objects.requireNonNull(resumeFunction, "resumeFunction is null"); - return RxJavaPlugins.onAssembly(new MaybeOnErrorNext(this, resumeFunction, true)); + return RxJavaPlugins.onAssembly(new MaybeOnErrorNext<>(this, resumeFunction, true)); } /** @@ -3815,9 +3840,9 @@ public final Maybe onErrorResumeNext(Function onErrorReturn(Function valueSupplier) { + public final Maybe onErrorReturn(@NonNull Function valueSupplier) { Objects.requireNonNull(valueSupplier, "valueSupplier is null"); - return RxJavaPlugins.onAssembly(new MaybeOnErrorReturn(this, valueSupplier)); + return RxJavaPlugins.onAssembly(new MaybeOnErrorReturn<>(this, valueSupplier)); } /** @@ -3841,7 +3866,7 @@ public final Maybe onErrorReturn(Function val @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe onErrorReturnItem(final T item) { + public final Maybe onErrorReturnItem(@NonNull T item) { Objects.requireNonNull(item, "item is null"); return onErrorReturn(Functions.justFunction(item)); } @@ -3871,9 +3896,9 @@ public final Maybe onErrorReturnItem(final T item) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe onExceptionResumeNext(final MaybeSource next) { + public final Maybe onExceptionResumeNext(@NonNull MaybeSource next) { Objects.requireNonNull(next, "next is null"); - return RxJavaPlugins.onAssembly(new MaybeOnErrorNext(this, Functions.justFunction(next), false)); + return RxJavaPlugins.onAssembly(new MaybeOnErrorNext<>(this, Functions.justFunction(next), false)); } /** @@ -3890,8 +3915,9 @@ public final Maybe onExceptionResumeNext(final MaybeSource next) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe onTerminateDetach() { - return RxJavaPlugins.onAssembly(new MaybeDetach(this)); + return RxJavaPlugins.onAssembly(new MaybeDetach<>(this)); } /** @@ -3911,6 +3937,7 @@ public final Maybe onTerminateDetach() { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable repeat() { return repeat(Long.MAX_VALUE); } @@ -3939,6 +3966,7 @@ public final Flowable repeat() { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable repeat(long times) { return toFlowable().repeat(times); } @@ -3966,7 +3994,8 @@ public final Flowable repeat(long times) { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable repeatUntil(BooleanSupplier stop) { + @NonNull + public final Flowable repeatUntil(@NonNull BooleanSupplier stop) { return toFlowable().repeatUntil(stop); } @@ -3995,7 +4024,8 @@ public final Flowable repeatUntil(BooleanSupplier stop) { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable repeatWhen(final Function, ? extends Publisher> handler) { + @NonNull + public final Flowable repeatWhen(@NonNull Function, ? extends Publisher> handler) { return toFlowable().repeatWhen(handler); } @@ -4017,6 +4047,7 @@ public final Flowable repeatWhen(final Function, ? e */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe retry() { return retry(Long.MAX_VALUE, Functions.alwaysTrue()); } @@ -4040,7 +4071,8 @@ public final Maybe retry() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe retry(BiPredicate predicate) { + @NonNull + public final Maybe retry(@NonNull BiPredicate predicate) { return toFlowable().retry(predicate).singleElement(); } @@ -4065,6 +4097,7 @@ public final Maybe retry(BiPredicate pred */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe retry(long count) { return retry(count, Functions.alwaysTrue()); } @@ -4082,7 +4115,8 @@ public final Maybe retry(long count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe retry(long times, Predicate predicate) { + @NonNull + public final Maybe retry(long times, @NonNull Predicate predicate) { return toFlowable().retry(times, predicate).singleElement(); } @@ -4098,7 +4132,8 @@ public final Maybe retry(long times, Predicate predicate) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe retry(Predicate predicate) { + @NonNull + public final Maybe retry(@NonNull Predicate predicate) { return retry(Long.MAX_VALUE, predicate); } @@ -4114,7 +4149,7 @@ public final Maybe retry(Predicate predicate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe retryUntil(final BooleanSupplier stop) { + public final Maybe retryUntil(@NonNull BooleanSupplier stop) { Objects.requireNonNull(stop, "stop is null"); return retry(Long.MAX_VALUE, Functions.predicateReverseFor(stop)); } @@ -4194,8 +4229,9 @@ public final Maybe retryUntil(final BooleanSupplier stop) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe retryWhen( - final Function, ? extends Publisher> handler) { + @NonNull Function, ? extends Publisher> handler) { return toFlowable().retryWhen(handler).singleElement(); } @@ -4215,6 +4251,7 @@ public final Maybe retryWhen( * @see ReactiveX operators documentation: Subscribe */ @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Disposable subscribe() { return subscribe(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -4240,7 +4277,8 @@ public final Disposable subscribe() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(Consumer onSuccess) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onSuccess) { return subscribe(onSuccess, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -4266,7 +4304,8 @@ public final Disposable subscribe(Consumer onSuccess) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(Consumer onSuccess, Consumer onError) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onSuccess, @NonNull Consumer onError) { return subscribe(onSuccess, onError, Functions.EMPTY_ACTION); } @@ -4297,8 +4336,8 @@ public final Disposable subscribe(Consumer onSuccess, Consumer onSuccess, Consumer onError, - Action onComplete) { + public final Disposable subscribe(@NonNull Consumer onSuccess, @NonNull Consumer onError, + @NonNull Action onComplete) { Objects.requireNonNull(onSuccess, "onSuccess is null"); Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); @@ -4307,7 +4346,7 @@ public final Disposable subscribe(Consumer onSuccess, Consumer observer) { + public final void subscribe(@NonNull MaybeObserver observer) { Objects.requireNonNull(observer, "observer is null"); observer = RxJavaPlugins.onSubscribe(this, observer); @@ -4333,7 +4372,7 @@ public final void subscribe(MaybeObserver observer) { * applied by {@link #subscribe(MaybeObserver)} before this method gets called. * @param observer the MaybeObserver to handle, not null */ - protected abstract void subscribeActual(MaybeObserver observer); + protected abstract void subscribeActual(@NonNull MaybeObserver observer); /** * Asynchronously subscribes subscribers to this Maybe on the specified {@link Scheduler}. @@ -4354,9 +4393,9 @@ public final void subscribe(MaybeObserver observer) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Maybe subscribeOn(Scheduler scheduler) { + public final Maybe subscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new MaybeSubscribeOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new MaybeSubscribeOn<>(this, scheduler)); } /** @@ -4384,7 +4423,8 @@ public final Maybe subscribeOn(Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > E subscribeWith(E observer) { + @NonNull + public final <@NonNull E extends MaybeObserver> E subscribeWith(E observer) { subscribe(observer); return observer; } @@ -4407,9 +4447,9 @@ public final > E subscribeWith(E observer) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe switchIfEmpty(MaybeSource other) { + public final Maybe switchIfEmpty(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new MaybeSwitchIfEmpty(this, other)); + return RxJavaPlugins.onAssembly(new MaybeSwitchIfEmpty<>(this, other)); } /** @@ -4431,9 +4471,9 @@ public final Maybe switchIfEmpty(MaybeSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single switchIfEmpty(SingleSource other) { + public final Single switchIfEmpty(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new MaybeSwitchIfEmptySingle(this, other)); + return RxJavaPlugins.onAssembly(new MaybeSwitchIfEmptySingle<>(this, other)); } /** @@ -4457,9 +4497,9 @@ public final Single switchIfEmpty(SingleSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe takeUntil(MaybeSource other) { + public final Maybe takeUntil(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new MaybeTakeUntilMaybe(this, other)); + return RxJavaPlugins.onAssembly(new MaybeTakeUntilMaybe<>(this, other)); } /** @@ -4487,9 +4527,9 @@ public final Maybe takeUntil(MaybeSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe takeUntil(Publisher other) { + public final Maybe takeUntil(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new MaybeTakeUntilPublisher(this, other)); + return RxJavaPlugins.onAssembly(new MaybeTakeUntilPublisher<>(this, other)); } /** @@ -4512,7 +4552,8 @@ public final Maybe takeUntil(Publisher other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Maybe timeout(long timeout, TimeUnit timeUnit) { + @NonNull + public final Maybe timeout(long timeout, @NonNull TimeUnit timeUnit) { return timeout(timeout, timeUnit, Schedulers.computation()); } @@ -4539,7 +4580,7 @@ public final Maybe timeout(long timeout, TimeUnit timeUnit) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Maybe timeout(long timeout, TimeUnit timeUnit, MaybeSource fallback) { + public final Maybe timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull MaybeSource fallback) { Objects.requireNonNull(fallback, "fallback is null"); return timeout(timeout, timeUnit, Schedulers.computation(), fallback); } @@ -4570,7 +4611,7 @@ public final Maybe timeout(long timeout, TimeUnit timeUnit, MaybeSource timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler, MaybeSource fallback) { + public final Maybe timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull Scheduler scheduler, @NonNull MaybeSource fallback) { Objects.requireNonNull(fallback, "fallback is null"); return timeout(timer(timeout, timeUnit, scheduler), fallback); } @@ -4598,7 +4639,8 @@ public final Maybe timeout(long timeout, TimeUnit timeUnit, Scheduler schedul */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Maybe timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler) { + @NonNull + public final Maybe timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull Scheduler scheduler) { return timeout(timer(timeout, timeUnit, scheduler)); } @@ -4617,9 +4659,9 @@ public final Maybe timeout(long timeout, TimeUnit timeUnit, Scheduler schedul @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe timeout(MaybeSource timeoutIndicator) { + public final Maybe timeout(@NonNull MaybeSource timeoutIndicator) { Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); - return RxJavaPlugins.onAssembly(new MaybeTimeoutMaybe(this, timeoutIndicator, null)); + return RxJavaPlugins.onAssembly(new MaybeTimeoutMaybe<>(this, timeoutIndicator, null)); } /** @@ -4639,10 +4681,10 @@ public final Maybe timeout(MaybeSource timeoutIndicator) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe timeout(MaybeSource timeoutIndicator, MaybeSource fallback) { + public final Maybe timeout(@NonNull MaybeSource timeoutIndicator, @NonNull MaybeSource fallback) { Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); Objects.requireNonNull(fallback, "fallback is null"); - return RxJavaPlugins.onAssembly(new MaybeTimeoutMaybe(this, timeoutIndicator, fallback)); + return RxJavaPlugins.onAssembly(new MaybeTimeoutMaybe<>(this, timeoutIndicator, fallback)); } /** @@ -4664,9 +4706,9 @@ public final Maybe timeout(MaybeSource timeoutIndicator, MaybeSource Maybe timeout(Publisher timeoutIndicator) { + public final Maybe timeout(@NonNull Publisher timeoutIndicator) { Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); - return RxJavaPlugins.onAssembly(new MaybeTimeoutPublisher(this, timeoutIndicator, null)); + return RxJavaPlugins.onAssembly(new MaybeTimeoutPublisher<>(this, timeoutIndicator, null)); } /** @@ -4690,10 +4732,10 @@ public final Maybe timeout(Publisher timeoutIndicator) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe timeout(Publisher timeoutIndicator, MaybeSource fallback) { + public final Maybe timeout(@NonNull Publisher timeoutIndicator, @NonNull MaybeSource fallback) { Objects.requireNonNull(timeoutIndicator, "timeoutIndicator is null"); Objects.requireNonNull(fallback, "fallback is null"); - return RxJavaPlugins.onAssembly(new MaybeTimeoutPublisher(this, timeoutIndicator, fallback)); + return RxJavaPlugins.onAssembly(new MaybeTimeoutPublisher<>(this, timeoutIndicator, fallback)); } /** @@ -4712,9 +4754,9 @@ public final Maybe timeout(Publisher timeoutIndicator, MaybeSource unsubscribeOn(final Scheduler scheduler) { + public final Maybe unsubscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new MaybeUnsubscribeOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new MaybeUnsubscribeOn<>(this, scheduler)); } /** @@ -4746,7 +4788,7 @@ public final Maybe unsubscribeOn(final Scheduler scheduler) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe zipWith(MaybeSource other, BiFunction zipper) { + public final Maybe zipWith(@NonNull MaybeSource other, @NonNull BiFunction zipper) { Objects.requireNonNull(other, "other is null"); return zip(this, other, zipper); } @@ -4766,8 +4808,9 @@ public final Maybe zipWith(MaybeSource other, BiFunction< */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestObserver test() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); subscribe(to); return to; } @@ -4784,8 +4827,9 @@ public final TestObserver test() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestObserver test(boolean dispose) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); if (dispose) { to.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/core/Notification.java b/src/main/java/io/reactivex/rxjava3/core/Notification.java index bb1c35c3f1..086ffafc8e 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Notification.java +++ b/src/main/java/io/reactivex/rxjava3/core/Notification.java @@ -27,7 +27,7 @@ public final class Notification { final Object value; /** Not meant to be implemented externally. */ - private Notification(Object value) { + private Notification(@Nullable Object value) { this.value = value; } diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index 0e2ff9dec1..42d7f3d00a 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -120,7 +120,7 @@ public abstract class Observable implements ObservableSource { @SchedulerSupport(SchedulerSupport.NONE) public static Observable amb(Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); - return RxJavaPlugins.onAssembly(new ObservableAmb(null, sources)); + return RxJavaPlugins.onAssembly(new ObservableAmb<>(null, sources)); } /** @@ -145,6 +145,7 @@ public static Observable amb(Iterable Observable ambArray(ObservableSource... sources) { Objects.requireNonNull(sources, "sources is null"); int len = sources.length; @@ -154,7 +155,7 @@ public static Observable ambArray(ObservableSource... source if (len == 1) { return (Observable)wrap(sources[0]); } - return RxJavaPlugins.onAssembly(new ObservableAmb(sources, null)); + return RxJavaPlugins.onAssembly(new ObservableAmb<>(sources, null)); } /** @@ -1243,6 +1244,7 @@ public static Observable concatArrayDelayError(ObservableSource Observable concatArrayEager(ObservableSource... sources) { return concatArrayEager(bufferSize(), bufferSize(), sources); } @@ -1294,6 +1296,7 @@ public static Observable concatArrayEager(int maxConcurrency, int prefetc */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Observable concatArrayEagerDelayError(ObservableSource... sources) { return concatArrayEagerDelayError(bufferSize(), bufferSize(), sources); } @@ -1550,7 +1553,7 @@ public static Observable concatEager(Iterable Observable create(ObservableOnSubscribe source) { Objects.requireNonNull(source, "source is null"); - return RxJavaPlugins.onAssembly(new ObservableCreate(source)); + return RxJavaPlugins.onAssembly(new ObservableCreate<>(source)); } /** @@ -1582,7 +1585,7 @@ public static Observable create(ObservableOnSubscribe source) { @SchedulerSupport(SchedulerSupport.NONE) public static Observable defer(Supplier> supplier) { Objects.requireNonNull(supplier, "supplier is null"); - return RxJavaPlugins.onAssembly(new ObservableDefer(supplier)); + return RxJavaPlugins.onAssembly(new ObservableDefer<>(supplier)); } /** @@ -1679,6 +1682,7 @@ public static Observable error(final Throwable exception) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @NonNull + @SafeVarargs public static Observable fromArray(T... items) { Objects.requireNonNull(items, "items is null"); if (items.length == 0) { @@ -1687,7 +1691,7 @@ public static Observable fromArray(T... items) { if (items.length == 1) { return just(items[0]); } - return RxJavaPlugins.onAssembly(new ObservableFromArray(items)); + return RxJavaPlugins.onAssembly(new ObservableFromArray<>(items)); } /** @@ -2138,7 +2142,7 @@ public static Observable generate(Supplier initialState, BiFunction Objects.requireNonNull(initialState, "initialState is null"); Objects.requireNonNull(generator, "generator is null"); Objects.requireNonNull(disposeState, "disposeState is null"); - return RxJavaPlugins.onAssembly(new ObservableGenerate(initialState, generator, disposeState)); + return RxJavaPlugins.onAssembly(new ObservableGenerate<>(initialState, generator, disposeState)); } /** @@ -2343,7 +2347,7 @@ public static Observable intervalRange(long start, long count, long initia @SchedulerSupport(SchedulerSupport.NONE) public static Observable just(T item) { Objects.requireNonNull(item, "item is null"); - return RxJavaPlugins.onAssembly(new ObservableJust(item)); + return RxJavaPlugins.onAssembly(new ObservableJust<>(item)); } /** @@ -2364,7 +2368,6 @@ public static Observable just(T item) { * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2395,7 +2398,6 @@ public static Observable just(T item1, T item2) { * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2429,7 +2431,6 @@ public static Observable just(T item1, T item2, T item3) { * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2466,7 +2467,6 @@ public static Observable just(T item1, T item2, T item3, T item4) { * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2506,7 +2506,6 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2549,7 +2548,6 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2595,7 +2593,6 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2644,7 +2641,6 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -2696,7 +2692,6 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 * @return an Observable that emits each item * @see ReactiveX operators documentation: Just */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) @@ -3713,7 +3708,7 @@ public static Single sequenceEqual(ObservableSource so Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(isEqual, "isEqual is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new ObservableSequenceEqualSingle(source1, source2, isEqual, bufferSize)); + return RxJavaPlugins.onAssembly(new ObservableSequenceEqualSingle<>(source1, source2, isEqual, bufferSize)); } /** @@ -3955,7 +3950,7 @@ public static Observable unsafeCreate(ObservableSource onSubscribe) { if (onSubscribe instanceof Observable) { throw new IllegalArgumentException("unsafeCreate(Observable) should be upgraded"); } - return RxJavaPlugins.onAssembly(new ObservableFromUnsafeSource(onSubscribe)); + return RxJavaPlugins.onAssembly(new ObservableFromUnsafeSource<>(onSubscribe)); } /** @@ -4044,7 +4039,7 @@ public static Observable wrap(ObservableSource source) { if (source instanceof Observable) { return RxJavaPlugins.onAssembly((Observable)source); } - return RxJavaPlugins.onAssembly(new ObservableFromUnsafeSource(source)); + return RxJavaPlugins.onAssembly(new ObservableFromUnsafeSource<>(source)); } /** @@ -4205,7 +4200,6 @@ public static Observable zip(IterableReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4261,7 +4255,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4318,7 +4311,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4377,7 +4369,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4440,7 +4431,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4508,7 +4498,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4579,7 +4568,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4654,7 +4642,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4734,7 +4721,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4818,7 +4804,6 @@ public static Observable zip( * @return an Observable that emits the zipped results * @see ReactiveX operators documentation: Zip */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public static Observable zip( @@ -4890,6 +4875,7 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @SafeVarargs public static Observable zipArray(Function zipper, boolean delayError, int bufferSize, ObservableSource... sources) { if (sources.length == 0) { @@ -4924,7 +4910,7 @@ public static Observable zipArray(Function all(Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new ObservableAllSingle(this, predicate)); + return RxJavaPlugins.onAssembly(new ObservableAllSingle<>(this, predicate)); } /** @@ -4944,7 +4930,6 @@ public final Single all(Predicate predicate) { * emitted an item or sent a termination notification * @see ReactiveX operators documentation: Amb */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable ambWith(ObservableSource other) { @@ -4976,7 +4961,7 @@ public final Observable ambWith(ObservableSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Single any(Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new ObservableAnySingle(this, predicate)); + return RxJavaPlugins.onAssembly(new ObservableAnySingle<>(this, predicate)); } /** @@ -4997,7 +4982,7 @@ public final Single any(Predicate predicate) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final T blockingFirst() { - BlockingFirstObserver observer = new BlockingFirstObserver(); + BlockingFirstObserver observer = new BlockingFirstObserver<>(); subscribe(observer); T v = observer.blockingGet(); if (v != null) { @@ -5025,7 +5010,7 @@ public final T blockingFirst() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final T blockingFirst(T defaultItem) { - BlockingFirstObserver observer = new BlockingFirstObserver(); + BlockingFirstObserver observer = new BlockingFirstObserver<>(); subscribe(observer); T v = observer.blockingGet(); return v != null ? v : defaultItem; @@ -5109,7 +5094,7 @@ public final Iterable blockingIterable() { @SchedulerSupport(SchedulerSupport.NONE) public final Iterable blockingIterable(int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return new BlockingObservableIterable(this, bufferSize); + return new BlockingObservableIterable<>(this, bufferSize); } /** @@ -5134,7 +5119,7 @@ public final Iterable blockingIterable(int bufferSize) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final T blockingLast() { - BlockingLastObserver observer = new BlockingLastObserver(); + BlockingLastObserver observer = new BlockingLastObserver<>(); subscribe(observer); T v = observer.blockingGet(); if (v != null) { @@ -5166,7 +5151,7 @@ public final T blockingLast() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final T blockingLast(T defaultItem) { - BlockingLastObserver observer = new BlockingLastObserver(); + BlockingLastObserver observer = new BlockingLastObserver<>(); subscribe(observer); T v = observer.blockingGet(); return v != null ? v : defaultItem; @@ -5194,7 +5179,7 @@ public final T blockingLast(T defaultItem) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Iterable blockingLatest() { - return new BlockingObservableLatest(this); + return new BlockingObservableLatest<>(this); } /** @@ -5217,7 +5202,7 @@ public final Iterable blockingLatest() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Iterable blockingMostRecent(T initialValue) { - return new BlockingObservableMostRecent(this, initialValue); + return new BlockingObservableMostRecent<>(this, initialValue); } /** @@ -5237,7 +5222,7 @@ public final Iterable blockingMostRecent(T initialValue) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Iterable blockingNext() { - return new BlockingObservableNext(this); + return new BlockingObservableNext<>(this); } /** @@ -5521,7 +5506,7 @@ public final > Observable buffer(int count, i ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(skip, "skip"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); - return RxJavaPlugins.onAssembly(new ObservableBuffer(this, count, skip, bufferSupplier)); + return RxJavaPlugins.onAssembly(new ObservableBuffer<>(this, count, skip, bufferSupplier)); } /** @@ -5653,7 +5638,7 @@ public final > Observable buffer(long timespa Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); - return RxJavaPlugins.onAssembly(new ObservableBufferTimed(this, timespan, timeskip, unit, scheduler, bufferSupplier, Integer.MAX_VALUE, false)); + return RxJavaPlugins.onAssembly(new ObservableBufferTimed<>(this, timespan, timeskip, unit, scheduler, bufferSupplier, Integer.MAX_VALUE, false)); } /** @@ -5799,7 +5784,7 @@ public final > Observable buffer( Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); ObjectHelper.verifyPositive(count, "count"); - return RxJavaPlugins.onAssembly(new ObservableBufferTimed(this, timespan, timespan, unit, scheduler, bufferSupplier, count, restartTimerOnMaxSize)); + return RxJavaPlugins.onAssembly(new ObservableBufferTimed<>(this, timespan, timespan, unit, scheduler, bufferSupplier, count, restartTimerOnMaxSize)); } /** @@ -6001,7 +5986,7 @@ public final Observable> buffer(ObservableSource boundary, final public final > Observable buffer(ObservableSource boundary, Supplier bufferSupplier) { Objects.requireNonNull(boundary, "boundary is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); - return RxJavaPlugins.onAssembly(new ObservableBufferExactBoundary(this, boundary, bufferSupplier)); + return RxJavaPlugins.onAssembly(new ObservableBufferExactBoundary<>(this, boundary, bufferSupplier)); } /** @@ -6114,7 +6099,7 @@ public final Observable cache() { @SchedulerSupport(SchedulerSupport.NONE) public final Observable cacheWithInitialCapacity(int initialCapacity) { ObjectHelper.verifyPositive(initialCapacity, "initialCapacity"); - return RxJavaPlugins.onAssembly(new ObservableCache(this, initialCapacity)); + return RxJavaPlugins.onAssembly(new ObservableCache<>(this, initialCapacity)); } /** @@ -6304,7 +6289,7 @@ public final Observable concatMap(Function(this, mapper, prefetch, ErrorMode.IMMEDIATE)); + return RxJavaPlugins.onAssembly(new ObservableConcatMap<>(this, mapper, prefetch, ErrorMode.IMMEDIATE)); } /** @@ -6339,7 +6324,7 @@ public final Observable concatMap(Function(this, mapper, prefetch, ErrorMode.IMMEDIATE, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapScheduler<>(this, mapper, prefetch, ErrorMode.IMMEDIATE, scheduler)); } /** @@ -6409,7 +6394,7 @@ public final Observable concatMapDelayError(Function(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); + return RxJavaPlugins.onAssembly(new ObservableConcatMap<>(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); } /** @@ -6444,7 +6429,7 @@ public final Observable concatMapDelayError(Function(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapScheduler<>(this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, scheduler)); } /** @@ -6500,7 +6485,7 @@ public final Observable concatMapEager(Function(this, mapper, ErrorMode.IMMEDIATE, maxConcurrency, prefetch)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapEager<>(this, mapper, ErrorMode.IMMEDIATE, maxConcurrency, prefetch)); } /** @@ -6565,7 +6550,7 @@ public final Observable concatMapEagerDelayError(Function(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, maxConcurrency, prefetch)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapEager<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, maxConcurrency, prefetch)); } /** @@ -6613,7 +6598,7 @@ public final Completable concatMapCompletable(Function mapper, int capacityHint) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); - return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable(this, mapper, ErrorMode.IMMEDIATE, capacityHint)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable<>(this, mapper, ErrorMode.IMMEDIATE, capacityHint)); } /** @@ -6703,7 +6688,7 @@ public final Completable concatMapCompletableDelayError(Function mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } /** @@ -6730,7 +6715,7 @@ public final Completable concatMapCompletableDelayError(Function Observable concatMapIterable(final Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableFlattenIterable(this, mapper)); + return RxJavaPlugins.onAssembly(new ObservableFlattenIterable<>(this, mapper)); } /** @@ -6818,7 +6803,7 @@ public final Observable concatMapMaybe(Function Observable concatMapMaybe(Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe(this, mapper, ErrorMode.IMMEDIATE, prefetch)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe<>(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } /** @@ -6913,7 +6898,7 @@ public final Observable concatMapMaybeDelayError(Function Observable concatMapMaybeDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } /** @@ -6971,7 +6956,7 @@ public final Observable concatMapSingle(Function Observable concatMapSingle(Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle(this, mapper, ErrorMode.IMMEDIATE, prefetch)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle<>(this, mapper, ErrorMode.IMMEDIATE, prefetch)); } /** @@ -7066,7 +7051,7 @@ public final Observable concatMapSingleDelayError(Function Observable concatMapSingleDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); + return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); } /** @@ -7110,7 +7095,7 @@ public final Observable concatWith(ObservableSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable concatWith(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableConcatWithSingle(this, other)); + return RxJavaPlugins.onAssembly(new ObservableConcatWithSingle<>(this, other)); } /** @@ -7131,7 +7116,7 @@ public final Observable concatWith(@NonNull SingleSource other) @SchedulerSupport(SchedulerSupport.NONE) public final Observable concatWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableConcatWithMaybe(this, other)); + return RxJavaPlugins.onAssembly(new ObservableConcatWithMaybe<>(this, other)); } /** @@ -7152,7 +7137,7 @@ public final Observable concatWith(@NonNull MaybeSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable concatWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableConcatWithCompletable(this, other)); + return RxJavaPlugins.onAssembly(new ObservableConcatWithCompletable<>(this, other)); } /** @@ -7195,7 +7180,7 @@ public final Single contains(final Object element) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single count() { - return RxJavaPlugins.onAssembly(new ObservableCountSingle(this)); + return RxJavaPlugins.onAssembly(new ObservableCountSingle<>(this)); } /** @@ -7228,7 +7213,7 @@ public final Single count() { @SchedulerSupport(SchedulerSupport.NONE) public final Observable debounce(Function> debounceSelector) { Objects.requireNonNull(debounceSelector, "debounceSelector is null"); - return RxJavaPlugins.onAssembly(new ObservableDebounce(this, debounceSelector)); + return RxJavaPlugins.onAssembly(new ObservableDebounce<>(this, debounceSelector)); } /** @@ -7308,7 +7293,7 @@ public final Observable debounce(long timeout, TimeUnit unit) { public final Observable debounce(long timeout, TimeUnit unit, Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableDebounceTimed(this, timeout, unit, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableDebounceTimed<>(this, timeout, unit, scheduler)); } /** @@ -7466,7 +7451,7 @@ public final Observable delay(long delay, TimeUnit unit, Scheduler scheduler, Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableDelay(this, delay, unit, scheduler, delayError)); + return RxJavaPlugins.onAssembly(new ObservableDelay<>(this, delay, unit, scheduler, delayError)); } /** @@ -7525,7 +7510,7 @@ public final Observable delay(ObservableSource subscriptionDelay, @SchedulerSupport(SchedulerSupport.NONE) public final Observable delaySubscription(ObservableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableDelaySubscriptionOther(this, other)); + return RxJavaPlugins.onAssembly(new ObservableDelaySubscriptionOther<>(this, other)); } /** @@ -7627,7 +7612,7 @@ public final Observable delaySubscription(long delay, TimeUnit unit, Schedule @SchedulerSupport(SchedulerSupport.NONE) public final Observable dematerialize(Function> selector) { Objects.requireNonNull(selector, "selector is null"); - return RxJavaPlugins.onAssembly(new ObservableDematerialize(this, selector)); + return RxJavaPlugins.onAssembly(new ObservableDematerialize<>(this, selector)); } /** @@ -7737,7 +7722,7 @@ public final Observable distinct(Function keySelector) { public final Observable distinct(Function keySelector, Supplier> collectionSupplier) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); - return RxJavaPlugins.onAssembly(new ObservableDistinct(this, keySelector, collectionSupplier)); + return RxJavaPlugins.onAssembly(new ObservableDistinct<>(this, keySelector, collectionSupplier)); } /** @@ -7817,7 +7802,7 @@ public final Observable distinctUntilChanged() { @SchedulerSupport(SchedulerSupport.NONE) public final Observable distinctUntilChanged(Function keySelector) { Objects.requireNonNull(keySelector, "keySelector is null"); - return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged(this, keySelector, ObjectHelper.equalsPredicate())); + return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged<>(this, keySelector, ObjectHelper.equalsPredicate())); } /** @@ -7851,7 +7836,7 @@ public final Observable distinctUntilChanged(Function keySe @SchedulerSupport(SchedulerSupport.NONE) public final Observable distinctUntilChanged(BiPredicate comparer) { Objects.requireNonNull(comparer, "comparer is null"); - return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged(this, Functions.identity(), comparer)); + return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged<>(this, Functions.identity(), comparer)); } /** @@ -7875,7 +7860,7 @@ public final Observable distinctUntilChanged(BiPredicate doAfterNext(Consumer onAfterNext) { Objects.requireNonNull(onAfterNext, "onAfterNext is null"); - return RxJavaPlugins.onAssembly(new ObservableDoAfterNext(this, onAfterNext)); + return RxJavaPlugins.onAssembly(new ObservableDoAfterNext<>(this, onAfterNext)); } /** @@ -7926,7 +7911,7 @@ public final Observable doAfterTerminate(Action onFinally) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable doFinally(Action onFinally) { Objects.requireNonNull(onFinally, "onFinally is null"); - return RxJavaPlugins.onAssembly(new ObservableDoFinally(this, onFinally)); + return RxJavaPlugins.onAssembly(new ObservableDoFinally<>(this, onFinally)); } /** @@ -7996,7 +7981,7 @@ private Observable doOnEach(Consumer onNext, Consumer(this, onNext, onError, onComplete, onAfterTerminate)); + return RxJavaPlugins.onAssembly(new ObservableDoOnEach<>(this, onNext, onError, onComplete, onAfterTerminate)); } /** @@ -8101,7 +8086,7 @@ public final Observable doOnError(Consumer onError) { public final Observable doOnLifecycle(final Consumer onSubscribe, final Action onDispose) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); Objects.requireNonNull(onDispose, "onDispose is null"); - return RxJavaPlugins.onAssembly(new ObservableDoOnLifecycle(this, onSubscribe, onDispose)); + return RxJavaPlugins.onAssembly(new ObservableDoOnLifecycle<>(this, onSubscribe, onDispose)); } /** @@ -8199,7 +8184,7 @@ public final Maybe elementAt(long index) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } - return RxJavaPlugins.onAssembly(new ObservableElementAtMaybe(this, index)); + return RxJavaPlugins.onAssembly(new ObservableElementAtMaybe<>(this, index)); } /** @@ -8229,7 +8214,7 @@ public final Single elementAt(long index, T defaultItem) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } Objects.requireNonNull(defaultItem, "defaultItem is null"); - return RxJavaPlugins.onAssembly(new ObservableElementAtSingle(this, index, defaultItem)); + return RxJavaPlugins.onAssembly(new ObservableElementAtSingle<>(this, index, defaultItem)); } /** @@ -8256,7 +8241,7 @@ public final Single elementAtOrError(long index) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } - return RxJavaPlugins.onAssembly(new ObservableElementAtSingle(this, index, null)); + return RxJavaPlugins.onAssembly(new ObservableElementAtSingle<>(this, index, null)); } /** @@ -8279,7 +8264,7 @@ public final Single elementAtOrError(long index) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable filter(Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new ObservableFilter(this, predicate)); + return RxJavaPlugins.onAssembly(new ObservableFilter<>(this, predicate)); } /** @@ -8473,7 +8458,7 @@ public final Observable flatMap(Function(this, mapper, delayErrors, maxConcurrency, bufferSize)); + return RxJavaPlugins.onAssembly(new ObservableFlatMap<>(this, mapper, delayErrors, maxConcurrency, bufferSize)); } /** @@ -8509,7 +8494,7 @@ public final Observable flatMap( Objects.requireNonNull(onNextMapper, "onNextMapper is null"); Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); - return merge(new ObservableMapNotification(this, onNextMapper, onErrorMapper, onCompleteSupplier)); + return merge(new ObservableMapNotification<>(this, onNextMapper, onErrorMapper, onCompleteSupplier)); } /** @@ -8550,7 +8535,7 @@ public final Observable flatMap( Objects.requireNonNull(onNextMapper, "onNextMapper is null"); Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); - return merge(new ObservableMapNotification(this, onNextMapper, onErrorMapper, onCompleteSupplier), maxConcurrency); + return merge(new ObservableMapNotification<>(this, onNextMapper, onErrorMapper, onCompleteSupplier), maxConcurrency); } /** @@ -8794,7 +8779,7 @@ public final Completable flatMapCompletable(Function mapper, boolean delayErrors) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableFlatMapCompletableCompletable(this, mapper, delayErrors)); + return RxJavaPlugins.onAssembly(new ObservableFlatMapCompletableCompletable<>(this, mapper, delayErrors)); } /** @@ -8820,7 +8805,7 @@ public final Completable flatMapCompletable(Function Observable flatMapIterable(final Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableFlattenIterable(this, mapper)); + return RxJavaPlugins.onAssembly(new ObservableFlattenIterable<>(this, mapper)); } /** @@ -8895,7 +8880,7 @@ public final Observable flatMapMaybe(Function Observable flatMapMaybe(Function> mapper, boolean delayErrors) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableFlatMapMaybe(this, mapper, delayErrors)); + return RxJavaPlugins.onAssembly(new ObservableFlatMapMaybe<>(this, mapper, delayErrors)); } /** @@ -8937,7 +8922,7 @@ public final Observable flatMapSingle(Function Observable flatMapSingle(Function> mapper, boolean delayErrors) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableFlatMapSingle(this, mapper, delayErrors)); + return RxJavaPlugins.onAssembly(new ObservableFlatMapSingle<>(this, mapper, delayErrors)); } /** @@ -9048,7 +9033,7 @@ public final Disposable forEachWhile(final Predicate onNext, Consumer Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); - ForEachWhileObserver o = new ForEachWhileObserver(onNext, onError, onComplete); + ForEachWhileObserver o = new ForEachWhileObserver<>(onNext, onError, onComplete); subscribe(o); return o; } @@ -9346,7 +9331,7 @@ public final Observable groupJoin( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable hide() { - return RxJavaPlugins.onAssembly(new ObservableHide(this)); + return RxJavaPlugins.onAssembly(new ObservableHide<>(this)); } /** @@ -9364,7 +9349,7 @@ public final Observable hide() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Completable ignoreElements() { - return RxJavaPlugins.onAssembly(new ObservableIgnoreElementsCompletable(this)); + return RxJavaPlugins.onAssembly(new ObservableIgnoreElementsCompletable<>(this)); } /** @@ -9452,7 +9437,7 @@ public final Observable join( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Maybe lastElement() { - return RxJavaPlugins.onAssembly(new ObservableLastMaybe(this)); + return RxJavaPlugins.onAssembly(new ObservableLastMaybe<>(this)); } /** @@ -9475,7 +9460,7 @@ public final Maybe lastElement() { @SchedulerSupport(SchedulerSupport.NONE) public final Single last(T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); - return RxJavaPlugins.onAssembly(new ObservableLastSingle(this, defaultItem)); + return RxJavaPlugins.onAssembly(new ObservableLastSingle<>(this, defaultItem)); } /** @@ -9495,7 +9480,7 @@ public final Single last(T defaultItem) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single lastOrError() { - return RxJavaPlugins.onAssembly(new ObservableLastSingle(this, null)); + return RxJavaPlugins.onAssembly(new ObservableLastSingle<>(this, null)); } /** @@ -9689,7 +9674,7 @@ public final Observable map(Function mapper) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable> materialize() { - return RxJavaPlugins.onAssembly(new ObservableMaterialize(this)); + return RxJavaPlugins.onAssembly(new ObservableMaterialize<>(this)); } /** @@ -9736,7 +9721,7 @@ public final Observable mergeWith(ObservableSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable mergeWith(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableMergeWithSingle(this, other)); + return RxJavaPlugins.onAssembly(new ObservableMergeWithSingle<>(this, other)); } /** @@ -9760,7 +9745,7 @@ public final Observable mergeWith(@NonNull SingleSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable mergeWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableMergeWithMaybe(this, other)); + return RxJavaPlugins.onAssembly(new ObservableMergeWithMaybe<>(this, other)); } /** @@ -9781,7 +9766,7 @@ public final Observable mergeWith(@NonNull MaybeSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable mergeWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableMergeWithCompletable(this, other)); + return RxJavaPlugins.onAssembly(new ObservableMergeWithCompletable<>(this, other)); } /** @@ -9897,7 +9882,7 @@ public final Observable observeOn(Scheduler scheduler, boolean delayError) { public final Observable observeOn(Scheduler scheduler, boolean delayError, int bufferSize) { Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new ObservableObserveOn(this, scheduler, delayError, bufferSize)); + return RxJavaPlugins.onAssembly(new ObservableObserveOn<>(this, scheduler, delayError, bufferSize)); } /** @@ -9955,7 +9940,7 @@ public final Observable ofType(final Class clazz) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable onErrorResumeNext(Function> resumeFunction) { Objects.requireNonNull(resumeFunction, "resumeFunction is null"); - return RxJavaPlugins.onAssembly(new ObservableOnErrorNext(this, resumeFunction)); + return RxJavaPlugins.onAssembly(new ObservableOnErrorNext<>(this, resumeFunction)); } /** @@ -10024,7 +10009,7 @@ public final Observable onErrorResumeWith(final ObservableSource @SchedulerSupport(SchedulerSupport.NONE) public final Observable onErrorReturn(Function valueSupplier) { Objects.requireNonNull(valueSupplier, "valueSupplier is null"); - return RxJavaPlugins.onAssembly(new ObservableOnErrorReturn(this, valueSupplier)); + return RxJavaPlugins.onAssembly(new ObservableOnErrorReturn<>(this, valueSupplier)); } /** @@ -10076,7 +10061,7 @@ public final Observable onErrorReturnItem(final T item) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Observable onTerminateDetach() { - return RxJavaPlugins.onAssembly(new ObservableDetach(this)); + return RxJavaPlugins.onAssembly(new ObservableDetach<>(this)); } /** @@ -10097,7 +10082,7 @@ public final Observable onTerminateDetach() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final ConnectableObservable publish() { - return RxJavaPlugins.onAssembly(new ObservablePublish(this)); + return RxJavaPlugins.onAssembly(new ObservablePublish<>(this)); } /** @@ -10123,7 +10108,7 @@ public final ConnectableObservable publish() { @SchedulerSupport(SchedulerSupport.NONE) public final Observable publish(Function, ? extends ObservableSource> selector) { Objects.requireNonNull(selector, "selector is null"); - return RxJavaPlugins.onAssembly(new ObservablePublishSelector(this, selector)); + return RxJavaPlugins.onAssembly(new ObservablePublishSelector<>(this, selector)); } /** @@ -10158,7 +10143,7 @@ public final Observable publish(Function, ? extends @SchedulerSupport(SchedulerSupport.NONE) public final Maybe reduce(BiFunction reducer) { Objects.requireNonNull(reducer, "reducer is null"); - return RxJavaPlugins.onAssembly(new ObservableReduceMaybe(this, reducer)); + return RxJavaPlugins.onAssembly(new ObservableReduceMaybe<>(this, reducer)); } /** @@ -10216,7 +10201,7 @@ public final Maybe reduce(BiFunction reducer) { public final Single reduce(R seed, BiFunction reducer) { Objects.requireNonNull(seed, "seed is null"); Objects.requireNonNull(reducer, "reducer is null"); - return RxJavaPlugins.onAssembly(new ObservableReduceSeedSingle(this, seed, reducer)); + return RxJavaPlugins.onAssembly(new ObservableReduceSeedSingle<>(this, seed, reducer)); } /** @@ -10256,7 +10241,7 @@ public final Single reduce(R seed, BiFunction reducer) { public final Single reduceWith(Supplier seedSupplier, BiFunction reducer) { Objects.requireNonNull(seedSupplier, "seedSupplier is null"); Objects.requireNonNull(reducer, "reducer is null"); - return RxJavaPlugins.onAssembly(new ObservableReduceWithSingle(this, seedSupplier, reducer)); + return RxJavaPlugins.onAssembly(new ObservableReduceWithSingle<>(this, seedSupplier, reducer)); } /** @@ -10305,7 +10290,7 @@ public final Observable repeat(long times) { if (times == 0) { return empty(); } - return RxJavaPlugins.onAssembly(new ObservableRepeat(this, times)); + return RxJavaPlugins.onAssembly(new ObservableRepeat<>(this, times)); } /** @@ -10331,7 +10316,7 @@ public final Observable repeat(long times) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable repeatUntil(BooleanSupplier stop) { Objects.requireNonNull(stop, "stop is null"); - return RxJavaPlugins.onAssembly(new ObservableRepeatUntil(this, stop)); + return RxJavaPlugins.onAssembly(new ObservableRepeatUntil<>(this, stop)); } /** @@ -10357,7 +10342,7 @@ public final Observable repeatUntil(BooleanSupplier stop) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable repeatWhen(final Function, ? extends ObservableSource> handler) { Objects.requireNonNull(handler, "handler is null"); - return RxJavaPlugins.onAssembly(new ObservableRepeatWhen(this, handler)); + return RxJavaPlugins.onAssembly(new ObservableRepeatWhen<>(this, handler)); } /** @@ -11048,7 +11033,7 @@ public final Observable retry() { public final Observable retry(BiPredicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new ObservableRetryBiPredicate(this, predicate)); + return RxJavaPlugins.onAssembly(new ObservableRetryBiPredicate<>(this, predicate)); } /** @@ -11101,7 +11086,7 @@ public final Observable retry(long times, Predicate predic } Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new ObservableRetryPredicate(this, times, predicate)); + return RxJavaPlugins.onAssembly(new ObservableRetryPredicate<>(this, times, predicate)); } /** @@ -11218,7 +11203,7 @@ public final Observable retryUntil(final BooleanSupplier stop) { public final Observable retryWhen( final Function, ? extends ObservableSource> handler) { Objects.requireNonNull(handler, "handler is null"); - return RxJavaPlugins.onAssembly(new ObservableRetryWhen(this, handler)); + return RxJavaPlugins.onAssembly(new ObservableRetryWhen<>(this, handler)); } /** @@ -11325,7 +11310,7 @@ public final Observable sample(long period, TimeUnit unit, boolean emitLast) public final Observable sample(long period, TimeUnit unit, Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableSampleTimed(this, period, unit, scheduler, false)); + return RxJavaPlugins.onAssembly(new ObservableSampleTimed<>(this, period, unit, scheduler, false)); } /** @@ -11361,7 +11346,7 @@ public final Observable sample(long period, TimeUnit unit, Scheduler schedule public final Observable sample(long period, TimeUnit unit, Scheduler scheduler, boolean emitLast) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableSampleTimed(this, period, unit, scheduler, emitLast)); + return RxJavaPlugins.onAssembly(new ObservableSampleTimed<>(this, period, unit, scheduler, emitLast)); } /** @@ -11386,7 +11371,7 @@ public final Observable sample(long period, TimeUnit unit, Scheduler schedule @SchedulerSupport(SchedulerSupport.NONE) public final Observable sample(ObservableSource sampler) { Objects.requireNonNull(sampler, "sampler is null"); - return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable(this, sampler, false)); + return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable<>(this, sampler, false)); } /** @@ -11418,7 +11403,7 @@ public final Observable sample(ObservableSource sampler) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable sample(ObservableSource sampler, boolean emitLast) { Objects.requireNonNull(sampler, "sampler is null"); - return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable(this, sampler, emitLast)); + return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable<>(this, sampler, emitLast)); } /** @@ -11446,7 +11431,7 @@ public final Observable sample(ObservableSource sampler, boolean emitL @SchedulerSupport(SchedulerSupport.NONE) public final Observable scan(BiFunction accumulator) { Objects.requireNonNull(accumulator, "accumulator is null"); - return RxJavaPlugins.onAssembly(new ObservableScan(this, accumulator)); + return RxJavaPlugins.onAssembly(new ObservableScan<>(this, accumulator)); } /** @@ -11531,7 +11516,7 @@ public final Observable scan(final R initialValue, BiFunction Observable scanWith(Supplier seedSupplier, BiFunction accumulator) { Objects.requireNonNull(seedSupplier, "seedSupplier is null"); Objects.requireNonNull(accumulator, "accumulator is null"); - return RxJavaPlugins.onAssembly(new ObservableScanSeed(this, seedSupplier, accumulator)); + return RxJavaPlugins.onAssembly(new ObservableScanSeed<>(this, seedSupplier, accumulator)); } /** @@ -11557,7 +11542,7 @@ public final Observable scanWith(Supplier seedSupplier, BiFunction serialize() { - return RxJavaPlugins.onAssembly(new ObservableSerialized(this)); + return RxJavaPlugins.onAssembly(new ObservableSerialized<>(this)); } /** @@ -11599,7 +11584,7 @@ public final Observable share() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Maybe singleElement() { - return RxJavaPlugins.onAssembly(new ObservableSingleMaybe(this)); + return RxJavaPlugins.onAssembly(new ObservableSingleMaybe<>(this)); } /** @@ -11622,7 +11607,7 @@ public final Maybe singleElement() { @SchedulerSupport(SchedulerSupport.NONE) public final Single single(T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); - return RxJavaPlugins.onAssembly(new ObservableSingleSingle(this, defaultItem)); + return RxJavaPlugins.onAssembly(new ObservableSingleSingle<>(this, defaultItem)); } /** @@ -11643,7 +11628,7 @@ public final Single single(T defaultItem) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Single singleOrError() { - return RxJavaPlugins.onAssembly(new ObservableSingleSingle(this, null)); + return RxJavaPlugins.onAssembly(new ObservableSingleSingle<>(this, null)); } /** @@ -11668,7 +11653,7 @@ public final Observable skip(long count) { if (count <= 0) { return RxJavaPlugins.onAssembly(this); } - return RxJavaPlugins.onAssembly(new ObservableSkip(this, count)); + return RxJavaPlugins.onAssembly(new ObservableSkip<>(this, count)); } /** @@ -11753,7 +11738,7 @@ public final Observable skipLast(int count) { if (count == 0) { return RxJavaPlugins.onAssembly(this); } - return RxJavaPlugins.onAssembly(new ObservableSkipLast(this, count)); + return RxJavaPlugins.onAssembly(new ObservableSkipLast<>(this, count)); } /** @@ -11907,7 +11892,7 @@ public final Observable skipLast(long time, TimeUnit unit, Scheduler schedule ObjectHelper.verifyPositive(bufferSize, "bufferSize"); // the internal buffer holds pairs of (timestamp, value) so double the default buffer size int s = bufferSize << 1; - return RxJavaPlugins.onAssembly(new ObservableSkipLastTimed(this, time, unit, scheduler, s, delayError)); + return RxJavaPlugins.onAssembly(new ObservableSkipLastTimed<>(this, time, unit, scheduler, s, delayError)); } /** @@ -11932,7 +11917,7 @@ public final Observable skipLast(long time, TimeUnit unit, Scheduler schedule @SchedulerSupport(SchedulerSupport.NONE) public final Observable skipUntil(ObservableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableSkipUntil(this, other)); + return RxJavaPlugins.onAssembly(new ObservableSkipUntil<>(this, other)); } /** @@ -11955,7 +11940,7 @@ public final Observable skipUntil(ObservableSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable skipWhile(Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new ObservableSkipWhile(this, predicate)); + return RxJavaPlugins.onAssembly(new ObservableSkipWhile<>(this, predicate)); } /** @@ -12217,7 +12202,7 @@ public final Disposable subscribe(Consumer onNext, Consumer ls = new LambdaObserver(onNext, onError, onComplete, Functions.emptyConsumer()); + LambdaObserver ls = new LambdaObserver<>(onNext, onError, onComplete, Functions.emptyConsumer()); subscribe(ls); @@ -12310,7 +12295,7 @@ public final > E subscribeWith(E observer) { @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable subscribeOn(Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableSubscribeOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableSubscribeOn<>(this, scheduler)); } /** @@ -12333,7 +12318,7 @@ public final Observable subscribeOn(Scheduler scheduler) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable switchIfEmpty(ObservableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableSwitchIfEmpty(this, other)); + return RxJavaPlugins.onAssembly(new ObservableSwitchIfEmpty<>(this, other)); } /** @@ -12401,7 +12386,7 @@ public final Observable switchMap(Function(this, mapper, bufferSize, false)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMap<>(this, mapper, bufferSize, false)); } /** @@ -12442,7 +12427,7 @@ public final Observable switchMap(Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable(this, mapper, false)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable<>(this, mapper, false)); } /** @@ -12484,7 +12469,7 @@ public final Completable switchMapCompletable(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable(this, mapper, true)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable<>(this, mapper, true)); } /** @@ -12520,7 +12505,7 @@ public final Completable switchMapCompletableDelayError(@NonNull Function Observable switchMapMaybe(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe(this, mapper, false)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe<>(this, mapper, false)); } /** @@ -12546,7 +12531,7 @@ public final Observable switchMapMaybe(@NonNull Function Observable switchMapMaybeDelayError(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe(this, mapper, true)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe<>(this, mapper, true)); } /** @@ -12577,7 +12562,7 @@ public final Observable switchMapMaybeDelayError(@NonNull Function Observable switchMapSingle(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableSwitchMapSingle(this, mapper, false)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMapSingle<>(this, mapper, false)); } /** @@ -12609,7 +12594,7 @@ public final Observable switchMapSingle(@NonNull Function Observable switchMapSingleDelayError(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableSwitchMapSingle(this, mapper, true)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMapSingle<>(this, mapper, true)); } /** @@ -12681,7 +12666,7 @@ public final Observable switchMapDelayError(Function(this, mapper, bufferSize, true)); + return RxJavaPlugins.onAssembly(new ObservableSwitchMap<>(this, mapper, bufferSize, true)); } /** @@ -12710,7 +12695,7 @@ public final Observable take(long count) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); } - return RxJavaPlugins.onAssembly(new ObservableTake(this, count)); + return RxJavaPlugins.onAssembly(new ObservableTake<>(this, count)); } /** @@ -12793,12 +12778,12 @@ public final Observable takeLast(int count) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); } if (count == 0) { - return RxJavaPlugins.onAssembly(new ObservableIgnoreElements(this)); + return RxJavaPlugins.onAssembly(new ObservableIgnoreElements<>(this)); } if (count == 1) { - return RxJavaPlugins.onAssembly(new ObservableTakeLastOne(this)); + return RxJavaPlugins.onAssembly(new ObservableTakeLastOne<>(this)); } - return RxJavaPlugins.onAssembly(new ObservableTakeLast(this, count)); + return RxJavaPlugins.onAssembly(new ObservableTakeLast<>(this, count)); } /** @@ -12900,7 +12885,7 @@ public final Observable takeLast(long count, long time, TimeUnit unit, Schedu if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); } - return RxJavaPlugins.onAssembly(new ObservableTakeLastTimed(this, count, time, unit, scheduler, bufferSize, delayError)); + return RxJavaPlugins.onAssembly(new ObservableTakeLastTimed<>(this, count, time, unit, scheduler, bufferSize, delayError)); } /** @@ -13068,7 +13053,7 @@ public final Observable takeLast(long time, TimeUnit unit, Scheduler schedule @SchedulerSupport(SchedulerSupport.NONE) public final Observable takeUntil(ObservableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new ObservableTakeUntil(this, other)); + return RxJavaPlugins.onAssembly(new ObservableTakeUntil<>(this, other)); } /** @@ -13097,7 +13082,7 @@ public final Observable takeUntil(ObservableSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable takeUntil(Predicate stopPredicate) { Objects.requireNonNull(stopPredicate, "stopPredicate is null"); - return RxJavaPlugins.onAssembly(new ObservableTakeUntilPredicate(this, stopPredicate)); + return RxJavaPlugins.onAssembly(new ObservableTakeUntilPredicate<>(this, stopPredicate)); } /** @@ -13121,7 +13106,7 @@ public final Observable takeUntil(Predicate stopPredicate) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable takeWhile(Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new ObservableTakeWhile(this, predicate)); + return RxJavaPlugins.onAssembly(new ObservableTakeWhile<>(this, predicate)); } /** @@ -13178,7 +13163,7 @@ public final Observable throttleFirst(long windowDuration, TimeUnit unit) { public final Observable throttleFirst(long skipDuration, TimeUnit unit, Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableThrottleFirstTimed(this, skipDuration, unit, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableThrottleFirstTimed<>(this, skipDuration, unit, scheduler)); } /** @@ -13365,7 +13350,7 @@ public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler scheduler, boolean emitLast) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableThrottleLatest(this, timeout, unit, scheduler, emitLast)); + return RxJavaPlugins.onAssembly(new ObservableThrottleLatest<>(this, timeout, unit, scheduler, emitLast)); } /** @@ -13518,7 +13503,7 @@ public final Observable> timeInterval(TimeUnit unit) { public final Observable> timeInterval(TimeUnit unit, Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableTimeInterval(this, unit, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableTimeInterval<>(this, unit, scheduler)); } /** @@ -13779,7 +13764,7 @@ private Observable timeout0(long timeout, TimeUnit timeUnit, ObservableSource Scheduler scheduler) { Objects.requireNonNull(timeUnit, "timeUnit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableTimeoutTimed(this, timeout, timeUnit, scheduler, other)); + return RxJavaPlugins.onAssembly(new ObservableTimeoutTimed<>(this, timeout, timeUnit, scheduler, other)); } private Observable timeout0( @@ -13787,7 +13772,7 @@ private Observable timeout0( Function> itemTimeoutIndicator, ObservableSource other) { Objects.requireNonNull(itemTimeoutIndicator, "itemTimeoutIndicator is null"); - return RxJavaPlugins.onAssembly(new ObservableTimeout(this, firstTimeoutIndicator, itemTimeoutIndicator, other)); + return RxJavaPlugins.onAssembly(new ObservableTimeout<>(this, firstTimeoutIndicator, itemTimeoutIndicator, other)); } /** @@ -13995,7 +13980,7 @@ public final Single> toList(final int capacityHint) { @SchedulerSupport(SchedulerSupport.NONE) public final > Single toList(Supplier collectionSupplier) { Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); - return RxJavaPlugins.onAssembly(new ObservableToListSingle(this, collectionSupplier)); + return RxJavaPlugins.onAssembly(new ObservableToListSingle<>(this, collectionSupplier)); } /** @@ -14284,7 +14269,7 @@ public final Single>> toMultimap( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final Flowable toFlowable(BackpressureStrategy strategy) { - Flowable f = new FlowableFromObservable(this); + Flowable f = new FlowableFromObservable<>(this); switch (strategy) { case DROP: @@ -14294,7 +14279,7 @@ public final Flowable toFlowable(BackpressureStrategy strategy) { case MISSING: return f; case ERROR: - return RxJavaPlugins.onAssembly(new FlowableOnBackpressureError(f)); + return RxJavaPlugins.onAssembly(new FlowableOnBackpressureError<>(f)); default: return f.onBackpressureBuffer(); } @@ -14439,7 +14424,7 @@ public final Single> toSortedList(int capacityHint) { @SchedulerSupport(SchedulerSupport.CUSTOM) public final Observable unsubscribeOn(Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableUnsubscribeOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableUnsubscribeOn<>(this, scheduler)); } /** @@ -14525,7 +14510,7 @@ public final Observable> window(long count, long skip, int bufferS ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(skip, "skip"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new ObservableWindow(this, count, skip, bufferSize)); + return RxJavaPlugins.onAssembly(new ObservableWindow<>(this, count, skip, bufferSize)); } /** @@ -14635,7 +14620,7 @@ public final Observable> window(long timespan, long timeskip, Time ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(unit, "unit is null"); - return RxJavaPlugins.onAssembly(new ObservableWindowTimed(this, timespan, timeskip, unit, scheduler, Long.MAX_VALUE, bufferSize, false)); + return RxJavaPlugins.onAssembly(new ObservableWindowTimed<>(this, timespan, timeskip, unit, scheduler, Long.MAX_VALUE, bufferSize, false)); } /** @@ -14906,7 +14891,7 @@ public final Observable> window( Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(unit, "unit is null"); ObjectHelper.verifyPositive(count, "count"); - return RxJavaPlugins.onAssembly(new ObservableWindowTimed(this, timespan, timespan, unit, scheduler, count, bufferSize, restart)); + return RxJavaPlugins.onAssembly(new ObservableWindowTimed<>(this, timespan, timespan, unit, scheduler, count, bufferSize, restart)); } /** @@ -14972,7 +14957,7 @@ public final Observable> window(ObservableSource boundary) public final Observable> window(ObservableSource boundary, int bufferSize) { Objects.requireNonNull(boundary, "boundary is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new ObservableWindowBoundary(this, boundary, bufferSize)); + return RxJavaPlugins.onAssembly(new ObservableWindowBoundary<>(this, boundary, bufferSize)); } /** @@ -15049,7 +15034,7 @@ public final Observable> window( Objects.requireNonNull(openingIndicator, "openingIndicator is null"); Objects.requireNonNull(closingIndicator, "closingIndicator is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new ObservableWindowBoundarySelector(this, openingIndicator, closingIndicator, bufferSize)); + return RxJavaPlugins.onAssembly(new ObservableWindowBoundarySelector<>(this, openingIndicator, closingIndicator, bufferSize)); } /** @@ -15230,7 +15215,7 @@ public final Observable withLatestFrom( public final Observable withLatestFrom(ObservableSource[] others, Function combiner) { Objects.requireNonNull(others, "others is null"); Objects.requireNonNull(combiner, "combiner is null"); - return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany(this, others, combiner)); + return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany<>(this, others, combiner)); } /** @@ -15259,7 +15244,7 @@ public final Observable withLatestFrom(ObservableSource[] others, Func public final Observable withLatestFrom(Iterable> others, Function combiner) { Objects.requireNonNull(others, "others is null"); Objects.requireNonNull(combiner, "combiner is null"); - return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany(this, others, combiner)); + return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany<>(this, others, combiner)); } /** @@ -15447,7 +15432,7 @@ public final Observable zipWith(ObservableSource other, @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final TestObserver test() { // NoPMD - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); subscribe(to); return to; } @@ -15467,7 +15452,7 @@ public final TestObserver test() { // NoPMD @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public final TestObserver test(boolean dispose) { // NoPMD - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); if (dispose) { to.dispose(); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Single.java b/src/main/java/io/reactivex/rxjava3/core/Single.java index bb742d4a03..ce7ef85608 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Single.java +++ b/src/main/java/io/reactivex/rxjava3/core/Single.java @@ -132,9 +132,9 @@ public abstract class Single implements SingleSource { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single amb(final Iterable> sources) { + public static Single amb(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); - return RxJavaPlugins.onAssembly(new SingleAmb(null, sources)); + return RxJavaPlugins.onAssembly(new SingleAmb<>(null, sources)); } /** @@ -154,15 +154,18 @@ public static Single amb(final Iterable Single ambArray(final SingleSource... sources) { + @SafeVarargs + @NonNull + public static Single ambArray(@NonNull SingleSource... sources) { if (sources.length == 0) { return error(SingleInternalHelper.emptyThrower()); } if (sources.length == 1) { - return wrap((SingleSource)sources[0]); + @SuppressWarnings("unchecked") + SingleSource source = (SingleSource)sources[0]; + return wrap(source); } - return RxJavaPlugins.onAssembly(new SingleAmb(sources, null)); + return RxJavaPlugins.onAssembly(new SingleAmb<>(sources, null)); } /** @@ -185,7 +188,7 @@ public static Single ambArray(final SingleSource... sources) @NonNull @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.FULL) - public static Flowable concat(Iterable> sources) { + public static Flowable concat(@NonNull Iterable> sources) { return concat(Flowable.fromIterable(sources)); } @@ -207,7 +210,7 @@ public static Flowable concat(Iterable Observable concat(ObservableSource> sources) { + public static Observable concat(@NonNull ObservableSource> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableConcatMap(sources, SingleInternalHelper.toObservable(), 2, ErrorMode.IMMEDIATE)); } @@ -233,7 +236,7 @@ public static Observable concat(ObservableSource Flowable concat(Publisher> sources) { + public static Flowable concat(@NonNull Publisher> sources) { return concat(sources, 2); } @@ -260,7 +263,7 @@ public static Flowable concat(Publisher Flowable concat(Publisher> sources, int prefetch) { + public static Flowable concat(@NonNull Publisher> sources, int prefetch) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new FlowableConcatMapPublisher(sources, SingleInternalHelper.toFlowable(), prefetch, ErrorMode.IMMEDIATE)); @@ -289,9 +292,8 @@ public static Flowable concat(Publisher Flowable concat( - SingleSource source1, SingleSource source2 + @NonNull SingleSource source1, @NonNull SingleSource source2 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -323,10 +325,9 @@ public static Flowable concat( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable concat( - SingleSource source1, SingleSource source2, - SingleSource source3 + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -361,10 +362,9 @@ public static Flowable concat( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable concat( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4 + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -394,7 +394,8 @@ public static Flowable concat( @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "unchecked", "rawtypes" }) - public static Flowable concatArray(SingleSource... sources) { + @SafeVarargs + public static Flowable concatArray(@NonNull SingleSource... sources) { return RxJavaPlugins.onAssembly(new FlowableConcatMap(Flowable.fromArray(sources), SingleInternalHelper.toFlowable(), 2, ErrorMode.BOUNDARY)); } @@ -420,7 +421,8 @@ public static Flowable concatArray(SingleSource... sources) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concatArrayEager(SingleSource... sources) { + @SafeVarargs + public static Flowable concatArrayEager(@NonNull SingleSource... sources) { return Flowable.fromArray(sources).concatMapEager(SingleInternalHelper.toFlowable()); } @@ -448,7 +450,7 @@ public static Flowable concatArrayEager(SingleSource... sour @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concatEager(Publisher> sources) { + public static Flowable concatEager(@NonNull Publisher> sources) { return Flowable.fromPublisher(sources).concatMapEager(SingleInternalHelper.toFlowable()); } @@ -474,7 +476,7 @@ public static Flowable concatEager(Publisher Flowable concatEager(Iterable> sources) { + public static Flowable concatEager(@NonNull Iterable> sources) { return Flowable.fromIterable(sources).concatMapEager(SingleInternalHelper.toFlowable()); } @@ -523,9 +525,9 @@ public static Flowable concatEager(Iterable Single create(SingleOnSubscribe source) { + public static <@NonNull T> Single create(@NonNull SingleOnSubscribe source) { Objects.requireNonNull(source, "source is null"); - return RxJavaPlugins.onAssembly(new SingleCreate(source)); + return RxJavaPlugins.onAssembly(new SingleCreate<>(source)); } /** @@ -545,9 +547,9 @@ public static Single create(SingleOnSubscribe source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single defer(final Supplier> singleSupplier) { + public static Single defer(@NonNull Supplier> singleSupplier) { Objects.requireNonNull(singleSupplier, "singleSupplier is null"); - return RxJavaPlugins.onAssembly(new SingleDefer(singleSupplier)); + return RxJavaPlugins.onAssembly(new SingleDefer<>(singleSupplier)); } /** @@ -566,7 +568,7 @@ public static Single defer(final Supplier Single error(final Supplier errorSupplier) { + public static Single error(@NonNull Supplier errorSupplier) { Objects.requireNonNull(errorSupplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new SingleError(errorSupplier)); } @@ -592,7 +594,7 @@ public static Single error(final Supplier errorSuppl @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single error(final Throwable exception) { + public static Single error(@NonNull Throwable exception) { Objects.requireNonNull(exception, "exception is null"); return error(Functions.justSupplier(exception)); } @@ -628,7 +630,7 @@ public static Single error(final Throwable exception) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single fromCallable(final Callable callable) { + public static <@NonNull T> Single fromCallable(@NonNull Callable callable) { Objects.requireNonNull(callable, "callable is null"); return RxJavaPlugins.onAssembly(new SingleFromCallable(callable)); } @@ -658,7 +660,8 @@ public static Single fromCallable(final Callable callable) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single fromFuture(Future future) { + @NonNull + public static <@NonNull T> Single fromFuture(@NonNull Future future) { return toSingle(Flowable.fromFuture(future)); } @@ -691,7 +694,8 @@ public static Single fromFuture(Future future) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single fromFuture(Future future, long timeout, TimeUnit unit) { + @NonNull + public static <@NonNull T> Single fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) { return toSingle(Flowable.fromFuture(future, timeout, unit)); } @@ -726,7 +730,8 @@ public static Single fromFuture(Future future, long timeout, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Single fromFuture(Future future, long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public static <@NonNull T> Single fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return toSingle(Flowable.fromFuture(future, timeout, unit, scheduler)); } @@ -756,7 +761,8 @@ public static Single fromFuture(Future future, long timeout, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Single fromFuture(Future future, Scheduler scheduler) { + @NonNull + public static <@NonNull T> Single fromFuture(@NonNull Future future, @NonNull Scheduler scheduler) { return toSingle(Flowable.fromFuture(future, scheduler)); } @@ -793,7 +799,7 @@ public static Single fromFuture(Future future, Scheduler sch @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single fromPublisher(final Publisher publisher) { + public static Single fromPublisher(@NonNull Publisher publisher) { Objects.requireNonNull(publisher, "publisher is null"); return RxJavaPlugins.onAssembly(new SingleFromPublisher(publisher)); } @@ -817,7 +823,7 @@ public static Single fromPublisher(final Publisher publisher @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single fromObservable(ObservableSource observableSource) { + public static Single fromObservable(@NonNull ObservableSource observableSource) { Objects.requireNonNull(observableSource, "observableSource is null"); return RxJavaPlugins.onAssembly(new ObservableSingleSingle(observableSource, null)); } @@ -855,7 +861,7 @@ public static Single fromObservable(ObservableSource observa @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single fromSupplier(final Supplier supplier) { + public static <@NonNull T> Single fromSupplier(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new SingleFromSupplier(supplier)); } @@ -882,9 +888,9 @@ public static Single fromSupplier(final Supplier supplier) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @NonNull - public static Single just(final T item) { + public static <@NonNull T> Single just(T item) { Objects.requireNonNull(item, "item is null"); - return RxJavaPlugins.onAssembly(new SingleJust(item)); + return RxJavaPlugins.onAssembly(new SingleJust<>(item)); } /** @@ -960,7 +966,7 @@ public static Flowable merge(Iterable @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings({ "unchecked", "rawtypes" }) - public static Flowable merge(Publisher> sources) { + public static Flowable merge(@NonNull Publisher> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, SingleInternalHelper.toFlowable(), false, Integer.MAX_VALUE, Flowable.bufferSize())); } @@ -991,7 +997,7 @@ public static Flowable merge(Publisher Single merge(SingleSource> source) { + public static Single merge(@NonNull SingleSource> source) { Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new SingleFlatMap, T>(source, (Function)Functions.identity())); } @@ -1036,9 +1042,8 @@ public static Single merge(SingleSource Flowable merge( - SingleSource source1, SingleSource source2 + @NonNull SingleSource source1, @NonNull SingleSource source2 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1087,10 +1092,9 @@ public static Flowable merge( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable merge( - SingleSource source1, SingleSource source2, - SingleSource source3 + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1142,10 +1146,9 @@ public static Flowable merge( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable merge( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4 + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1176,7 +1179,7 @@ public static Flowable merge( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable mergeDelayError(Iterable> sources) { + public static Flowable mergeDelayError(@NonNull Iterable> sources) { return mergeDelayError(Flowable.fromIterable(sources)); } @@ -1203,7 +1206,7 @@ public static Flowable mergeDelayError(Iterable Flowable mergeDelayError(Publisher> sources) { + public static Flowable mergeDelayError(@NonNull Publisher> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, SingleInternalHelper.toFlowable(), true, Integer.MAX_VALUE, Flowable.bufferSize())); } @@ -1237,9 +1240,8 @@ public static Flowable mergeDelayError(Publisher Flowable mergeDelayError( - SingleSource source1, SingleSource source2 + @NonNull SingleSource source1, @NonNull SingleSource source2 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1277,10 +1279,9 @@ public static Flowable mergeDelayError( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable mergeDelayError( - SingleSource source1, SingleSource source2, - SingleSource source3 + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1321,10 +1322,9 @@ public static Flowable mergeDelayError( @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Flowable mergeDelayError( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4 + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4 ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1348,6 +1348,7 @@ public static Flowable mergeDelayError( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") + @NonNull public static Single never() { return RxJavaPlugins.onAssembly((Single) SingleNever.INSTANCE); } @@ -1367,7 +1368,8 @@ public static Single never() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Single timer(long delay, TimeUnit unit) { + @NonNull + public static Single timer(long delay, @NonNull TimeUnit unit) { return timer(delay, unit, Schedulers.computation()); } @@ -1391,7 +1393,7 @@ public static Single timer(long delay, TimeUnit unit) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Single timer(final long delay, final TimeUnit unit, final Scheduler scheduler) { + public static Single timer(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new SingleTimer(delay, unit, scheduler)); @@ -1414,10 +1416,10 @@ public static Single timer(final long delay, final TimeUnit unit, final Sc @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single equals(final SingleSource first, final SingleSource second) { // NOPMD + public static Single equals(@NonNull SingleSource first, @NonNull SingleSource second) { // NOPMD Objects.requireNonNull(first, "first is null"); Objects.requireNonNull(second, "second is null"); - return RxJavaPlugins.onAssembly(new SingleEquals(first, second)); + return RxJavaPlugins.onAssembly(new SingleEquals<>(first, second)); } /** @@ -1440,12 +1442,12 @@ public static Single equals(final SingleSource first, @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single unsafeCreate(SingleSource onSubscribe) { + public static Single unsafeCreate(@NonNull SingleSource onSubscribe) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); if (onSubscribe instanceof Single) { throw new IllegalArgumentException("unsafeCreate(Single) should be upgraded"); } - return RxJavaPlugins.onAssembly(new SingleFromUnsafeSource(onSubscribe)); + return RxJavaPlugins.onAssembly(new SingleFromUnsafeSource<>(onSubscribe)); } /** @@ -1471,9 +1473,10 @@ public static Single unsafeCreate(SingleSource onSubscribe) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single using(Supplier resourceSupplier, - Function> singleFunction, - Consumer disposer) { + @NonNull + public static Single using(@NonNull Supplier resourceSupplier, + @NonNull Function> singleFunction, + @NonNull Consumer disposer) { return using(resourceSupplier, singleFunction, disposer, true); } @@ -1507,15 +1510,15 @@ public static Single using(Supplier resourceSupplier, @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Single using( - final Supplier resourceSupplier, - final Function> singleFunction, - final Consumer disposer, - final boolean eager) { + @NonNull Supplier resourceSupplier, + @NonNull Function> singleFunction, + @NonNull Consumer disposer, + boolean eager) { Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); Objects.requireNonNull(singleFunction, "singleFunction is null"); Objects.requireNonNull(disposer, "disposer is null"); - return RxJavaPlugins.onAssembly(new SingleUsing(resourceSupplier, singleFunction, disposer, eager)); + return RxJavaPlugins.onAssembly(new SingleUsing<>(resourceSupplier, singleFunction, disposer, eager)); } /** @@ -1534,12 +1537,12 @@ public static Single using( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single wrap(SingleSource source) { + public static Single wrap(@NonNull SingleSource source) { Objects.requireNonNull(source, "source is null"); if (source instanceof Single) { return RxJavaPlugins.onAssembly((Single)source); } - return RxJavaPlugins.onAssembly(new SingleFromUnsafeSource(source)); + return RxJavaPlugins.onAssembly(new SingleFromUnsafeSource<>(source)); } /** @@ -1574,7 +1577,7 @@ public static Single wrap(SingleSource source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single zip(final Iterable> sources, Function zipper) { + public static Single zip(@NonNull Iterable> sources, @NonNull Function zipper) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new SingleZipIterable(sources, zipper)); @@ -1606,10 +1609,9 @@ public static Single zip(final Iterable Single zip( - SingleSource source1, SingleSource source2, - BiFunction zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull BiFunction zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1645,11 +1647,10 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Single zip( - SingleSource source1, SingleSource source2, - SingleSource source3, - Function3 zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, + @NonNull Function3 zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1689,11 +1690,10 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Single zip( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4, - Function4 zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4, + @NonNull Function4 zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1737,12 +1737,11 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Single zip( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4, - SingleSource source5, - Function5 zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4, + @NonNull SingleSource source5, + @NonNull Function5 zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1790,12 +1789,11 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Single zip( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4, - SingleSource source5, SingleSource source6, - Function6 zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4, + @NonNull SingleSource source5, @NonNull SingleSource source6, + @NonNull Function6 zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1847,13 +1845,12 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Single zip( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4, - SingleSource source5, SingleSource source6, - SingleSource source7, - Function7 zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4, + @NonNull SingleSource source5, @NonNull SingleSource source6, + @NonNull SingleSource source7, + @NonNull Function7 zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1909,13 +1906,12 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Single zip( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4, - SingleSource source5, SingleSource source6, - SingleSource source7, SingleSource source8, - Function8 zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4, + @NonNull SingleSource source5, @NonNull SingleSource source6, + @NonNull SingleSource source7, @NonNull SingleSource source8, + @NonNull Function8 zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -1975,14 +1971,13 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") public static Single zip( - SingleSource source1, SingleSource source2, - SingleSource source3, SingleSource source4, - SingleSource source5, SingleSource source6, - SingleSource source7, SingleSource source8, - SingleSource source9, - Function9 zipper + @NonNull SingleSource source1, @NonNull SingleSource source2, + @NonNull SingleSource source3, @NonNull SingleSource source4, + @NonNull SingleSource source5, @NonNull SingleSource source6, + @NonNull SingleSource source7, @NonNull SingleSource source8, + @NonNull SingleSource source9, + @NonNull Function9 zipper ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); @@ -2027,7 +2022,8 @@ public static Single zip( @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Single zipArray(Function zipper, SingleSource... sources) { + @SafeVarargs + public static Single zipArray(@NonNull Function zipper, @NonNull SingleSource... sources) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { @@ -2052,8 +2048,7 @@ public static Single zipArray(Function @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - @SuppressWarnings("unchecked") - public final Single ambWith(SingleSource other) { + public final Single ambWith(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -2072,8 +2067,9 @@ public final Single ambWith(SingleSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single hide() { - return RxJavaPlugins.onAssembly(new SingleHide(this)); + return RxJavaPlugins.onAssembly(new SingleHide<>(this)); } /** @@ -2099,7 +2095,8 @@ public final Single hide() { @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single compose(SingleTransformer transformer) { + @NonNull + public final Single compose(@NonNull SingleTransformer transformer) { return wrap(((SingleTransformer) Objects.requireNonNull(transformer, "transformer is null")).apply(this)); } @@ -2119,8 +2116,9 @@ public final Single compose(SingleTransformer tra */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single cache() { - return RxJavaPlugins.onAssembly(new SingleCache(this)); + return RxJavaPlugins.onAssembly(new SingleCache<>(this)); } /** @@ -2140,7 +2138,7 @@ public final Single cache() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single cast(final Class clazz) { + public final Single cast(@NonNull Class clazz) { Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -2166,7 +2164,8 @@ public final Single cast(final Class clazz) { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable concatWith(SingleSource other) { + @NonNull + public final Flowable concatWith(@NonNull SingleSource other) { return concat(this, other); } @@ -2187,7 +2186,8 @@ public final Flowable concatWith(SingleSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Single delay(long time, TimeUnit unit) { + @NonNull + public final Single delay(long time, @NonNull TimeUnit unit) { return delay(time, unit, Schedulers.computation(), false); } @@ -2208,7 +2208,8 @@ public final Single delay(long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Single delay(long time, TimeUnit unit, boolean delayError) { + @NonNull + public final Single delay(long time, @NonNull TimeUnit unit, boolean delayError) { return delay(time, unit, Schedulers.computation(), delayError); } @@ -2233,7 +2234,8 @@ public final Single delay(long time, TimeUnit unit, boolean delayError) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single delay(final long time, final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final Single delay(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delay(time, unit, scheduler, false); } @@ -2259,10 +2261,10 @@ public final Single delay(final long time, final TimeUnit unit, final Schedul @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single delay(final long time, final TimeUnit unit, final Scheduler scheduler, boolean delayError) { + public final Single delay(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new SingleDelay(this, time, unit, scheduler, delayError)); + return RxJavaPlugins.onAssembly(new SingleDelay<>(this, time, unit, scheduler, delayError)); } /** @@ -2284,9 +2286,9 @@ public final Single delay(final long time, final TimeUnit unit, final Schedul @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single delaySubscription(CompletableSource other) { + public final Single delaySubscription(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new SingleDelayWithCompletable(this, other)); + return RxJavaPlugins.onAssembly(new SingleDelayWithCompletable<>(this, other)); } /** @@ -2309,9 +2311,9 @@ public final Single delaySubscription(CompletableSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single delaySubscription(SingleSource other) { + public final Single delaySubscription(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new SingleDelayWithSingle(this, other)); + return RxJavaPlugins.onAssembly(new SingleDelayWithSingle<>(this, other)); } /** @@ -2334,9 +2336,9 @@ public final Single delaySubscription(SingleSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single delaySubscription(ObservableSource other) { + public final Single delaySubscription(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new SingleDelayWithObservable(this, other)); + return RxJavaPlugins.onAssembly(new SingleDelayWithObservable<>(this, other)); } /** @@ -2364,9 +2366,9 @@ public final Single delaySubscription(ObservableSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single delaySubscription(Publisher other) { + public final Single delaySubscription(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new SingleDelayWithPublisher(this, other)); + return RxJavaPlugins.onAssembly(new SingleDelayWithPublisher<>(this, other)); } /** @@ -2385,7 +2387,8 @@ public final Single delaySubscription(Publisher other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Single delaySubscription(long time, TimeUnit unit) { + @NonNull + public final Single delaySubscription(long time, @NonNull TimeUnit unit) { return delaySubscription(time, unit, Schedulers.computation()); } @@ -2406,7 +2409,8 @@ public final Single delaySubscription(long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single delaySubscription(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Single delaySubscription(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delaySubscription(Observable.timer(time, unit, scheduler)); } @@ -2445,9 +2449,9 @@ public final Single delaySubscription(long time, TimeUnit unit, Scheduler sch @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe dematerialize(Function> selector) { + public final <@NonNull R> Maybe dematerialize(@NonNull Function> selector) { Objects.requireNonNull(selector, "selector is null"); - return RxJavaPlugins.onAssembly(new SingleDematerialize(this, selector)); + return RxJavaPlugins.onAssembly(new SingleDematerialize<>(this, selector)); } /** @@ -2469,9 +2473,9 @@ public final Maybe dematerialize(Function> sel @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doAfterSuccess(Consumer onAfterSuccess) { + public final Single doAfterSuccess(@NonNull Consumer onAfterSuccess) { Objects.requireNonNull(onAfterSuccess, "onAfterSuccess is null"); - return RxJavaPlugins.onAssembly(new SingleDoAfterSuccess(this, onAfterSuccess)); + return RxJavaPlugins.onAssembly(new SingleDoAfterSuccess<>(this, onAfterSuccess)); } /** @@ -2498,9 +2502,9 @@ public final Single doAfterSuccess(Consumer onAfterSuccess) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doAfterTerminate(Action onAfterTerminate) { + public final Single doAfterTerminate(@NonNull Action onAfterTerminate) { Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); - return RxJavaPlugins.onAssembly(new SingleDoAfterTerminate(this, onAfterTerminate)); + return RxJavaPlugins.onAssembly(new SingleDoAfterTerminate<>(this, onAfterTerminate)); } /** @@ -2525,9 +2529,9 @@ public final Single doAfterTerminate(Action onAfterTerminate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doFinally(Action onFinally) { + public final Single doFinally(@NonNull Action onFinally) { Objects.requireNonNull(onFinally, "onFinally is null"); - return RxJavaPlugins.onAssembly(new SingleDoFinally(this, onFinally)); + return RxJavaPlugins.onAssembly(new SingleDoFinally<>(this, onFinally)); } /** @@ -2547,9 +2551,9 @@ public final Single doFinally(Action onFinally) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doOnSubscribe(final Consumer onSubscribe) { + public final Single doOnSubscribe(@NonNull Consumer onSubscribe) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); - return RxJavaPlugins.onAssembly(new SingleDoOnSubscribe(this, onSubscribe)); + return RxJavaPlugins.onAssembly(new SingleDoOnSubscribe<>(this, onSubscribe)); } /** @@ -2574,9 +2578,9 @@ public final Single doOnSubscribe(final Consumer onSubscr @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doOnTerminate(final Action onTerminate) { + public final Single doOnTerminate(@NonNull Action onTerminate) { Objects.requireNonNull(onTerminate, "onTerminate is null"); - return RxJavaPlugins.onAssembly(new SingleDoOnTerminate(this, onTerminate)); + return RxJavaPlugins.onAssembly(new SingleDoOnTerminate<>(this, onTerminate)); } /** @@ -2596,9 +2600,9 @@ public final Single doOnTerminate(final Action onTerminate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doOnSuccess(final Consumer onSuccess) { + public final Single doOnSuccess(@NonNull Consumer onSuccess) { Objects.requireNonNull(onSuccess, "onSuccess is null"); - return RxJavaPlugins.onAssembly(new SingleDoOnSuccess(this, onSuccess)); + return RxJavaPlugins.onAssembly(new SingleDoOnSuccess<>(this, onSuccess)); } /** @@ -2617,9 +2621,9 @@ public final Single doOnSuccess(final Consumer onSuccess) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doOnEvent(final BiConsumer onEvent) { + public final Single doOnEvent(@NonNull BiConsumer onEvent) { Objects.requireNonNull(onEvent, "onEvent is null"); - return RxJavaPlugins.onAssembly(new SingleDoOnEvent(this, onEvent)); + return RxJavaPlugins.onAssembly(new SingleDoOnEvent<>(this, onEvent)); } /** @@ -2639,9 +2643,9 @@ public final Single doOnEvent(final BiConsumer @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doOnError(final Consumer onError) { + public final Single doOnError(@NonNull Consumer onError) { Objects.requireNonNull(onError, "onError is null"); - return RxJavaPlugins.onAssembly(new SingleDoOnError(this, onError)); + return RxJavaPlugins.onAssembly(new SingleDoOnError<>(this, onError)); } /** @@ -2662,9 +2666,9 @@ public final Single doOnError(final Consumer onError) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single doOnDispose(final Action onDispose) { + public final Single doOnDispose(@NonNull Action onDispose) { Objects.requireNonNull(onDispose, "onDispose is null"); - return RxJavaPlugins.onAssembly(new SingleDoOnDispose(this, onDispose)); + return RxJavaPlugins.onAssembly(new SingleDoOnDispose<>(this, onDispose)); } /** @@ -2687,9 +2691,9 @@ public final Single doOnDispose(final Action onDispose) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe filter(Predicate predicate) { + public final Maybe filter(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); - return RxJavaPlugins.onAssembly(new MaybeFilterSingle(this, predicate)); + return RxJavaPlugins.onAssembly(new MaybeFilterSingle<>(this, predicate)); } /** @@ -2711,9 +2715,9 @@ public final Maybe filter(Predicate predicate) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single flatMap(Function> mapper) { + public final Single flatMap(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMap(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleFlatMap<>(this, mapper)); } /** @@ -2735,9 +2739,9 @@ public final Single flatMap(Function Maybe flatMapMaybe(final Function> mapper) { + public final Maybe flatMapMaybe(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMapMaybe(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleFlatMapMaybe<>(this, mapper)); } /** @@ -2764,9 +2768,9 @@ public final Maybe flatMapMaybe(final Function Flowable flatMapPublisher(Function> mapper) { + public final Flowable flatMapPublisher(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMapPublisher(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleFlatMapPublisher<>(this, mapper)); } /** @@ -2793,9 +2797,9 @@ public final Flowable flatMapPublisher(Function Flowable flattenAsFlowable(final Function> mapper) { + public final Flowable flattenAsFlowable(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMapIterableFlowable(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleFlatMapIterableFlowable<>(this, mapper)); } /** @@ -2819,9 +2823,9 @@ public final Flowable flattenAsFlowable(final Function Observable flattenAsObservable(final Function> mapper) { + public final <@NonNull U> Observable flattenAsObservable(@NonNull Function<@NonNull ? super T, @NonNull ? extends Iterable> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMapIterableObservable(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleFlatMapIterableObservable<>(this, mapper)); } /** @@ -2843,9 +2847,9 @@ public final Observable flattenAsObservable(final Function Observable flatMapObservable(Function> mapper) { + public final <@NonNull R> Observable flatMapObservable(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMapObservable(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleFlatMapObservable<>(this, mapper)); } /** @@ -2868,9 +2872,9 @@ public final Observable flatMapObservable(Function mapper) { + public final Completable flatMapCompletable(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMapCompletable(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleFlatMapCompletable<>(this, mapper)); } /** @@ -2890,8 +2894,9 @@ public final Completable flatMapCompletable(final Function observer = new BlockingMultiObserver(); + BlockingMultiObserver observer = new BlockingMultiObserver<>(); subscribe(observer); return observer.blockingGet(); } @@ -3042,7 +3047,7 @@ public final T blockingGet() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single lift(final SingleOperator lift) { + public final Single lift(@NonNull SingleOperator lift) { Objects.requireNonNull(lift, "lift is null"); return RxJavaPlugins.onAssembly(new SingleLift(this, lift)); } @@ -3066,7 +3071,7 @@ public final Single lift(final SingleOperator lif @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single map(Function mapper) { + public final <@NonNull R> Single map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new SingleMap(this, mapper)); } @@ -3087,8 +3092,9 @@ public final Single map(Function mapper) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> materialize() { - return RxJavaPlugins.onAssembly(new SingleMaterialize(this)); + return RxJavaPlugins.onAssembly(new SingleMaterialize<>(this)); } /** @@ -3108,7 +3114,8 @@ public final Single> materialize() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single contains(Object value) { + @NonNull + public final Single contains(@NonNull Object value) { return contains(value, ObjectHelper.equalsPredicate()); } @@ -3130,10 +3137,10 @@ public final Single contains(Object value) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single contains(final Object value, final BiPredicate comparer) { + public final Single contains(@NonNull Object value, @NonNull BiPredicate comparer) { Objects.requireNonNull(value, "value is null"); Objects.requireNonNull(comparer, "comparer is null"); - return RxJavaPlugins.onAssembly(new SingleContains(this, value, comparer)); + return RxJavaPlugins.onAssembly(new SingleContains<>(this, value, comparer)); } /** @@ -3158,7 +3165,8 @@ public final Single contains(final Object value, final BiPredicate mergeWith(SingleSource other) { + @NonNull + public final Flowable mergeWith(@NonNull SingleSource other) { return merge(this, other); } @@ -3184,9 +3192,9 @@ public final Flowable mergeWith(SingleSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single observeOn(final Scheduler scheduler) { + public final Single observeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new SingleObserveOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new SingleObserveOn<>(this, scheduler)); } /** @@ -3218,9 +3226,9 @@ public final Single observeOn(final Scheduler scheduler) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single onErrorReturn(final Function resumeFunction) { + public final Single onErrorReturn(@NonNull Function resumeFunction) { Objects.requireNonNull(resumeFunction, "resumeFunction is null"); - return RxJavaPlugins.onAssembly(new SingleOnErrorReturn(this, resumeFunction, null)); + return RxJavaPlugins.onAssembly(new SingleOnErrorReturn<>(this, resumeFunction, null)); } /** @@ -3238,9 +3246,9 @@ public final Single onErrorReturn(final Function resu @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single onErrorReturnItem(final T value) { + public final Single onErrorReturnItem(@NonNull T value) { Objects.requireNonNull(value, "value is null"); - return RxJavaPlugins.onAssembly(new SingleOnErrorReturn(this, null, value)); + return RxJavaPlugins.onAssembly(new SingleOnErrorReturn<>(this, null, value)); } /** @@ -3273,7 +3281,7 @@ public final Single onErrorReturnItem(final T value) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single onErrorResumeWith(final SingleSource resumeSingleInCaseOfError) { + public final Single onErrorResumeWith(@NonNull SingleSource resumeSingleInCaseOfError) { Objects.requireNonNull(resumeSingleInCaseOfError, "resumeSingleInCaseOfError is null"); return onErrorResumeNext(Functions.justFunction(resumeSingleInCaseOfError)); } @@ -3310,9 +3318,9 @@ public final Single onErrorResumeWith(final SingleSource resumeS @NonNull @SchedulerSupport(SchedulerSupport.NONE) public final Single onErrorResumeNext( - final Function> resumeFunctionInCaseOfError) { + @NonNull Function> resumeFunctionInCaseOfError) { Objects.requireNonNull(resumeFunctionInCaseOfError, "resumeFunctionInCaseOfError is null"); - return RxJavaPlugins.onAssembly(new SingleResumeNext(this, resumeFunctionInCaseOfError)); + return RxJavaPlugins.onAssembly(new SingleResumeNext<>(this, resumeFunctionInCaseOfError)); } /** @@ -3331,8 +3339,9 @@ public final Single onErrorResumeNext( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single onTerminateDetach() { - return RxJavaPlugins.onAssembly(new SingleDetach(this)); + return RxJavaPlugins.onAssembly(new SingleDetach<>(this)); } /** @@ -3351,6 +3360,7 @@ public final Single onTerminateDetach() { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable repeat() { return toFlowable().repeat(); } @@ -3372,6 +3382,7 @@ public final Flowable repeat() { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Flowable repeat(long times) { return toFlowable().repeat(times); } @@ -3399,7 +3410,8 @@ public final Flowable repeat(long times) { @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable repeatWhen(Function, ? extends Publisher> handler) { + @NonNull + public final Flowable repeatWhen(@NonNull Function, ? extends Publisher> handler) { return toFlowable().repeatWhen(handler); } @@ -3421,7 +3433,8 @@ public final Flowable repeatWhen(Function, ? extends @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable repeatUntil(BooleanSupplier stop) { + @NonNull + public final Flowable repeatUntil(@NonNull BooleanSupplier stop) { return toFlowable().repeatUntil(stop); } @@ -3438,6 +3451,7 @@ public final Flowable repeatUntil(BooleanSupplier stop) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single retry() { return toSingle(toFlowable().retry()); } @@ -3457,6 +3471,7 @@ public final Single retry() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single retry(long times) { return toSingle(toFlowable().retry(times)); } @@ -3477,7 +3492,8 @@ public final Single retry(long times) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single retry(BiPredicate predicate) { + @NonNull + public final Single retry(@NonNull BiPredicate predicate) { return toSingle(toFlowable().retry(predicate)); } @@ -3499,7 +3515,8 @@ public final Single retry(BiPredicate pre */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single retry(long times, Predicate predicate) { + @NonNull + public final Single retry(long times, @NonNull Predicate predicate) { return toSingle(toFlowable().retry(times, predicate)); } @@ -3519,7 +3536,8 @@ public final Single retry(long times, Predicate predicate) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single retry(Predicate predicate) { + @NonNull + public final Single retry(@NonNull Predicate predicate) { return toSingle(toFlowable().retry(predicate)); } @@ -3568,7 +3586,8 @@ public final Single retry(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single retryWhen(Function, ? extends Publisher> handler) { + @NonNull + public final Single retryWhen(@NonNull Function, ? extends Publisher> handler) { return toSingle(toFlowable().retryWhen(handler)); } @@ -3589,6 +3608,7 @@ public final Single retryWhen(Function, ? extends * @see ReactiveX operators documentation: Subscribe */ @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Disposable subscribe() { return subscribe(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING); } @@ -3614,10 +3634,10 @@ public final Disposable subscribe() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(final BiConsumer onCallback) { + public final Disposable subscribe(@NonNull BiConsumer onCallback) { Objects.requireNonNull(onCallback, "onCallback is null"); - BiConsumerSingleObserver observer = new BiConsumerSingleObserver(onCallback); + BiConsumerSingleObserver observer = new BiConsumerSingleObserver<>(onCallback); subscribe(observer); return observer; } @@ -3644,7 +3664,8 @@ public final Disposable subscribe(final BiConsumer */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(Consumer onSuccess) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onSuccess) { return subscribe(onSuccess, Functions.ON_ERROR_MISSING); } @@ -3672,18 +3693,18 @@ public final Disposable subscribe(Consumer onSuccess) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(final Consumer onSuccess, final Consumer onError) { + public final Disposable subscribe(@NonNull Consumer onSuccess, @NonNull Consumer onError) { Objects.requireNonNull(onSuccess, "onSuccess is null"); Objects.requireNonNull(onError, "onError is null"); - ConsumerSingleObserver observer = new ConsumerSingleObserver(onSuccess, onError); + ConsumerSingleObserver observer = new ConsumerSingleObserver<>(onSuccess, onError); subscribe(observer); return observer; } @SchedulerSupport(SchedulerSupport.NONE) @Override - public final void subscribe(SingleObserver observer) { + public final void subscribe(@NonNull SingleObserver observer) { Objects.requireNonNull(observer, "observer is null"); observer = RxJavaPlugins.onSubscribe(this, observer); @@ -3739,7 +3760,8 @@ public final void subscribe(SingleObserver observer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > E subscribeWith(E observer) { + @NonNull + public final <@NonNull E extends SingleObserver> E subscribeWith(E observer) { subscribe(observer); return observer; } @@ -3763,9 +3785,9 @@ public final > E subscribeWith(E observer) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single subscribeOn(final Scheduler scheduler) { + public final Single subscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new SingleSubscribeOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new SingleSubscribeOn<>(this, scheduler)); } /** @@ -3788,7 +3810,7 @@ public final Single subscribeOn(final Scheduler scheduler) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single takeUntil(final CompletableSource other) { + public final Single takeUntil(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); return takeUntil(new CompletableToFlowable(other)); } @@ -3820,9 +3842,9 @@ public final Single takeUntil(final CompletableSource other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single takeUntil(final Publisher other) { + public final Single takeUntil(@NonNull Publisher other) { Objects.requireNonNull(other, "other is null"); - return RxJavaPlugins.onAssembly(new SingleTakeUntil(this, other)); + return RxJavaPlugins.onAssembly(new SingleTakeUntil<>(this, other)); } /** @@ -3846,7 +3868,7 @@ public final Single takeUntil(final Publisher other) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public final Single takeUntil(final SingleSource other) { + public final Single takeUntil(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); return takeUntil(new SingleToFlowable(other)); } @@ -3867,7 +3889,8 @@ public final Single takeUntil(final SingleSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Single timeout(long timeout, TimeUnit unit) { + @NonNull + public final Single timeout(long timeout, @NonNull TimeUnit unit) { return timeout0(timeout, unit, Schedulers.computation(), null); } @@ -3889,7 +3912,8 @@ public final Single timeout(long timeout, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single timeout(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Single timeout(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return timeout0(timeout, unit, scheduler, null); } @@ -3912,7 +3936,7 @@ public final Single timeout(long timeout, TimeUnit unit, Scheduler scheduler) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single timeout(long timeout, TimeUnit unit, Scheduler scheduler, SingleSource other) { + public final Single timeout(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, @NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); return timeout0(timeout, unit, scheduler, other); } @@ -3940,7 +3964,7 @@ public final Single timeout(long timeout, TimeUnit unit, Scheduler scheduler, @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Single timeout(long timeout, TimeUnit unit, SingleSource other) { + public final Single timeout(long timeout, @NonNull TimeUnit unit, @NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); return timeout0(timeout, unit, Schedulers.computation(), other); } @@ -3948,7 +3972,7 @@ public final Single timeout(long timeout, TimeUnit unit, SingleSource timeout0(final long timeout, final TimeUnit unit, final Scheduler scheduler, final SingleSource other) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new SingleTimeout(this, timeout, unit, scheduler, other)); + return RxJavaPlugins.onAssembly(new SingleTimeout<>(this, timeout, unit, scheduler, other)); } /** @@ -3990,8 +4014,9 @@ public final R to(@NonNull SingleConverter converter) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable ignoreElement() { - return RxJavaPlugins.onAssembly(new CompletableFromSingle(this)); + return RxJavaPlugins.onAssembly(new CompletableFromSingle<>(this)); } /** @@ -4011,11 +4036,12 @@ public final Completable ignoreElement() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") + @NonNull public final Flowable toFlowable() { if (this instanceof FuseToFlowable) { return ((FuseToFlowable)this).fuseToFlowable(); } - return RxJavaPlugins.onAssembly(new SingleToFlowable(this)); + return RxJavaPlugins.onAssembly(new SingleToFlowable<>(this)); } /** @@ -4032,6 +4058,7 @@ public final Flowable toFlowable() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Future toFuture() { return subscribeWith(new FutureSingleObserver()); } @@ -4050,11 +4077,12 @@ public final Future toFuture() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") + @NonNull public final Maybe toMaybe() { if (this instanceof FuseToMaybe) { return ((FuseToMaybe)this).fuseToMaybe(); } - return RxJavaPlugins.onAssembly(new MaybeFromSingle(this)); + return RxJavaPlugins.onAssembly(new MaybeFromSingle<>(this)); } /** * Converts this Single into an {@link Observable}. @@ -4070,11 +4098,12 @@ public final Maybe toMaybe() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @SuppressWarnings("unchecked") + @NonNull public final Observable toObservable() { if (this instanceof FuseToObservable) { return ((FuseToObservable)this).fuseToObservable(); } - return RxJavaPlugins.onAssembly(new SingleToObservable(this)); + return RxJavaPlugins.onAssembly(new SingleToObservable<>(this)); } /** @@ -4095,9 +4124,9 @@ public final Observable toObservable() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Single unsubscribeOn(final Scheduler scheduler) { + public final Single unsubscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new SingleUnsubscribeOn(this, scheduler)); + return RxJavaPlugins.onAssembly(new SingleUnsubscribeOn<>(this, scheduler)); } /** @@ -4125,7 +4154,8 @@ public final Single unsubscribeOn(final Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single zipWith(SingleSource other, BiFunction zipper) { + @NonNull + public final Single zipWith(@NonNull SingleSource other, @NonNull BiFunction zipper) { return zip(this, other, zipper); } @@ -4146,8 +4176,9 @@ public final Single zipWith(SingleSource other, BiFunction test() { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); subscribe(to); return to; } @@ -4167,8 +4198,9 @@ public final TestObserver test() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestObserver test(boolean dispose) { - TestObserver to = new TestObserver(); + TestObserver to = new TestObserver<>(); if (dispose) { to.dispose(); @@ -4178,8 +4210,9 @@ public final TestObserver test(boolean dispose) { return to; } - private static Single toSingle(Flowable source) { - return RxJavaPlugins.onAssembly(new FlowableSingleSingle(source, null)); + @NonNull + private static Single toSingle(@NonNull Flowable source) { + return RxJavaPlugins.onAssembly(new FlowableSingleSingle<>(source, null)); } // ------------------------------------------------------------------------- diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java index 668a7acf92..fd9712b64d 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableNullTests.java @@ -483,7 +483,6 @@ public void mergeArrayNull() { Flowable.mergeArray(128, 128, (Publisher[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeArrayOneIsNull() { Flowable.mergeArray(128, 128, just1, null).blockingLast(); @@ -514,7 +513,6 @@ public void mergeDelayErrorArrayNull() { Flowable.mergeArrayDelayError(128, 128, (Publisher[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeDelayErrorArrayOneIsNull() { Flowable.mergeArrayDelayError(128, 128, just1, null).blockingLast(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java index 9289288ccc..e7625da6f9 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java @@ -1210,7 +1210,6 @@ public void arrayDelayErrorMaxConcurrency() { PublishProcessor pp2 = PublishProcessor.create(); PublishProcessor pp3 = PublishProcessor.create(); - @SuppressWarnings("unchecked") TestSubscriber ts = Flowable.concatArrayEagerDelayError(2, 2, pp1, pp2, pp3) .test(); @@ -1246,7 +1245,6 @@ public void arrayDelayErrorMaxConcurrencyErrorDelayed() { PublishProcessor pp2 = PublishProcessor.create(); PublishProcessor pp3 = PublishProcessor.create(); - @SuppressWarnings("unchecked") TestSubscriber ts = Flowable.concatArrayEagerDelayError(2, 2, pp1, pp2, pp3) .test(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java index 118f249091..64a2206e84 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeDelayErrorTest.java @@ -622,7 +622,6 @@ public void array() { } } - @SuppressWarnings("unchecked") @Test public void mergeArrayDelayError() { Flowable.mergeArrayDelayError(Flowable.just(1), Flowable.just(2)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java index 19d0ac86a0..3a1d65fc33 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeTest.java @@ -1518,7 +1518,6 @@ public void array() { } } - @SuppressWarnings("unchecked") @Test public void mergeArray2() { Flowable.mergeArray(Flowable.just(1), Flowable.just(2)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java index a660eb7429..69ed3a4aed 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishFunctionTest.java @@ -464,7 +464,6 @@ public void run() { public void longFlow() { Flowable.range(1, 1000000) .publish(new Function, Publisher>() { - @SuppressWarnings("unchecked") @Override public Publisher apply(Flowable v) throws Exception { return Flowable.mergeArray( @@ -491,7 +490,6 @@ public boolean test(Integer w) throws Exception { public void longFlow2() { Flowable.range(1, 100000) .publish(new Function, Publisher>() { - @SuppressWarnings("unchecked") @Override public Publisher apply(Flowable v) throws Exception { return Flowable.mergeArray( @@ -519,7 +517,6 @@ public boolean test(Integer w) throws Exception { public void longFlowHidden() { Flowable.range(1, 1000000).hide() .publish(new Function, Publisher>() { - @SuppressWarnings("unchecked") @Override public Publisher apply(Flowable v) throws Exception { return Flowable.mergeArray( diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java index dec723d651..fb3a6aff5b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeAmbTest.java @@ -116,7 +116,6 @@ public void run() { @Test public void disposeNoFurtherSignals() { - @SuppressWarnings("unchecked") TestObserver to = Maybe.ambArray(new Maybe() { @Override protected void subscribeActual( @@ -134,7 +133,6 @@ protected void subscribeActual( to.assertResult(1); } - @SuppressWarnings("unchecked") @Test public void noWinnerSuccessDispose() throws Exception { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { @@ -160,7 +158,6 @@ public void accept(Object v) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void noWinnerErrorDispose() throws Exception { final TestException ex = new TestException(); @@ -187,7 +184,6 @@ public void accept(Throwable e) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void noWinnerCompleteDispose() throws Exception { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { @@ -223,7 +219,6 @@ public void nullSourceSuccessRace() { final Subject ps = ReplaySubject.create(); ps.onNext(1); - @SuppressWarnings("unchecked") final Maybe source = Maybe.ambArray(ps.singleElement(), Maybe.never(), Maybe.never(), null); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java index 3fa9235fbd..301a8a35f5 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayTest.java @@ -29,7 +29,6 @@ public class MaybeConcatArrayTest extends RxJavaTest { - @SuppressWarnings("unchecked") @Test public void cancel() { Maybe.concatArray(Maybe.just(1), Maybe.just(2)) @@ -38,7 +37,6 @@ public void cancel() { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void cancelDelayError() { Maybe.concatArrayDelayError(Maybe.just(1), Maybe.just(2)) @@ -47,7 +45,6 @@ public void cancelDelayError() { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void backpressure() { TestSubscriber ts = Maybe.concatArray(Maybe.just(1), Maybe.just(2)) @@ -64,7 +61,6 @@ public void backpressure() { ts.assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void backpressureDelayError() { TestSubscriber ts = Maybe.concatArrayDelayError(Maybe.just(1), Maybe.just(2)) @@ -81,7 +77,6 @@ public void backpressureDelayError() { ts.assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void requestCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -106,7 +101,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void requestCancelRaceDelayError() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -131,7 +125,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void errorAfterTermination() { List errors = TestHelper.trackPluginErrors(); @@ -158,7 +151,6 @@ protected void subscribeActual(MaybeObserver observer) { } } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscription() { final int[] calls = { 0 }; @@ -178,7 +170,6 @@ public void subscribe(MaybeEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionDelayError() { final int[] calls = { 0 }; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java index 86261efc4f..6b886fd894 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArrayTest.java @@ -32,7 +32,6 @@ public class MaybeMergeArrayTest extends RxJavaTest { - @SuppressWarnings("unchecked") @Test public void normal() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.SYNC); @@ -45,7 +44,6 @@ public void normal() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void fusedPollMixed() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); @@ -92,7 +90,6 @@ public void onComplete() { }); } - @SuppressWarnings("unchecked") @Test public void cancel() { TestSubscriber ts = new TestSubscriber<>(0L); @@ -106,7 +103,6 @@ public void cancel() { ts.assertEmpty(); } - @SuppressWarnings("unchecked") @Test public void firstErrors() { TestSubscriber ts = new TestSubscriber<>(0L); @@ -117,7 +113,6 @@ public void firstErrors() { ts.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorFused() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); @@ -130,7 +125,6 @@ public void errorFused() { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void errorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -172,7 +166,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void mergeBadSource() { Maybe.mergeArray(new Maybe() { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArrayTest.java index aea7a478ce..c685e79cae 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArrayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArrayTest.java @@ -151,7 +151,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipArrayOneIsNull() { Maybe.zipArray(new Function() { @@ -163,7 +162,6 @@ public Object apply(Object[] v) { .blockingGet(); } - @SuppressWarnings("unchecked") @Test public void singleSourceZipperReturnsNull() { Maybe.zipArray(Functions.justFunction(null), Maybe.just(1)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java index aae11b288e..4f6e07f00a 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterableTest.java @@ -205,7 +205,6 @@ public Object apply(Object[] v) { .blockingGet(); } - @SuppressWarnings("unchecked") @Test public void singleSourceZipperReturnsNull() { Maybe.zipArray(Functions.justFunction(null), Maybe.just(1)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java index 4764b3e50e..323cbc3270 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybeTest.java @@ -399,7 +399,6 @@ public void cancelNoConcurrentClean() { public void checkUnboundedInnerQueue() { MaybeSubject ms = MaybeSubject.create(); - @SuppressWarnings("unchecked") TestObserver to = Observable .fromArray(ms, Maybe.just(2), Maybe.just(3), Maybe.just(4)) .concatMapMaybe(Functions.>identity(), 2) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java index 41659020e5..90f15ac250 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingleTest.java @@ -339,7 +339,6 @@ public void cancelNoConcurrentClean() { public void checkUnboundedInnerQueue() { SingleSubject ss = SingleSubject.create(); - @SuppressWarnings("unchecked") TestObserver to = Observable .fromArray(ss, Single.just(2), Single.just(3), Single.just(4)) .concatMapSingle(Functions.>identity(), 2) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java index 4e8cbd34b6..4948f1a13d 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmbTest.java @@ -92,7 +92,6 @@ public void amb() { Observable observable3 = createObservable(new String[] { "3", "33", "333", "3333" }, 3000, null); - @SuppressWarnings("unchecked") Observable o = Observable.ambArray(observable1, observable2, observable3); @@ -121,7 +120,6 @@ public void amb2() { Observable observable3 = createObservable(new String[] {}, 3000, new IOException("fake exception")); - @SuppressWarnings("unchecked") Observable o = Observable.ambArray(observable1, observable2, observable3); @@ -148,7 +146,6 @@ public void amb3() { Observable observable3 = createObservable(new String[] { "3" }, 3000, null); - @SuppressWarnings("unchecked") Observable o = Observable.ambArray(observable1, observable2, observable3); @@ -161,7 +158,6 @@ public void amb3() { inOrder.verifyNoMoreInteractions(); } - @SuppressWarnings("unchecked") @Test public void subscriptionOnlyHappensOnce() throws InterruptedException { final AtomicLong count = new AtomicLong(); @@ -205,7 +201,6 @@ public void accept(Integer t) { assertEquals(1, result); } - @SuppressWarnings("unchecked") @Test public void ambCancelsOthers() { PublishSubject source1 = PublishSubject.create(); @@ -228,13 +223,11 @@ public void ambCancelsOthers() { } - @SuppressWarnings("unchecked") @Test public void ambArrayEmpty() { assertSame(Observable.empty(), Observable.ambArray()); } - @SuppressWarnings("unchecked") @Test public void ambArraySingleElement() { assertSame(Observable.never(), Observable.ambArray(Observable.never())); @@ -265,7 +258,6 @@ public void singleIterable() { .assertResult(1); } - @SuppressWarnings("unchecked") @Test public void disposed() { TestHelper.checkDisposed(Observable.ambArray(Observable.never(), Observable.never())); @@ -277,7 +269,6 @@ public void onNextRace() { final PublishSubject ps1 = PublishSubject.create(); final PublishSubject ps2 = PublishSubject.create(); - @SuppressWarnings("unchecked") TestObserverEx to = Observable.ambArray(ps1, ps2).to(TestHelper.testConsumer()); Runnable r1 = new Runnable() { @@ -308,7 +299,6 @@ public void onCompleteRace() { final PublishSubject ps1 = PublishSubject.create(); final PublishSubject ps2 = PublishSubject.create(); - @SuppressWarnings("unchecked") TestObserver to = Observable.ambArray(ps1, ps2).test(); Runnable r1 = new Runnable() { @@ -336,7 +326,6 @@ public void onErrorRace() { final PublishSubject ps1 = PublishSubject.create(); final PublishSubject ps2 = PublishSubject.create(); - @SuppressWarnings("unchecked") TestObserver to = Observable.ambArray(ps1, ps2).test(); final Throwable ex = new TestException(); @@ -380,14 +369,12 @@ public void ambIterableOrder() { Observable.amb(Arrays.asList(Observable.just(1), error)).test().assertValue(1).assertComplete(); } - @SuppressWarnings("unchecked") @Test public void ambArrayOrder() { Observable error = Observable.error(new RuntimeException()); Observable.ambArray(Observable.just(1), error).test().assertValue(1).assertComplete(); } - @SuppressWarnings("unchecked") @Test public void noWinnerSuccessDispose() throws Exception { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { @@ -413,7 +400,6 @@ public void accept(Object v) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void noWinnerErrorDispose() throws Exception { final TestException ex = new TestException(); @@ -440,7 +426,6 @@ public void accept(Throwable e) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void noWinnerCompleteDispose() throws Exception { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java index 6ad8b9c649..6a88930b75 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java @@ -218,7 +218,6 @@ public void simple2() { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness2() { final AtomicInteger count = new AtomicInteger(); @@ -238,7 +237,6 @@ public void accept(Integer t) { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness3() { final AtomicInteger count = new AtomicInteger(); @@ -258,7 +256,6 @@ public void accept(Integer t) { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness4() { final AtomicInteger count = new AtomicInteger(); @@ -278,7 +275,6 @@ public void accept(Integer t) { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness5() { final AtomicInteger count = new AtomicInteger(); @@ -298,7 +294,6 @@ public void accept(Integer t) { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness6() { final AtomicInteger count = new AtomicInteger(); @@ -318,7 +313,6 @@ public void accept(Integer t) { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness7() { final AtomicInteger count = new AtomicInteger(); @@ -338,7 +332,6 @@ public void accept(Integer t) { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness8() { final AtomicInteger count = new AtomicInteger(); @@ -358,7 +351,6 @@ public void accept(Integer t) { to.assertComplete(); } - @SuppressWarnings("unchecked") @Test public void eagerness9() { final AtomicInteger count = new AtomicInteger(); @@ -387,7 +379,6 @@ public void mainError() { to.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void innerError() { // TODO verify: concatMapEager subscribes first then consumes the sources is okay @@ -404,7 +395,6 @@ public void innerError() { to.assertNotComplete(); } - @SuppressWarnings("unchecked") @Test public void innerEmpty() { Observable.concatArrayEager(Observable.empty(), Observable.empty()).subscribe(to); @@ -870,7 +860,6 @@ public void arrayDelayErrorDefault() { PublishSubject ps2 = PublishSubject.create(); PublishSubject ps3 = PublishSubject.create(); - @SuppressWarnings("unchecked") TestObserver to = Observable.concatArrayEagerDelayError(ps1, ps2, ps3) .test(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java index 769f42f984..a5d2a6aa4c 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapTest.java @@ -372,7 +372,6 @@ protected void subscribeActual(Observer observer) { } } - @SuppressWarnings("unchecked") @Test public void concatReportsDisposedOnComplete() { final Disposable[] disposable = { null }; @@ -404,7 +403,6 @@ public void onComplete() { } @Test - @SuppressWarnings("unchecked") public void concatReportsDisposedOnError() { final Disposable[] disposable = { null }; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java index 1de6190f05..d481c16662 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZipTest.java @@ -1299,7 +1299,6 @@ public Object apply(Integer a, Integer b) throws Exception { .assertFailure(TestException.class, "929"); } - @SuppressWarnings("unchecked") @Test public void zipArrayEmpty() { assertSame(Observable.empty(), Observable.zipArray(Functions.identity(), false, 16)); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java index 7faf26454e..acfa6528a6 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleAmbTest.java @@ -112,7 +112,6 @@ public void ambIterableWithSecondFires() { to.assertResult(2); } - @SuppressWarnings("unchecked") @Test public void ambArrayEmpty() { Single.ambArray() @@ -120,13 +119,11 @@ public void ambArrayEmpty() { .assertFailure(NoSuchElementException.class); } - @SuppressWarnings("unchecked") @Test public void ambSingleSource() { assertSame(Single.never(), Single.ambArray(Single.never())); } - @SuppressWarnings("unchecked") @Test public void error() { Single.ambArray(Single.error(new TestException()), Single.just(1)) @@ -144,7 +141,6 @@ public void nullSourceSuccessRace() { final Subject ps = ReplaySubject.create(); ps.onNext(1); - @SuppressWarnings("unchecked") final Single source = Single.ambArray(ps.singleOrError(), Single.never(), Single.never(), null); Runnable r1 = new Runnable() { @@ -172,7 +168,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void multipleErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -212,7 +207,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void successErrorRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -276,14 +270,12 @@ public void ambIterableOrder() { Single.amb(Arrays.asList(Single.just(1), error)).test().assertValue(1); } - @SuppressWarnings("unchecked") @Test public void ambArrayOrder() { Single error = Single.error(new RuntimeException()); Single.ambArray(Single.just(1), error).test().assertValue(1); } - @SuppressWarnings("unchecked") @Test public void noWinnerSuccessDispose() throws Exception { for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { @@ -311,7 +303,6 @@ public void accept(Object v, Throwable e) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void noWinnerErrorDispose() throws Exception { final TestException ex = new TestException(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java index 348810be1f..bf8c534f46 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleConcatTest.java @@ -70,7 +70,6 @@ public void concatArray() { } } - @SuppressWarnings("unchecked") @Test public void concatArrayEagerTest() { PublishProcessor pp1 = PublishProcessor.create(); @@ -142,7 +141,6 @@ public void concatObservable() { } } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscription() { final int[] calls = { 0 }; diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArrayTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArrayTest.java index 18f799b3a2..50989db3ec 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArrayTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArrayTest.java @@ -151,7 +151,6 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipArrayOneIsNull() { Single.zipArray(new Function() { @@ -176,7 +175,6 @@ public Object[] apply(Object[] a) throws Exception { .assertFailure(NoSuchElementException.class); } - @SuppressWarnings("unchecked") @Test public void oneArray() { Single.zipArray(new Function() { @@ -189,7 +187,6 @@ public Object apply(Object[] a) throws Exception { .assertResult(2); } - @SuppressWarnings("unchecked") @Test public void singleSourceZipperReturnsNull() { Single.zipArray(Functions.justFunction(null), Single.just(1)) diff --git a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java index c3335c0bab..d222f92392 100644 --- a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java @@ -1382,13 +1382,11 @@ public void concat3Backpressured() { ts.assertResult(1, 2, 3); } - @SuppressWarnings("unchecked") @Test public void concatArrayZero() { assertSame(Flowable.empty(), Maybe.concatArray()); } - @SuppressWarnings("unchecked") @Test public void concatArrayOne() { Maybe.concatArray(Maybe.just(1)).test().assertResult(1); @@ -1594,7 +1592,6 @@ public void maybeToPublisherEnum() { TestHelper.checkEnum(MaybeToPublisher.class); } - @SuppressWarnings("unchecked") @Test public void ambArrayOneIsNull() { Maybe.ambArray(null, Maybe.just(1)) @@ -1602,13 +1599,11 @@ public void ambArrayOneIsNull() { .assertError(NullPointerException.class); } - @SuppressWarnings("unchecked") @Test public void ambArrayEmpty() { assertSame(Maybe.empty(), Maybe.ambArray()); } - @SuppressWarnings("unchecked") @Test public void ambArrayOne() { assertSame(Maybe.never(), Maybe.ambArray(Maybe.never())); @@ -1626,14 +1621,12 @@ public void ambIterableOrder() { Maybe.amb(Arrays.asList(Maybe.just(1), error)).test().assertValue(1); } - @SuppressWarnings("unchecked") @Test public void ambArrayOrder() { Maybe error = Maybe.error(new RuntimeException()); Maybe.ambArray(Maybe.just(1), error).test().assertValue(1); } - @SuppressWarnings("unchecked") @Test public void ambArray1SignalsSuccess() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1656,7 +1649,6 @@ public void ambArray1SignalsSuccess() { to.assertResult(1); } - @SuppressWarnings("unchecked") @Test public void ambArray2SignalsSuccess() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1679,7 +1671,6 @@ public void ambArray2SignalsSuccess() { to.assertResult(2); } - @SuppressWarnings("unchecked") @Test public void ambArray1SignalsError() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1701,7 +1692,6 @@ public void ambArray1SignalsError() { to.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void ambArray2SignalsError() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1723,7 +1713,6 @@ public void ambArray2SignalsError() { to.assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void ambArray1SignalsComplete() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1745,7 +1734,6 @@ public void ambArray1SignalsComplete() { to.assertResult(); } - @SuppressWarnings("unchecked") @Test public void ambArray2SignalsComplete() { PublishProcessor pp1 = PublishProcessor.create(); @@ -1972,7 +1960,6 @@ public void ambIterableOne() { Maybe.amb(Collections.singleton(Maybe.just(1))).test().assertResult(1); } - @SuppressWarnings("unchecked") @Test public void mergeArray() { Maybe.mergeArray(Maybe.just(1), Maybe.just(2), Maybe.just(3)) @@ -2009,7 +1996,6 @@ public void merge4Take2() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeArrayBackpressured() { TestSubscriber ts = Maybe.mergeArray(Maybe.just(1), Maybe.just(2), Maybe.just(3)) @@ -2029,7 +2015,6 @@ public void mergeArrayBackpressured() { ts.assertResult(1, 2, 3); } - @SuppressWarnings("unchecked") @Test public void mergeArrayBackpressuredMixed1() { TestSubscriber ts = Maybe.mergeArray(Maybe.just(1), Maybe.empty(), Maybe.just(3)) @@ -2046,7 +2031,6 @@ public void mergeArrayBackpressuredMixed1() { ts.assertResult(1, 3); } - @SuppressWarnings("unchecked") @Test public void mergeArrayBackpressuredMixed2() { TestSubscriber ts = Maybe.mergeArray(Maybe.just(1), Maybe.just(2), Maybe.empty()) @@ -2063,7 +2047,6 @@ public void mergeArrayBackpressuredMixed2() { ts.assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeArrayBackpressuredMixed3() { TestSubscriber ts = Maybe.mergeArray(Maybe.empty(), Maybe.just(2), Maybe.just(3)) @@ -2080,7 +2063,6 @@ public void mergeArrayBackpressuredMixed3() { ts.assertResult(2, 3); } - @SuppressWarnings("unchecked") @Test public void mergeArrayFused() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); @@ -2093,7 +2075,6 @@ public void mergeArrayFused() { .assertResult(1, 2, 3); } - @SuppressWarnings("unchecked") @Test public void mergeArrayFusedRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { @@ -2129,13 +2110,11 @@ public void run() { } } - @SuppressWarnings("unchecked") @Test public void mergeArrayZero() { assertSame(Flowable.empty(), Maybe.mergeArray()); } - @SuppressWarnings("unchecked") @Test public void mergeArrayOne() { Maybe.mergeArray(Maybe.just(1)).test().assertResult(1); @@ -2462,7 +2441,6 @@ public void accept(Integer v, Throwable e) throws Exception { .assertFailure(TestException.class); } - @SuppressWarnings("unchecked") @Test public void concatArrayDelayError() { Maybe.concatArrayDelayError(Maybe.empty(), Maybe.just(1), Maybe.error(new TestException())) @@ -2500,7 +2478,6 @@ public void concatPublisherDelayError() { .assertFailure(TestException.class, 1); } - @SuppressWarnings("unchecked") @Test public void concatEagerArray() { PublishProcessor pp1 = PublishProcessor.create(); @@ -2607,7 +2584,6 @@ public void fromFuture() { ; } - @SuppressWarnings("unchecked") @Test public void mergeArrayDelayError() { Maybe.mergeArrayDelayError(Maybe.empty(), Maybe.just(1), Maybe.error(new TestException())) diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java index 9c112b0f7d..074057a8d3 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java @@ -45,7 +45,6 @@ public void ambVarargsNull() { Observable.ambArray((Observable[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void ambVarargsOneIsNull() { Observable.ambArray(Observable.never(), null).blockingLast(); diff --git a/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java b/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java index 98ba085b19..6ffdb2ebf8 100644 --- a/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/single/SingleNullTests.java @@ -61,7 +61,6 @@ public void ambArrayNull() { Single.ambArray((Single[])null); } - @SuppressWarnings("unchecked") @Test public void ambArrayOneIsNull() { Single.ambArray(null, just1) @@ -534,7 +533,6 @@ public Object apply(Object[] v) { .blockingGet(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipArrayOneIsNull() { Single.zipArray(new Function() { @@ -546,13 +544,11 @@ public Object apply(Object[] v) { .blockingGet(); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipArrayFunctionNull() { Single.zipArray(null, just1, just1); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void zipArrayFunctionReturnsNull() { Single.zipArray(new Function() { diff --git a/src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java b/src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java new file mode 100644 index 0000000000..8cde4fac7e --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java @@ -0,0 +1,257 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.validators; + +import java.io.File; +import java.nio.file.Files; +import java.util.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.parallel.ParallelFlowable; +import io.reactivex.rxjava3.testsupport.TestHelper; + +/** + * Parse the given files and check if all public final and public static methods have + * @NonNull or @Nullable annotations specified on their return type and object-type parameters + * as well as @SafeVarargs for varargs. + */ +public class SourceAnnotationCheck { + + @Test + public void checkCompletable() throws Exception { + processFile(Completable.class); + } + + @Test + public void checkSingle() throws Exception { + processFile(Single.class); + } + + @Test + public void checkMaybe() throws Exception { + processFile(Maybe.class); + } + + // TODO later + // @Test + public void checkObservable() throws Exception { + processFile(Observable.class); + } + + @Test + public void checkFlowable() throws Exception { + processFile(Flowable.class); + } + + // TODO later + // @Test + public void checkParallelFlowable() throws Exception { + processFile(ParallelFlowable.class); + } + + static void processFile(Class clazz) throws Exception { + String baseClassName = clazz.getSimpleName(); + File f = TestHelper.findSource(baseClassName); + if (f == null) { + return; + } + String fullClassName = clazz.getName(); + + int errorCount = 0; + StringBuilder errors = new StringBuilder(); + + List lines = Files.readAllLines(f.toPath()); + + for (int j = 0; j < lines.size(); j++) { + String line = lines.get(j).trim(); + + if (line.startsWith("public static") || line.startsWith("public final")) { + int methodArgStart = line.indexOf("("); + + int isBoolean = line.indexOf(" boolean "); + int isInt = line.indexOf(" int "); + int isLong = line.indexOf(" long "); + int isVoid = line.indexOf(" void "); + int isElementType = line.indexOf(" R "); + + boolean hasSafeVarargsAnnotation = false; + + if (!((isBoolean > 0 && isBoolean < methodArgStart) + || (isInt > 0 && isInt < methodArgStart) + || (isLong > 0 && isLong < methodArgStart) + || (isVoid > 0 && isVoid < methodArgStart) + || (isElementType > 0 && isElementType < methodArgStart) + )) { + + boolean annotationFound = false; + for (int k = j - 1; k >= 0; k--) { + + String prevLine = lines.get(k).trim(); + + if (prevLine.startsWith("}") || prevLine.startsWith("*/")) { + break; + } + if (prevLine.startsWith("@NonNull") || prevLine.startsWith("@Nullable")) { + annotationFound = true; + } + if (prevLine.startsWith("@SafeVarargs")) { + hasSafeVarargsAnnotation = true; + } + } + + if (!annotationFound) { + errorCount++; + errors.append("L") + .append(j) + .append(" : Missing return type nullability annotation | ") + .append(line) + .append("\r\n") + .append(" at ") + .append(fullClassName) + .append(".method(") + .append(f.getName()) + .append(":") + .append(j + 1) + .append(")\r\n") + ; + } + } + + // Extract arguments + StringBuilder arguments = new StringBuilder(); + int methodArgEnd = line.indexOf(")", methodArgStart); + if (methodArgEnd > 0) { + arguments.append(line.substring(methodArgStart + 1, methodArgEnd)); + } else { + arguments.append(line.substring(methodArgStart + 1)); + for (int k = j + 1; k < lines.size(); k++) { + String ln = lines.get(k).trim(); + int idx = ln.indexOf(")"); + if (idx > 0) { + arguments.append(ln.substring(0, idx)); + break; + } + arguments.append(ln).append(" "); + } + } + + // Strip generics arguments + StringBuilder strippedArguments = new StringBuilder(); + int skippingDepth = 0; + for (int k = 0; k < arguments.length(); k++) { + char c = arguments.charAt(k); + if (c == '<') { + skippingDepth++; + } + else if (c == '>') { + skippingDepth--; + } + else if (skippingDepth == 0) { + strippedArguments.append(c); + } + } + + String strippedArgumentsStr = strippedArguments.toString(); + String[] args = strippedArgumentsStr.split("\\s*,\\s*"); + + for (String typeName : CLASS_NAMES) { + String typeNameSpaced = typeName + " "; + for (int k = 0; k < args.length; k++) { + String typeDef = args[k]; + if (typeDef.contains(typeNameSpaced) + && !typeDef.contains("@NonNull") + && !typeDef.contains("@Nullable")) { + + if (!line.contains("@Nullable " + typeName) + && !line.contains("@NonNull " + typeName)) { + errorCount++; + errors.append("L") + .append(j) + .append(" - argument ").append(k + 1).append(" - ").append(typeDef) + .append(" : Missing argument type nullability annotation |\r\n ") + .append(strippedArgumentsStr) + .append("\r\n") + .append(" at ") + .append(fullClassName) + .append(".method(") + .append(f.getName()) + .append(":") + .append(j + 1) + .append(")\r\n") + ; + } + } + } + } + + if (strippedArgumentsStr.contains("...") && !hasSafeVarargsAnnotation) { + errorCount++; + errors.append("L") + .append(j) + .append(" : Missing @SafeVarargs annotation |\r\n ") + .append(strippedArgumentsStr) + .append("\r\n") + .append(" at ") + .append(fullClassName) + .append(".method(") + .append(f.getName()) + .append(":") + .append(j + 1) + .append(")\r\n") + ; + } + } + } + + if (errorCount != 0) { + errors.insert(0, errorCount + " missing annotations\r\n"); + errors.setLength(errors.length() - 2); + throw new AssertionError(errors.toString()); + } + } + + static final List CLASS_NAMES = Arrays.asList( + "TimeUnit", "Scheduler", "Emitter", + + "Completable", "CompletableSource", "CompletableObserver", "CompletableOnSubscribe", + "CompletableTransformer", "CompletableOperator", "CompletableEmitter", "CompletableConverter", + + "Single", "SingleSource", "SingleObserver", "SingleOnSubscribe", + "SingleTransformer", "SingleOperator", "SingleEmitter", "SingleConverter", + + "Maybe", "MaybeSource", "MaybeObserver", "MaybeOnSubscribe", + "MaybeTransformer", "MaybeOperator", "MaybeEmitter", "MaybeConverter", + + "Observable", "ObservableSource", "Observer", "ObservableOnSubscribe", + "ObservableTransformer", "ObservableOperator", "ObservableEmitter", "ObservableConverter", + + "Flowable", "Publisher", "Subscriber", "FlowableSubscriber", "FlowableOnSubscribe", + "FlowableTransformer", "FlowableOperator", "FlowableEmitter", "FlowableConverter", + + "Function", "BiFunction", "Function3", "Function4", "Function5", "Function6", + "Function7", "Function8", "Function9", + + "Action", "Runnable", "Consumer", "BiConsumer", "Supplier", "Callable", "Void", + "Throwable", "Optional", "CompletionStage", "BooleanSupplier", "LongConsumer", + "Predicate", "BiPredicate", "Object", + + "BackpressureOverflowStrategy", "BackpressureStrategy", + "Subject", "Processor", "FlowableProcessor", + + "T", "R", "U", "V" + ); +} From 480889c58d3b762b073ef80229bf27f9d9429dd7 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Wed, 25 Dec 2019 23:52:13 +0100 Subject: [PATCH 023/665] 3.x: Add missing annotations, fix many diamonds (#6795) * 3.x: Add missing annotations, fix many diamonds * Fix accidental JDK 9 API usage * Fix NPE in unicast due to null-cleanup --- .../reactivex/rxjava3/core/Completable.java | 2 +- .../io/reactivex/rxjava3/core/Flowable.java | 36 +- .../java/io/reactivex/rxjava3/core/Maybe.java | 26 +- .../io/reactivex/rxjava3/core/Observable.java | 1565 +++++++++++------ .../io/reactivex/rxjava3/core/Single.java | 10 +- .../disposables/CompositeDisposable.java | 10 +- .../rxjava3/disposables/SerialDisposable.java | 4 +- .../exceptions/CompositeException.java | 6 +- .../flowables/ConnectableFlowable.java | 65 +- .../rxjava3/internal/functions/Functions.java | 53 +- .../observables/ConnectableObservable.java | 53 +- .../rxjava3/parallel/ParallelFlowable.java | 479 ++++- .../rxjava3/processors/AsyncProcessor.java | 12 +- .../rxjava3/processors/BehaviorProcessor.java | 24 +- .../rxjava3/processors/FlowableProcessor.java | 6 +- .../processors/MulticastProcessor.java | 25 +- .../rxjava3/processors/PublishProcessor.java | 11 +- .../rxjava3/processors/ReplayProcessor.java | 49 +- .../processors/SerializedProcessor.java | 8 +- .../rxjava3/processors/UnicastProcessor.java | 58 +- .../rxjava3/schedulers/TestScheduler.java | 2 +- .../rxjava3/subjects/AsyncSubject.java | 12 +- .../rxjava3/subjects/BehaviorSubject.java | 23 +- .../rxjava3/subjects/CompletableSubject.java | 2 +- .../rxjava3/subjects/MaybeSubject.java | 6 +- .../rxjava3/subjects/PublishSubject.java | 10 +- .../rxjava3/subjects/ReplaySubject.java | 52 +- .../rxjava3/subjects/SerializedSubject.java | 8 +- .../rxjava3/subjects/SingleSubject.java | 6 +- .../reactivex/rxjava3/subjects/Subject.java | 7 +- .../rxjava3/subjects/UnicastSubject.java | 60 +- .../reactivex/rxjava3/core/ConverterTest.java | 10 + .../rxjava3/core/TransformerTest.java | 4 +- .../flowable/FlowableConversionTest.java | 2 +- .../internal/functions/FunctionsTest.java | 56 - .../flowable/FlowableCombineLatestTest.java | 2 +- .../operators/flowable/FlowableRetryTest.java | 2 +- .../operators/flowable/FlowableZipTest.java | 2 +- .../ObservableConcatMapEagerTest.java | 2 - .../observable/ObservableConcatTest.java | 11 - .../ObservableMergeDelayErrorTest.java | 1 - .../observable/ObservableMergeTest.java | 1 - .../observable/ObservableRetryTest.java | 2 +- .../io/reactivex/rxjava3/maybe/MaybeTest.java | 2 +- .../observable/ObservableNullTests.java | 3 - .../parallel/ParallelFlowableTest.java | 1 - .../parallel/ParallelReduceFullTest.java | 1 - .../parallel/ParallelSortedJoinTest.java | 1 - .../rxjava3/subjects/UnicastSubjectTest.java | 2 +- .../rxjava3/testsupport/TestHelper.java | 25 +- .../validators/BaseTypeAnnotations.java | 142 +- .../validators/ParamValidationNaming.java | 235 +++ .../validators/SourceAnnotationCheck.java | 191 +- 53 files changed, 2441 insertions(+), 947 deletions(-) create mode 100644 src/test/java/io/reactivex/rxjava3/validators/ParamValidationNaming.java diff --git a/src/main/java/io/reactivex/rxjava3/core/Completable.java b/src/main/java/io/reactivex/rxjava3/core/Completable.java index 46ec666337..00591742e4 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Completable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Completable.java @@ -2335,7 +2335,7 @@ public final void subscribe(@NonNull CompletableObserver observer) { * applied by {@link #subscribe(CompletableObserver)} before this method gets called. * @param observer the CompletableObserver instance, never null */ - protected abstract void subscribeActual(CompletableObserver observer); + protected abstract void subscribeActual(@NonNull CompletableObserver observer); /** * Subscribes a given CompletableObserver (subclass) to this Completable and returns the given diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index c5bdb06b2c..15b1864d8b 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -183,7 +183,7 @@ public abstract class Flowable implements Publisher { @NonNull @BackpressureSupport(BackpressureKind.PASS_THROUGH) @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable amb(Iterable> sources) { + public static Flowable amb(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new FlowableAmb<>(null, sources)); } @@ -232,6 +232,7 @@ public static Flowable ambArray(Publisher... sources) { * before the Flowable class is loaded. * @return the default internal buffer size. */ + @CheckReturnValue public static int bufferSize() { return BUFFER_SIZE; } @@ -673,6 +674,7 @@ public static Flowable combineLatest( @NonNull BiFunction combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2 }, Functions.toFunction(combiner), bufferSize()); } @@ -723,6 +725,7 @@ public static Flowable combineLatest( Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2, source3 }, Functions.toFunction(combiner), bufferSize()); } @@ -777,6 +780,7 @@ public static Flowable combineLatest( Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4 }, Functions.toFunction(combiner), bufferSize()); } @@ -836,6 +840,7 @@ public static Flowable combineLatest( Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5 }, Functions.toFunction(combiner), bufferSize()); } @@ -899,6 +904,7 @@ public static Flowable combineLatest( Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6 }, Functions.toFunction(combiner), bufferSize()); } @@ -967,6 +973,7 @@ public static Flowable combineLatest( Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6, source7 }, Functions.toFunction(combiner), bufferSize()); } @@ -1039,6 +1046,7 @@ public static Flowable combineLatest( Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6, source7, source8 }, Functions.toFunction(combiner), bufferSize()); } @@ -1116,6 +1124,7 @@ public static Flowable combineLatest( Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); Objects.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new Publisher[] { source1, source2, source3, source4, source5, source6, source7, source8, source9 }, Functions.toFunction(combiner), bufferSize()); } @@ -4735,6 +4744,7 @@ public static Flowable zip( @NonNull BiFunction zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2); } @@ -4796,6 +4806,7 @@ public static Flowable zip( @NonNull BiFunction zipper, boolean delayError) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize(), source1, source2); } @@ -4858,6 +4869,7 @@ public static Flowable zip( @NonNull BiFunction zipper, boolean delayError, int bufferSize) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize, source1, source2); } @@ -4923,6 +4935,7 @@ public static Flowable zip( Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3); } @@ -4993,6 +5006,7 @@ public static Flowable zip( Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4); } @@ -5067,6 +5081,7 @@ public static Flowable zip( Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5); } @@ -5144,6 +5159,7 @@ public static Flowable zip( Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6); } @@ -5226,6 +5242,7 @@ public static Flowable zip( Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7); } @@ -5312,6 +5329,7 @@ public static Flowable zip( Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -5403,6 +5421,7 @@ public static Flowable zip( Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); Objects.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8, source9); } @@ -6791,7 +6810,7 @@ public final Flowable> buffer(@NonNull Publisher boundaryIndicato @BackpressureSupport(BackpressureKind.ERROR) @SchedulerSupport(SchedulerSupport.NONE) @NonNull - public final Flowable> buffer(@NonNull Publisher boundaryIndicator, final int initialCapacity) { + public final Flowable> buffer(@NonNull Publisher boundaryIndicator, int initialCapacity) { ObjectHelper.verifyPositive(initialCapacity, "initialCapacity"); return buffer(boundaryIndicator, Functions.createArrayList(initialCapacity)); } @@ -12899,7 +12918,7 @@ public final Flowable replay(@NonNull Function, ? ext @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable replay(@NonNull Function, ? extends Publisher> selector, final int bufferSize) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, int bufferSize) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, bufferSize, false), selector); @@ -12942,7 +12961,7 @@ public final Flowable replay(@NonNull Function, ? ext @NonNull @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable replay(@NonNull Function, ? extends Publisher> selector, final int bufferSize, boolean eagerTruncate) { + public final Flowable replay(@NonNull Function, ? extends Publisher> selector, int bufferSize, boolean eagerTruncate) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return FlowableReplay.multicastSelector(FlowableInternalHelper.replaySupplier(this, bufferSize, eagerTruncate), selector); @@ -13551,7 +13570,7 @@ public final ConnectableFlowable replay(long time, @NonNull TimeUnit unit, @N @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.CUSTOM) @NonNull - public final ConnectableFlowable replay(final long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { + public final ConnectableFlowable replay(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return FlowableReplay.create(this, time, unit, scheduler, eagerTruncate); @@ -16930,7 +16949,7 @@ public final Flowable timeout( @NonNull Publisher firstTimeoutIndicator, @NonNull Function> itemTimeoutIndicator, @NonNull Publisher other) { - Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutSelector is null"); + Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); Objects.requireNonNull(other, "other is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, other); } @@ -17152,7 +17171,7 @@ public final Single> toList() { @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @SchedulerSupport(SchedulerSupport.NONE) @NonNull - public final Single> toList(final int capacityHint) { + public final Single> toList(int capacityHint) { ObjectHelper.verifyPositive(capacityHint, "capacityHint"); return RxJavaPlugins.onAssembly(new FlowableToListSingle<>(this, Functions.createArrayList(capacityHint))); } @@ -18470,6 +18489,7 @@ public final Flowable> window( @NonNull Function3 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new Publisher[] { source1, source2 }, f); } @@ -18513,6 +18533,7 @@ public final Flowable> window( Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new Publisher[] { source1, source2, source3 }, f); } @@ -18559,6 +18580,7 @@ public final Flowable> window( Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); return withLatestFrom(new Publisher[] { source1, source2, source3, source4 }, f); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Maybe.java b/src/main/java/io/reactivex/rxjava3/core/Maybe.java index bb0a6dcbdb..d5336f96ce 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Maybe.java +++ b/src/main/java/io/reactivex/rxjava3/core/Maybe.java @@ -127,7 +127,7 @@ public abstract class Maybe implements MaybeSource { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Maybe amb(final Iterable> sources) { + public static Maybe amb(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new MaybeAmb<>(null, sources)); } @@ -452,7 +452,7 @@ public static Flowable concatArrayEager(@NonNull MaybeSource @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Flowable concatDelayError(Iterable> sources) { + public static Flowable concatDelayError(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); return Flowable.fromIterable(sources).concatMapDelayError((Function)MaybeToPublisher.instance()); } @@ -678,7 +678,7 @@ public static Maybe error(@NonNull Throwable exception) { @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Maybe error(@NonNull Supplier supplier) { - Objects.requireNonNull(supplier, "errorSupplier is null"); + Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new MaybeErrorCallable(supplier)); } @@ -788,7 +788,7 @@ public static Maybe fromSingle(@NonNull SingleSource singleSource) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static <@NonNull T> Maybe fromCallable(@NonNull final Callable callable) { + public static <@NonNull T> Maybe fromCallable(@NonNull Callable callable) { Objects.requireNonNull(callable, "callable is null"); return RxJavaPlugins.onAssembly(new MaybeFromCallable(callable)); } @@ -928,7 +928,7 @@ public static Maybe fromRunnable(@NonNull Runnable run) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static <@NonNull T> Maybe fromSupplier(@NonNull final Supplier supplier) { + public static <@NonNull T> Maybe fromSupplier(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new MaybeFromSupplier(supplier)); } @@ -1064,7 +1064,7 @@ public static Flowable merge(@NonNull Publisher Flowable merge(@NonNull Publisher> sources, int maxConcurrency) { - Objects.requireNonNull(sources, "source is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, MaybeToPublisher.instance(), false, maxConcurrency, 1)); } @@ -1449,7 +1449,7 @@ public static Flowable mergeDelayError(@NonNull Publisher Flowable mergeDelayError(@NonNull Publisher> sources, int maxConcurrency) { - Objects.requireNonNull(sources, "source is null"); + Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new FlowableFlatMapPublisher(sources, MaybeToPublisher.instance(), true, maxConcurrency, 1)); } @@ -1806,7 +1806,7 @@ public static Maybe using(@NonNull Supplier resourceSuppl @NonNull Consumer resourceDisposer, boolean eager) { Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); Objects.requireNonNull(sourceSupplier, "sourceSupplier is null"); - Objects.requireNonNull(resourceDisposer, "disposer is null"); + Objects.requireNonNull(resourceDisposer, "resourceDisposer is null"); return RxJavaPlugins.onAssembly(new MaybeUsing(resourceSupplier, sourceSupplier, resourceDisposer, eager)); } @@ -1828,7 +1828,7 @@ public static Maybe wrap(@NonNull MaybeSource source) { if (source instanceof Maybe) { return RxJavaPlugins.onAssembly((Maybe)source); } - Objects.requireNonNull(source, "onSubscribe is null"); + Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new MaybeUnsafeCreate<>(source)); } @@ -1902,6 +1902,7 @@ public static Maybe zip( @NonNull BiFunction zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2); } @@ -1943,6 +1944,7 @@ public static Maybe zip( Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3); } @@ -1989,6 +1991,7 @@ public static Maybe zip( Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4); } @@ -2039,6 +2042,7 @@ public static Maybe zip( Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5); } @@ -2093,6 +2097,7 @@ public static Maybe zip( Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6); } @@ -2152,6 +2157,7 @@ public static Maybe zip( Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7); } @@ -2215,6 +2221,7 @@ public static Maybe zip( Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -2282,6 +2289,7 @@ public static Maybe zip( Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); Objects.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8, source9); } diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index 42d7f3d00a..16e1454af3 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -118,7 +118,7 @@ public abstract class Observable implements ObservableSource { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable amb(Iterable> sources) { + public static Observable amb(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableAmb<>(null, sources)); } @@ -146,7 +146,7 @@ public static Observable amb(Iterable Observable ambArray(ObservableSource... sources) { + public static Observable ambArray(@NonNull ObservableSource... sources) { Objects.requireNonNull(sources, "sources is null"); int len = sources.length; if (len == 0) { @@ -165,6 +165,7 @@ public static Observable ambArray(ObservableSource... source * before the {@link Flowable} class is loaded. * @return the default 'island' size or capacity-increment hint */ + @CheckReturnValue public static int bufferSize() { return Flowable.bufferSize(); } @@ -206,8 +207,10 @@ public static int bufferSize() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable combineLatest(Iterable> sources, - Function combiner) { + @NonNull + public static Observable combineLatest( + @NonNull Iterable> sources, + @NonNull Function combiner) { return combineLatest(sources, combiner, bufferSize()); } @@ -251,8 +254,9 @@ public static Observable combineLatest(Iterable Observable combineLatest(Iterable> sources, - Function combiner, int bufferSize) { + public static Observable combineLatest( + @NonNull Iterable> sources, + @NonNull Function combiner, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -299,8 +303,10 @@ public static Observable combineLatest(Iterable Observable combineLatestArray(ObservableSource[] sources, - Function combiner) { + @NonNull + public static Observable combineLatestArray( + @NonNull ObservableSource[] sources, + @NonNull Function combiner) { return combineLatestArray(sources, combiner, bufferSize()); } @@ -344,8 +350,9 @@ public static Observable combineLatestArray(ObservableSource Observable combineLatestArray(ObservableSource[] sources, - Function combiner, int bufferSize) { + public static Observable combineLatestArray( + @NonNull ObservableSource[] sources, + @NonNull Function combiner, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); if (sources.length == 0) { return empty(); @@ -391,10 +398,11 @@ public static Observable combineLatestArray(ObservableSource Observable combineLatest( - ObservableSource source1, ObservableSource source2, - BiFunction combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull BiFunction combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2 }, Functions.toFunction(combiner), bufferSize()); } @@ -434,12 +442,13 @@ public static Observable combineLatest( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable combineLatest( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, - Function3 combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, + @NonNull Function3 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3 }, Functions.toFunction(combiner), bufferSize()); } @@ -482,13 +491,14 @@ public static Observable combineLatest( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable combineLatest( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4, - Function4 combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull Function4 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4 }, Functions.toFunction(combiner), bufferSize()); } @@ -534,15 +544,16 @@ public static Observable combineLatest( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable combineLatest( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4, - ObservableSource source5, - Function5 combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull ObservableSource source5, + @NonNull Function5 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5 }, Functions.toFunction(combiner), bufferSize()); } @@ -591,16 +602,17 @@ public static Observable combineLatest( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable combineLatest( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4, - ObservableSource source5, ObservableSource source6, - Function6 combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull Function6 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6 }, Functions.toFunction(combiner), bufferSize()); } @@ -652,11 +664,11 @@ public static Observable combineLatest( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable combineLatest( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4, - ObservableSource source5, ObservableSource source6, - ObservableSource source7, - Function7 combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull ObservableSource source7, + @NonNull Function7 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -664,6 +676,7 @@ public static Observable combineLatest( Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6, source7 }, Functions.toFunction(combiner), bufferSize()); } @@ -718,11 +731,11 @@ public static Observable combineLatest( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable combineLatest( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4, - ObservableSource source5, ObservableSource source6, - ObservableSource source7, ObservableSource source8, - Function8 combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull ObservableSource source7, @NonNull ObservableSource source8, + @NonNull Function8 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -731,6 +744,7 @@ public static Observable combineLatest( Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6, source7, source8 }, Functions.toFunction(combiner), bufferSize()); } @@ -788,12 +802,12 @@ public static Observable combineLatest( @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable combineLatest( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4, - ObservableSource source5, ObservableSource source6, - ObservableSource source7, ObservableSource source8, - ObservableSource source9, - Function9 combiner) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull ObservableSource source7, @NonNull ObservableSource source8, + @NonNull ObservableSource source9, + @NonNull Function9 combiner) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -803,6 +817,7 @@ public static Observable combineLates Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); Objects.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(combiner, "combiner is null"); return combineLatestArray(new ObservableSource[] { source1, source2, source3, source4, source5, source6, source7, source8, source9 }, Functions.toFunction(combiner), bufferSize()); } @@ -843,8 +858,10 @@ public static Observable combineLates */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable combineLatestDelayError(ObservableSource[] sources, - Function combiner) { + @NonNull + public static Observable combineLatestDelayError( + @NonNull ObservableSource[] sources, + @NonNull Function combiner) { return combineLatestDelayError(sources, combiner, bufferSize()); } @@ -889,8 +906,8 @@ public static Observable combineLatestDelayError(ObservableSource Observable combineLatestDelayError(ObservableSource[] sources, - Function combiner, int bufferSize) { + public static Observable combineLatestDelayError(@NonNull ObservableSource[] sources, + @NonNull Function combiner, int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(combiner, "combiner is null"); if (sources.length == 0) { @@ -939,8 +956,9 @@ public static Observable combineLatestDelayError(ObservableSource Observable combineLatestDelayError(Iterable> sources, - Function combiner) { + @NonNull + public static Observable combineLatestDelayError(@NonNull Iterable> sources, + @NonNull Function combiner) { return combineLatestDelayError(sources, combiner, bufferSize()); } @@ -985,8 +1003,8 @@ public static Observable combineLatestDelayError(Iterable Observable combineLatestDelayError(Iterable> sources, - Function combiner, int bufferSize) { + public static Observable combineLatestDelayError(@NonNull Iterable> sources, + @NonNull Function combiner, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); Objects.requireNonNull(combiner, "combiner is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -1013,7 +1031,7 @@ public static Observable combineLatestDelayError(Iterable Observable concat(Iterable> sources) { + public static Observable concat(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); return fromIterable(sources).concatMapDelayError((Function)Functions.identity(), false, bufferSize()); } @@ -1037,7 +1055,8 @@ public static Observable concat(Iterable Observable concat(ObservableSource> sources) { + @NonNull + public static Observable concat(@NonNull ObservableSource> sources) { return concat(sources, bufferSize()); } @@ -1064,7 +1083,7 @@ public static Observable concat(ObservableSource Observable concat(ObservableSource> sources, int prefetch) { + public static Observable concat(@NonNull ObservableSource> sources, int prefetch) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMap(sources, Functions.identity(), prefetch, ErrorMode.IMMEDIATE)); @@ -1089,11 +1108,10 @@ public static Observable concat(ObservableSourceReactiveX operators documentation: Concat */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable concat(ObservableSource source1, ObservableSource source2) { + public static Observable concat(@NonNull ObservableSource source1, ObservableSource source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return concatArray(source1, source2); @@ -1120,13 +1138,12 @@ public static Observable concat(ObservableSource source1, Ob * without interleaving them * @see ReactiveX operators documentation: Concat */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable concat( - ObservableSource source1, ObservableSource source2, - ObservableSource source3) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1156,13 +1173,12 @@ public static Observable concat( * without interleaving them * @see ReactiveX operators documentation: Concat */ - @SuppressWarnings("unchecked") @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable concat( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -1188,7 +1204,9 @@ public static Observable concat( @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable concatArray(ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable concatArray(@NonNull ObservableSource... sources) { if (sources.length == 0) { return empty(); } @@ -1212,15 +1230,18 @@ public static Observable concatArray(ObservableSource... sou * @return the new Observable instance * @throws NullPointerException if sources is null */ - @SuppressWarnings({ "unchecked" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable concatArrayDelayError(ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable concatArrayDelayError(@NonNull ObservableSource... sources) { if (sources.length == 0) { return empty(); } if (sources.length == 1) { - return (Observable)wrap(sources[0]); + @SuppressWarnings("unchecked") + Observable source = (Observable)wrap(sources[0]); + return source; } return concatDelayError(fromArray(sources)); } @@ -1245,7 +1266,8 @@ public static Observable concatArrayDelayError(ObservableSource Observable concatArrayEager(ObservableSource... sources) { + @NonNull + public static Observable concatArrayEager(@NonNull ObservableSource... sources) { return concatArrayEager(bufferSize(), bufferSize(), sources); } @@ -1272,7 +1294,9 @@ public static Observable concatArrayEager(ObservableSource.. @SuppressWarnings({ "rawtypes", "unchecked" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable concatArrayEager(int maxConcurrency, int prefetch, ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable concatArrayEager(int maxConcurrency, int prefetch, @NonNull ObservableSource... sources) { return fromArray(sources).concatMapEagerDelayError((Function)Functions.identity(), false, maxConcurrency, prefetch); } @@ -1297,7 +1321,8 @@ public static Observable concatArrayEager(int maxConcurrency, int prefetc @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs - public static Observable concatArrayEagerDelayError(ObservableSource... sources) { + @NonNull + public static Observable concatArrayEagerDelayError(@NonNull ObservableSource... sources) { return concatArrayEagerDelayError(bufferSize(), bufferSize(), sources); } @@ -1325,7 +1350,9 @@ public static Observable concatArrayEagerDelayError(ObservableSource Observable concatArrayEagerDelayError(int maxConcurrency, int prefetch, ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable concatArrayEagerDelayError(int maxConcurrency, int prefetch, @NonNull ObservableSource... sources) { return fromArray(sources).concatMapEagerDelayError((Function)Functions.identity(), true, maxConcurrency, prefetch); } @@ -1346,7 +1373,7 @@ public static Observable concatArrayEagerDelayError(int maxConcurrency, i @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable concatDelayError(Iterable> sources) { + public static Observable concatDelayError(@NonNull Iterable> sources) { Objects.requireNonNull(sources, "sources is null"); return concatDelayError(fromIterable(sources)); } @@ -1367,7 +1394,8 @@ public static Observable concatDelayError(Iterable Observable concatDelayError(ObservableSource> sources) { + @NonNull + public static Observable concatDelayError(@NonNull ObservableSource> sources) { return concatDelayError(sources, bufferSize(), true); } @@ -1392,7 +1420,7 @@ public static Observable concatDelayError(ObservableSource Observable concatDelayError(ObservableSource> sources, int prefetch, boolean tillTheEnd) { + public static Observable concatDelayError(@NonNull ObservableSource> sources, int prefetch, boolean tillTheEnd) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch is null"); return RxJavaPlugins.onAssembly(new ObservableConcatMap(sources, Functions.identity(), prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); @@ -1417,7 +1445,8 @@ public static Observable concatDelayError(ObservableSource Observable concatEager(ObservableSource> sources) { + @NonNull + public static Observable concatEager(@NonNull ObservableSource> sources) { return concatEager(sources, bufferSize(), bufferSize()); } @@ -1444,7 +1473,8 @@ public static Observable concatEager(ObservableSource Observable concatEager(ObservableSource> sources, int maxConcurrency, int prefetch) { + @NonNull + public static Observable concatEager(@NonNull ObservableSource> sources, int maxConcurrency, int prefetch) { return wrap(sources).concatMapEager((Function)Functions.identity(), maxConcurrency, prefetch); } @@ -1467,7 +1497,8 @@ public static Observable concatEager(ObservableSource Observable concatEager(Iterable> sources) { + @NonNull + public static Observable concatEager(@NonNull Iterable> sources) { return concatEager(sources, bufferSize(), bufferSize()); } @@ -1494,7 +1525,8 @@ public static Observable concatEager(Iterable Observable concatEager(Iterable> sources, int maxConcurrency, int prefetch) { + @NonNull + public static Observable concatEager(@NonNull Iterable> sources, int maxConcurrency, int prefetch) { return fromIterable(sources).concatMapEagerDelayError((Function)Functions.identity(), false, maxConcurrency, prefetch); } @@ -1551,7 +1583,7 @@ public static Observable concatEager(Iterable Observable create(ObservableOnSubscribe source) { + public static Observable create(@NonNull ObservableOnSubscribe source) { Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new ObservableCreate<>(source)); } @@ -1583,7 +1615,7 @@ public static Observable create(ObservableOnSubscribe source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable defer(Supplier> supplier) { + public static Observable defer(@NonNull Supplier> supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new ObservableDefer<>(supplier)); } @@ -1607,6 +1639,7 @@ public static Observable defer(Supplier Observable empty() { return RxJavaPlugins.onAssembly((Observable) ObservableEmpty.INSTANCE); } @@ -1632,7 +1665,7 @@ public static Observable empty() { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable error(Supplier errorSupplier) { + public static Observable error(@NonNull Supplier errorSupplier) { Objects.requireNonNull(errorSupplier, "errorSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableError(errorSupplier)); } @@ -1658,7 +1691,7 @@ public static Observable error(Supplier errorSupplie @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable error(final Throwable exception) { + public static Observable error(@NonNull Throwable exception) { Objects.requireNonNull(exception, "exception is null"); return error(Functions.justSupplier(exception)); } @@ -1683,7 +1716,7 @@ public static Observable error(final Throwable exception) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull @SafeVarargs - public static Observable fromArray(T... items) { + public static Observable fromArray(@NonNull T... items) { Objects.requireNonNull(items, "items is null"); if (items.length == 0) { return empty(); @@ -1726,7 +1759,7 @@ public static Observable fromArray(T... items) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable fromCallable(Callable supplier) { + public static Observable fromCallable(@NonNull Callable supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new ObservableFromCallable(supplier)); } @@ -1760,7 +1793,7 @@ public static Observable fromCallable(Callable supplier) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable fromFuture(Future future) { + public static Observable fromFuture(@NonNull Future future) { Objects.requireNonNull(future, "future is null"); return RxJavaPlugins.onAssembly(new ObservableFromFuture(future, 0L, null)); } @@ -1798,7 +1831,7 @@ public static Observable fromFuture(Future future) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable fromFuture(Future future, long timeout, TimeUnit unit) { + public static Observable fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) { Objects.requireNonNull(future, "future is null"); Objects.requireNonNull(unit, "unit is null"); return RxJavaPlugins.onAssembly(new ObservableFromFuture(future, timeout, unit)); @@ -1840,7 +1873,7 @@ public static Observable fromFuture(Future future, long time @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Observable fromFuture(Future future, long timeout, TimeUnit unit, Scheduler scheduler) { + public static Observable fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); Observable o = fromFuture(future, timeout, unit); return o.subscribeOn(scheduler); @@ -1876,7 +1909,7 @@ public static Observable fromFuture(Future future, long time @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Observable fromFuture(Future future, Scheduler scheduler) { + public static Observable fromFuture(@NonNull Future future, @NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); Observable o = fromFuture(future); return o.subscribeOn(scheduler); @@ -1902,7 +1935,7 @@ public static Observable fromFuture(Future future, Scheduler @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable fromIterable(Iterable source) { + public static Observable fromIterable(@NonNull Iterable source) { Objects.requireNonNull(source, "source is null"); return RxJavaPlugins.onAssembly(new ObservableFromIterable(source)); } @@ -1939,7 +1972,7 @@ public static Observable fromIterable(Iterable source) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable fromPublisher(Publisher publisher) { + public static Observable fromPublisher(@NonNull Publisher publisher) { Objects.requireNonNull(publisher, "publisher is null"); return RxJavaPlugins.onAssembly(new ObservableFromPublisher(publisher)); } @@ -1976,7 +2009,7 @@ public static Observable fromPublisher(Publisher publisher) @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable fromSupplier(Supplier supplier) { + public static Observable fromSupplier(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); return RxJavaPlugins.onAssembly(new ObservableFromSupplier(supplier)); } @@ -2005,7 +2038,7 @@ public static Observable fromSupplier(Supplier supplier) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable generate(final Consumer> generator) { + public static Observable generate(@NonNull Consumer> generator) { Objects.requireNonNull(generator, "generator is null"); return generate(Functions.nullSupplier(), ObservableInternalHelper.simpleGenerator(generator), Functions.emptyConsumer()); @@ -2037,7 +2070,7 @@ public static Observable generate(final Consumer> generator) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable generate(Supplier initialState, final BiConsumer> generator) { + public static Observable generate(@NonNull Supplier initialState, @NonNull BiConsumer> generator) { Objects.requireNonNull(generator, "generator is null"); return generate(initialState, ObservableInternalHelper.simpleBiGenerator(generator), Functions.emptyConsumer()); } @@ -2071,9 +2104,9 @@ public static Observable generate(Supplier initialState, final BiCo @NonNull @SchedulerSupport(SchedulerSupport.NONE) public static Observable generate( - final Supplier initialState, - final BiConsumer> generator, - Consumer disposeState) { + @NonNull Supplier initialState, + @NonNull BiConsumer> generator, + @NonNull Consumer disposeState) { Objects.requireNonNull(generator, "generator is null"); return generate(initialState, ObservableInternalHelper.simpleBiGenerator(generator), disposeState); } @@ -2104,7 +2137,8 @@ public static Observable generate( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable generate(Supplier initialState, BiFunction, S> generator) { + @NonNull + public static Observable generate(@NonNull Supplier initialState, @NonNull BiFunction, S> generator) { return generate(initialState, generator, Functions.emptyConsumer()); } @@ -2137,8 +2171,8 @@ public static Observable generate(Supplier initialState, BiFunction @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable generate(Supplier initialState, BiFunction, S> generator, - Consumer disposeState) { + public static Observable generate(@NonNull Supplier initialState, @NonNull BiFunction, S> generator, + @NonNull Consumer disposeState) { Objects.requireNonNull(initialState, "initialState is null"); Objects.requireNonNull(generator, "generator is null"); Objects.requireNonNull(disposeState, "disposeState is null"); @@ -2168,7 +2202,8 @@ public static Observable generate(Supplier initialState, BiFunction */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Observable interval(long initialDelay, long period, TimeUnit unit) { + @NonNull + public static Observable interval(long initialDelay, long period, @NonNull TimeUnit unit) { return interval(initialDelay, period, unit, Schedulers.computation()); } @@ -2198,7 +2233,7 @@ public static Observable interval(long initialDelay, long period, TimeUnit @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Observable interval(long initialDelay, long period, TimeUnit unit, Scheduler scheduler) { + public static Observable interval(long initialDelay, long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -2223,7 +2258,8 @@ public static Observable interval(long initialDelay, long period, TimeUnit */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Observable interval(long period, TimeUnit unit) { + @NonNull + public static Observable interval(long period, @NonNull TimeUnit unit) { return interval(period, period, unit, Schedulers.computation()); } @@ -2248,7 +2284,8 @@ public static Observable interval(long period, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Observable interval(long period, TimeUnit unit, Scheduler scheduler) { + @NonNull + public static Observable interval(long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return interval(period, period, unit, scheduler); } @@ -2271,7 +2308,8 @@ public static Observable interval(long period, TimeUnit unit, Scheduler sc */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public static Observable intervalRange(long start, long count, long initialDelay, long period, TimeUnit unit) { + @NonNull + public static Observable intervalRange(long start, long count, long initialDelay, long period, @NonNull TimeUnit unit) { return intervalRange(start, count, initialDelay, period, unit, Schedulers.computation()); } @@ -2295,7 +2333,7 @@ public static Observable intervalRange(long start, long count, long initia @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Observable intervalRange(long start, long count, long initialDelay, long period, TimeUnit unit, Scheduler scheduler) { + public static Observable intervalRange(long start, long count, long initialDelay, long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); } @@ -2345,7 +2383,7 @@ public static Observable intervalRange(long start, long count, long initia @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item) { + public static Observable just(@NonNull T item) { Objects.requireNonNull(item, "item is null"); return RxJavaPlugins.onAssembly(new ObservableJust<>(item)); } @@ -2371,7 +2409,7 @@ public static Observable just(T item) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2) { + public static Observable just(@NonNull T item1, @NonNull T item2) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); @@ -2401,7 +2439,7 @@ public static Observable just(T item1, T item2) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2434,7 +2472,7 @@ public static Observable just(T item1, T item2, T item3) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3, T item4) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3, @NonNull T item4) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2470,7 +2508,7 @@ public static Observable just(T item1, T item2, T item3, T item4) { @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3, T item4, T item5) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3, @NonNull T item4, @NonNull T item5) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2509,7 +2547,7 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3, @NonNull T item4, @NonNull T item5, @NonNull T item6) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2551,7 +2589,7 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3, @NonNull T item4, @NonNull T item5, @NonNull T item6, @NonNull T item7) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2596,7 +2634,7 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3, @NonNull T item4, @NonNull T item5, @NonNull T item6, @NonNull T item7, @NonNull T item8) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2644,7 +2682,7 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3, @NonNull T item4, @NonNull T item5, @NonNull T item6, @NonNull T item7, @NonNull T item8, @NonNull T item9) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2695,7 +2733,7 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @CheckReturnValue @NonNull @SchedulerSupport(SchedulerSupport.NONE) - public static Observable just(T item1, T item2, T item3, T item4, T item5, T item6, T item7, T item8, T item9, T item10) { + public static Observable just(@NonNull T item1, @NonNull T item2, @NonNull T item3, @NonNull T item4, @NonNull T item5, @NonNull T item6, @NonNull T item7, @NonNull T item8, @NonNull T item9, @NonNull T item10) { Objects.requireNonNull(item1, "item1 is null"); Objects.requireNonNull(item2, "item2 is null"); Objects.requireNonNull(item3, "item3 is null"); @@ -2753,7 +2791,8 @@ public static Observable just(T item1, T item2, T item3, T item4, T item5 @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable merge(Iterable> sources, int maxConcurrency, int bufferSize) { + @NonNull + public static Observable merge(@NonNull Iterable> sources, int maxConcurrency, int bufferSize) { return fromIterable(sources).flatMap((Function)Functions.identity(), false, maxConcurrency, bufferSize); } @@ -2800,7 +2839,9 @@ public static Observable merge(Iterable Observable mergeArray(int maxConcurrency, int bufferSize, ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable mergeArray(int maxConcurrency, int bufferSize, @NonNull ObservableSource... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), false, maxConcurrency, bufferSize); } @@ -2840,7 +2881,8 @@ public static Observable mergeArray(int maxConcurrency, int bufferSize, O @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable merge(Iterable> sources) { + @NonNull + public static Observable merge(@NonNull Iterable> sources) { return fromIterable(sources).flatMap((Function)Functions.identity()); } @@ -2885,7 +2927,8 @@ public static Observable merge(Iterable Observable merge(Iterable> sources, int maxConcurrency) { + @NonNull + public static Observable merge(@NonNull Iterable> sources, int maxConcurrency) { return fromIterable(sources).flatMap((Function)Functions.identity(), maxConcurrency); } @@ -2926,7 +2969,8 @@ public static Observable merge(Iterable Observable merge(ObservableSource> sources) { + @NonNull + public static Observable merge(@NonNull ObservableSource> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), false, Integer.MAX_VALUE, bufferSize())); } @@ -2974,7 +3018,8 @@ public static Observable merge(ObservableSource Observable merge(ObservableSource> sources, int maxConcurrency) { + @NonNull + public static Observable merge(@NonNull ObservableSource> sources, int maxConcurrency) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), false, maxConcurrency, bufferSize())); @@ -3017,7 +3062,8 @@ public static Observable merge(ObservableSource Observable merge(ObservableSource source1, ObservableSource source2) { + @NonNull + public static Observable merge(@NonNull ObservableSource source1, @NonNull ObservableSource source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), false, 2); @@ -3062,7 +3108,10 @@ public static Observable merge(ObservableSource source1, Obs @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable merge(ObservableSource source1, ObservableSource source2, ObservableSource source3) { + @NonNull + public static Observable merge( + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3110,9 +3159,10 @@ public static Observable merge(ObservableSource source1, Obs @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable merge( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3155,7 +3205,9 @@ public static Observable merge( @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable mergeArray(ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable mergeArray(@NonNull ObservableSource... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), sources.length); } @@ -3187,7 +3239,8 @@ public static Observable mergeArray(ObservableSource... sour @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable mergeDelayError(Iterable> sources) { + @NonNull + public static Observable mergeDelayError(@NonNull Iterable> sources) { return fromIterable(sources).flatMap((Function)Functions.identity(), true); } @@ -3223,7 +3276,8 @@ public static Observable mergeDelayError(Iterable Observable mergeDelayError(Iterable> sources, int maxConcurrency, int bufferSize) { + @NonNull + public static Observable mergeDelayError(@NonNull Iterable> sources, int maxConcurrency, int bufferSize) { return fromIterable(sources).flatMap((Function)Functions.identity(), true, maxConcurrency, bufferSize); } @@ -3259,7 +3313,9 @@ public static Observable mergeDelayError(Iterable Observable mergeArrayDelayError(int maxConcurrency, int bufferSize, ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable mergeArrayDelayError(int maxConcurrency, int bufferSize, @NonNull ObservableSource... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), true, maxConcurrency, bufferSize); } @@ -3293,7 +3349,8 @@ public static Observable mergeArrayDelayError(int maxConcurrency, int buf @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable mergeDelayError(Iterable> sources, int maxConcurrency) { + @NonNull + public static Observable mergeDelayError(@NonNull Iterable> sources, int maxConcurrency) { return fromIterable(sources).flatMap((Function)Functions.identity(), true, maxConcurrency); } @@ -3325,7 +3382,8 @@ public static Observable mergeDelayError(Iterable Observable mergeDelayError(ObservableSource> sources) { + @NonNull + public static Observable mergeDelayError(@NonNull ObservableSource> sources) { Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), true, Integer.MAX_VALUE, bufferSize())); } @@ -3362,7 +3420,8 @@ public static Observable mergeDelayError(ObservableSource Observable mergeDelayError(ObservableSource> sources, int maxConcurrency) { + @NonNull + public static Observable mergeDelayError(@NonNull ObservableSource> sources, int maxConcurrency) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); return RxJavaPlugins.onAssembly(new ObservableFlatMap(sources, Functions.identity(), true, maxConcurrency, bufferSize())); @@ -3397,7 +3456,9 @@ public static Observable mergeDelayError(ObservableSource Observable mergeDelayError(ObservableSource source1, ObservableSource source2) { + @NonNull + public static Observable mergeDelayError( + @NonNull ObservableSource source1, @NonNull ObservableSource source2) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); return fromArray(source1, source2).flatMap((Function)Functions.identity(), true, 2); @@ -3435,7 +3496,10 @@ public static Observable mergeDelayError(ObservableSource so @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable mergeDelayError(ObservableSource source1, ObservableSource source2, ObservableSource source3) { + @NonNull + public static Observable mergeDelayError( + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3476,9 +3540,10 @@ public static Observable mergeDelayError(ObservableSource so @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable mergeDelayError( - ObservableSource source1, ObservableSource source2, - ObservableSource source3, ObservableSource source4) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -3514,7 +3579,9 @@ public static Observable mergeDelayError( @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable mergeArrayDelayError(ObservableSource... sources) { + @NonNull + @SafeVarargs + public static Observable mergeArrayDelayError(@NonNull ObservableSource... sources) { return fromArray(sources).flatMap((Function)Functions.identity(), true, sources.length); } @@ -3537,6 +3604,7 @@ public static Observable mergeArrayDelayError(ObservableSource Observable never() { return RxJavaPlugins.onAssembly((Observable) ObservableNever.INSTANCE); } @@ -3562,7 +3630,8 @@ public static Observable never() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable range(final int start, final int count) { + @NonNull + public static Observable range(int start, int count) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); } @@ -3599,6 +3668,7 @@ public static Observable range(final int start, final int count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable rangeLong(long start, long count) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); @@ -3641,7 +3711,8 @@ public static Observable rangeLong(long start, long count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(ObservableSource source1, ObservableSource source2) { + @NonNull + public static Single sequenceEqual(@NonNull ObservableSource source1, @NonNull ObservableSource source2) { return sequenceEqual(source1, source2, ObjectHelper.equalsPredicate(), bufferSize()); } @@ -3670,8 +3741,10 @@ public static Single sequenceEqual(ObservableSource so */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(ObservableSource source1, ObservableSource source2, - BiPredicate isEqual) { + @NonNull + public static Single sequenceEqual( + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull BiPredicate isEqual) { return sequenceEqual(source1, source2, isEqual, bufferSize()); } @@ -3702,8 +3775,10 @@ public static Single sequenceEqual(ObservableSource so */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(ObservableSource source1, ObservableSource source2, - BiPredicate isEqual, int bufferSize) { + @NonNull + public static Single sequenceEqual( + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull BiPredicate isEqual, int bufferSize) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(isEqual, "isEqual is null"); @@ -3734,7 +3809,8 @@ public static Single sequenceEqual(ObservableSource so */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Single sequenceEqual(ObservableSource source1, ObservableSource source2, + @NonNull + public static Single sequenceEqual(@NonNull ObservableSource source1, @NonNull ObservableSource source2, int bufferSize) { return sequenceEqual(source1, source2, ObjectHelper.equalsPredicate(), bufferSize); } @@ -3769,7 +3845,8 @@ public static Single sequenceEqual(ObservableSource so @SuppressWarnings({ "rawtypes", "unchecked" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable switchOnNext(ObservableSource> sources, int bufferSize) { + @NonNull + public static Observable switchOnNext(@NonNull ObservableSource> sources, int bufferSize) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableSwitchMap(sources, Functions.identity(), bufferSize, false)); @@ -3802,7 +3879,8 @@ public static Observable switchOnNext(ObservableSource Observable switchOnNext(ObservableSource> sources) { + @NonNull + public static Observable switchOnNext(@NonNull ObservableSource> sources) { return switchOnNext(sources, bufferSize()); } @@ -3835,7 +3913,8 @@ public static Observable switchOnNext(ObservableSource Observable switchOnNextDelayError(ObservableSource> sources) { + @NonNull + public static Observable switchOnNextDelayError(@NonNull ObservableSource> sources) { return switchOnNextDelayError(sources, bufferSize()); } @@ -3871,7 +3950,8 @@ public static Observable switchOnNextDelayError(ObservableSource Observable switchOnNextDelayError(ObservableSource> sources, int prefetch) { + @NonNull + public static Observable switchOnNextDelayError(@NonNull ObservableSource> sources, int prefetch) { Objects.requireNonNull(sources, "sources is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableSwitchMap(sources, Functions.identity(), prefetch, true)); @@ -3895,7 +3975,8 @@ public static Observable switchOnNextDelayError(ObservableSource timer(long delay, TimeUnit unit) { + @NonNull + public static Observable timer(long delay, @NonNull TimeUnit unit) { return timer(delay, unit, Schedulers.computation()); } @@ -3924,7 +4005,8 @@ public static Observable timer(long delay, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public static Observable timer(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public static Observable timer(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -3945,7 +4027,8 @@ public static Observable timer(long delay, TimeUnit unit, Scheduler schedu */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable unsafeCreate(ObservableSource onSubscribe) { + @NonNull + public static Observable unsafeCreate(@NonNull ObservableSource onSubscribe) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); if (onSubscribe instanceof Observable) { throw new IllegalArgumentException("unsafeCreate(Observable) should be upgraded"); @@ -3976,7 +4059,11 @@ public static Observable unsafeCreate(ObservableSource onSubscribe) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable using(Supplier resourceSupplier, Function> sourceSupplier, Consumer disposer) { + @NonNull + public static Observable using( + @NonNull Supplier resourceSupplier, + @NonNull Function> sourceSupplier, + @NonNull Consumer disposer) { return using(resourceSupplier, sourceSupplier, disposer, true); } @@ -4012,7 +4099,11 @@ public static Observable using(Supplier resourceSupplier, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable using(Supplier resourceSupplier, Function> sourceSupplier, Consumer disposer, boolean eager) { + @NonNull + public static Observable using( + @NonNull Supplier resourceSupplier, + @NonNull Function> sourceSupplier, + @NonNull Consumer disposer, boolean eager) { Objects.requireNonNull(resourceSupplier, "resourceSupplier is null"); Objects.requireNonNull(sourceSupplier, "sourceSupplier is null"); Objects.requireNonNull(disposer, "disposer is null"); @@ -4034,7 +4125,8 @@ public static Observable using(Supplier resourceSupplier, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable wrap(ObservableSource source) { + @NonNull + public static Observable wrap(@NonNull ObservableSource source) { Objects.requireNonNull(source, "source is null"); if (source instanceof Observable) { return RxJavaPlugins.onAssembly((Observable)source); @@ -4089,7 +4181,8 @@ public static Observable wrap(ObservableSource source) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public static Observable zip(Iterable> sources, Function zipper) { + @NonNull + public static Observable zip(@NonNull Iterable> sources, @NonNull Function zipper) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); return RxJavaPlugins.onAssembly(new ObservableZip(null, sources, zipper, bufferSize(), false)); @@ -4147,8 +4240,9 @@ public static Observable zip(Iterable Observable zip(Iterable> sources, - Function zipper, boolean delayError, + @NonNull + public static Observable zip(@NonNull Iterable> sources, + @NonNull Function zipper, boolean delayError, int bufferSize) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); @@ -4202,11 +4296,13 @@ public static Observable zip(Iterable Observable zip( - ObservableSource source1, ObservableSource source2, - BiFunction zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull BiFunction zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2); } @@ -4257,11 +4353,13 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, - BiFunction zipper, boolean delayError) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull BiFunction zipper, boolean delayError) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize(), source1, source2); } @@ -4313,11 +4411,13 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, - BiFunction zipper, boolean delayError, int bufferSize) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull BiFunction zipper, boolean delayError, int bufferSize) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), delayError, bufferSize, source1, source2); } @@ -4371,12 +4471,15 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, ObservableSource source3, - Function3 zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, + @NonNull Function3 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3); } @@ -4433,14 +4536,16 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, ObservableSource source3, - ObservableSource source4, - Function4 zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull Function4 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4); } @@ -4500,15 +4605,17 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, ObservableSource source3, - ObservableSource source4, ObservableSource source5, - Function5 zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, @NonNull ObservableSource source3, + @NonNull ObservableSource source4, @NonNull ObservableSource source5, + @NonNull Function5 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5); } @@ -4570,16 +4677,18 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, ObservableSource source3, - ObservableSource source4, ObservableSource source5, ObservableSource source6, - Function6 zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, @NonNull ObservableSource source3, + @NonNull ObservableSource source4, @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull Function6 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6); } @@ -4644,11 +4753,12 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, ObservableSource source3, - ObservableSource source4, ObservableSource source5, ObservableSource source6, - ObservableSource source7, - Function7 zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, @NonNull ObservableSource source3, + @NonNull ObservableSource source4, @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull ObservableSource source7, + @NonNull Function7 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -4656,6 +4766,7 @@ public static Observable zip( Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7); } @@ -4723,11 +4834,12 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, ObservableSource source3, - ObservableSource source4, ObservableSource source5, ObservableSource source6, - ObservableSource source7, ObservableSource source8, - Function8 zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, @NonNull ObservableSource source3, + @NonNull ObservableSource source4, @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull ObservableSource source7, @NonNull ObservableSource source8, + @NonNull Function8 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -4736,6 +4848,7 @@ public static Observable zip( Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -4806,11 +4919,12 @@ public static Observable zip( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public static Observable zip( - ObservableSource source1, ObservableSource source2, ObservableSource source3, - ObservableSource source4, ObservableSource source5, ObservableSource source6, - ObservableSource source7, ObservableSource source8, ObservableSource source9, - Function9 zipper) { + @NonNull ObservableSource source1, @NonNull ObservableSource source2, @NonNull ObservableSource source3, + @NonNull ObservableSource source4, @NonNull ObservableSource source5, @NonNull ObservableSource source6, + @NonNull ObservableSource source7, @NonNull ObservableSource source8, @NonNull ObservableSource source9, + @NonNull Function9 zipper) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); @@ -4820,6 +4934,7 @@ public static Observable zip( Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); Objects.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), false, bufferSize(), source1, source2, source3, source4, source5, source6, source7, source8, source9); } @@ -4876,8 +4991,11 @@ public static Observable zip( @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @SafeVarargs - public static Observable zipArray(Function zipper, - boolean delayError, int bufferSize, ObservableSource... sources) { + @NonNull + public static Observable zipArray( + @NonNull Function zipper, + boolean delayError, int bufferSize, + @NonNull ObservableSource... sources) { if (sources.length == 0) { return empty(); } @@ -4908,7 +5026,8 @@ public static Observable zipArray(Function all(Predicate predicate) { + @NonNull + public final Single all(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableAllSingle<>(this, predicate)); } @@ -4932,7 +5051,8 @@ public final Single all(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable ambWith(ObservableSource other) { + @NonNull + public final Observable ambWith(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return ambArray(this, other); } @@ -4959,7 +5079,8 @@ public final Observable ambWith(ObservableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single any(Predicate predicate) { + @NonNull + public final Single any(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableAnySingle<>(this, predicate)); } @@ -4981,6 +5102,7 @@ public final Single any(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final T blockingFirst() { BlockingFirstObserver observer = new BlockingFirstObserver<>(); subscribe(observer); @@ -5009,7 +5131,8 @@ public final T blockingFirst() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final T blockingFirst(T defaultItem) { + @NonNull + public final T blockingFirst(@NonNull T defaultItem) { BlockingFirstObserver observer = new BlockingFirstObserver<>(); subscribe(observer); T v = observer.blockingGet(); @@ -5046,7 +5169,8 @@ public final T blockingFirst(T defaultItem) { * @see #subscribe(Consumer) */ @SchedulerSupport(SchedulerSupport.NONE) - public final void blockingForEach(Consumer onNext) { + @NonNull + public final void blockingForEach(@NonNull Consumer onNext) { Iterator it = blockingIterable().iterator(); while (it.hasNext()) { try { @@ -5073,6 +5197,7 @@ public final void blockingForEach(Consumer onNext) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingIterable() { return blockingIterable(bufferSize()); } @@ -5092,6 +5217,7 @@ public final Iterable blockingIterable() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingIterable(int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return new BlockingObservableIterable<>(this, bufferSize); @@ -5118,6 +5244,7 @@ public final Iterable blockingIterable(int bufferSize) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final T blockingLast() { BlockingLastObserver observer = new BlockingLastObserver<>(); subscribe(observer); @@ -5150,7 +5277,8 @@ public final T blockingLast() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final T blockingLast(T defaultItem) { + @NonNull + public final T blockingLast(@NonNull T defaultItem) { BlockingLastObserver observer = new BlockingLastObserver<>(); subscribe(observer); T v = observer.blockingGet(); @@ -5178,6 +5306,7 @@ public final T blockingLast(T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingLatest() { return new BlockingObservableLatest<>(this); } @@ -5201,7 +5330,8 @@ public final Iterable blockingLatest() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Iterable blockingMostRecent(T initialValue) { + @NonNull + public final Iterable blockingMostRecent(@NonNull T initialValue) { return new BlockingObservableMostRecent<>(this, initialValue); } @@ -5221,6 +5351,7 @@ public final Iterable blockingMostRecent(T initialValue) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Iterable blockingNext() { return new BlockingObservableNext<>(this); } @@ -5244,6 +5375,7 @@ public final Iterable blockingNext() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final T blockingSingle() { T v = singleElement().blockingGet(); if (v == null) { @@ -5275,7 +5407,8 @@ public final T blockingSingle() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final T blockingSingle(T defaultItem) { + @NonNull + public final T blockingSingle(@NonNull T defaultItem) { return single(defaultItem).blockingGet(); } @@ -5300,6 +5433,7 @@ public final T blockingSingle(T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Future toFuture() { return subscribeWith(new FutureObserver()); } @@ -5350,7 +5484,7 @@ public final void blockingSubscribe() { * @see #blockingSubscribe(Consumer, Consumer, Action) */ @SchedulerSupport(SchedulerSupport.NONE) - public final void blockingSubscribe(Consumer onNext) { + public final void blockingSubscribe(@NonNull Consumer onNext) { ObservableBlockingSubscribe.subscribe(this, onNext, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -5372,7 +5506,7 @@ public final void blockingSubscribe(Consumer onNext) { * @see #blockingSubscribe(Consumer, Consumer, Action) */ @SchedulerSupport(SchedulerSupport.NONE) - public final void blockingSubscribe(Consumer onNext, Consumer onError) { + public final void blockingSubscribe(@NonNull Consumer onNext, @NonNull Consumer onError) { ObservableBlockingSubscribe.subscribe(this, onNext, onError, Functions.EMPTY_ACTION); } @@ -5394,7 +5528,7 @@ public final void blockingSubscribe(Consumer onNext, Consumer onNext, Consumer onError, Action onComplete) { + public final void blockingSubscribe(@NonNull Consumer onNext, @NonNull Consumer onError, @NonNull Action onComplete) { ObservableBlockingSubscribe.subscribe(this, onNext, onError, onComplete); } @@ -5415,7 +5549,7 @@ public final void blockingSubscribe(Consumer onNext, Consumer observer) { + public final void blockingSubscribe(@NonNull Observer observer) { ObservableBlockingSubscribe.subscribe(this, observer); } @@ -5440,7 +5574,8 @@ public final void blockingSubscribe(Observer observer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> buffer(int count) { + @NonNull + public final Observable<@NonNull List> buffer(int count) { return buffer(count, count); } @@ -5469,7 +5604,8 @@ public final Observable> buffer(int count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> buffer(int count, int skip) { + @NonNull + public final Observable<@NonNull List> buffer(int count, int skip) { return buffer(count, skip, ArrayListSupplier.asSupplier()); } @@ -5502,7 +5638,8 @@ public final Observable> buffer(int count, int skip) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > Observable buffer(int count, int skip, Supplier bufferSupplier) { + @NonNull + public final > Observable buffer(int count, int skip, @NonNull Supplier bufferSupplier) { ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(skip, "skip"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); @@ -5534,7 +5671,8 @@ public final > Observable buffer(int count, i */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > Observable buffer(int count, Supplier bufferSupplier) { + @NonNull + public final <@NonNull U extends Collection> Observable buffer(int count, @NonNull Supplier bufferSupplier) { return buffer(count, count, bufferSupplier); } @@ -5564,7 +5702,8 @@ public final > Observable buffer(int count, S */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable> buffer(long timespan, long timeskip, TimeUnit unit) { + @NonNull + public final Observable<@NonNull List> buffer(long timespan, long timeskip, @NonNull TimeUnit unit) { return buffer(timespan, timeskip, unit, Schedulers.computation(), ArrayListSupplier.asSupplier()); } @@ -5597,7 +5736,8 @@ public final Observable> buffer(long timespan, long timeskip, TimeUnit u */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> buffer(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable<@NonNull List> buffer(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return buffer(timespan, timeskip, unit, scheduler, ArrayListSupplier.asSupplier()); } @@ -5634,7 +5774,8 @@ public final Observable> buffer(long timespan, long timeskip, TimeUnit u */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final > Observable buffer(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler, Supplier bufferSupplier) { + @NonNull + public final <@NonNull U extends Collection> Observable buffer(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, @NonNull Supplier bufferSupplier) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); @@ -5666,7 +5807,8 @@ public final > Observable buffer(long timespa */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable> buffer(long timespan, TimeUnit unit) { + @NonNull + public final Observable<@NonNull List> buffer(long timespan, @NonNull TimeUnit unit) { return buffer(timespan, unit, Schedulers.computation(), Integer.MAX_VALUE); } @@ -5699,7 +5841,8 @@ public final Observable> buffer(long timespan, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable> buffer(long timespan, TimeUnit unit, int count) { + @NonNull + public final Observable<@NonNull List> buffer(long timespan, @NonNull TimeUnit unit, int count) { return buffer(timespan, unit, Schedulers.computation(), count); } @@ -5734,7 +5877,8 @@ public final Observable> buffer(long timespan, TimeUnit unit, int count) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> buffer(long timespan, TimeUnit unit, Scheduler scheduler, int count) { + @NonNull + public final Observable<@NonNull List> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int count) { return buffer(timespan, unit, scheduler, count, ArrayListSupplier.asSupplier(), false); } @@ -5775,10 +5919,11 @@ public final Observable> buffer(long timespan, TimeUnit unit, Scheduler */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final > Observable buffer( - long timespan, TimeUnit unit, - Scheduler scheduler, int count, - Supplier bufferSupplier, + @NonNull + public final <@NonNull U extends Collection> Observable buffer( + long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler, int count, + @NonNull Supplier bufferSupplier, boolean restartTimerOnMaxSize) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -5814,7 +5959,8 @@ public final > Observable buffer( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> buffer(long timespan, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable<@NonNull List> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return buffer(timespan, unit, scheduler, Integer.MAX_VALUE, ArrayListSupplier.asSupplier(), false); } @@ -5844,9 +5990,10 @@ public final Observable> buffer(long timespan, TimeUnit unit, Scheduler */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> buffer( - ObservableSource openingIndicator, - Function> closingIndicator) { + @NonNull + public final Observable<@NonNull List> buffer( + @NonNull ObservableSource openingIndicator, + @NonNull Function> closingIndicator) { return buffer(openingIndicator, closingIndicator, ArrayListSupplier.asSupplier()); } @@ -5880,10 +6027,11 @@ public final Observable> buffer( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > Observable buffer( - ObservableSource openingIndicator, - Function> closingIndicator, - Supplier bufferSupplier) { + @NonNull + public final > Observable buffer( + @NonNull ObservableSource openingIndicator, + @NonNull Function> closingIndicator, + @NonNull Supplier bufferSupplier) { Objects.requireNonNull(openingIndicator, "openingIndicator is null"); Objects.requireNonNull(closingIndicator, "closingIndicator is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); @@ -5916,7 +6064,8 @@ public final > Observable */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> buffer(ObservableSource boundary) { + @NonNull + public final Observable<@NonNull List> buffer(@NonNull ObservableSource boundary) { return buffer(boundary, ArrayListSupplier.asSupplier()); } @@ -5948,7 +6097,8 @@ public final Observable> buffer(ObservableSource boundary) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> buffer(ObservableSource boundary, final int initialCapacity) { + @NonNull + public final Observable<@NonNull List> buffer(@NonNull ObservableSource boundary, int initialCapacity) { ObjectHelper.verifyPositive(initialCapacity, "initialCapacity"); return buffer(boundary, Functions.createArrayList(initialCapacity)); } @@ -5983,7 +6133,8 @@ public final Observable> buffer(ObservableSource boundary, final */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > Observable buffer(ObservableSource boundary, Supplier bufferSupplier) { + @NonNull + public final > Observable buffer(@NonNull ObservableSource boundary, @NonNull Supplier bufferSupplier) { Objects.requireNonNull(boundary, "boundary is null"); Objects.requireNonNull(bufferSupplier, "bufferSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableBufferExactBoundary<>(this, boundary, bufferSupplier)); @@ -6039,6 +6190,7 @@ public final > Observable buffer(Observabl */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable cache() { return cacheWithInitialCapacity(16); } @@ -6097,6 +6249,7 @@ public final Observable cache() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable cacheWithInitialCapacity(int initialCapacity) { ObjectHelper.verifyPositive(initialCapacity, "initialCapacity"); return RxJavaPlugins.onAssembly(new ObservableCache<>(this, initialCapacity)); @@ -6122,7 +6275,8 @@ public final Observable cacheWithInitialCapacity(int initialCapacity) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable cast(final Class clazz) { + @NonNull + public final Observable cast(@NonNull Class clazz) { Objects.requireNonNull(clazz, "clazz is null"); return map(Functions.castFunction(clazz)); } @@ -6155,7 +6309,8 @@ public final Observable cast(final Class clazz) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single collect(Supplier initialValueSupplier, BiConsumer collector) { + @NonNull + public final Single collect(@NonNull Supplier initialValueSupplier, @NonNull BiConsumer collector) { Objects.requireNonNull(initialValueSupplier, "initialValueSupplier is null"); Objects.requireNonNull(collector, "collector is null"); return RxJavaPlugins.onAssembly(new ObservableCollectSingle(this, initialValueSupplier, collector)); @@ -6189,7 +6344,8 @@ public final Single collect(Supplier initialValueSupplier, B */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single collectInto(final U initialValue, BiConsumer collector) { + @NonNull + public final Single collectInto(@NonNull U initialValue, @NonNull BiConsumer collector) { Objects.requireNonNull(initialValue, "initialValue is null"); return collect(Functions.justSupplier(initialValue), collector); } @@ -6216,7 +6372,8 @@ public final Single collectInto(final U initialValue, BiConsumer Observable compose(ObservableTransformer composer) { + @NonNull + public final Observable compose(@NonNull ObservableTransformer composer) { return wrap(((ObservableTransformer) Objects.requireNonNull(composer, "composer is null")).apply(this)); } @@ -6246,7 +6403,8 @@ public final Observable compose(ObservableTransformer Observable concatMap(Function> mapper) { + @NonNull + public final Observable concatMap(@NonNull Function> mapper) { return concatMap(mapper, 2); } @@ -6278,7 +6436,8 @@ public final Observable concatMap(Function Observable concatMap(Function> mapper, int prefetch) { + @NonNull + public final Observable concatMap(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); if (this instanceof ScalarSupplier) { @@ -6320,7 +6479,8 @@ public final Observable concatMap(Function Observable concatMap(Function> mapper, int prefetch, Scheduler scheduler) { + @NonNull + public final Observable concatMap(@NonNull Function> mapper, int prefetch, @NonNull Scheduler scheduler) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -6350,7 +6510,8 @@ public final Observable concatMap(Function Observable concatMapDelayError(Function> mapper) { + @NonNull + public final Observable concatMapDelayError(@NonNull Function> mapper) { return concatMapDelayError(mapper, true, bufferSize()); } @@ -6382,7 +6543,8 @@ public final Observable concatMapDelayError(Function Observable concatMapDelayError(Function> mapper, + @NonNull + public final Observable concatMapDelayError(@NonNull Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); @@ -6424,8 +6586,9 @@ public final Observable concatMapDelayError(Function Observable concatMapDelayError(Function> mapper, - boolean tillTheEnd, int prefetch, Scheduler scheduler) { + @NonNull + public final Observable concatMapDelayError(@NonNull Function> mapper, + boolean tillTheEnd, int prefetch, @NonNull Scheduler scheduler) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -6453,7 +6616,8 @@ public final Observable concatMapDelayError(Function Observable concatMapEager(Function> mapper) { + @NonNull + public final Observable concatMapEager(@NonNull Function> mapper) { return concatMapEager(mapper, Integer.MAX_VALUE, bufferSize()); } @@ -6480,7 +6644,8 @@ public final Observable concatMapEager(Function Observable concatMapEager(Function> mapper, + @NonNull + public final Observable concatMapEager(@NonNull Function> mapper, int maxConcurrency, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); @@ -6512,7 +6677,8 @@ public final Observable concatMapEager(Function Observable concatMapEagerDelayError(Function> mapper, + @NonNull + public final Observable concatMapEagerDelayError(@NonNull Function> mapper, boolean tillTheEnd) { return concatMapEagerDelayError(mapper, tillTheEnd, Integer.MAX_VALUE, bufferSize()); } @@ -6545,7 +6711,8 @@ public final Observable concatMapEagerDelayError(Function Observable concatMapEagerDelayError(Function> mapper, + @NonNull + public final Observable concatMapEagerDelayError(@NonNull Function> mapper, boolean tillTheEnd, int maxConcurrency, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); @@ -6570,7 +6737,8 @@ public final Observable concatMapEagerDelayError(Function mapper) { + @NonNull + public final Completable concatMapCompletable(@NonNull Function mapper) { return concatMapCompletable(mapper, 2); } @@ -6595,7 +6763,8 @@ public final Completable concatMapCompletable(Function mapper, int capacityHint) { + @NonNull + public final Completable concatMapCompletable(@NonNull Function mapper, int capacityHint) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable<>(this, mapper, ErrorMode.IMMEDIATE, capacityHint)); @@ -6621,7 +6790,8 @@ public final Completable concatMapCompletable(Function mapper) { + @NonNull + public final Completable concatMapCompletableDelayError(@NonNull Function mapper) { return concatMapCompletableDelayError(mapper, true, 2); } @@ -6651,7 +6821,8 @@ public final Completable concatMapCompletableDelayError(Function mapper, boolean tillTheEnd) { + @NonNull + public final Completable concatMapCompletableDelayError(@NonNull Function mapper, boolean tillTheEnd) { return concatMapCompletableDelayError(mapper, tillTheEnd, 2); } @@ -6685,7 +6856,8 @@ public final Completable concatMapCompletableDelayError(Function mapper, boolean tillTheEnd, int prefetch) { + @NonNull + public final Completable concatMapCompletableDelayError(@NonNull Function mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapCompletable<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); @@ -6713,7 +6885,8 @@ public final Completable concatMapCompletableDelayError(Function Observable concatMapIterable(final Function> mapper) { + @NonNull + public final Observable concatMapIterable(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlattenIterable<>(this, mapper)); } @@ -6742,7 +6915,8 @@ public final Observable concatMapIterable(final Function Observable concatMapIterable(final Function> mapper, int prefetch) { + @NonNull + public final Observable concatMapIterable(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return concatMap(ObservableInternalHelper.flatMapIntoIterable(mapper), prefetch); @@ -6770,7 +6944,8 @@ public final Observable concatMapIterable(final Function Observable concatMapMaybe(Function> mapper) { + @NonNull + public final Observable concatMapMaybe(@NonNull Function> mapper) { return concatMapMaybe(mapper, 2); } @@ -6800,7 +6975,8 @@ public final Observable concatMapMaybe(Function Observable concatMapMaybe(Function> mapper, int prefetch) { + @NonNull + public final Observable concatMapMaybe(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe<>(this, mapper, ErrorMode.IMMEDIATE, prefetch)); @@ -6828,7 +7004,8 @@ public final Observable concatMapMaybe(Function Observable concatMapMaybeDelayError(Function> mapper) { + @NonNull + public final Observable concatMapMaybeDelayError(@NonNull Function> mapper) { return concatMapMaybeDelayError(mapper, true, 2); } @@ -6860,7 +7037,8 @@ public final Observable concatMapMaybeDelayError(Function Observable concatMapMaybeDelayError(Function> mapper, boolean tillTheEnd) { + @NonNull + public final Observable concatMapMaybeDelayError(@NonNull Function> mapper, boolean tillTheEnd) { return concatMapMaybeDelayError(mapper, tillTheEnd, 2); } @@ -6895,7 +7073,8 @@ public final Observable concatMapMaybeDelayError(Function Observable concatMapMaybeDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { + @NonNull + public final Observable concatMapMaybeDelayError(@NonNull Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapMaybe<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); @@ -6923,7 +7102,8 @@ public final Observable concatMapMaybeDelayError(Function Observable concatMapSingle(Function> mapper) { + @NonNull + public final Observable concatMapSingle(@NonNull Function> mapper) { return concatMapSingle(mapper, 2); } @@ -6953,7 +7133,8 @@ public final Observable concatMapSingle(Function Observable concatMapSingle(Function> mapper, int prefetch) { + @NonNull + public final Observable concatMapSingle(@NonNull Function> mapper, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle<>(this, mapper, ErrorMode.IMMEDIATE, prefetch)); @@ -6981,7 +7162,8 @@ public final Observable concatMapSingle(Function Observable concatMapSingleDelayError(Function> mapper) { + @NonNull + public final Observable concatMapSingleDelayError(@NonNull Function> mapper) { return concatMapSingleDelayError(mapper, true, 2); } @@ -7013,7 +7195,8 @@ public final Observable concatMapSingleDelayError(Function Observable concatMapSingleDelayError(Function> mapper, boolean tillTheEnd) { + @NonNull + public final Observable concatMapSingleDelayError(@NonNull Function> mapper, boolean tillTheEnd) { return concatMapSingleDelayError(mapper, tillTheEnd, 2); } @@ -7048,7 +7231,8 @@ public final Observable concatMapSingleDelayError(Function Observable concatMapSingleDelayError(Function> mapper, boolean tillTheEnd, int prefetch) { + @NonNull + public final Observable concatMapSingleDelayError(@NonNull Function> mapper, boolean tillTheEnd, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); return RxJavaPlugins.onAssembly(new ObservableConcatMapSingle<>(this, mapper, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY, prefetch)); @@ -7072,7 +7256,8 @@ public final Observable concatMapSingleDelayError(Function concatWith(ObservableSource other) { + @NonNull + public final Observable concatWith(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return concat(this, other); } @@ -7093,6 +7278,7 @@ public final Observable concatWith(ObservableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable concatWith(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableConcatWithSingle<>(this, other)); @@ -7114,6 +7300,7 @@ public final Observable concatWith(@NonNull SingleSource other) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable concatWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableConcatWithMaybe<>(this, other)); @@ -7135,6 +7322,7 @@ public final Observable concatWith(@NonNull MaybeSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable concatWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableConcatWithCompletable<>(this, other)); @@ -7158,7 +7346,8 @@ public final Observable concatWith(@NonNull CompletableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single contains(final Object element) { + @NonNull + public final Single contains(@NonNull Object element) { Objects.requireNonNull(element, "element is null"); return any(Functions.equalsWith(element)); } @@ -7179,6 +7368,7 @@ public final Single contains(final Object element) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single count() { return RxJavaPlugins.onAssembly(new ObservableCountSingle<>(this)); } @@ -7211,7 +7401,8 @@ public final Single count() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable debounce(Function> debounceSelector) { + @NonNull + public final Observable debounce(@NonNull Function> debounceSelector) { Objects.requireNonNull(debounceSelector, "debounceSelector is null"); return RxJavaPlugins.onAssembly(new ObservableDebounce<>(this, debounceSelector)); } @@ -7250,7 +7441,8 @@ public final Observable debounce(Function debounce(long timeout, TimeUnit unit) { + @NonNull + public final Observable debounce(long timeout, @NonNull TimeUnit unit) { return debounce(timeout, unit, Schedulers.computation()); } @@ -7290,7 +7482,8 @@ public final Observable debounce(long timeout, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable debounce(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable debounce(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableDebounceTimed<>(this, timeout, unit, scheduler)); @@ -7314,7 +7507,8 @@ public final Observable debounce(long timeout, TimeUnit unit, Scheduler sched */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable defaultIfEmpty(T defaultItem) { + @NonNull + public final Observable defaultIfEmpty(@NonNull T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); return switchIfEmpty(just(defaultItem)); } @@ -7344,7 +7538,8 @@ public final Observable defaultIfEmpty(T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable delay(final Function> itemDelay) { + @NonNull + public final Observable delay(@NonNull Function> itemDelay) { Objects.requireNonNull(itemDelay, "itemDelay is null"); return flatMap(ObservableInternalHelper.itemDelay(itemDelay)); } @@ -7368,7 +7563,8 @@ public final Observable delay(final Function delay(long delay, TimeUnit unit) { + @NonNull + public final Observable delay(long delay, @NonNull TimeUnit unit) { return delay(delay, unit, Schedulers.computation(), false); } @@ -7394,7 +7590,8 @@ public final Observable delay(long delay, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable delay(long delay, TimeUnit unit, boolean delayError) { + @NonNull + public final Observable delay(long delay, @NonNull TimeUnit unit, boolean delayError) { return delay(delay, unit, Schedulers.computation(), delayError); } @@ -7419,7 +7616,8 @@ public final Observable delay(long delay, TimeUnit unit, boolean delayError) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable delay(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable delay(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delay(delay, unit, scheduler, false); } @@ -7447,7 +7645,8 @@ public final Observable delay(long delay, TimeUnit unit, Scheduler scheduler) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable delay(long delay, TimeUnit unit, Scheduler scheduler, boolean delayError) { + @NonNull + public final Observable delay(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -7484,8 +7683,9 @@ public final Observable delay(long delay, TimeUnit unit, Scheduler scheduler, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable delay(ObservableSource subscriptionDelay, - Function> itemDelay) { + @NonNull + public final Observable delay(@NonNull ObservableSource subscriptionDelay, + @NonNull Function> itemDelay) { return delaySubscription(subscriptionDelay).delay(itemDelay); } @@ -7508,7 +7708,8 @@ public final Observable delay(ObservableSource subscriptionDelay, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable delaySubscription(ObservableSource other) { + @NonNull + public final Observable delaySubscription(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableDelaySubscriptionOther<>(this, other)); } @@ -7531,7 +7732,8 @@ public final Observable delaySubscription(ObservableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable delaySubscription(long delay, TimeUnit unit) { + @NonNull + public final Observable delaySubscription(long delay, @NonNull TimeUnit unit) { return delaySubscription(delay, unit, Schedulers.computation()); } @@ -7557,7 +7759,8 @@ public final Observable delaySubscription(long delay, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable delaySubscription(long delay, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable delaySubscription(long delay, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return delaySubscription(timer(delay, unit, scheduler)); } @@ -7610,7 +7813,8 @@ public final Observable delaySubscription(long delay, TimeUnit unit, Schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable dematerialize(Function> selector) { + @NonNull + public final Observable dematerialize(@NonNull Function> selector) { Objects.requireNonNull(selector, "selector is null"); return RxJavaPlugins.onAssembly(new ObservableDematerialize<>(this, selector)); } @@ -7648,6 +7852,7 @@ public final Observable dematerialize(Function */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable distinct() { return distinct(Functions.identity(), Functions.createHashSet()); } @@ -7688,7 +7893,8 @@ public final Observable distinct() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable distinct(Function keySelector) { + @NonNull + public final Observable distinct(@NonNull Function keySelector) { return distinct(keySelector, Functions.createHashSet()); } @@ -7719,7 +7925,8 @@ public final Observable distinct(Function keySelector) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable distinct(Function keySelector, Supplier> collectionSupplier) { + @NonNull + public final Observable distinct(@NonNull Function keySelector, @NonNull Supplier> collectionSupplier) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableDistinct<>(this, keySelector, collectionSupplier)); @@ -7758,6 +7965,7 @@ public final Observable distinct(Function keySelector, Supp */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable distinctUntilChanged() { return distinctUntilChanged(Functions.identity()); } @@ -7800,7 +8008,8 @@ public final Observable distinctUntilChanged() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable distinctUntilChanged(Function keySelector) { + @NonNull + public final Observable distinctUntilChanged(@NonNull Function keySelector) { Objects.requireNonNull(keySelector, "keySelector is null"); return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged<>(this, keySelector, ObjectHelper.equalsPredicate())); } @@ -7834,7 +8043,8 @@ public final Observable distinctUntilChanged(Function keySe */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable distinctUntilChanged(BiPredicate comparer) { + @NonNull + public final Observable distinctUntilChanged(@NonNull BiPredicate comparer) { Objects.requireNonNull(comparer, "comparer is null"); return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged<>(this, Functions.identity(), comparer)); } @@ -7858,7 +8068,8 @@ public final Observable distinctUntilChanged(BiPredicate doAfterNext(Consumer onAfterNext) { + @NonNull + public final Observable doAfterNext(@NonNull Consumer onAfterNext) { Objects.requireNonNull(onAfterNext, "onAfterNext is null"); return RxJavaPlugins.onAssembly(new ObservableDoAfterNext<>(this, onAfterNext)); } @@ -7882,7 +8093,8 @@ public final Observable doAfterNext(Consumer onAfterNext) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doAfterTerminate(Action onFinally) { + @NonNull + public final Observable doAfterTerminate(@NonNull Action onFinally) { Objects.requireNonNull(onFinally, "onFinally is null"); return doOnEach(Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.EMPTY_ACTION, onFinally); } @@ -7909,7 +8121,8 @@ public final Observable doAfterTerminate(Action onFinally) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doFinally(Action onFinally) { + @NonNull + public final Observable doFinally(@NonNull Action onFinally) { Objects.requireNonNull(onFinally, "onFinally is null"); return RxJavaPlugins.onAssembly(new ObservableDoFinally<>(this, onFinally)); } @@ -7937,7 +8150,8 @@ public final Observable doFinally(Action onFinally) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnDispose(Action onDispose) { + @NonNull + public final Observable doOnDispose(@NonNull Action onDispose) { return doOnLifecycle(Functions.emptyConsumer(), onDispose); } @@ -7957,7 +8171,8 @@ public final Observable doOnDispose(Action onDispose) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnComplete(Action onComplete) { + @NonNull + public final Observable doOnComplete(@NonNull Action onComplete) { return doOnEach(Functions.emptyConsumer(), Functions.emptyConsumer(), onComplete, Functions.EMPTY_ACTION); } @@ -7976,7 +8191,8 @@ public final Observable doOnComplete(Action onComplete) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - private Observable doOnEach(Consumer onNext, Consumer onError, Action onComplete, Action onAfterTerminate) { + @NonNull + private Observable doOnEach(@NonNull Consumer onNext, @NonNull Consumer onError, @NonNull Action onComplete, @NonNull Action onAfterTerminate) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); @@ -8000,7 +8216,8 @@ private Observable doOnEach(Consumer onNext, Consumer doOnEach(final Consumer> onNotification) { + @NonNull + public final Observable doOnEach(@NonNull Consumer> onNotification) { Objects.requireNonNull(onNotification, "onNotification is null"); return doOnEach( Functions.notificationOnNext(onNotification), @@ -8032,7 +8249,8 @@ public final Observable doOnEach(final Consumer> onNo */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnEach(final Observer observer) { + @NonNull + public final Observable doOnEach(@NonNull Observer observer) { Objects.requireNonNull(observer, "observer is null"); return doOnEach( ObservableInternalHelper.observerOnNext(observer), @@ -8060,7 +8278,8 @@ public final Observable doOnEach(final Observer observer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnError(Consumer onError) { + @NonNull + public final Observable doOnError(@NonNull Consumer onError) { return doOnEach(Functions.emptyConsumer(), onError, Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); } @@ -8083,7 +8302,8 @@ public final Observable doOnError(Consumer onError) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnLifecycle(final Consumer onSubscribe, final Action onDispose) { + @NonNull + public final Observable doOnLifecycle(@NonNull Consumer onSubscribe, @NonNull Action onDispose) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); Objects.requireNonNull(onDispose, "onDispose is null"); return RxJavaPlugins.onAssembly(new ObservableDoOnLifecycle<>(this, onSubscribe, onDispose)); @@ -8105,7 +8325,8 @@ public final Observable doOnLifecycle(final Consumer onSu */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnNext(Consumer onNext) { + @NonNull + public final Observable doOnNext(@NonNull Consumer onNext) { return doOnEach(onNext, Functions.emptyConsumer(), Functions.EMPTY_ACTION, Functions.EMPTY_ACTION); } @@ -8128,7 +8349,8 @@ public final Observable doOnNext(Consumer onNext) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnSubscribe(Consumer onSubscribe) { + @NonNull + public final Observable doOnSubscribe(@NonNull Consumer onSubscribe) { return doOnLifecycle(onSubscribe, Functions.EMPTY_ACTION); } @@ -8153,7 +8375,8 @@ public final Observable doOnSubscribe(Consumer onSubscrib */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable doOnTerminate(final Action onTerminate) { + @NonNull + public final Observable doOnTerminate(@NonNull Action onTerminate) { Objects.requireNonNull(onTerminate, "onTerminate is null"); return doOnEach(Functions.emptyConsumer(), Functions.actionConsumer(onTerminate), onTerminate, @@ -8180,6 +8403,7 @@ public final Observable doOnTerminate(final Action onTerminate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe elementAt(long index) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); @@ -8209,7 +8433,8 @@ public final Maybe elementAt(long index) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single elementAt(long index, T defaultItem) { + @NonNull + public final Single elementAt(long index, @NonNull T defaultItem) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); } @@ -8237,6 +8462,7 @@ public final Single elementAt(long index, T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single elementAtOrError(long index) { if (index < 0) { throw new IndexOutOfBoundsException("index >= 0 required but it was " + index); @@ -8262,7 +8488,8 @@ public final Single elementAtOrError(long index) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable filter(Predicate predicate) { + @NonNull + public final Observable filter(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableFilter<>(this, predicate)); } @@ -8282,6 +8509,7 @@ public final Observable filter(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe firstElement() { return elementAt(0L); } @@ -8303,7 +8531,8 @@ public final Maybe firstElement() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single first(T defaultItem) { + @NonNull + public final Single first(@NonNull T defaultItem) { return elementAt(0L, defaultItem); } @@ -8322,6 +8551,7 @@ public final Single first(T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single firstOrError() { return elementAtOrError(0L); } @@ -8348,7 +8578,8 @@ public final Single firstOrError() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable flatMap(Function> mapper) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper) { return flatMap(mapper, false); } @@ -8377,7 +8608,8 @@ public final Observable flatMap(Function Observable flatMap(Function> mapper, boolean delayErrors) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, boolean delayErrors) { return flatMap(mapper, delayErrors, Integer.MAX_VALUE); } @@ -8410,7 +8642,8 @@ public final Observable flatMap(Function Observable flatMap(Function> mapper, boolean delayErrors, int maxConcurrency) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, boolean delayErrors, int maxConcurrency) { return flatMap(mapper, delayErrors, maxConcurrency, bufferSize()); } @@ -8445,7 +8678,8 @@ public final Observable flatMap(Function Observable flatMap(Function> mapper, + @NonNull + public final Observable flatMap(@NonNull Function> mapper, boolean delayErrors, int maxConcurrency, int bufferSize) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); @@ -8487,10 +8721,11 @@ public final Observable flatMap(Function Observable flatMap( - Function> onNextMapper, - Function> onErrorMapper, - Supplier> onCompleteSupplier) { + @NonNull Function> onNextMapper, + @NonNull Function> onErrorMapper, + @NonNull Supplier> onCompleteSupplier) { Objects.requireNonNull(onNextMapper, "onNextMapper is null"); Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); Objects.requireNonNull(onCompleteSupplier, "onCompleteSupplier is null"); @@ -8527,10 +8762,11 @@ public final Observable flatMap( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable flatMap( - Function> onNextMapper, - Function> onErrorMapper, - Supplier> onCompleteSupplier, + @NonNull Function> onNextMapper, + @NonNull Function> onErrorMapper, + @NonNull Supplier> onCompleteSupplier, int maxConcurrency) { Objects.requireNonNull(onNextMapper, "onNextMapper is null"); Objects.requireNonNull(onErrorMapper, "onErrorMapper is null"); @@ -8564,7 +8800,8 @@ public final Observable flatMap( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable flatMap(Function> mapper, int maxConcurrency) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, int maxConcurrency) { return flatMap(mapper, false, maxConcurrency, bufferSize()); } @@ -8593,8 +8830,9 @@ public final Observable flatMap(Function Observable flatMap(Function> mapper, - BiFunction resultSelector) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, + @NonNull BiFunction resultSelector) { return flatMap(mapper, resultSelector, false, bufferSize(), bufferSize()); } @@ -8626,8 +8864,9 @@ public final Observable flatMap(Function Observable flatMap(Function> mapper, - BiFunction combiner, boolean delayErrors) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, boolean delayErrors) { return flatMap(mapper, combiner, delayErrors, bufferSize(), bufferSize()); } @@ -8663,8 +8902,9 @@ public final Observable flatMap(Function Observable flatMap(Function> mapper, - BiFunction combiner, boolean delayErrors, int maxConcurrency) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, boolean delayErrors, int maxConcurrency) { return flatMap(mapper, combiner, delayErrors, maxConcurrency, bufferSize()); } @@ -8702,8 +8942,9 @@ public final Observable flatMap(Function Observable flatMap(final Function> mapper, - final BiFunction combiner, boolean delayErrors, int maxConcurrency, int bufferSize) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, boolean delayErrors, int maxConcurrency, int bufferSize) { Objects.requireNonNull(mapper, "mapper is null"); Objects.requireNonNull(combiner, "combiner is null"); return flatMap(ObservableInternalHelper.flatMapWithCombiner(mapper, combiner), delayErrors, maxConcurrency, bufferSize); @@ -8738,8 +8979,9 @@ public final Observable flatMap(final Function Observable flatMap(Function> mapper, - BiFunction combiner, int maxConcurrency) { + @NonNull + public final Observable flatMap(@NonNull Function> mapper, + @NonNull BiFunction combiner, int maxConcurrency) { return flatMap(mapper, combiner, false, maxConcurrency, bufferSize()); } @@ -8757,7 +8999,8 @@ public final Observable flatMap(Function mapper) { + @NonNull + public final Completable flatMapCompletable(@NonNull Function mapper) { return flatMapCompletable(mapper, false); } @@ -8777,7 +9020,8 @@ public final Completable flatMapCompletable(Function mapper, boolean delayErrors) { + @NonNull + public final Completable flatMapCompletable(@NonNull Function mapper, boolean delayErrors) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMapCompletableCompletable<>(this, mapper, delayErrors)); } @@ -8803,7 +9047,8 @@ public final Completable flatMapCompletable(Function Observable flatMapIterable(final Function> mapper) { + @NonNull + public final Observable flatMapIterable(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlattenIterable<>(this, mapper)); } @@ -8834,8 +9079,9 @@ public final Observable flatMapIterable(final Function Observable flatMapIterable(final Function> mapper, - BiFunction resultSelector) { + @NonNull + public final Observable flatMapIterable(@NonNull Function> mapper, + @NonNull BiFunction resultSelector) { Objects.requireNonNull(mapper, "mapper is null"); Objects.requireNonNull(resultSelector, "resultSelector is null"); return flatMap(ObservableInternalHelper.flatMapIntoIterable(mapper), resultSelector, false, bufferSize(), bufferSize()); @@ -8856,7 +9102,8 @@ public final Observable flatMapIterable(final Function Observable flatMapMaybe(Function> mapper) { + @NonNull + public final Observable flatMapMaybe(@NonNull Function> mapper) { return flatMapMaybe(mapper, false); } @@ -8878,7 +9125,8 @@ public final Observable flatMapMaybe(Function Observable flatMapMaybe(Function> mapper, boolean delayErrors) { + @NonNull + public final Observable flatMapMaybe(@NonNull Function> mapper, boolean delayErrors) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMapMaybe<>(this, mapper, delayErrors)); } @@ -8898,7 +9146,8 @@ public final Observable flatMapMaybe(Function Observable flatMapSingle(Function> mapper) { + @NonNull + public final Observable flatMapSingle(@NonNull Function> mapper) { return flatMapSingle(mapper, false); } @@ -8920,7 +9169,8 @@ public final Observable flatMapSingle(Function Observable flatMapSingle(Function> mapper, boolean delayErrors) { + @NonNull + public final Observable flatMapSingle(@NonNull Function> mapper, boolean delayErrors) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableFlatMapSingle<>(this, mapper, delayErrors)); } @@ -8946,7 +9196,8 @@ public final Observable flatMapSingle(Function onNext) { + @NonNull + public final Disposable forEach(@NonNull Consumer onNext) { return subscribe(onNext); } @@ -8974,7 +9225,8 @@ public final Disposable forEach(Consumer onNext) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable forEachWhile(Predicate onNext) { + @NonNull + public final Disposable forEachWhile(@NonNull Predicate onNext) { return forEachWhile(onNext, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -8999,7 +9251,8 @@ public final Disposable forEachWhile(Predicate onNext) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable forEachWhile(Predicate onNext, Consumer onError) { + @NonNull + public final Disposable forEachWhile(@NonNull Predicate onNext, @NonNull Consumer onError) { return forEachWhile(onNext, onError, Functions.EMPTY_ACTION); } @@ -9027,8 +9280,9 @@ public final Disposable forEachWhile(Predicate onNext, Consumer onNext, Consumer onError, - final Action onComplete) { + @NonNull + public final Disposable forEachWhile(@NonNull Predicate onNext, @NonNull Consumer onError, + @NonNull Action onComplete) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); @@ -9074,7 +9328,8 @@ public final Disposable forEachWhile(final Predicate onNext, Consumer @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> groupBy(Function keySelector) { + @NonNull + public final Observable> groupBy(@NonNull Function keySelector) { return groupBy(keySelector, (Function)Functions.identity(), false, bufferSize()); } @@ -9117,7 +9372,8 @@ public final Observable> groupBy(Function Observable> groupBy(Function keySelector, boolean delayError) { + @NonNull + public final Observable> groupBy(@NonNull Function keySelector, boolean delayError) { return groupBy(keySelector, (Function)Functions.identity(), delayError, bufferSize()); } @@ -9160,7 +9416,8 @@ public final Observable> groupBy(Function Observable> groupBy(Function keySelector, + @NonNull + public final Observable> groupBy(@NonNull Function keySelector, Function valueSelector) { return groupBy(keySelector, valueSelector, false, bufferSize()); } @@ -9207,8 +9464,9 @@ public final Observable> groupBy(Function Observable> groupBy(Function keySelector, - Function valueSelector, boolean delayError) { + @NonNull + public final Observable> groupBy(@NonNull Function keySelector, + @NonNull Function valueSelector, boolean delayError) { return groupBy(keySelector, valueSelector, delayError, bufferSize()); } @@ -9256,8 +9514,9 @@ public final Observable> groupBy(Function Observable> groupBy(Function keySelector, - Function valueSelector, + @NonNull + public final Observable> groupBy(@NonNull Function keySelector, + @NonNull Function valueSelector, boolean delayError, int bufferSize) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); @@ -9299,12 +9558,13 @@ public final Observable> groupBy(Function Observable groupJoin( - ObservableSource other, - Function> leftEnd, - Function> rightEnd, - BiFunction, ? extends R> resultSelector - ) { + @NonNull ObservableSource other, + @NonNull Function> leftEnd, + @NonNull Function> rightEnd, + @NonNull BiFunction, ? extends R> resultSelector + ) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(leftEnd, "leftEnd is null"); Objects.requireNonNull(rightEnd, "rightEnd is null"); @@ -9330,6 +9590,7 @@ public final Observable groupJoin( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable hide() { return RxJavaPlugins.onAssembly(new ObservableHide<>(this)); } @@ -9348,6 +9609,7 @@ public final Observable hide() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Completable ignoreElements() { return RxJavaPlugins.onAssembly(new ObservableIgnoreElementsCompletable<>(this)); } @@ -9369,6 +9631,7 @@ public final Completable ignoreElements() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single isEmpty() { return all(Functions.alwaysFalse()); } @@ -9406,12 +9669,13 @@ public final Single isEmpty() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable join( - ObservableSource other, - Function> leftEnd, - Function> rightEnd, - BiFunction resultSelector - ) { + @NonNull ObservableSource other, + @NonNull Function> leftEnd, + @NonNull Function> rightEnd, + @NonNull BiFunction resultSelector + ) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(leftEnd, "leftEnd is null"); Objects.requireNonNull(rightEnd, "rightEnd is null"); @@ -9436,6 +9700,7 @@ public final Observable join( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe lastElement() { return RxJavaPlugins.onAssembly(new ObservableLastMaybe<>(this)); } @@ -9458,7 +9723,8 @@ public final Maybe lastElement() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single last(T defaultItem) { + @NonNull + public final Single last(@NonNull T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new ObservableLastSingle<>(this, defaultItem)); } @@ -9479,6 +9745,7 @@ public final Single last(T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single lastOrError() { return RxJavaPlugins.onAssembly(new ObservableLastSingle<>(this, null)); } @@ -9627,7 +9894,8 @@ public final Single lastOrError() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable lift(ObservableOperator lifter) { + @NonNull + public final Observable lift(@NonNull ObservableOperator lifter) { Objects.requireNonNull(lifter, "lifter is null"); return RxJavaPlugins.onAssembly(new ObservableLift(this, lifter)); } @@ -9651,7 +9919,8 @@ public final Observable lift(ObservableOperator l */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable map(Function mapper) { + @NonNull + public final Observable map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableMap(this, mapper)); } @@ -9673,6 +9942,7 @@ public final Observable map(Function mapper) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> materialize() { return RxJavaPlugins.onAssembly(new ObservableMaterialize<>(this)); } @@ -9696,7 +9966,8 @@ public final Observable> materialize() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable mergeWith(ObservableSource other) { + @NonNull + public final Observable mergeWith(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return merge(this, other); } @@ -9719,6 +9990,7 @@ public final Observable mergeWith(ObservableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable mergeWith(@NonNull SingleSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableMergeWithSingle<>(this, other)); @@ -9743,6 +10015,7 @@ public final Observable mergeWith(@NonNull SingleSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable mergeWith(@NonNull MaybeSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableMergeWithMaybe<>(this, other)); @@ -9764,6 +10037,7 @@ public final Observable mergeWith(@NonNull MaybeSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable mergeWith(@NonNull CompletableSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableMergeWithCompletable<>(this, other)); @@ -9802,7 +10076,8 @@ public final Observable mergeWith(@NonNull CompletableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable observeOn(Scheduler scheduler) { + @NonNull + public final Observable observeOn(@NonNull Scheduler scheduler) { return observeOn(scheduler, false, bufferSize()); } @@ -9840,7 +10115,8 @@ public final Observable observeOn(Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable observeOn(Scheduler scheduler, boolean delayError) { + @NonNull + public final Observable observeOn(@NonNull Scheduler scheduler, boolean delayError) { return observeOn(scheduler, delayError, bufferSize()); } @@ -9879,7 +10155,8 @@ public final Observable observeOn(Scheduler scheduler, boolean delayError) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable observeOn(Scheduler scheduler, boolean delayError, int bufferSize) { + @NonNull + public final Observable observeOn(@NonNull Scheduler scheduler, boolean delayError, int bufferSize) { Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableObserveOn<>(this, scheduler, delayError, bufferSize)); @@ -9902,7 +10179,8 @@ public final Observable observeOn(Scheduler scheduler, boolean delayError, in */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable ofType(final Class clazz) { + @NonNull + public final Observable ofType(@NonNull Class clazz) { Objects.requireNonNull(clazz, "clazz is null"); return filter(Functions.isInstanceOf(clazz)).cast(clazz); } @@ -9938,7 +10216,8 @@ public final Observable ofType(final Class clazz) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable onErrorResumeNext(Function> resumeFunction) { + @NonNull + public final Observable onErrorResumeNext(@NonNull Function> resumeFunction) { Objects.requireNonNull(resumeFunction, "resumeFunction is null"); return RxJavaPlugins.onAssembly(new ObservableOnErrorNext<>(this, resumeFunction)); } @@ -9974,7 +10253,8 @@ public final Observable onErrorResumeNext(Function onErrorResumeWith(final ObservableSource next) { + @NonNull + public final Observable onErrorResumeWith(@NonNull ObservableSource next) { Objects.requireNonNull(next, "next is null"); return onErrorResumeNext(Functions.justFunction(next)); } @@ -10007,7 +10287,8 @@ public final Observable onErrorResumeWith(final ObservableSource */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable onErrorReturn(Function valueSupplier) { + @NonNull + public final Observable onErrorReturn(@NonNull Function valueSupplier) { Objects.requireNonNull(valueSupplier, "valueSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableOnErrorReturn<>(this, valueSupplier)); } @@ -10040,7 +10321,8 @@ public final Observable onErrorReturn(Function onErrorReturnItem(final T item) { + @NonNull + public final Observable onErrorReturnItem(@NonNull T item) { Objects.requireNonNull(item, "item is null"); return onErrorReturn(Functions.justFunction(item)); } @@ -10060,6 +10342,7 @@ public final Observable onErrorReturnItem(final T item) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable onTerminateDetach() { return RxJavaPlugins.onAssembly(new ObservableDetach<>(this)); } @@ -10081,6 +10364,7 @@ public final Observable onTerminateDetach() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final ConnectableObservable publish() { return RxJavaPlugins.onAssembly(new ObservablePublish<>(this)); } @@ -10106,7 +10390,8 @@ public final ConnectableObservable publish() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable publish(Function, ? extends ObservableSource> selector) { + @NonNull + public final Observable publish(@NonNull Function, ? extends ObservableSource> selector) { Objects.requireNonNull(selector, "selector is null"); return RxJavaPlugins.onAssembly(new ObservablePublishSelector<>(this, selector)); } @@ -10141,7 +10426,8 @@ public final Observable publish(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Maybe reduce(BiFunction reducer) { + @NonNull + public final Maybe reduce(@NonNull BiFunction reducer) { Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new ObservableReduceMaybe<>(this, reducer)); } @@ -10198,7 +10484,8 @@ public final Maybe reduce(BiFunction reducer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single reduce(R seed, BiFunction reducer) { + @NonNull + public final <@NonNull R> Single reduce(R seed, @NonNull BiFunction reducer) { Objects.requireNonNull(seed, "seed is null"); Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new ObservableReduceSeedSingle<>(this, seed, reducer)); @@ -10238,7 +10525,8 @@ public final Single reduce(R seed, BiFunction reducer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single reduceWith(Supplier seedSupplier, BiFunction reducer) { + @NonNull + public final <@NonNull R> Single reduceWith(@NonNull Supplier seedSupplier, @NonNull BiFunction reducer) { Objects.requireNonNull(seedSupplier, "seedSupplier is null"); Objects.requireNonNull(reducer, "reducer is null"); return RxJavaPlugins.onAssembly(new ObservableReduceWithSingle<>(this, seedSupplier, reducer)); @@ -10258,6 +10546,7 @@ public final Single reduceWith(Supplier seedSupplier, BiFunction repeat() { return repeat(Long.MAX_VALUE); } @@ -10283,6 +10572,7 @@ public final Observable repeat() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable repeat(long times) { if (times < 0) { throw new IllegalArgumentException("times >= 0 required but it was " + times); @@ -10314,7 +10604,8 @@ public final Observable repeat(long times) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable repeatUntil(BooleanSupplier stop) { + @NonNull + public final Observable repeatUntil(@NonNull BooleanSupplier stop) { Objects.requireNonNull(stop, "stop is null"); return RxJavaPlugins.onAssembly(new ObservableRepeatUntil<>(this, stop)); } @@ -10340,7 +10631,8 @@ public final Observable repeatUntil(BooleanSupplier stop) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable repeatWhen(final Function, ? extends ObservableSource> handler) { + @NonNull + public final Observable repeatWhen(@NonNull Function, ? extends ObservableSource> handler) { Objects.requireNonNull(handler, "handler is null"); return RxJavaPlugins.onAssembly(new ObservableRepeatWhen<>(this, handler)); } @@ -10363,6 +10655,7 @@ public final Observable repeatWhen(final Function, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final ConnectableObservable replay() { return ObservableReplay.createFrom(this); } @@ -10388,7 +10681,8 @@ public final ConnectableObservable replay() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable replay(Function, ? extends ObservableSource> selector) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector) { Objects.requireNonNull(selector, "selector is null"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this), selector); } @@ -10422,7 +10716,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, int bufferSize) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this, bufferSize, false), selector); @@ -10459,7 +10754,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize, boolean eagerTruncate) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, int bufferSize, boolean eagerTruncate) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return ObservableReplay.multicastSelector(ObservableInternalHelper.replaySupplier(this, bufferSize, eagerTruncate), selector); @@ -10498,7 +10794,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable replay(Function, ? extends ObservableSource> selector, int bufferSize, long time, TimeUnit unit) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, int bufferSize, long time, @NonNull TimeUnit unit) { return replay(selector, bufferSize, time, unit, Schedulers.computation()); } @@ -10540,7 +10837,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(unit, "unit is null"); @@ -10588,7 +10886,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable replay(Function, ? extends ObservableSource> selector, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { Objects.requireNonNull(selector, "selector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(unit, "unit is null"); @@ -10624,7 +10923,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable replay(Function, ? extends ObservableSource> selector, long time, TimeUnit unit) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, long time, @NonNull TimeUnit unit) { return replay(selector, time, unit, Schedulers.computation()); } @@ -10658,7 +10958,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable replay(Function, ? extends ObservableSource> selector, final long time, final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(selector, "selector is null"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -10697,7 +10998,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable replay(Function, ? extends ObservableSource> selector, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + @NonNull + public final Observable replay(@NonNull Function, ? extends ObservableSource> selector, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { Objects.requireNonNull(selector, "selector is null"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -10730,7 +11032,8 @@ public final Observable replay(Function, ? extends */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final ConnectableObservable replay(final int bufferSize) { + @NonNull + public final ConnectableObservable replay(int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return ObservableReplay.create(this, bufferSize, false); } @@ -10762,7 +11065,8 @@ public final ConnectableObservable replay(final int bufferSize) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final ConnectableObservable replay(final int bufferSize, boolean eagerTruncate) { + @NonNull + public final ConnectableObservable replay(int bufferSize, boolean eagerTruncate) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return ObservableReplay.create(this, bufferSize, eagerTruncate); } @@ -10798,7 +11102,8 @@ public final ConnectableObservable replay(final int bufferSize, boolean eager */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final ConnectableObservable replay(int bufferSize, long time, TimeUnit unit) { + @NonNull + public final ConnectableObservable replay(int bufferSize, long time, @NonNull TimeUnit unit) { return replay(bufferSize, time, unit, Schedulers.computation()); } @@ -10837,7 +11142,8 @@ public final ConnectableObservable replay(int bufferSize, long time, TimeUnit */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableObservable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final ConnectableObservable replay(int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -10881,7 +11187,8 @@ public final ConnectableObservable replay(final int bufferSize, final long ti */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableObservable replay(final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + @NonNull + public final ConnectableObservable replay(int bufferSize, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -10910,7 +11217,8 @@ public final ConnectableObservable replay(final int bufferSize, final long ti */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final ConnectableObservable replay(long time, TimeUnit unit) { + @NonNull + public final ConnectableObservable replay(long time, @NonNull TimeUnit unit) { return replay(time, unit, Schedulers.computation()); } @@ -10942,7 +11250,8 @@ public final ConnectableObservable replay(long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableObservable replay(final long time, final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final ConnectableObservable replay(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.create(this, time, unit, scheduler, false); @@ -10978,7 +11287,8 @@ public final ConnectableObservable replay(final long time, final TimeUnit uni */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final ConnectableObservable replay(final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { + @NonNull + public final ConnectableObservable replay(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean eagerTruncate) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return ObservableReplay.create(this, time, unit, scheduler, eagerTruncate); @@ -11007,6 +11317,7 @@ public final ConnectableObservable replay(final long time, final TimeUnit uni */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable retry() { return retry(Long.MAX_VALUE, Functions.alwaysTrue()); } @@ -11030,7 +11341,8 @@ public final Observable retry() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable retry(BiPredicate predicate) { + @NonNull + public final Observable retry(@NonNull BiPredicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableRetryBiPredicate<>(this, predicate)); @@ -11062,6 +11374,7 @@ public final Observable retry(BiPredicate */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable retry(long times) { return retry(times, Functions.alwaysTrue()); } @@ -11080,7 +11393,8 @@ public final Observable retry(long times) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable retry(long times, Predicate predicate) { + @NonNull + public final Observable retry(long times, @NonNull Predicate predicate) { if (times < 0) { throw new IllegalArgumentException("times >= 0 required but it was " + times); } @@ -11103,7 +11417,8 @@ public final Observable retry(long times, Predicate predic */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable retry(Predicate predicate) { + @NonNull + public final Observable retry(@NonNull Predicate predicate) { return retry(Long.MAX_VALUE, predicate); } @@ -11120,7 +11435,8 @@ public final Observable retry(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable retryUntil(final BooleanSupplier stop) { + @NonNull + public final Observable retryUntil(@NonNull BooleanSupplier stop) { Objects.requireNonNull(stop, "stop is null"); return retry(Long.MAX_VALUE, Functions.predicateReverseFor(stop)); } @@ -11200,8 +11516,9 @@ public final Observable retryUntil(final BooleanSupplier stop) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable retryWhen( - final Function, ? extends ObservableSource> handler) { + @NonNull Function, ? extends ObservableSource> handler) { Objects.requireNonNull(handler, "handler is null"); return RxJavaPlugins.onAssembly(new ObservableRetryWhen<>(this, handler)); } @@ -11219,7 +11536,8 @@ public final Observable retryWhen( * @throws NullPointerException if s is null */ @SchedulerSupport(SchedulerSupport.NONE) - public final void safeSubscribe(Observer observer) { + @NonNull + public final void safeSubscribe(@NonNull Observer observer) { Objects.requireNonNull(observer, "observer is null"); if (observer instanceof SafeObserver) { subscribe(observer); @@ -11249,7 +11567,8 @@ public final void safeSubscribe(Observer observer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable sample(long period, TimeUnit unit) { + @NonNull + public final Observable sample(long period, @NonNull TimeUnit unit) { return sample(period, unit, Schedulers.computation()); } @@ -11280,7 +11599,8 @@ public final Observable sample(long period, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable sample(long period, TimeUnit unit, boolean emitLast) { + @NonNull + public final Observable sample(long period, @NonNull TimeUnit unit, boolean emitLast) { return sample(period, unit, Schedulers.computation(), emitLast); } @@ -11307,7 +11627,8 @@ public final Observable sample(long period, TimeUnit unit, boolean emitLast) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable sample(long period, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable sample(long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleTimed<>(this, period, unit, scheduler, false)); @@ -11343,7 +11664,8 @@ public final Observable sample(long period, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable sample(long period, TimeUnit unit, Scheduler scheduler, boolean emitLast) { + @NonNull + public final Observable sample(long period, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean emitLast) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleTimed<>(this, period, unit, scheduler, emitLast)); @@ -11369,7 +11691,8 @@ public final Observable sample(long period, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable sample(ObservableSource sampler) { + @NonNull + public final Observable sample(@NonNull ObservableSource sampler) { Objects.requireNonNull(sampler, "sampler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable<>(this, sampler, false)); } @@ -11401,7 +11724,8 @@ public final Observable sample(ObservableSource sampler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable sample(ObservableSource sampler, boolean emitLast) { + @NonNull + public final Observable sample(@NonNull ObservableSource sampler, boolean emitLast) { Objects.requireNonNull(sampler, "sampler is null"); return RxJavaPlugins.onAssembly(new ObservableSampleWithObservable<>(this, sampler, emitLast)); } @@ -11429,7 +11753,8 @@ public final Observable sample(ObservableSource sampler, boolean emitL */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable scan(BiFunction accumulator) { + @NonNull + public final Observable scan(@NonNull BiFunction accumulator) { Objects.requireNonNull(accumulator, "accumulator is null"); return RxJavaPlugins.onAssembly(new ObservableScan<>(this, accumulator)); } @@ -11478,7 +11803,8 @@ public final Observable scan(BiFunction accumulator) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable scan(final R initialValue, BiFunction accumulator) { + @NonNull + public final Observable scan(@NonNull R initialValue, @NonNull BiFunction accumulator) { Objects.requireNonNull(initialValue, "initialValue is null"); return scanWith(Functions.justSupplier(initialValue), accumulator); } @@ -11513,7 +11839,8 @@ public final Observable scan(final R initialValue, BiFunction Observable scanWith(Supplier seedSupplier, BiFunction accumulator) { + @NonNull + public final Observable scanWith(@NonNull Supplier seedSupplier, @NonNull BiFunction accumulator) { Objects.requireNonNull(seedSupplier, "seedSupplier is null"); Objects.requireNonNull(accumulator, "accumulator is null"); return RxJavaPlugins.onAssembly(new ObservableScanSeed<>(this, seedSupplier, accumulator)); @@ -11541,6 +11868,7 @@ public final Observable scanWith(Supplier seedSupplier, BiFunction serialize() { return RxJavaPlugins.onAssembly(new ObservableSerialized<>(this)); } @@ -11564,6 +11892,7 @@ public final Observable serialize() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable share() { return publish().refCount(); } @@ -11583,6 +11912,7 @@ public final Observable share() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Maybe singleElement() { return RxJavaPlugins.onAssembly(new ObservableSingleMaybe<>(this)); } @@ -11605,7 +11935,8 @@ public final Maybe singleElement() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single single(T defaultItem) { + @NonNull + public final Single single(@NonNull T defaultItem) { Objects.requireNonNull(defaultItem, "defaultItem is null"); return RxJavaPlugins.onAssembly(new ObservableSingleSingle<>(this, defaultItem)); } @@ -11627,6 +11958,7 @@ public final Single single(T defaultItem) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single singleOrError() { return RxJavaPlugins.onAssembly(new ObservableSingleSingle<>(this, null)); } @@ -11649,6 +11981,7 @@ public final Single singleOrError() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable skip(long count) { if (count <= 0) { return RxJavaPlugins.onAssembly(this); @@ -11677,7 +12010,8 @@ public final Observable skip(long count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable skip(long time, TimeUnit unit) { + @NonNull + public final Observable skip(long time, @NonNull TimeUnit unit) { return skipUntil(timer(time, unit)); } @@ -11703,7 +12037,8 @@ public final Observable skip(long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable skip(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable skip(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return skipUntil(timer(time, unit, scheduler)); } @@ -11731,6 +12066,7 @@ public final Observable skip(long time, TimeUnit unit, Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable skipLast(int count) { if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); @@ -11764,7 +12100,8 @@ public final Observable skipLast(int count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.TRAMPOLINE) - public final Observable skipLast(long time, TimeUnit unit) { + @NonNull + public final Observable skipLast(long time, @NonNull TimeUnit unit) { return skipLast(time, unit, Schedulers.trampoline(), false, bufferSize()); } @@ -11794,7 +12131,8 @@ public final Observable skipLast(long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.TRAMPOLINE) - public final Observable skipLast(long time, TimeUnit unit, boolean delayError) { + @NonNull + public final Observable skipLast(long time, @NonNull TimeUnit unit, boolean delayError) { return skipLast(time, unit, Schedulers.trampoline(), delayError, bufferSize()); } @@ -11822,7 +12160,8 @@ public final Observable skipLast(long time, TimeUnit unit, boolean delayError */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable skipLast(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable skipLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return skipLast(time, unit, scheduler, false, bufferSize()); } @@ -11853,7 +12192,8 @@ public final Observable skipLast(long time, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable skipLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError) { + @NonNull + public final Observable skipLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { return skipLast(time, unit, scheduler, delayError, bufferSize()); } @@ -11886,7 +12226,8 @@ public final Observable skipLast(long time, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable skipLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { + @NonNull + public final Observable skipLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError, int bufferSize) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -11915,7 +12256,8 @@ public final Observable skipLast(long time, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable skipUntil(ObservableSource other) { + @NonNull + public final Observable skipUntil(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableSkipUntil<>(this, other)); } @@ -11938,7 +12280,8 @@ public final Observable skipUntil(ObservableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable skipWhile(Predicate predicate) { + @NonNull + public final Observable skipWhile(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableSkipWhile<>(this, predicate)); } @@ -11965,6 +12308,7 @@ public final Observable skipWhile(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable sorted() { return toList().toObservable().map(Functions.listSorter(Functions.naturalComparator())).flatMapIterable(Functions.>identity()); } @@ -11988,7 +12332,8 @@ public final Observable sorted() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable sorted(Comparator sortFunction) { + @NonNull + public final Observable sorted(@NonNull Comparator sortFunction) { Objects.requireNonNull(sortFunction, "sortFunction is null"); return toList().toObservable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.>identity()); } @@ -12012,10 +12357,10 @@ public final Observable sorted(Comparator sortFunction) { * @see #startWithItem(Object) * @see #startWithArray(Object...) */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable startWithIterable(Iterable items) { + @NonNull + public final Observable startWithIterable(@NonNull Iterable items) { return concatArray(fromIterable(items), this); } @@ -12035,10 +12380,10 @@ public final Observable startWithIterable(Iterable items) { * emitted by the source ObservableSource * @see ReactiveX operators documentation: StartWith */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable startWith(ObservableSource other) { + @NonNull + public final Observable startWith(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return concatArray(other, this); } @@ -12062,10 +12407,10 @@ public final Observable startWith(ObservableSource other) { * @see #startWithIterable(Iterable) * @since 3.0.0 */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable startWithItem(T item) { + @NonNull + public final Observable startWithItem(@NonNull T item) { Objects.requireNonNull(item, "item is null"); return concatArray(just(item), this); } @@ -12088,10 +12433,11 @@ public final Observable startWithItem(T item) { * @see #startWithItem(Object) * @see #startWithIterable(Iterable) */ - @SuppressWarnings("unchecked") @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable startWithArray(T... items) { + @SafeVarargs + @NonNull + public final Observable startWithArray(@NonNull T... items) { Observable fromArray = fromArray(items); if (fromArray == empty()) { return RxJavaPlugins.onAssembly(this); @@ -12115,6 +12461,7 @@ public final Observable startWithArray(T... items) { * @see ReactiveX operators documentation: Subscribe */ @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Disposable subscribe() { return subscribe(Functions.emptyConsumer(), Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -12140,7 +12487,8 @@ public final Disposable subscribe() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(Consumer onNext) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onNext) { return subscribe(onNext, Functions.ON_ERROR_MISSING, Functions.EMPTY_ACTION); } @@ -12166,7 +12514,8 @@ public final Disposable subscribe(Consumer onNext) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Disposable subscribe(Consumer onNext, Consumer onError) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onNext, @NonNull Consumer onError) { return subscribe(onNext, onError, Functions.EMPTY_ACTION); } @@ -12196,8 +12545,9 @@ public final Disposable subscribe(Consumer onNext, Consumer onNext, Consumer onError, - Action onComplete) { + @NonNull + public final Disposable subscribe(@NonNull Consumer onNext, @NonNull Consumer onError, + @NonNull Action onComplete) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(onError, "onError is null"); Objects.requireNonNull(onComplete, "onComplete is null"); @@ -12211,7 +12561,7 @@ public final Disposable subscribe(Consumer onNext, Consumer observer) { + public final void subscribe(@NonNull Observer observer) { Objects.requireNonNull(observer, "observer is null"); try { observer = RxJavaPlugins.onSubscribe(this, observer); @@ -12241,7 +12591,7 @@ public final void subscribe(Observer observer) { * applied by {@link #subscribe(Observer)} before this method gets called. * @param observer the incoming Observer, never null */ - protected abstract void subscribeActual(Observer observer); + protected abstract void subscribeActual(@NonNull Observer observer); /** * Subscribes a given Observer (subclass) to this Observable and returns the given @@ -12269,7 +12619,8 @@ public final void subscribe(Observer observer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > E subscribeWith(E observer) { + @NonNull + public final <@NonNull E extends Observer> E subscribeWith(E observer) { subscribe(observer); return observer; } @@ -12293,7 +12644,8 @@ public final > E subscribeWith(E observer) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable subscribeOn(Scheduler scheduler) { + @NonNull + public final Observable subscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableSubscribeOn<>(this, scheduler)); } @@ -12316,7 +12668,8 @@ public final Observable subscribeOn(Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable switchIfEmpty(ObservableSource other) { + @NonNull + public final Observable switchIfEmpty(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchIfEmpty<>(this, other)); } @@ -12345,7 +12698,8 @@ public final Observable switchIfEmpty(ObservableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable switchMap(Function> mapper) { + @NonNull + public final Observable switchMap(@NonNull Function> mapper) { return switchMap(mapper, bufferSize()); } @@ -12375,7 +12729,8 @@ public final Observable switchMap(Function Observable switchMap(Function> mapper, int bufferSize) { + @NonNull + public final Observable switchMap(@NonNull Function> mapper, int bufferSize) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (this instanceof ScalarSupplier) { @@ -12425,6 +12780,7 @@ public final Observable switchMap(Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable<>(this, mapper, false)); @@ -12467,6 +12823,7 @@ public final Completable switchMapCompletable(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapCompletable<>(this, mapper, true)); @@ -12503,6 +12860,7 @@ public final Completable switchMapCompletableDelayError(@NonNull Function Observable switchMapMaybe(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe<>(this, mapper, false)); @@ -12529,6 +12887,7 @@ public final Observable switchMapMaybe(@NonNull Function Observable switchMapMaybeDelayError(@NonNull Function> mapper) { Objects.requireNonNull(mapper, "mapper is null"); return RxJavaPlugins.onAssembly(new ObservableSwitchMapMaybe<>(this, mapper, true)); @@ -12623,7 +12982,8 @@ public final Observable switchMapSingleDelayError(@NonNull Function Observable switchMapDelayError(Function> mapper) { + @NonNull + public final Observable switchMapDelayError(@NonNull Function> mapper) { return switchMapDelayError(mapper, bufferSize()); } @@ -12655,7 +13015,8 @@ public final Observable switchMapDelayError(Function Observable switchMapDelayError(Function> mapper, int bufferSize) { + @NonNull + public final Observable switchMapDelayError(@NonNull Function> mapper, int bufferSize) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); if (this instanceof ScalarSupplier) { @@ -12691,6 +13052,7 @@ public final Observable switchMapDelayError(Function take(long count) { if (count < 0) { throw new IllegalArgumentException("count >= 0 required but it was " + count); @@ -12720,7 +13082,8 @@ public final Observable take(long count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable take(long time, TimeUnit unit) { + @NonNull + public final Observable take(long time, @NonNull TimeUnit unit) { return takeUntil(timer(time, unit)); } @@ -12749,7 +13112,8 @@ public final Observable take(long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable take(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable take(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return takeUntil(timer(time, unit, scheduler)); } @@ -12773,6 +13137,7 @@ public final Observable take(long time, TimeUnit unit, Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable takeLast(int count) { if (count < 0) { throw new IndexOutOfBoundsException("count >= 0 required but it was " + count); @@ -12809,7 +13174,8 @@ public final Observable takeLast(int count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.TRAMPOLINE) - public final Observable takeLast(long count, long time, TimeUnit unit) { + @NonNull + public final Observable takeLast(long count, long time, @NonNull TimeUnit unit) { return takeLast(count, time, unit, Schedulers.trampoline(), false, bufferSize()); } @@ -12841,7 +13207,8 @@ public final Observable takeLast(long count, long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable takeLast(long count, long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable takeLast(long count, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return takeLast(count, time, unit, scheduler, false, bufferSize()); } @@ -12878,7 +13245,8 @@ public final Observable takeLast(long count, long time, TimeUnit unit, Schedu */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable takeLast(long count, long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { + @NonNull + public final Observable takeLast(long count, long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError, int bufferSize) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -12908,7 +13276,8 @@ public final Observable takeLast(long count, long time, TimeUnit unit, Schedu */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.TRAMPOLINE) - public final Observable takeLast(long time, TimeUnit unit) { + @NonNull + public final Observable takeLast(long time, @NonNull TimeUnit unit) { return takeLast(time, unit, Schedulers.trampoline(), false, bufferSize()); } @@ -12935,7 +13304,8 @@ public final Observable takeLast(long time, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.TRAMPOLINE) - public final Observable takeLast(long time, TimeUnit unit, boolean delayError) { + @NonNull + public final Observable takeLast(long time, @NonNull TimeUnit unit, boolean delayError) { return takeLast(time, unit, Schedulers.trampoline(), delayError, bufferSize()); } @@ -12963,7 +13333,8 @@ public final Observable takeLast(long time, TimeUnit unit, boolean delayError */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable takeLast(long time, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable takeLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return takeLast(time, unit, scheduler, false, bufferSize()); } @@ -12994,7 +13365,8 @@ public final Observable takeLast(long time, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable takeLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError) { + @NonNull + public final Observable takeLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError) { return takeLast(time, unit, scheduler, delayError, bufferSize()); } @@ -13027,7 +13399,8 @@ public final Observable takeLast(long time, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable takeLast(long time, TimeUnit unit, Scheduler scheduler, boolean delayError, int bufferSize) { + @NonNull + public final Observable takeLast(long time, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean delayError, int bufferSize) { return takeLast(Long.MAX_VALUE, time, unit, scheduler, delayError, bufferSize); } @@ -13051,7 +13424,8 @@ public final Observable takeLast(long time, TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable takeUntil(ObservableSource other) { + @NonNull + public final Observable takeUntil(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return RxJavaPlugins.onAssembly(new ObservableTakeUntil<>(this, other)); } @@ -13080,7 +13454,8 @@ public final Observable takeUntil(ObservableSource other) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable takeUntil(Predicate stopPredicate) { + @NonNull + public final Observable takeUntil(@NonNull Predicate stopPredicate) { Objects.requireNonNull(stopPredicate, "stopPredicate is null"); return RxJavaPlugins.onAssembly(new ObservableTakeUntilPredicate<>(this, stopPredicate)); } @@ -13104,7 +13479,8 @@ public final Observable takeUntil(Predicate stopPredicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable takeWhile(Predicate predicate) { + @NonNull + public final Observable takeWhile(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate is null"); return RxJavaPlugins.onAssembly(new ObservableTakeWhile<>(this, predicate)); } @@ -13131,7 +13507,8 @@ public final Observable takeWhile(Predicate predicate) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable throttleFirst(long windowDuration, TimeUnit unit) { + @NonNull + public final Observable throttleFirst(long windowDuration, @NonNull TimeUnit unit) { return throttleFirst(windowDuration, unit, Schedulers.computation()); } @@ -13160,7 +13537,8 @@ public final Observable throttleFirst(long windowDuration, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable throttleFirst(long skipDuration, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable throttleFirst(long skipDuration, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableThrottleFirstTimed<>(this, skipDuration, unit, scheduler)); @@ -13190,7 +13568,8 @@ public final Observable throttleFirst(long skipDuration, TimeUnit unit, Sched */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable throttleLast(long intervalDuration, TimeUnit unit) { + @NonNull + public final Observable throttleLast(long intervalDuration, @NonNull TimeUnit unit) { return sample(intervalDuration, unit); } @@ -13221,7 +13600,8 @@ public final Observable throttleLast(long intervalDuration, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable throttleLast(long intervalDuration, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable throttleLast(long intervalDuration, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return sample(intervalDuration, unit, scheduler); } @@ -13252,7 +13632,8 @@ public final Observable throttleLast(long intervalDuration, TimeUnit unit, Sc */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable throttleLatest(long timeout, TimeUnit unit) { + @NonNull + public final Observable throttleLatest(long timeout, @NonNull TimeUnit unit) { return throttleLatest(timeout, unit, Schedulers.computation(), false); } @@ -13283,7 +13664,8 @@ public final Observable throttleLatest(long timeout, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable throttleLatest(long timeout, TimeUnit unit, boolean emitLast) { + @NonNull + public final Observable throttleLatest(long timeout, @NonNull TimeUnit unit, boolean emitLast) { return throttleLatest(timeout, unit, Schedulers.computation(), emitLast); } @@ -13315,7 +13697,8 @@ public final Observable throttleLatest(long timeout, TimeUnit unit, boolean e */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable throttleLatest(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return throttleLatest(timeout, unit, scheduler, false); } @@ -13347,7 +13730,8 @@ public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler scheduler, boolean emitLast) { + @NonNull + public final Observable throttleLatest(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, boolean emitLast) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableThrottleLatest<>(this, timeout, unit, scheduler, emitLast)); @@ -13380,7 +13764,8 @@ public final Observable throttleLatest(long timeout, TimeUnit unit, Scheduler */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable throttleWithTimeout(long timeout, TimeUnit unit) { + @NonNull + public final Observable throttleWithTimeout(long timeout, @NonNull TimeUnit unit) { return debounce(timeout, unit); } @@ -13414,7 +13799,8 @@ public final Observable throttleWithTimeout(long timeout, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable throttleWithTimeout(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable throttleWithTimeout(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return debounce(timeout, unit, scheduler); } @@ -13434,6 +13820,7 @@ public final Observable throttleWithTimeout(long timeout, TimeUnit unit, Sche */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> timeInterval() { return timeInterval(TimeUnit.MILLISECONDS, Schedulers.computation()); } @@ -13456,7 +13843,8 @@ public final Observable> timeInterval() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Observable> timeInterval(Scheduler scheduler) { + @NonNull + public final Observable> timeInterval(@NonNull Scheduler scheduler) { return timeInterval(TimeUnit.MILLISECONDS, scheduler); } @@ -13477,7 +13865,8 @@ public final Observable> timeInterval(Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> timeInterval(TimeUnit unit) { + @NonNull + public final Observable> timeInterval(@NonNull TimeUnit unit) { return timeInterval(unit, Schedulers.computation()); } @@ -13500,7 +13889,8 @@ public final Observable> timeInterval(TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Observable> timeInterval(TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable> timeInterval(@NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableTimeInterval<>(this, unit, scheduler)); @@ -13532,7 +13922,8 @@ public final Observable> timeInterval(TimeUnit unit, Scheduler schedule */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable timeout(Function> itemTimeoutIndicator) { + @NonNull + public final Observable timeout(@NonNull Function> itemTimeoutIndicator) { return timeout0(null, itemTimeoutIndicator, null); } @@ -13564,8 +13955,9 @@ public final Observable timeout(Function Observable timeout(Function> itemTimeoutIndicator, - ObservableSource other) { + @NonNull + public final Observable timeout(@NonNull Function> itemTimeoutIndicator, + @NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return timeout0(null, itemTimeoutIndicator, other); } @@ -13591,7 +13983,8 @@ public final Observable timeout(Function timeout(long timeout, TimeUnit timeUnit) { + @NonNull + public final Observable timeout(long timeout, @NonNull TimeUnit timeUnit) { return timeout0(timeout, timeUnit, null, Schedulers.computation()); } @@ -13618,7 +14011,8 @@ public final Observable timeout(long timeout, TimeUnit timeUnit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable timeout(long timeout, TimeUnit timeUnit, ObservableSource other) { + @NonNull + public final Observable timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, Schedulers.computation()); } @@ -13649,7 +14043,8 @@ public final Observable timeout(long timeout, TimeUnit timeUnit, ObservableSo */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler, ObservableSource other) { + @NonNull + public final Observable timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull Scheduler scheduler, @NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); return timeout0(timeout, timeUnit, other, scheduler); } @@ -13678,7 +14073,8 @@ public final Observable timeout(long timeout, TimeUnit timeUnit, Scheduler sc */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable timeout(long timeout, TimeUnit timeUnit, Scheduler scheduler) { + @NonNull + public final Observable timeout(long timeout, @NonNull TimeUnit timeUnit, @NonNull Scheduler scheduler) { return timeout0(timeout, timeUnit, null, scheduler); } @@ -13711,8 +14107,9 @@ public final Observable timeout(long timeout, TimeUnit timeUnit, Scheduler sc */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable timeout(ObservableSource firstTimeoutIndicator, - Function> itemTimeoutIndicator) { + @NonNull + public final Observable timeout(@NonNull ObservableSource firstTimeoutIndicator, + @NonNull Function> itemTimeoutIndicator) { Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, null); } @@ -13751,26 +14148,30 @@ public final Observable timeout(ObservableSource firstTimeoutIndica */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable timeout( - ObservableSource firstTimeoutIndicator, - Function> itemTimeoutIndicator, - ObservableSource other) { + @NonNull ObservableSource firstTimeoutIndicator, + @NonNull Function> itemTimeoutIndicator, + @NonNull ObservableSource other) { Objects.requireNonNull(firstTimeoutIndicator, "firstTimeoutIndicator is null"); Objects.requireNonNull(other, "other is null"); return timeout0(firstTimeoutIndicator, itemTimeoutIndicator, other); } - private Observable timeout0(long timeout, TimeUnit timeUnit, ObservableSource other, - Scheduler scheduler) { + @NonNull + private Observable timeout0(long timeout, @NonNull TimeUnit timeUnit, + @Nullable ObservableSource other, + @NonNull Scheduler scheduler) { Objects.requireNonNull(timeUnit, "timeUnit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableTimeoutTimed<>(this, timeout, timeUnit, scheduler, other)); } + @NonNull private Observable timeout0( - ObservableSource firstTimeoutIndicator, - Function> itemTimeoutIndicator, - ObservableSource other) { + @NonNull ObservableSource firstTimeoutIndicator, + @NonNull Function> itemTimeoutIndicator, + @Nullable ObservableSource other) { Objects.requireNonNull(itemTimeoutIndicator, "itemTimeoutIndicator is null"); return RxJavaPlugins.onAssembly(new ObservableTimeout<>(this, firstTimeoutIndicator, itemTimeoutIndicator, other)); } @@ -13791,6 +14192,7 @@ private Observable timeout0( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> timestamp() { return timestamp(TimeUnit.MILLISECONDS, Schedulers.computation()); } @@ -13814,7 +14216,8 @@ public final Observable> timestamp() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Observable> timestamp(Scheduler scheduler) { + @NonNull + public final Observable> timestamp(@NonNull Scheduler scheduler) { return timestamp(TimeUnit.MILLISECONDS, scheduler); } @@ -13835,7 +14238,8 @@ public final Observable> timestamp(Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> timestamp(TimeUnit unit) { + @NonNull + public final Observable> timestamp(@NonNull TimeUnit unit) { return timestamp(unit, Schedulers.computation()); } @@ -13859,7 +14263,8 @@ public final Observable> timestamp(TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) // Supplied scheduler is only used for creating timestamps. - public final Observable> timestamp(final TimeUnit unit, final Scheduler scheduler) { + @NonNull + public final Observable> timestamp(@NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); return map(Functions.timestampWith(unit, scheduler)); @@ -13882,6 +14287,7 @@ public final Observable> timestamp(final TimeUnit unit, final Scheduler */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final R to(@NonNull ObservableConverter converter) { return Objects.requireNonNull(converter, "converter is null").apply(this); } @@ -13912,7 +14318,8 @@ public final R to(@NonNull ObservableConverter converter) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toList() { + @NonNull + public final Single<@NonNull List> toList() { return toList(16); } @@ -13944,7 +14351,8 @@ public final Single> toList() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toList(final int capacityHint) { + @NonNull + public final Single<@NonNull List> toList(int capacityHint) { ObjectHelper.verifyPositive(capacityHint, "capacityHint"); return RxJavaPlugins.onAssembly(new ObservableToListSingle>(this, capacityHint)); } @@ -13978,7 +14386,8 @@ public final Single> toList(final int capacityHint) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final > Single toList(Supplier collectionSupplier) { + @NonNull + public final <@NonNull U extends Collection> Single toList(@NonNull Supplier collectionSupplier) { Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); return RxJavaPlugins.onAssembly(new ObservableToListSingle<>(this, collectionSupplier)); } @@ -14009,7 +14418,8 @@ public final > Single toList(Supplier coll */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toMap(final Function keySelector) { + @NonNull + public final Single<@NonNull Map> toMap(@NonNull Function keySelector) { Objects.requireNonNull(keySelector, "keySelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); } @@ -14043,9 +14453,10 @@ public final Single> toMap(final Function */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> toMap( - final Function keySelector, - final Function valueSelector) { + @NonNull Function keySelector, + @NonNull Function valueSelector) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); @@ -14079,10 +14490,11 @@ public final Single> toMap( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Single> toMap( - final Function keySelector, - final Function valueSelector, - Supplier> mapSupplier) { + @NonNull Function keySelector, + @NonNull Function valueSelector, + @NonNull Supplier> mapSupplier) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); Objects.requireNonNull(mapSupplier, "mapSupplier is null"); @@ -14112,7 +14524,8 @@ public final Single> toMap( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single>> toMultimap(Function keySelector) { + @NonNull + public final Single<@NonNull Map>> toMultimap(@NonNull Function keySelector) { @SuppressWarnings({ "rawtypes", "unchecked" }) Function valueSelector = (Function)Functions.identity(); Supplier>> mapSupplier = HashMapSupplier.asSupplier(); @@ -14147,7 +14560,8 @@ public final Single>> toMultimap(Function Single>> toMultimap(Function keySelector, Function valueSelector) { + @NonNull + public final Single<@NonNull Map>> toMultimap(@NonNull Function keySelector, Function valueSelector) { Supplier>> mapSupplier = HashMapSupplier.asSupplier(); Function> collectionFactory = ArrayListSupplier.asFunction(); return toMultimap(keySelector, valueSelector, mapSupplier, collectionFactory); @@ -14180,11 +14594,12 @@ public final Single>> toMultimap(Function Single>> toMultimap( - final Function keySelector, - final Function valueSelector, - final Supplier>> mapSupplier, - final Function> collectionFactory) { + @NonNull + public final Single<@NonNull Map>> toMultimap( + @NonNull Function keySelector, + @NonNull Function valueSelector, + @NonNull Supplier>> mapSupplier, + @NonNull Function> collectionFactory) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); Objects.requireNonNull(mapSupplier, "mapSupplier is null"); @@ -14221,11 +14636,12 @@ public final Single>> toMultimap( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single>> toMultimap( - Function keySelector, - Function valueSelector, - Supplier>> mapSupplier - ) { + @NonNull + public final Single<@NonNull Map>> toMultimap( + @NonNull Function keySelector, + @NonNull Function valueSelector, + @NonNull Supplier>> mapSupplier + ) { return toMultimap(keySelector, valueSelector, mapSupplier, ArrayListSupplier.asFunction()); } @@ -14268,7 +14684,8 @@ public final Single>> toMultimap( @BackpressureSupport(BackpressureKind.SPECIAL) @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Flowable toFlowable(BackpressureStrategy strategy) { + @NonNull + public final Flowable toFlowable(@NonNull BackpressureStrategy strategy) { Flowable f = new FlowableFromObservable<>(this); switch (strategy) { @@ -14309,7 +14726,8 @@ public final Flowable toFlowable(BackpressureStrategy strategy) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toSortedList() { + @NonNull + public final Single<@NonNull List> toSortedList() { return toSortedList(Functions.naturalOrder()); } @@ -14336,7 +14754,8 @@ public final Single> toSortedList() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toSortedList(final Comparator comparator) { + @NonNull + public final Single<@NonNull List> toSortedList(@NonNull Comparator comparator) { Objects.requireNonNull(comparator, "comparator is null"); return toList().map(Functions.listSorter(comparator)); } @@ -14367,7 +14786,8 @@ public final Single> toSortedList(final Comparator comparator */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toSortedList(final Comparator comparator, int capacityHint) { + @NonNull + public final Single<@NonNull List> toSortedList(@NonNull Comparator comparator, int capacityHint) { Objects.requireNonNull(comparator, "comparator is null"); return toList(capacityHint).map(Functions.listSorter(comparator)); } @@ -14400,7 +14820,8 @@ public final Single> toSortedList(final Comparator comparator */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Single> toSortedList(int capacityHint) { + @NonNull + public final Single<@NonNull List> toSortedList(int capacityHint) { return toSortedList(Functions.naturalOrder(), capacityHint); } @@ -14422,7 +14843,8 @@ public final Single> toSortedList(int capacityHint) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable unsubscribeOn(Scheduler scheduler) { + @NonNull + public final Observable unsubscribeOn(@NonNull Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler is null"); return RxJavaPlugins.onAssembly(new ObservableUnsubscribeOn<>(this, scheduler)); } @@ -14448,6 +14870,7 @@ public final Observable unsubscribeOn(Scheduler scheduler) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> window(long count) { return window(count, count, bufferSize()); } @@ -14476,6 +14899,7 @@ public final Observable> window(long count) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> window(long count, long skip) { return window(count, skip, bufferSize()); } @@ -14506,6 +14930,7 @@ public final Observable> window(long count, long skip) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> window(long count, long skip, int bufferSize) { ObjectHelper.verifyPositive(count, "count"); ObjectHelper.verifyPositive(skip, "skip"); @@ -14542,7 +14967,8 @@ public final Observable> window(long count, long skip, int bufferS */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable> window(long timespan, long timeskip, TimeUnit unit) { + @NonNull + public final Observable> window(long timespan, long timeskip, @NonNull TimeUnit unit) { return window(timespan, timeskip, unit, Schedulers.computation(), bufferSize()); } @@ -14577,7 +15003,8 @@ public final Observable> window(long timespan, long timeskip, Time */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> window(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable> window(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return window(timespan, timeskip, unit, scheduler, bufferSize()); } @@ -14614,7 +15041,8 @@ public final Observable> window(long timespan, long timeskip, Time */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> window(long timespan, long timeskip, TimeUnit unit, Scheduler scheduler, int bufferSize) { + @NonNull + public final Observable> window(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int bufferSize) { ObjectHelper.verifyPositive(timespan, "timespan"); ObjectHelper.verifyPositive(timeskip, "timeskip"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -14651,7 +15079,8 @@ public final Observable> window(long timespan, long timeskip, Time */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable> window(long timespan, TimeUnit unit) { + @NonNull + public final Observable> window(long timespan, @NonNull TimeUnit unit) { return window(timespan, unit, Schedulers.computation(), Long.MAX_VALUE, false); } @@ -14687,7 +15116,8 @@ public final Observable> window(long timespan, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable> window(long timespan, TimeUnit unit, + @NonNull + public final Observable> window(long timespan, @NonNull TimeUnit unit, long count) { return window(timespan, unit, Schedulers.computation(), count, false); } @@ -14726,7 +15156,8 @@ public final Observable> window(long timespan, TimeUnit unit, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable> window(long timespan, TimeUnit unit, + @NonNull + public final Observable> window(long timespan, @NonNull TimeUnit unit, long count, boolean restart) { return window(timespan, unit, Schedulers.computation(), count, restart); } @@ -14761,8 +15192,9 @@ public final Observable> window(long timespan, TimeUnit unit, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> window(long timespan, TimeUnit unit, - Scheduler scheduler) { + @NonNull + public final Observable> window(long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler) { return window(timespan, unit, scheduler, Long.MAX_VALUE, false); } @@ -14800,8 +15232,9 @@ public final Observable> window(long timespan, TimeUnit unit, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> window(long timespan, TimeUnit unit, - Scheduler scheduler, long count) { + @NonNull + public final Observable> window(long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler, long count) { return window(timespan, unit, scheduler, count, false); } @@ -14841,8 +15274,9 @@ public final Observable> window(long timespan, TimeUnit unit, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable> window(long timespan, TimeUnit unit, - Scheduler scheduler, long count, boolean restart) { + @NonNull + public final Observable> window(long timespan, @NonNull TimeUnit unit, + @NonNull Scheduler scheduler, long count, boolean restart) { return window(timespan, unit, scheduler, count, restart, bufferSize()); } @@ -14884,8 +15318,9 @@ public final Observable> window(long timespan, TimeUnit unit, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) + @NonNull public final Observable> window( - long timespan, TimeUnit unit, Scheduler scheduler, + long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, long count, boolean restart, int bufferSize) { ObjectHelper.verifyPositive(bufferSize, "bufferSize"); Objects.requireNonNull(scheduler, "scheduler is null"); @@ -14921,7 +15356,8 @@ public final Observable> window( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> window(ObservableSource boundary) { + @NonNull + public final Observable> window(@NonNull ObservableSource boundary) { return window(boundary, bufferSize()); } @@ -14954,7 +15390,8 @@ public final Observable> window(ObservableSource boundary) */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable> window(ObservableSource boundary, int bufferSize) { + @NonNull + public final Observable> window(@NonNull ObservableSource boundary, int bufferSize) { Objects.requireNonNull(boundary, "boundary is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); return RxJavaPlugins.onAssembly(new ObservableWindowBoundary<>(this, boundary, bufferSize)); @@ -14990,9 +15427,10 @@ public final Observable> window(ObservableSource boundary, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> window( - ObservableSource openingIndicator, - Function> closingIndicator) { + @NonNull ObservableSource openingIndicator, + @NonNull Function> closingIndicator) { return window(openingIndicator, closingIndicator, bufferSize()); } @@ -15028,9 +15466,10 @@ public final Observable> window( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable> window( - ObservableSource openingIndicator, - Function> closingIndicator, int bufferSize) { + @NonNull ObservableSource openingIndicator, + @NonNull Function> closingIndicator, int bufferSize) { Objects.requireNonNull(openingIndicator, "openingIndicator is null"); Objects.requireNonNull(closingIndicator, "closingIndicator is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); @@ -15063,7 +15502,8 @@ public final Observable> window( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable withLatestFrom(ObservableSource other, BiFunction combiner) { + @NonNull + public final Observable withLatestFrom(@NonNull ObservableSource other, @NonNull BiFunction combiner) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(combiner, "combiner is null"); @@ -15088,22 +15528,23 @@ public final Observable withLatestFrom(ObservableSource o * @param the first other source's value type * @param the second other source's value type * @param the result value type - * @param o1 the first other ObservableSource - * @param o2 the second other ObservableSource + * @param source1 the first other ObservableSource + * @param source2 the second other ObservableSource * @param combiner the function called with an array of values from each participating ObservableSource * @return the new ObservableSource instance * @since 2.0 */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable withLatestFrom( - ObservableSource o1, ObservableSource o2, - Function3 combiner) { - Objects.requireNonNull(o1, "o1 is null"); - Objects.requireNonNull(o2, "o2 is null"); + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull Function3 combiner) { + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); - return withLatestFrom(new ObservableSource[] { o1, o2 }, f); + return withLatestFrom(new ObservableSource[] { source1, source2 }, f); } /** @@ -15125,25 +15566,26 @@ public final Observable withLatestFrom( * @param the second other source's value type * @param the third other source's value type * @param the result value type - * @param o1 the first other ObservableSource - * @param o2 the second other ObservableSource - * @param o3 the third other ObservableSource + * @param source1 the first other ObservableSource + * @param source2 the second other ObservableSource + * @param source3 the third other ObservableSource * @param combiner the function called with an array of values from each participating ObservableSource * @return the new ObservableSource instance * @since 2.0 */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable withLatestFrom( - ObservableSource o1, ObservableSource o2, - ObservableSource o3, - Function4 combiner) { - Objects.requireNonNull(o1, "o1 is null"); - Objects.requireNonNull(o2, "o2 is null"); - Objects.requireNonNull(o3, "o3 is null"); + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, + @NonNull Function4 combiner) { + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); - return withLatestFrom(new ObservableSource[] { o1, o2, o3 }, f); + return withLatestFrom(new ObservableSource[] { source1, source2, source3 }, f); } /** @@ -15166,27 +15608,28 @@ public final Observable withLatestFrom( * @param the third other source's value type * @param the fourth other source's value type * @param the result value type - * @param o1 the first other ObservableSource - * @param o2 the second other ObservableSource - * @param o3 the third other ObservableSource - * @param o4 the fourth other ObservableSource + * @param source1 the first other ObservableSource + * @param source2 the second other ObservableSource + * @param source3 the third other ObservableSource + * @param source4 the fourth other ObservableSource * @param combiner the function called with an array of values from each participating ObservableSource * @return the new ObservableSource instance * @since 2.0 */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable withLatestFrom( - ObservableSource o1, ObservableSource o2, - ObservableSource o3, ObservableSource o4, - Function5 combiner) { - Objects.requireNonNull(o1, "o1 is null"); - Objects.requireNonNull(o2, "o2 is null"); - Objects.requireNonNull(o3, "o3 is null"); - Objects.requireNonNull(o4, "o4 is null"); + @NonNull ObservableSource source1, @NonNull ObservableSource source2, + @NonNull ObservableSource source3, @NonNull ObservableSource source4, + @NonNull Function5 combiner) { + Objects.requireNonNull(source1, "source1 is null"); + Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(combiner, "combiner is null"); Function f = Functions.toFunction(combiner); - return withLatestFrom(new ObservableSource[] { o1, o2, o3, o4 }, f); + return withLatestFrom(new ObservableSource[] { source1, source2, source3, source4 }, f); } /** @@ -15212,7 +15655,8 @@ public final Observable withLatestFrom( */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable withLatestFrom(ObservableSource[] others, Function combiner) { + @NonNull + public final Observable withLatestFrom(@NonNull ObservableSource[] others, @NonNull Function combiner) { Objects.requireNonNull(others, "others is null"); Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany<>(this, others, combiner)); @@ -15241,7 +15685,8 @@ public final Observable withLatestFrom(ObservableSource[] others, Func */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable withLatestFrom(Iterable> others, Function combiner) { + @NonNull + public final Observable withLatestFrom(@NonNull Iterable> others, @NonNull Function combiner) { Objects.requireNonNull(others, "others is null"); Objects.requireNonNull(combiner, "combiner is null"); return RxJavaPlugins.onAssembly(new ObservableWithLatestFromMany<>(this, others, combiner)); @@ -15275,7 +15720,8 @@ public final Observable withLatestFrom(Iterable Observable zipWith(Iterable other, BiFunction zipper) { + @NonNull + public final Observable zipWith(@NonNull Iterable other, @NonNull BiFunction zipper) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(zipper, "zipper is null"); return RxJavaPlugins.onAssembly(new ObservableZipIterable(this, other, zipper)); @@ -15318,8 +15764,9 @@ public final Observable zipWith(Iterable other, BiFunction Observable zipWith(ObservableSource other, - BiFunction zipper) { + @NonNull + public final Observable zipWith(@NonNull ObservableSource other, + @NonNull BiFunction zipper) { Objects.requireNonNull(other, "other is null"); return zip(this, other, zipper); } @@ -15364,8 +15811,9 @@ public final Observable zipWith(ObservableSource other, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable zipWith(ObservableSource other, - BiFunction zipper, boolean delayError) { + @NonNull + public final Observable zipWith(@NonNull ObservableSource other, + @NonNull BiFunction zipper, boolean delayError) { return zip(this, other, zipper, delayError); } @@ -15411,8 +15859,9 @@ public final Observable zipWith(ObservableSource other, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) - public final Observable zipWith(ObservableSource other, - BiFunction zipper, boolean delayError, int bufferSize) { + @NonNull + public final Observable zipWith(@NonNull ObservableSource other, + @NonNull BiFunction zipper, boolean delayError, int bufferSize) { return zip(this, other, zipper, delayError, bufferSize); } @@ -15431,6 +15880,7 @@ public final Observable zipWith(ObservableSource other, */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestObserver test() { // NoPMD TestObserver to = new TestObserver<>(); subscribe(to); @@ -15451,6 +15901,7 @@ public final TestObserver test() { // NoPMD */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final TestObserver test(boolean dispose) { // NoPMD TestObserver to = new TestObserver<>(); if (dispose) { diff --git a/src/main/java/io/reactivex/rxjava3/core/Single.java b/src/main/java/io/reactivex/rxjava3/core/Single.java index ce7ef85608..8bd0f1db4e 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Single.java +++ b/src/main/java/io/reactivex/rxjava3/core/Single.java @@ -927,7 +927,7 @@ public static Single fromObservable(@NonNull ObservableSource Flowable merge(Iterable> sources) { + public static Flowable merge(@NonNull Iterable> sources) { return merge(Flowable.fromIterable(sources)); } @@ -1615,6 +1615,7 @@ public static Single zip( ) { Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2); } @@ -1655,6 +1656,7 @@ public static Single zip( Objects.requireNonNull(source1, "source1 is null"); Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3); } @@ -1699,6 +1701,7 @@ public static Single zip( Objects.requireNonNull(source2, "source2 is null"); Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4); } @@ -1748,6 +1751,7 @@ public static Single zip( Objects.requireNonNull(source3, "source3 is null"); Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5); } @@ -1801,6 +1805,7 @@ public static Single zip( Objects.requireNonNull(source4, "source4 is null"); Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6); } @@ -1859,6 +1864,7 @@ public static Single zip( Objects.requireNonNull(source5, "source5 is null"); Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7); } @@ -1921,6 +1927,7 @@ public static Single zip( Objects.requireNonNull(source6, "source6 is null"); Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8); } @@ -1988,6 +1995,7 @@ public static Single zip( Objects.requireNonNull(source7, "source7 is null"); Objects.requireNonNull(source8, "source8 is null"); Objects.requireNonNull(source9, "source9 is null"); + Objects.requireNonNull(zipper, "zipper is null"); return zipArray(Functions.toFunction(zipper), source1, source2, source3, source4, source5, source6, source7, source8, source9); } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java index 3a75ac5f3e..eaf067c177 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/CompositeDisposable.java @@ -41,7 +41,7 @@ public CompositeDisposable() { */ public CompositeDisposable(@NonNull Disposable... disposables) { Objects.requireNonNull(disposables, "disposables is null"); - this.resources = new OpenHashSet(disposables.length + 1); + this.resources = new OpenHashSet<>(disposables.length + 1); for (Disposable d : disposables) { Objects.requireNonNull(d, "A Disposable in the disposables array is null"); this.resources.add(d); @@ -55,7 +55,7 @@ public CompositeDisposable(@NonNull Disposable... disposables) { */ public CompositeDisposable(@NonNull Iterable disposables) { Objects.requireNonNull(disposables, "disposables is null"); - this.resources = new OpenHashSet(); + this.resources = new OpenHashSet<>(); for (Disposable d : disposables) { Objects.requireNonNull(d, "A Disposable item in the disposables sequence is null"); this.resources.add(d); @@ -100,7 +100,7 @@ public boolean add(@NonNull Disposable disposable) { if (!disposed) { OpenHashSet set = resources; if (set == null) { - set = new OpenHashSet(); + set = new OpenHashSet<>(); resources = set; } set.add(disposable); @@ -126,7 +126,7 @@ public boolean addAll(@NonNull Disposable... disposables) { if (!disposed) { OpenHashSet set = resources; if (set == null) { - set = new OpenHashSet(disposables.length + 1); + set = new OpenHashSet<>(disposables.length + 1); resources = set; } for (Disposable d : disposables) { @@ -239,7 +239,7 @@ void dispose(OpenHashSet set) { } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (errors == null) { - errors = new ArrayList(); + errors = new ArrayList<>(); } errors.add(ex); } diff --git a/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java index d2c7ddee4b..1f408f8c94 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/SerialDisposable.java @@ -30,7 +30,7 @@ public final class SerialDisposable implements Disposable { * Constructs an empty SerialDisposable. */ public SerialDisposable() { - this.resource = new AtomicReference(); + this.resource = new AtomicReference<>(); } /** @@ -38,7 +38,7 @@ public SerialDisposable() { * @param initialDisposable the initial Disposable instance to use, null allowed */ public SerialDisposable(@Nullable Disposable initialDisposable) { - this.resource = new AtomicReference(initialDisposable); + this.resource = new AtomicReference<>(initialDisposable); } /** diff --git a/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java b/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java index aa08128917..85b06bd313 100644 --- a/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java +++ b/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java @@ -62,8 +62,8 @@ public CompositeException(@NonNull Throwable... exceptions) { * @throws IllegalArgumentException if errors is empty. */ public CompositeException(@NonNull Iterable errors) { - Set deDupedExceptions = new LinkedHashSet(); - List localExceptions = new ArrayList(); + Set deDupedExceptions = new LinkedHashSet<>(); + List localExceptions = new ArrayList<>(); if (errors != null) { for (Throwable ex : errors) { if (ex instanceof CompositeException) { @@ -108,7 +108,7 @@ public synchronized Throwable getCause() { // NOPMD if (cause == null) { String separator = System.getProperty("line.separator"); if (exceptions.size() > 1) { - Map seenCauses = new IdentityHashMap(); + Map seenCauses = new IdentityHashMap<>(); StringBuilder aggregateMessage = new StringBuilder(); aggregateMessage.append("Multiple exceptions (").append(exceptions.size()).append(")").append(separator); diff --git a/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java b/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java index 6916d219bf..3c51f1c3bc 100644 --- a/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/flowables/ConnectableFlowable.java @@ -59,20 +59,30 @@ public abstract class ConnectableFlowable extends Flowable { /** * Instructs the {@code ConnectableFlowable} to begin emitting the items from its underlying * {@link Flowable} to its {@link Subscriber}s. + *
+ *
Scheduler:
+ *
The behavior is determined by the implementor of this abstract class.
+ *
* * @param connection * the action that receives the connection subscription before the subscription to source happens * allowing the caller to synchronously disconnect a synchronous source * @see ReactiveX documentation: Connect */ + @SchedulerSupport(SchedulerSupport.NONE) public abstract void connect(@NonNull Consumer connection); /** * Resets this ConnectableFlowable into its fresh state if it has terminated. *

* Calling this method on a fresh or active {@code ConnectableFlowable} has no effect. + *

+ *
Scheduler:
+ *
The behavior is determined by the implementor of this abstract class.
+ *
* @since 3.0.0 */ + @SchedulerSupport(SchedulerSupport.NONE) public abstract void reset(); /** @@ -80,10 +90,16 @@ public abstract class ConnectableFlowable extends Flowable { * {@link Flowable} to its {@link Subscriber}s. *

* To disconnect from a synchronous source, use the {@link #connect(io.reactivex.rxjava3.functions.Consumer)} method. + *

+ *
Scheduler:
+ *
The behavior is determined by the implementor of this abstract class.
+ *
* * @return the subscription representing the connection * @see ReactiveX documentation: Connect */ + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) public final Disposable connect() { ConnectConsumer cc = new ConnectConsumer(); connect(cc); @@ -111,7 +127,7 @@ public final Disposable connect() { @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.PASS_THROUGH) public Flowable refCount() { - return RxJavaPlugins.onAssembly(new FlowableRefCount(this)); + return RxJavaPlugins.onAssembly(new FlowableRefCount<>(this)); } /** @@ -132,6 +148,7 @@ public Flowable refCount() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @NonNull public final Flowable refCount(int subscriberCount) { return refCount(subscriberCount, 0, TimeUnit.NANOSECONDS, Schedulers.trampoline()); } @@ -157,7 +174,8 @@ public final Flowable refCount(int subscriberCount) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) @BackpressureSupport(BackpressureKind.PASS_THROUGH) - public final Flowable refCount(long timeout, TimeUnit unit) { + @NonNull + public final Flowable refCount(long timeout, @NonNull TimeUnit unit) { return refCount(1, timeout, unit, Schedulers.computation()); } @@ -182,7 +200,8 @@ public final Flowable refCount(long timeout, TimeUnit unit) { @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) @BackpressureSupport(BackpressureKind.PASS_THROUGH) - public final Flowable refCount(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable refCount(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return refCount(1, timeout, unit, scheduler); } @@ -208,7 +227,8 @@ public final Flowable refCount(long timeout, TimeUnit unit, Scheduler schedul @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) @BackpressureSupport(BackpressureKind.PASS_THROUGH) - public final Flowable refCount(int subscriberCount, long timeout, TimeUnit unit) { + @NonNull + public final Flowable refCount(int subscriberCount, long timeout, @NonNull TimeUnit unit) { return refCount(subscriberCount, timeout, unit, Schedulers.computation()); } @@ -234,11 +254,12 @@ public final Flowable refCount(int subscriberCount, long timeout, TimeUnit un @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) @BackpressureSupport(BackpressureKind.PASS_THROUGH) - public final Flowable refCount(int subscriberCount, long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Flowable refCount(int subscriberCount, long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { ObjectHelper.verifyPositive(subscriberCount, "subscriberCount"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new FlowableRefCount(this, subscriberCount, timeout, unit, scheduler)); + return RxJavaPlugins.onAssembly(new FlowableRefCount<>(this, subscriberCount, timeout, unit, scheduler)); } /** @@ -256,6 +277,13 @@ public final Flowable refCount(int subscriberCount, long timeout, TimeUnit un * This overload does not allow disconnecting the connection established via * {@link #connect(Consumer)}. Use the {@link #autoConnect(int, Consumer)} overload * to gain access to the {@code Disposable} representing the only connection. + *
+ *
Backpressure:
+ *
The operator itself doesn't interfere with backpressure which is determined by + * the upstream {@code ConnectableFlowable}'s behavior.
+ *
Scheduler:
+ *
{@code autoConnect} does not operate by default on a particular {@link Scheduler}.
+ *
* * @return a Flowable that automatically connects to this ConnectableFlowable * when the first Subscriber subscribes @@ -263,6 +291,9 @@ public final Flowable refCount(int subscriberCount, long timeout, TimeUnit un * @see #autoConnect(int, Consumer) */ @NonNull + @CheckReturnValue + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public Flowable autoConnect() { return autoConnect(1); } @@ -281,6 +312,13 @@ public Flowable autoConnect() { * This overload does not allow disconnecting the connection established via * {@link #connect(Consumer)}. Use the {@link #autoConnect(int, Consumer)} overload * to gain access to the {@code Disposable} representing the only connection. + *
+ *
Backpressure:
+ *
The operator itself doesn't interfere with backpressure which is determined by + * the upstream {@code ConnectableFlowable}'s behavior.
+ *
Scheduler:
+ *
{@code autoConnect} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param numberOfSubscribers the number of subscribers to await before calling connect * on the ConnectableFlowable. A non-positive value indicates @@ -289,6 +327,9 @@ public Flowable autoConnect() { * when the specified number of Subscribers subscribe to it */ @NonNull + @CheckReturnValue + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public Flowable autoConnect(int numberOfSubscribers) { return autoConnect(numberOfSubscribers, Functions.emptyConsumer()); } @@ -305,6 +346,13 @@ public Flowable autoConnect(int numberOfSubscribers) { * terminates, the connection is never renewed, no matter how Subscribers come * and go. Use {@link #refCount()} to renew a connection or dispose an active * connection when all {@code Subscriber}s have cancelled their {@code Subscription}s. + *
+ *
Backpressure:
+ *
The operator itself doesn't interfere with backpressure which is determined by + * the upstream {@code ConnectableFlowable}'s behavior.
+ *
Scheduler:
+ *
{@code autoConnect} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param numberOfSubscribers the number of subscribers to await before calling connect * on the ConnectableFlowable. A non-positive value indicates @@ -316,11 +364,14 @@ public Flowable autoConnect(int numberOfSubscribers) { * specified callback with the Subscription associated with the established connection */ @NonNull + @CheckReturnValue + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public Flowable autoConnect(int numberOfSubscribers, @NonNull Consumer connection) { if (numberOfSubscribers <= 0) { this.connect(connection); return RxJavaPlugins.onAssembly(this); } - return RxJavaPlugins.onAssembly(new FlowableAutoConnect(this, numberOfSubscribers, connection)); + return RxJavaPlugins.onAssembly(new FlowableAutoConnect<>(this, numberOfSubscribers, connection)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java index 2dcbbdea50..05d1997893 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java +++ b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java @@ -17,6 +17,7 @@ import org.reactivestreams.Subscription; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.exceptions.OnErrorNotImplementedException; import io.reactivex.rxjava3.functions.*; @@ -33,47 +34,47 @@ private Functions() { throw new IllegalStateException("No instances!"); } - public static Function toFunction(final BiFunction f) { - Objects.requireNonNull(f, "f is null"); + @NonNull + public static Function toFunction(@NonNull BiFunction f) { return new Array2Func(f); } - public static Function toFunction(final Function3 f) { - Objects.requireNonNull(f, "f is null"); + @NonNull + public static Function toFunction(@NonNull Function3 f) { return new Array3Func(f); } - public static Function toFunction(final Function4 f) { - Objects.requireNonNull(f, "f is null"); + @NonNull + public static Function toFunction(@NonNull Function4 f) { return new Array4Func(f); } - public static Function toFunction(final Function5 f) { - Objects.requireNonNull(f, "f is null"); + @NonNull + public static Function toFunction(@NonNull Function5 f) { return new Array5Func(f); } + @NonNull public static Function toFunction( - final Function6 f) { - Objects.requireNonNull(f, "f is null"); + @NonNull Function6 f) { return new Array6Func(f); } + @NonNull public static Function toFunction( - final Function7 f) { - Objects.requireNonNull(f, "f is null"); + @NonNull Function7 f) { return new Array7Func(f); } + @NonNull public static Function toFunction( - final Function8 f) { - Objects.requireNonNull(f, "f is null"); + @NonNull Function8 f) { return new Array8Func(f); } + @NonNull public static Function toFunction( - final Function9 f) { - Objects.requireNonNull(f, "f is null"); + @NonNull Function9 f) { return new Array9Func(f); } @@ -86,6 +87,7 @@ public static Function toFu * @return the identity function */ @SuppressWarnings("unchecked") + @NonNull public static Function identity() { return (Function)IDENTITY; } @@ -125,16 +127,19 @@ public static Consumer emptyConsumer() { static final Comparator NATURAL_COMPARATOR = new NaturalObjectComparator(); @SuppressWarnings("unchecked") + @NonNull public static Predicate alwaysTrue() { return (Predicate)ALWAYS_TRUE; } @SuppressWarnings("unchecked") + @NonNull public static Predicate alwaysFalse() { return (Predicate)ALWAYS_FALSE; } @SuppressWarnings("unchecked") + @NonNull public static Supplier nullSupplier() { return (Supplier)NULL_SUPPLIER; } @@ -145,6 +150,7 @@ public static Supplier nullSupplier() { * @return a natural order comparator which casts the parameters to Comparable */ @SuppressWarnings("unchecked") + @NonNull public static Comparator naturalOrder() { return (Comparator)NATURAL_COMPARATOR; } @@ -167,7 +173,8 @@ public void run() throws Exception { * @param future the future to call get() on, not null * @return the new Action instance */ - public static Action futureAction(Future future) { + @NonNull + public static Action futureAction(@NonNull Future future) { return new FutureAction(future); } @@ -200,7 +207,8 @@ public U get() throws Throwable { * @param value the value to return * @return the new Callable instance */ - public static Callable justCallable(T value) { + @NonNull + public static Callable justCallable(@NonNull T value) { return new JustValue(value); } @@ -210,7 +218,8 @@ public static Callable justCallable(T value) { * @param value the value to return * @return the new Callable instance */ - public static Supplier justSupplier(T value) { + @NonNull + public static Supplier justSupplier(@NonNull T value) { return new JustValue(value); } @@ -221,7 +230,8 @@ public static Supplier justSupplier(T value) { * @param value the value to return * @return the new Function instance */ - public static Function justFunction(U value) { + @NonNull + public static Function justFunction(@NonNull U value) { return new JustValue(value); } @@ -245,7 +255,8 @@ public U apply(T t) throws Exception { * @param target the target class * @return the new Function instance */ - public static Function castFunction(Class target) { + @NonNull + public static Function castFunction(@NonNull Class target) { return new CastToClass(target); } diff --git a/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java b/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java index 1a9b942011..192743187e 100644 --- a/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java +++ b/src/main/java/io/reactivex/rxjava3/observables/ConnectableObservable.java @@ -56,12 +56,17 @@ public abstract class ConnectableObservable extends Observable { /** * Instructs the {@code ConnectableObservable} to begin emitting the items from its underlying * {@link Observable} to its {@link Observer}s. + *
+ *
Scheduler:
+ *
The behavior is determined by the implementor of this abstract class.
+ *
* * @param connection * the action that receives the connection subscription before the subscription to source happens * allowing the caller to synchronously disconnect a synchronous source * @see ReactiveX documentation: Connect */ + @SchedulerSupport(SchedulerSupport.NONE) public abstract void connect(@NonNull Consumer connection); /** @@ -69,8 +74,13 @@ public abstract class ConnectableObservable extends Observable { * or has been disposed. *

* Calling this method on a fresh or active {@code ConnectableObservable} has no effect. + *

+ *
Scheduler:
+ *
The behavior is determined by the implementor of this abstract class.
+ *
* @since 3.0.0 */ + @SchedulerSupport(SchedulerSupport.NONE) public abstract void reset(); /** @@ -78,10 +88,16 @@ public abstract class ConnectableObservable extends Observable { * {@link Observable} to its {@link Observer}s. *

* To disconnect from a synchronous source, use the {@link #connect(Consumer)} method. + *

+ *
Scheduler:
+ *
The behavior is determined by the implementor of this abstract class.
+ *
* * @return the subscription representing the connection * @see ReactiveX documentation: Connect */ + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) public final Disposable connect() { ConnectConsumer cc = new ConnectConsumer(); connect(cc); @@ -105,7 +121,7 @@ public final Disposable connect() { @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) public Observable refCount() { - return RxJavaPlugins.onAssembly(new ObservableRefCount(this)); + return RxJavaPlugins.onAssembly(new ObservableRefCount<>(this)); } /** @@ -122,6 +138,7 @@ public Observable refCount() { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) + @NonNull public final Observable refCount(int subscriberCount) { return refCount(subscriberCount, 0, TimeUnit.NANOSECONDS, Schedulers.trampoline()); } @@ -143,7 +160,8 @@ public final Observable refCount(int subscriberCount) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable refCount(long timeout, TimeUnit unit) { + @NonNull + public final Observable refCount(long timeout, @NonNull TimeUnit unit) { return refCount(1, timeout, unit, Schedulers.computation()); } @@ -164,7 +182,8 @@ public final Observable refCount(long timeout, TimeUnit unit) { */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable refCount(long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable refCount(long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { return refCount(1, timeout, unit, scheduler); } @@ -186,7 +205,8 @@ public final Observable refCount(long timeout, TimeUnit unit, Scheduler sched */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.COMPUTATION) - public final Observable refCount(int subscriberCount, long timeout, TimeUnit unit) { + @NonNull + public final Observable refCount(int subscriberCount, long timeout, @NonNull TimeUnit unit) { return refCount(subscriberCount, timeout, unit, Schedulers.computation()); } @@ -208,11 +228,12 @@ public final Observable refCount(int subscriberCount, long timeout, TimeUnit */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.CUSTOM) - public final Observable refCount(int subscriberCount, long timeout, TimeUnit unit, Scheduler scheduler) { + @NonNull + public final Observable refCount(int subscriberCount, long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { ObjectHelper.verifyPositive(subscriberCount, "subscriberCount"); Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return RxJavaPlugins.onAssembly(new ObservableRefCount(this, subscriberCount, timeout, unit, scheduler)); + return RxJavaPlugins.onAssembly(new ObservableRefCount<>(this, subscriberCount, timeout, unit, scheduler)); } /** @@ -230,11 +251,17 @@ public final Observable refCount(int subscriberCount, long timeout, TimeUnit * This overload does not allow disconnecting the connection established via * {@link #connect(Consumer)}. Use the {@link #autoConnect(int, Consumer)} overload * to gain access to the {@code Disposable} representing the only connection. + *
+ *
Scheduler:
+ *
{@code autoConnect} overload does not operate on any particular {@link Scheduler}.
+ *
* * @return an Observable that automatically connects to this ConnectableObservable * when the first Observer subscribes */ @NonNull + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) public Observable autoConnect() { return autoConnect(1); } @@ -254,6 +281,10 @@ public Observable autoConnect() { * This overload does not allow disconnecting the connection established via * {@link #connect(Consumer)}. Use the {@link #autoConnect(int, Consumer)} overload * to gain access to the {@code Disposable} representing the only connection. + *
+ *
Scheduler:
+ *
{@code autoConnect} overload does not operate on any particular {@link Scheduler}.
+ *
* * @param numberOfSubscribers the number of subscribers to await before calling connect * on the ConnectableObservable. A non-positive value indicates @@ -262,6 +293,8 @@ public Observable autoConnect() { * when the specified number of Subscribers subscribe to it */ @NonNull + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) public Observable autoConnect(int numberOfSubscribers) { return autoConnect(numberOfSubscribers, Functions.emptyConsumer()); } @@ -278,6 +311,10 @@ public Observable autoConnect(int numberOfSubscribers) { * terminates, the connection is never renewed, no matter how Observers come * and go. Use {@link #refCount()} to renew a connection or dispose an active * connection when all {@code Observer}s have disposed their {@code Disposable}s. + *
+ *
Scheduler:
+ *
{@code autoConnect} overload does not operate on any particular {@link Scheduler}.
+ *
* * @param numberOfSubscribers the number of subscribers to await before calling connect * on the ConnectableObservable. A non-positive value indicates @@ -289,11 +326,13 @@ public Observable autoConnect(int numberOfSubscribers) { * specified callback with the Subscription associated with the established connection */ @NonNull + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) public Observable autoConnect(int numberOfSubscribers, @NonNull Consumer connection) { if (numberOfSubscribers <= 0) { this.connect(connection); return RxJavaPlugins.onAssembly(this); } - return RxJavaPlugins.onAssembly(new ObservableAutoConnect(this, numberOfSubscribers, connection)); + return RxJavaPlugins.onAssembly(new ObservableAutoConnect<>(this, numberOfSubscribers, connection)); } } diff --git a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java index afa2eddcf0..d4dff1cbb0 100644 --- a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java @@ -37,22 +37,31 @@ * @param the value type * @since 2.2 */ -public abstract class ParallelFlowable { +public abstract class ParallelFlowable<@NonNull T> { /** * Subscribes an array of Subscribers to this ParallelFlowable and triggers * the execution chain for all 'rails'. + *
+ *
Backpressure:
+ *
The backpressure behavior/expectation is determined by the supplied {@code Subscriber}.
+ *
Scheduler:
+ *
{@code subscribe} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param subscribers the subscribers array to run in parallel, the number * of items must be equal to the parallelism level of this ParallelFlowable * @see #parallelism() */ + @BackpressureSupport(BackpressureKind.SPECIAL) + @SchedulerSupport(SchedulerSupport.NONE) public abstract void subscribe(@NonNull Subscriber[] subscribers); /** * Returns the number of expected parallel Subscribers. * @return the number of expected parallel Subscribers */ + @CheckReturnValue public abstract int parallelism(); /** @@ -77,24 +86,46 @@ protected final boolean validate(@NonNull Subscriber[] subscribers) { /** * Take a Publisher and prepare to consume it on multiple 'rails' (number of CPUs) * in a round-robin fashion. + *
+ *
Backpressure:
+ *
The operator honors the backpressure of the parallel rails and + * requests {@link Flowable#bufferSize} amount from the upstream, followed + * by 75% of that amount requested after every 75% received.
+ *
Scheduler:
+ *
{@code from} does not operate by default on a particular {@link Scheduler}.
+ *
* @param the value type * @param source the source Publisher * @return the ParallelFlowable instance */ @CheckReturnValue + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.FULL) public static ParallelFlowable from(@NonNull Publisher source) { return from(source, Runtime.getRuntime().availableProcessors(), Flowable.bufferSize()); } /** * Take a Publisher and prepare to consume it on parallelism number of 'rails' in a round-robin fashion. + *
+ *
Backpressure:
+ *
The operator honors the backpressure of the parallel rails and + * requests {@link Flowable#bufferSize} amount from the upstream, followed + * by 75% of that amount requested after every 75% received.
+ *
Scheduler:
+ *
{@code from} does not operate by default on a particular {@link Scheduler}.
+ *
* @param the value type * @param source the source Publisher * @param parallelism the number of parallel rails * @return the new ParallelFlowable instance */ @CheckReturnValue - public static ParallelFlowable from(@NonNull Publisher source, int parallelism) { + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.FULL) + public static <@NonNull T> ParallelFlowable from(@NonNull Publisher source, int parallelism) { return from(source, parallelism, Flowable.bufferSize()); } @@ -102,6 +133,14 @@ public static ParallelFlowable from(@NonNull Publisher sourc * Take a Publisher and prepare to consume it on parallelism number of 'rails' , * possibly ordered and round-robin fashion and use custom prefetch amount and queue * for dealing with the source Publisher's values. + *
+ *
Backpressure:
+ *
The operator honors the backpressure of the parallel rails and + * requests the {@code prefetch} amount from the upstream, followed + * by 75% of that amount requested after every 75% received.
+ *
Scheduler:
+ *
{@code from} does not operate by default on a particular {@link Scheduler}.
+ *
* @param the value type * @param source the source Publisher * @param parallelism the number of parallel rails @@ -111,7 +150,9 @@ public static ParallelFlowable from(@NonNull Publisher sourc */ @CheckReturnValue @NonNull - public static ParallelFlowable from(@NonNull Publisher source, + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.FULL) + public static <@NonNull T> ParallelFlowable from(@NonNull Publisher source, int parallelism, int prefetch) { Objects.requireNonNull(source, "source"); ObjectHelper.verifyPositive(parallelism, "parallelism"); @@ -124,12 +165,21 @@ public static ParallelFlowable from(@NonNull Publisher sourc * Maps the source values on each 'rail' to another value. *

* Note that the same mapper function may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* @param the output value type * @param mapper the mapper function turning Ts into Us. * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.PASS_THROUGH) public final ParallelFlowable map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper"); return RxJavaPlugins.onAssembly(new ParallelMap(this, mapper)); @@ -140,6 +190,13 @@ public final ParallelFlowable map(@NonNull Function * Note that the same mapper function may be called from multiple threads concurrently. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
*

History: 2.0.8 - experimental * @param the output value type * @param mapper the mapper function turning Ts into Us. @@ -150,6 +207,8 @@ public final ParallelFlowable map(@NonNull Function ParallelFlowable map(@NonNull Function mapper, @NonNull ParallelFailureHandling errorHandler) { Objects.requireNonNull(mapper, "mapper"); Objects.requireNonNull(errorHandler, "errorHandler is null"); @@ -161,6 +220,13 @@ public final ParallelFlowable map(@NonNull Function * Note that the same mapper function may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
*

History: 2.0.8 - experimental * @param the output value type * @param mapper the mapper function turning Ts into Us. @@ -172,6 +238,8 @@ public final ParallelFlowable map(@NonNull Function ParallelFlowable map(@NonNull Function mapper, @NonNull BiFunction errorHandler) { Objects.requireNonNull(mapper, "mapper"); Objects.requireNonNull(errorHandler, "errorHandler is null"); @@ -182,13 +250,23 @@ public final ParallelFlowable map(@NonNull Function * Note that the same predicate may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code filter} does not operate by default on a particular {@link Scheduler}.
+ *
* @param predicate the function returning true to keep a value or false to drop a value * @return the new ParallelFlowable instance */ @CheckReturnValue + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.PASS_THROUGH) public final ParallelFlowable filter(@NonNull Predicate predicate) { Objects.requireNonNull(predicate, "predicate"); - return RxJavaPlugins.onAssembly(new ParallelFilter(this, predicate)); + return RxJavaPlugins.onAssembly(new ParallelFilter<>(this, predicate)); } /** @@ -196,6 +274,13 @@ public final ParallelFlowable filter(@NonNull Predicate predicate) * handles errors based on the given {@link ParallelFailureHandling} enumeration value. *

* Note that the same predicate may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code filter} does not operate by default on a particular {@link Scheduler}.
+ *
*

History: 2.0.8 - experimental * @param predicate the function returning true to keep a value or false to drop a value * @param errorHandler the enumeration that defines how to handle errors thrown @@ -204,10 +289,13 @@ public final ParallelFlowable filter(@NonNull Predicate predicate) * @since 2.2 */ @CheckReturnValue + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.PASS_THROUGH) public final ParallelFlowable filter(@NonNull Predicate predicate, @NonNull ParallelFailureHandling errorHandler) { Objects.requireNonNull(predicate, "predicate"); Objects.requireNonNull(errorHandler, "errorHandler is null"); - return RxJavaPlugins.onAssembly(new ParallelFilterTry(this, predicate, errorHandler)); + return RxJavaPlugins.onAssembly(new ParallelFilterTry<>(this, predicate, errorHandler)); } /** @@ -215,6 +303,13 @@ public final ParallelFlowable filter(@NonNull Predicate predicate, * handles errors based on the returned value by the handler function. *

* Note that the same predicate may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
*

History: 2.0.8 - experimental * @param predicate the function returning true to keep a value or false to drop a value * @param errorHandler the function called with the current repeat count and @@ -224,10 +319,13 @@ public final ParallelFlowable filter(@NonNull Predicate predicate, * @since 2.2 */ @CheckReturnValue + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.PASS_THROUGH) public final ParallelFlowable filter(@NonNull Predicate predicate, @NonNull BiFunction errorHandler) { Objects.requireNonNull(predicate, "predicate"); Objects.requireNonNull(errorHandler, "errorHandler is null"); - return RxJavaPlugins.onAssembly(new ParallelFilterTry(this, predicate, errorHandler)); + return RxJavaPlugins.onAssembly(new ParallelFilterTry<>(this, predicate, errorHandler)); } /** @@ -245,12 +343,23 @@ public final ParallelFlowable filter(@NonNull Predicate predicate, *

* This operator doesn't require the Scheduler to be trampolining as it * does its own built-in trampolining logic. + *

+ *
Backpressure:
+ *
The operator honors the backpressure of the parallel rails and + * requests {@link Flowable#bufferSize} amount from the upstream, followed + * by 75% of that amount requested after every 75% received.
+ *
Scheduler:
+ *
{@code runOn} drains the upstream rails on the specified {@link Scheduler}'s + * {@link io.reactivex.rxjava3.core.Scheduler.Worker Worker}s.
+ *
* * @param scheduler the scheduler to use * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.CUSTOM) public final ParallelFlowable runOn(@NonNull Scheduler scheduler) { return runOn(scheduler, Flowable.bufferSize()); } @@ -270,6 +379,15 @@ public final ParallelFlowable runOn(@NonNull Scheduler scheduler) { *

* This operator doesn't require the Scheduler to be trampolining as it * does its own built-in trampolining logic. + *

+ *
Backpressure:
+ *
The operator honors the backpressure of the parallel rails and + * requests the {@code prefetch} amount from the upstream, followed + * by 75% of that amount requested after every 75% received.
+ *
Scheduler:
+ *
{@code runOn} drains the upstream rails on the specified {@link Scheduler}'s + * {@link io.reactivex.rxjava3.core.Scheduler.Worker Worker}s.
+ *
* * @param scheduler the scheduler to use * that rail's worker has run out of work. @@ -278,10 +396,12 @@ public final ParallelFlowable runOn(@NonNull Scheduler scheduler) { */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.CUSTOM) public final ParallelFlowable runOn(@NonNull Scheduler scheduler, int prefetch) { Objects.requireNonNull(scheduler, "scheduler"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ParallelRunOn(this, scheduler, prefetch)); + return RxJavaPlugins.onAssembly(new ParallelRunOn<>(this, scheduler, prefetch)); } /** @@ -289,14 +409,23 @@ public final ParallelFlowable runOn(@NonNull Scheduler scheduler, int prefetc * sequential value. *

* Note that the same reducer function may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream and consumes + * the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code reduce} does not operate by default on a particular {@link Scheduler}.
+ *
* @param reducer the function to reduce two values into one. * @return the new Flowable instance emitting the reduced value or empty if the ParallelFlowable was empty */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) public final Flowable reduce(@NonNull BiFunction reducer) { Objects.requireNonNull(reducer, "reducer"); - return RxJavaPlugins.onAssembly(new ParallelReduceFull(this, reducer)); + return RxJavaPlugins.onAssembly(new ParallelReduceFull<>(this, reducer)); } /** @@ -304,6 +433,13 @@ public final Flowable reduce(@NonNull BiFunction reducer) { * a reducer function that is initialized on each rail from an initialSupplier value. *

* Note that the same mapper function may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and consumes + * the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code reduce} does not operate by default on a particular {@link Scheduler}.
+ *
* @param the reduced output type * @param initialSupplier the supplier for the initial value * @param reducer the function to reduce a previous output of reduce (or the initial value supplied) @@ -312,10 +448,12 @@ public final Flowable reduce(@NonNull BiFunction reducer) { */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable reduce(@NonNull Supplier initialSupplier, @NonNull BiFunction reducer) { Objects.requireNonNull(initialSupplier, "initialSupplier"); Objects.requireNonNull(reducer, "reducer"); - return RxJavaPlugins.onAssembly(new ParallelReduce(this, initialSupplier, reducer)); + return RxJavaPlugins.onAssembly(new ParallelReduce<>(this, initialSupplier, reducer)); } /** @@ -327,7 +465,9 @@ public final ParallelFlowable reduce(@NonNull Supplier initialSupplier * *
*
Backpressure:
- *
The operator honors backpressure.
+ *
The operator honors backpressure from the downstream and + * requests {@link Flowable#bufferSize()} amount from each rail, then + * requests from each rail 75% of this amount after 75% received.
*
Scheduler:
*
{@code sequential} does not operate by default on a particular {@link Scheduler}.
*
@@ -338,6 +478,7 @@ public final ParallelFlowable reduce(@NonNull Supplier initialSupplier @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue + @NonNull public final Flowable sequential() { return sequential(Flowable.bufferSize()); } @@ -349,7 +490,9 @@ public final Flowable sequential() { * *
*
Backpressure:
- *
The operator honors backpressure.
+ *
The operator honors backpressure from the downstream and + * requests the {@code prefetch} amount from each rail, then + * requests from each rail 75% of this amount after 75% received.
*
Scheduler:
*
{@code sequential} does not operate by default on a particular {@link Scheduler}.
*
@@ -364,7 +507,7 @@ public final Flowable sequential() { @NonNull public final Flowable sequential(int prefetch) { ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ParallelJoin(this, prefetch, false)); + return RxJavaPlugins.onAssembly(new ParallelJoin<>(this, prefetch, false)); } /** @@ -376,7 +519,9 @@ public final Flowable sequential(int prefetch) { * *
*
Backpressure:
- *
The operator honors backpressure.
+ *
The operator honors backpressure from the downstream and + * requests {@link Flowable#bufferSize()} amount from each rail, then + * requests from each rail 75% of this amount after 75% received.
*
Scheduler:
*
{@code sequentialDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -401,7 +546,9 @@ public final Flowable sequentialDelayError() { * *
*
Backpressure:
- *
The operator honors backpressure.
+ *
The operator honors backpressure from the downstream and + * requests the {@code prefetch} amount from each rail, then + * requests from each rail 75% of this amount after 75% received.
*
Scheduler:
*
{@code sequentialDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -418,7 +565,7 @@ public final Flowable sequentialDelayError() { @NonNull public final Flowable sequentialDelayError(int prefetch) { ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ParallelJoin(this, prefetch, true)); + return RxJavaPlugins.onAssembly(new ParallelJoin<>(this, prefetch, true)); } /** @@ -426,12 +573,21 @@ public final Flowable sequentialDelayError(int prefetch) { * picks the smallest next value from the rails. *

* This operator requires a finite source ParallelFlowable. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream and + * consumes the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code sorted} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param comparator the comparator to use * @return the new Flowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) public final Flowable sorted(@NonNull Comparator comparator) { return sorted(comparator, 16); } @@ -441,6 +597,13 @@ public final Flowable sorted(@NonNull Comparator comparator) { * picks the smallest next value from the rails. *

* This operator requires a finite source ParallelFlowable. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream and + * consumes the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code sorted} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param comparator the comparator to use * @param capacityHint the expected number of total elements @@ -448,6 +611,8 @@ public final Flowable sorted(@NonNull Comparator comparator) { */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) public final Flowable sorted(@NonNull Comparator comparator, int capacityHint) { Objects.requireNonNull(comparator, "comparator is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); @@ -455,19 +620,28 @@ public final Flowable sorted(@NonNull Comparator comparator, int c ParallelFlowable> railReduced = reduce(Functions.createArrayList(ch), ListAddBiConsumer.instance()); ParallelFlowable> railSorted = railReduced.map(new SorterFunction(comparator)); - return RxJavaPlugins.onAssembly(new ParallelSortedJoin(railSorted, comparator)); + return RxJavaPlugins.onAssembly(new ParallelSortedJoin<>(railSorted, comparator)); } /** * Sorts the 'rails' according to the comparator and returns a full sorted list as a Publisher. *

* This operator requires a finite source ParallelFlowable. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream and + * consumes the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code toSortedList} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param comparator the comparator to compare elements * @return the new Flowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) public final Flowable> toSortedList(@NonNull Comparator comparator) { return toSortedList(comparator, 16); } @@ -475,6 +649,13 @@ public final Flowable> toSortedList(@NonNull Comparator compa * Sorts the 'rails' according to the comparator and returns a full sorted list as a Publisher. *

* This operator requires a finite source ParallelFlowable. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream and + * consumes the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code toSortedList} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param comparator the comparator to compare elements * @param capacityHint the expected number of total elements @@ -482,7 +663,9 @@ public final Flowable> toSortedList(@NonNull Comparator compa */ @CheckReturnValue @NonNull - public final Flowable> toSortedList(@NonNull Comparator comparator, int capacityHint) { + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) + public final Flowable<@NonNull List> toSortedList(@NonNull Comparator comparator, int capacityHint) { Objects.requireNonNull(comparator, "comparator is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); @@ -497,15 +680,24 @@ public final Flowable> toSortedList(@NonNull Comparator compa /** * Call the specified consumer with the current element passing through any 'rail'. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param onNext the callback * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doOnNext(@NonNull Consumer onNext) { Objects.requireNonNull(onNext, "onNext is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, onNext, Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -520,6 +712,13 @@ public final ParallelFlowable doOnNext(@NonNull Consumer onNext) { /** * Call the specified consumer with the current element passing through any 'rail' and * handles errors based on the given {@link ParallelFailureHandling} enumeration value. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
*

History: 2.0.8 - experimental * @param onNext the callback * @param errorHandler the enumeration that defines how to handle errors thrown @@ -529,15 +728,24 @@ public final ParallelFlowable doOnNext(@NonNull Consumer onNext) { */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doOnNext(@NonNull Consumer onNext, @NonNull ParallelFailureHandling errorHandler) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(errorHandler, "errorHandler is null"); - return RxJavaPlugins.onAssembly(new ParallelDoOnNextTry(this, onNext, errorHandler)); + return RxJavaPlugins.onAssembly(new ParallelDoOnNextTry<>(this, onNext, errorHandler)); } /** * Call the specified consumer with the current element passing through any 'rail' and * handles errors based on the returned value by the handler function. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
*

History: 2.0.8 - experimental * @param onNext the callback * @param errorHandler the function called with the current repeat count and @@ -548,24 +756,35 @@ public final ParallelFlowable doOnNext(@NonNull Consumer onNext, @ */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doOnNext(@NonNull Consumer onNext, @NonNull BiFunction errorHandler) { Objects.requireNonNull(onNext, "onNext is null"); Objects.requireNonNull(errorHandler, "errorHandler is null"); - return RxJavaPlugins.onAssembly(new ParallelDoOnNextTry(this, onNext, errorHandler)); + return RxJavaPlugins.onAssembly(new ParallelDoOnNextTry<>(this, onNext, errorHandler)); } /** * Call the specified consumer with the current element passing through any 'rail' * after it has been delivered to downstream within the rail. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param onAfterNext the callback * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doAfterNext(@NonNull Consumer onAfterNext) { Objects.requireNonNull(onAfterNext, "onAfterNext is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, Functions.emptyConsumer(), onAfterNext, Functions.emptyConsumer(), @@ -579,15 +798,24 @@ public final ParallelFlowable doAfterNext(@NonNull Consumer onAfte /** * Call the specified consumer with the exception passing through any 'rail'. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param onError the callback * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doOnError(@NonNull Consumer onError) { Objects.requireNonNull(onError, "onError is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, Functions.emptyConsumer(), Functions.emptyConsumer(), onError, @@ -601,15 +829,24 @@ public final ParallelFlowable doOnError(@NonNull Consumer onError) /** * Run the specified Action when a 'rail' completes. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param onComplete the callback * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doOnComplete(@NonNull Action onComplete) { Objects.requireNonNull(onComplete, "onComplete is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -623,15 +860,24 @@ public final ParallelFlowable doOnComplete(@NonNull Action onComplete) { /** * Run the specified Action when a 'rail' completes or signals an error. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param onAfterTerminate the callback * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doAfterTerminated(@NonNull Action onAfterTerminate) { Objects.requireNonNull(onAfterTerminate, "onAfterTerminate is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -645,15 +891,24 @@ public final ParallelFlowable doAfterTerminated(@NonNull Action onAfterTermin /** * Call the specified callback when a 'rail' receives a Subscription from its upstream. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param onSubscribe the callback * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doOnSubscribe(@NonNull Consumer onSubscribe) { Objects.requireNonNull(onSubscribe, "onSubscribe is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -667,15 +922,24 @@ public final ParallelFlowable doOnSubscribe(@NonNull Consumer + *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ * * * @param onRequest the callback * @return the new ParallelFlowable instance */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable doOnRequest(@NonNull LongConsumer onRequest) { Objects.requireNonNull(onRequest, "onRequest is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -689,15 +953,24 @@ public final ParallelFlowable doOnRequest(@NonNull LongConsumer onRequest) { /** * Run the specified Action when a 'rail' receives a cancellation. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param onCancel the callback * @return the new ParallelFlowable instance */ + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) @CheckReturnValue @NonNull public final ParallelFlowable doOnCancel(@NonNull Action onCancel) { Objects.requireNonNull(onCancel, "onCancel is null"); - return RxJavaPlugins.onAssembly(new ParallelPeek(this, + return RxJavaPlugins.onAssembly(new ParallelPeek<>(this, Functions.emptyConsumer(), Functions.emptyConsumer(), Functions.emptyConsumer(), @@ -712,6 +985,13 @@ public final ParallelFlowable doOnCancel(@NonNull Action onCancel) { /** * Collect the elements in each rail into a collection supplied via a collectionSupplier * and collected into with a collector action, emitting the collection at the end. + *
+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * consumes the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the collection type * @param collectionSupplier the supplier of the collection in each rail @@ -720,6 +1000,8 @@ public final ParallelFlowable doOnCancel(@NonNull Action onCancel) { */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable collect(@NonNull Supplier collectionSupplier, @NonNull BiConsumer collector) { Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); Objects.requireNonNull(collector, "collector is null"); @@ -729,6 +1011,13 @@ public final ParallelFlowable collect(@NonNull Supplier coll /** * Wraps multiple Publishers into a ParallelFlowable which runs them * in parallel and unordered. + *
+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the value type * @param publishers the array of publishers @@ -736,17 +1025,27 @@ public final ParallelFlowable collect(@NonNull Supplier coll */ @CheckReturnValue @NonNull - public static ParallelFlowable fromArray(@NonNull Publisher... publishers) { + @SafeVarargs + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) + public static <@NonNull T> ParallelFlowable fromArray(@NonNull Publisher... publishers) { if (publishers.length == 0) { throw new IllegalArgumentException("Zero publishers not supported"); } - return RxJavaPlugins.onAssembly(new ParallelFromArray(publishers)); + return RxJavaPlugins.onAssembly(new ParallelFromArray<>(publishers)); } /** * Calls the specified converter function during assembly time and returns its resulting value. *

* This allows fluent conversion to any other type. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by how the converter function composes over the upstream source.
+ *
Scheduler:
+ *
{@code to} does not operate by default on a particular {@link Scheduler}.
+ *
*

History: 2.1.7 - experimental * @param the resulting object type * @param converter the function that receives the current ParallelFlowable instance and returns a value @@ -756,13 +1055,22 @@ public static ParallelFlowable fromArray(@NonNull Publisher... publish */ @CheckReturnValue @NonNull - public final R to(@NonNull ParallelFlowableConverter converter) { + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) + public final <@NonNull R> R to(@NonNull ParallelFlowableConverter converter) { return Objects.requireNonNull(converter, "converter is null").apply(this); } /** * Allows composing operators, in assembly time, on top of this ParallelFlowable * and returns another ParallelFlowable with composed features. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by how the converter function composes over the upstream source.
+ *
Scheduler:
+ *
{@code compose} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the output value type * @param composer the composer function from ParallelFlowable (this) to another ParallelFlowable @@ -770,6 +1078,8 @@ public final R to(@NonNull ParallelFlowableConverter converter) { */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable compose(@NonNull ParallelTransformer composer) { return RxJavaPlugins.onAssembly(Objects.requireNonNull(composer, "composer is null").apply(this)); } @@ -778,6 +1088,16 @@ public final ParallelFlowable compose(@NonNull ParallelTransformer * Generates and flattens Publishers on each 'rail'. *

* Errors are not delayed and uses unbounded concurrency along with default inner prefetch. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests {@link Flowable#bufferSize()} amount from each rail upfront + * and keeps requesting as many items per rail as many inner sources on + * that rail completed. The inner sources are requested {@link Flowable#bufferSize()} + * amount upfront, then 75% of this amount requested after 75% received. + *
Scheduler:
+ *
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -785,14 +1105,27 @@ public final ParallelFlowable compose(@NonNull ParallelTransformer */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable flatMap(@NonNull Function> mapper) { - return flatMap(mapper, false, Integer.MAX_VALUE, Flowable.bufferSize()); + return flatMap(mapper, false, Flowable.bufferSize(), Flowable.bufferSize()); } /** * Generates and flattens Publishers on each 'rail', optionally delaying errors. *

* It uses unbounded concurrency along with default inner prefetch. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests {@link Flowable#bufferSize()} amount from each rail upfront + * and keeps requesting as many items per rail as many inner sources on + * that rail completed. The inner sources are requested {@link Flowable#bufferSize()} + * amount upfront, then 75% of this amount requested after 75% received. + *
+ *
Scheduler:
+ *
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -801,9 +1134,11 @@ public final ParallelFlowable flatMap(@NonNull Function ParallelFlowable flatMap( @NonNull Function> mapper, boolean delayError) { - return flatMap(mapper, delayError, Integer.MAX_VALUE, Flowable.bufferSize()); + return flatMap(mapper, delayError, Flowable.bufferSize(), Flowable.bufferSize()); } /** @@ -811,6 +1146,17 @@ public final ParallelFlowable flatMap( * and having a total number of simultaneous subscriptions to the inner Publishers. *

* It uses a default inner prefetch. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests {@code maxConcurrency} amount from each rail upfront + * and keeps requesting as many items per rail as many inner sources on + * that rail completed. The inner sources are requested {@link Flowable#bufferSize()} + * amount upfront, then 75% of this amount requested after 75% received. + *
+ *
Scheduler:
+ *
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -820,6 +1166,8 @@ public final ParallelFlowable flatMap( */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable flatMap( @NonNull Function> mapper, boolean delayError, int maxConcurrency) { return flatMap(mapper, delayError, maxConcurrency, Flowable.bufferSize()); @@ -829,6 +1177,17 @@ public final ParallelFlowable flatMap( * Generates and flattens Publishers on each 'rail', optionally delaying errors, * having a total number of simultaneous subscriptions to the inner Publishers * and using the given prefetch amount for the inner Publishers. + *
+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests {@code maxConcurrency} amount from each rail upfront + * and keeps requesting as many items per rail as many inner sources on + * that rail completed. The inner sources are requested the {@code prefetch} + * amount upfront, then 75% of this amount requested after 75% received. + *
+ *
Scheduler:
+ *
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -839,18 +1198,29 @@ public final ParallelFlowable flatMap( */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable flatMap( @NonNull Function> mapper, boolean delayError, int maxConcurrency, int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(maxConcurrency, "maxConcurrency"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ParallelFlatMap(this, mapper, delayError, maxConcurrency, prefetch)); + return RxJavaPlugins.onAssembly(new ParallelFlatMap<>(this, mapper, delayError, maxConcurrency, prefetch)); } /** * Generates and concatenates Publishers on each 'rail', signalling errors immediately * and generating 2 publishers upfront. + *
+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests 2 from each rail upfront and keeps requesting 1 when the inner source complete. + * Requests for the inner sources are determined by the downstream rails' + * backpressure behavior.
+ *
Scheduler:
+ *
{@code concatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -859,6 +1229,8 @@ public final ParallelFlowable flatMap( */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable concatMap( @NonNull Function> mapper) { return concatMap(mapper, 2); @@ -867,6 +1239,16 @@ public final ParallelFlowable concatMap( /** * Generates and concatenates Publishers on each 'rail', signalling errors immediately * and using the given prefetch amount for generating Publishers upfront. + *
+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests the {@code prefetch} amount from each rail upfront and keeps + * requesting 75% of this amount after 75% received and the inner sources completed. + * Requests for the inner sources are determined by the downstream rails' + * backpressure behavior.
+ *
Scheduler:
+ *
{@code concatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -876,17 +1258,28 @@ public final ParallelFlowable concatMap( */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable concatMap( @NonNull Function> mapper, - int prefetch) { + int prefetch) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ParallelConcatMap(this, mapper, prefetch, ErrorMode.IMMEDIATE)); + return RxJavaPlugins.onAssembly(new ParallelConcatMap<>(this, mapper, prefetch, ErrorMode.IMMEDIATE)); } /** * Generates and concatenates Publishers on each 'rail', optionally delaying errors * and generating 2 publishers upfront. + *
+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests 2 from each rail upfront and keeps requesting 1 when the inner source complete. + * Requests for the inner sources are determined by the downstream rails' + * backpressure behavior.
+ *
Scheduler:
+ *
{@code concatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -897,6 +1290,8 @@ public final ParallelFlowable concatMap( */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable concatMapDelayError( @NonNull Function> mapper, boolean tillTheEnd) { @@ -906,6 +1301,16 @@ public final ParallelFlowable concatMapDelayError( /** * Generates and concatenates Publishers on each 'rail', optionally delaying errors * and using the given prefetch amount for generating Publishers upfront. + *
+ *
Backpressure:
+ *
The operator honors backpressure from the downstream rails and + * requests the {@code prefetch} amount from each rail upfront and keeps + * requesting 75% of this amount after 75% received and the inner sources completed. + * Requests for the inner sources are determined by the downstream rails' + * backpressure behavior.
+ *
Scheduler:
+ *
{@code concatMap} does not operate by default on a particular {@link Scheduler}.
+ *
* * @param the result type * @param mapper the function to map each rail's value into a Publisher @@ -916,12 +1321,14 @@ public final ParallelFlowable concatMapDelayError( */ @CheckReturnValue @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) public final ParallelFlowable concatMapDelayError( @NonNull Function> mapper, int prefetch, boolean tillTheEnd) { Objects.requireNonNull(mapper, "mapper is null"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ParallelConcatMap( + return RxJavaPlugins.onAssembly(new ParallelConcatMap<>( this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); } } diff --git a/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java index 9491773080..3461c146d6 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/AsyncProcessor.java @@ -137,7 +137,7 @@ public final class AsyncProcessor extends FlowableProcessor { @CheckReturnValue @NonNull public static AsyncProcessor create() { - return new AsyncProcessor(); + return new AsyncProcessor<>(); } /** @@ -146,7 +146,7 @@ public static AsyncProcessor create() { */ @SuppressWarnings("unchecked") AsyncProcessor() { - this.subscribers = new AtomicReference[]>(EMPTY); + this.subscribers = new AtomicReference<>(EMPTY); } @Override @@ -203,29 +203,33 @@ public void onComplete() { } @Override + @CheckReturnValue public boolean hasSubscribers() { return subscribers.get().length != 0; } @Override + @CheckReturnValue public boolean hasThrowable() { return subscribers.get() == TERMINATED && error != null; } @Override + @CheckReturnValue public boolean hasComplete() { return subscribers.get() == TERMINATED && error == null; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { return subscribers.get() == TERMINATED ? error : null; } @Override protected void subscribeActual(Subscriber s) { - AsyncSubscription as = new AsyncSubscription(s, this); + AsyncSubscription as = new AsyncSubscription<>(s, this); s.onSubscribe(as); if (add(as)) { if (as.isCancelled()) { @@ -316,6 +320,7 @@ void remove(AsyncSubscription ps) { *

The method is thread-safe. * @return true if this processor has any value */ + @CheckReturnValue public boolean hasValue() { return subscribers.get() == TERMINATED && value != null; } @@ -326,6 +331,7 @@ public boolean hasValue() { * @return a single value this processor currently has or null if no such value exists */ @Nullable + @CheckReturnValue public T getValue() { return subscribers.get() == TERMINATED ? value : null; } diff --git a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java index 4b85ff5ab0..ab4c517e65 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java @@ -191,7 +191,7 @@ public final class BehaviorProcessor extends FlowableProcessor { @CheckReturnValue @NonNull public static BehaviorProcessor create() { - return new BehaviorProcessor(); + return new BehaviorProcessor<>(); } /** @@ -207,9 +207,9 @@ public static BehaviorProcessor create() { */ @CheckReturnValue @NonNull - public static BehaviorProcessor createDefault(T defaultValue) { + public static <@NonNull T> BehaviorProcessor createDefault(T defaultValue) { Objects.requireNonNull(defaultValue, "defaultValue is null"); - return new BehaviorProcessor(defaultValue); + return new BehaviorProcessor<>(defaultValue); } /** @@ -218,12 +218,12 @@ public static BehaviorProcessor createDefault(T defaultValue) { */ @SuppressWarnings("unchecked") BehaviorProcessor() { - this.value = new AtomicReference(); + this.value = new AtomicReference<>(); this.lock = new ReentrantReadWriteLock(); this.readLock = lock.readLock(); this.writeLock = lock.writeLock(); - this.subscribers = new AtomicReference[]>(EMPTY); - this.terminalEvent = new AtomicReference(); + this.subscribers = new AtomicReference<>(EMPTY); + this.terminalEvent = new AtomicReference<>(); } /** @@ -239,7 +239,7 @@ public static BehaviorProcessor createDefault(T defaultValue) { @Override protected void subscribeActual(Subscriber s) { - BehaviorSubscription bs = new BehaviorSubscription(s, this); + BehaviorSubscription bs = new BehaviorSubscription<>(s, this); s.onSubscribe(bs); if (add(bs)) { if (bs.cancelled) { @@ -318,6 +318,7 @@ public void onComplete() { * @return true if the item was emitted to all Subscribers * @since 2.2 */ + @CheckReturnValue public boolean offer(T t) { if (t == null) { onError(ExceptionHelper.createNullPointerException("offer called with a null value.")); @@ -340,16 +341,19 @@ public boolean offer(T t) { } @Override + @CheckReturnValue public boolean hasSubscribers() { return subscribers.get().length != 0; } + @CheckReturnValue /* test support*/ int subscriberCount() { return subscribers.get().length; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { Object o = value.get(); if (NotificationLite.isError(o)) { @@ -364,6 +368,7 @@ public Throwable getThrowable() { * @return a single value the BehaviorProcessor currently has or null if no such value exists */ @Nullable + @CheckReturnValue public T getValue() { Object o = value.get(); if (NotificationLite.isComplete(o) || NotificationLite.isError(o)) { @@ -373,12 +378,14 @@ public T getValue() { } @Override + @CheckReturnValue public boolean hasComplete() { Object o = value.get(); return NotificationLite.isComplete(o); } @Override + @CheckReturnValue public boolean hasThrowable() { Object o = value.get(); return NotificationLite.isError(o); @@ -389,6 +396,7 @@ public boolean hasThrowable() { *

The method is thread-safe. * @return true if the BehaviorProcessor has any value */ + @CheckReturnValue public boolean hasValue() { Object o = value.get(); return o != null && !NotificationLite.isComplete(o) && !NotificationLite.isError(o); @@ -554,7 +562,7 @@ void emitNext(Object value, long stateIndex) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(value); diff --git a/src/main/java/io/reactivex/rxjava3/processors/FlowableProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/FlowableProcessor.java index abe5e2ad08..80db65e652 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/FlowableProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/FlowableProcessor.java @@ -33,6 +33,7 @@ public abstract class FlowableProcessor extends Flowable implements Proces *

The method is thread-safe. * @return true if the FlowableProcessor has subscribers */ + @CheckReturnValue public abstract boolean hasSubscribers(); /** @@ -42,6 +43,7 @@ public abstract class FlowableProcessor extends Flowable implements Proces * @see #getThrowable() * @see #hasComplete() */ + @CheckReturnValue public abstract boolean hasThrowable(); /** @@ -50,6 +52,7 @@ public abstract class FlowableProcessor extends Flowable implements Proces * @return true if the FlowableProcessor has reached a terminal state through a complete event * @see #hasThrowable() */ + @CheckReturnValue public abstract boolean hasComplete(); /** @@ -60,6 +63,7 @@ public abstract class FlowableProcessor extends Flowable implements Proces * hasn't terminated yet */ @Nullable + @CheckReturnValue public abstract Throwable getThrowable(); /** @@ -74,6 +78,6 @@ public final FlowableProcessor toSerialized() { if (this instanceof SerializedProcessor) { return this; } - return new SerializedProcessor(this); + return new SerializedProcessor<>(this); } } diff --git a/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java index 50497473ac..d258ba1718 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java @@ -170,7 +170,7 @@ public final class MulticastProcessor extends FlowableProcessor { @CheckReturnValue @NonNull public static MulticastProcessor create() { - return new MulticastProcessor(bufferSize(), false); + return new MulticastProcessor<>(bufferSize(), false); } /** @@ -184,7 +184,7 @@ public static MulticastProcessor create() { @CheckReturnValue @NonNull public static MulticastProcessor create(boolean refCount) { - return new MulticastProcessor(bufferSize(), refCount); + return new MulticastProcessor<>(bufferSize(), refCount); } /** @@ -196,7 +196,7 @@ public static MulticastProcessor create(boolean refCount) { @CheckReturnValue @NonNull public static MulticastProcessor create(int bufferSize) { - return new MulticastProcessor(bufferSize, false); + return new MulticastProcessor<>(bufferSize, false); } /** @@ -211,7 +211,7 @@ public static MulticastProcessor create(int bufferSize) { @CheckReturnValue @NonNull public static MulticastProcessor create(int bufferSize, boolean refCount) { - return new MulticastProcessor(bufferSize, refCount); + return new MulticastProcessor<>(bufferSize, refCount); } /** @@ -227,8 +227,8 @@ public static MulticastProcessor create(int bufferSize, boolean refCount) this.bufferSize = bufferSize; this.limit = bufferSize - (bufferSize >> 2); this.wip = new AtomicInteger(); - this.subscribers = new AtomicReference[]>(EMPTY); - this.upstream = new AtomicReference(); + this.subscribers = new AtomicReference<>(EMPTY); + this.upstream = new AtomicReference<>(); this.refcount = refCount; this.once = new AtomicBoolean(); } @@ -241,7 +241,7 @@ public static MulticastProcessor create(int bufferSize, boolean refCount) */ public void start() { if (SubscriptionHelper.setOnce(upstream, EmptySubscription.INSTANCE)) { - queue = new SpscArrayQueue(bufferSize); + queue = new SpscArrayQueue<>(bufferSize); } } @@ -253,7 +253,7 @@ public void start() { */ public void startUnbounded() { if (SubscriptionHelper.setOnce(upstream, EmptySubscription.INSTANCE)) { - queue = new SpscLinkedArrayQueue(bufferSize); + queue = new SpscLinkedArrayQueue<>(bufferSize); } } @@ -281,7 +281,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(bufferSize); + queue = new SpscArrayQueue<>(bufferSize); s.request(bufferSize); } @@ -309,6 +309,7 @@ public void onNext(T t) { * @param t the item to offer, not null * @return true if successful, false if the queue is full */ + @CheckReturnValue public boolean offer(T t) { if (once.get()) { return false; @@ -344,28 +345,32 @@ public void onComplete() { } @Override + @CheckReturnValue public boolean hasSubscribers() { return subscribers.get().length != 0; } @Override + @CheckReturnValue public boolean hasThrowable() { return once.get() && error != null; } @Override + @CheckReturnValue public boolean hasComplete() { return once.get() && error == null; } @Override + @CheckReturnValue public Throwable getThrowable() { return once.get() ? error : null; } @Override protected void subscribeActual(Subscriber s) { - MulticastSubscription ms = new MulticastSubscription(s, this); + MulticastSubscription ms = new MulticastSubscription<>(s, this); s.onSubscribe(ms); if (add(ms)) { if (ms.get() == Long.MIN_VALUE) { diff --git a/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java index 4dc9bc9371..36bc4eff34 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java @@ -128,7 +128,7 @@ public final class PublishProcessor extends FlowableProcessor { @CheckReturnValue @NonNull public static PublishProcessor create() { - return new PublishProcessor(); + return new PublishProcessor<>(); } /** @@ -137,12 +137,12 @@ public static PublishProcessor create() { */ @SuppressWarnings("unchecked") PublishProcessor() { - subscribers = new AtomicReference[]>(EMPTY); + subscribers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(Subscriber t) { - PublishSubscription ps = new PublishSubscription(t, this); + PublishSubscription ps = new PublishSubscription<>(t, this); t.onSubscribe(ps); if (add(ps)) { // if cancellation happened while a successful add, the remove() didn't work @@ -283,6 +283,7 @@ public void onComplete() { * @return true if the item was emitted to all Subscribers * @since 2.2 */ + @CheckReturnValue public boolean offer(T t) { if (t == null) { onError(ExceptionHelper.createNullPointerException("offer called with a null value.")); @@ -303,12 +304,14 @@ public boolean offer(T t) { } @Override + @CheckReturnValue public boolean hasSubscribers() { return subscribers.get().length != 0; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { if (subscribers.get() == TERMINATED) { return error; @@ -317,11 +320,13 @@ public Throwable getThrowable() { } @Override + @CheckReturnValue public boolean hasThrowable() { return subscribers.get() == TERMINATED && error != null; } @Override + @CheckReturnValue public boolean hasComplete() { return subscribers.get() == TERMINATED && error == null; } diff --git a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java index 04de41a77f..9b722057b9 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java @@ -173,7 +173,7 @@ public final class ReplayProcessor extends FlowableProcessor { @CheckReturnValue @NonNull public static ReplayProcessor create() { - return new ReplayProcessor(new UnboundedReplayBuffer(16)); + return new ReplayProcessor<>(new UnboundedReplayBuffer(16)); } /** @@ -194,7 +194,7 @@ public static ReplayProcessor create() { @CheckReturnValue @NonNull public static ReplayProcessor create(int capacityHint) { - return new ReplayProcessor(new UnboundedReplayBuffer(capacityHint)); + return new ReplayProcessor<>(new UnboundedReplayBuffer(capacityHint)); } /** @@ -220,7 +220,7 @@ public static ReplayProcessor create(int capacityHint) { @CheckReturnValue @NonNull public static ReplayProcessor createWithSize(int maxSize) { - return new ReplayProcessor(new SizeBoundReplayBuffer(maxSize)); + return new ReplayProcessor<>(new SizeBoundReplayBuffer(maxSize)); } /** @@ -236,8 +236,9 @@ public static ReplayProcessor createWithSize(int maxSize) { * the type of items observed and emitted by this type of processor * @return the created processor */ + @CheckReturnValue /* test */ static ReplayProcessor createUnbounded() { - return new ReplayProcessor(new SizeBoundReplayBuffer(Integer.MAX_VALUE)); + return new ReplayProcessor<>(new SizeBoundReplayBuffer(Integer.MAX_VALUE)); } /** @@ -274,8 +275,8 @@ public static ReplayProcessor createWithSize(int maxSize) { */ @CheckReturnValue @NonNull - public static ReplayProcessor createWithTime(long maxAge, TimeUnit unit, Scheduler scheduler) { - return new ReplayProcessor(new SizeAndTimeBoundReplayBuffer(Integer.MAX_VALUE, maxAge, unit, scheduler)); + public static ReplayProcessor createWithTime(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { + return new ReplayProcessor<>(new SizeAndTimeBoundReplayBuffer(Integer.MAX_VALUE, maxAge, unit, scheduler)); } /** @@ -314,8 +315,8 @@ public static ReplayProcessor createWithTime(long maxAge, TimeUnit unit, */ @CheckReturnValue @NonNull - public static ReplayProcessor createWithTimeAndSize(long maxAge, TimeUnit unit, Scheduler scheduler, int maxSize) { - return new ReplayProcessor(new SizeAndTimeBoundReplayBuffer(maxSize, maxAge, unit, scheduler)); + public static ReplayProcessor createWithTimeAndSize(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int maxSize) { + return new ReplayProcessor<>(new SizeAndTimeBoundReplayBuffer(maxSize, maxAge, unit, scheduler)); } /** @@ -325,12 +326,12 @@ public static ReplayProcessor createWithTimeAndSize(long maxAge, TimeUnit @SuppressWarnings("unchecked") ReplayProcessor(ReplayBuffer buffer) { this.buffer = buffer; - this.subscribers = new AtomicReference[]>(EMPTY); + this.subscribers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(Subscriber s) { - ReplaySubscription rs = new ReplaySubscription(s, this); + ReplaySubscription rs = new ReplaySubscription<>(s, this); s.onSubscribe(rs); if (add(rs)) { @@ -404,16 +405,19 @@ public void onComplete() { } @Override + @CheckReturnValue public boolean hasSubscribers() { return subscribers.get().length != 0; } + @CheckReturnValue /* test */ int subscriberCount() { return subscribers.get().length; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { ReplayBuffer b = buffer; if (b.isDone()) { @@ -445,6 +449,7 @@ public void cleanupBuffer() { *

The method is thread-safe. * @return the latest value this processor currently has or null if no such value exists */ + @CheckReturnValue public T getValue() { return buffer.getValue(); } @@ -454,6 +459,7 @@ public T getValue() { *

The method is thread-safe. * @return the array containing the snapshot of all values of this processor */ + @CheckReturnValue public Object[] getValues() { @SuppressWarnings("unchecked") T[] a = (T[])EMPTY_ARRAY; @@ -473,17 +479,20 @@ public Object[] getValues() { * @param array the target array to copy values into if it fits * @return the given array if the values fit into it or a new array containing all values */ + @CheckReturnValue public T[] getValues(T[] array) { return buffer.getValues(array); } @Override + @CheckReturnValue public boolean hasComplete() { ReplayBuffer b = buffer; return b.isDone() && b.getError() == null; } @Override + @CheckReturnValue public boolean hasThrowable() { ReplayBuffer b = buffer; return b.isDone() && b.getError() != null; @@ -494,10 +503,12 @@ public boolean hasThrowable() { *

The method is thread-safe. * @return true if the processor has any value */ + @CheckReturnValue public boolean hasValue() { return buffer.size() != 0; // NOPMD } + @CheckReturnValue /* test*/ int size() { return buffer.size(); } @@ -634,7 +645,7 @@ static final class UnboundedReplayBuffer volatile int size; UnboundedReplayBuffer(int capacityHint) { - this.buffer = new ArrayList(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); + this.buffer = new ArrayList<>(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); } @Override @@ -835,7 +846,7 @@ static final class SizeBoundReplayBuffer SizeBoundReplayBuffer(int maxSize) { this.maxSize = ObjectHelper.verifyPositive(maxSize, "maxSize"); - Node h = new Node(null); + Node h = new Node<>(null); this.tail = h; this.head = h; } @@ -850,7 +861,7 @@ void trim() { @Override public void next(T value) { - Node n = new Node(value); + Node n = new Node<>(value); Node t = tail; tail = n; @@ -876,7 +887,7 @@ public void complete() { @Override public void trimHead() { if (head.value != null) { - Node n = new Node(null); + Node n = new Node<>(null); n.lazySet(head.get()); head = n; } @@ -1054,7 +1065,7 @@ static final class SizeAndTimeBoundReplayBuffer this.maxAge = ObjectHelper.verifyPositive(maxAge, "maxAge"); this.unit = Objects.requireNonNull(unit, "unit is null"); this.scheduler = Objects.requireNonNull(scheduler, "scheduler is null"); - TimedNode h = new TimedNode(null, 0L); + TimedNode h = new TimedNode<>(null, 0L); this.tail = h; this.head = h; } @@ -1100,7 +1111,7 @@ void trimFinal() { TimedNode next = h.get(); if (next == null) { if (h.value != null) { - head = new TimedNode(null, 0L); + head = new TimedNode<>(null, 0L); } else { head = h; } @@ -1109,7 +1120,7 @@ void trimFinal() { if (next.time > limit) { if (h.value != null) { - TimedNode n = new TimedNode(null, 0L); + TimedNode n = new TimedNode<>(null, 0L); n.lazySet(h.get()); head = n; } else { @@ -1125,7 +1136,7 @@ void trimFinal() { @Override public void trimHead() { if (head.value != null) { - TimedNode n = new TimedNode(null, 0L); + TimedNode n = new TimedNode<>(null, 0L); n.lazySet(head.get()); head = n; } @@ -1133,7 +1144,7 @@ public void trimHead() { @Override public void next(T value) { - TimedNode n = new TimedNode(value, scheduler.now(unit)); + TimedNode n = new TimedNode<>(value, scheduler.now(unit)); TimedNode t = tail; tail = n; diff --git a/src/main/java/io/reactivex/rxjava3/processors/SerializedProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/SerializedProcessor.java index 6096398618..e7740b6523 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/SerializedProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/SerializedProcessor.java @@ -59,7 +59,7 @@ public void onSubscribe(Subscription s) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(NotificationLite.subscription(s)); @@ -92,7 +92,7 @@ public void onNext(T t) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(NotificationLite.next(t)); @@ -119,7 +119,7 @@ public void onError(Throwable t) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.setFirst(NotificationLite.error(t)); @@ -149,7 +149,7 @@ public void onComplete() { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(NotificationLite.complete()); diff --git a/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java index ee93a5ac9c..eab7da4f05 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/UnicastProcessor.java @@ -19,7 +19,7 @@ import org.reactivestreams.*; import io.reactivex.rxjava3.annotations.*; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; +import io.reactivex.rxjava3.internal.functions.*; import io.reactivex.rxjava3.internal.fuseable.QueueSubscription; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; import io.reactivex.rxjava3.internal.subscriptions.*; @@ -179,7 +179,7 @@ public final class UnicastProcessor extends FlowableProcessor { @CheckReturnValue @NonNull public static UnicastProcessor create() { - return new UnicastProcessor(bufferSize()); + return create(bufferSize(), Functions.EMPTY_RUNNABLE, true); } /** @@ -191,7 +191,7 @@ public static UnicastProcessor create() { @CheckReturnValue @NonNull public static UnicastProcessor create(int capacityHint) { - return new UnicastProcessor(capacityHint); + return create(capacityHint, Functions.EMPTY_RUNNABLE, true); } /** @@ -205,67 +205,47 @@ public static UnicastProcessor create(int capacityHint) { @CheckReturnValue @NonNull public static UnicastProcessor create(boolean delayError) { - return new UnicastProcessor(bufferSize(), null, delayError); + return create(bufferSize(), Functions.EMPTY_RUNNABLE, delayError); } /** * Creates an UnicastProcessor with the given internal buffer capacity hint and a callback for - * the case when the single Subscriber cancels its subscription. + * the case when the single Subscriber cancels its subscription or the + * processor is terminated. * *

The callback, if not null, is called exactly once and * non-overlapped with any active replay. * * @param the value type * @param capacityHint the hint to size the internal unbounded buffer - * @param onCancelled the non null callback + * @param onTerminate the non null callback * @return an UnicastProcessor instance */ @CheckReturnValue @NonNull - public static UnicastProcessor create(int capacityHint, Runnable onCancelled) { - Objects.requireNonNull(onCancelled, "onTerminate"); - return new UnicastProcessor(capacityHint, onCancelled); + public static UnicastProcessor create(int capacityHint, @NonNull Runnable onTerminate) { + return create(capacityHint, onTerminate, true); } /** * Creates an UnicastProcessor with the given internal buffer capacity hint, delay error flag and a callback for - * the case when the single Subscriber cancels its subscription. + * the case when the single Subscriber cancels its subscription or + * the processor is terminated. * *

The callback, if not null, is called exactly once and * non-overlapped with any active replay. *

History: 2.0.8 - experimental * @param the value type * @param capacityHint the hint to size the internal unbounded buffer - * @param onCancelled the non null callback + * @param onTerminate the non null callback * @param delayError deliver pending onNext events before onError * @return an UnicastProcessor instance * @since 2.2 */ @CheckReturnValue @NonNull - public static UnicastProcessor create(int capacityHint, Runnable onCancelled, boolean delayError) { - Objects.requireNonNull(onCancelled, "onTerminate"); - return new UnicastProcessor(capacityHint, onCancelled, delayError); - } - - /** - * Creates an UnicastProcessor with the given capacity hint. - * @param capacityHint the capacity hint for the internal, unbounded queue - * @since 2.0 - */ - UnicastProcessor(int capacityHint) { - this(capacityHint, null, true); - } - - /** - * Creates an UnicastProcessor with the given capacity hint and callback - * for when the Processor is terminated normally or its single Subscriber cancels. - * @param capacityHint the capacity hint for the internal, unbounded queue - * @param onTerminate the callback to run when the Processor is terminated or cancelled, null not allowed - * @since 2.0 - */ - UnicastProcessor(int capacityHint, Runnable onTerminate) { - this(capacityHint, onTerminate, true); + public static UnicastProcessor create(int capacityHint, @NonNull Runnable onTerminate, boolean delayError) { + return new UnicastProcessor<>(capacityHint, onTerminate, delayError); } /** @@ -278,10 +258,10 @@ public static UnicastProcessor create(int capacityHint, Runnable onCancel * @since 2.2 */ UnicastProcessor(int capacityHint, Runnable onTerminate, boolean delayError) { - this.queue = new SpscLinkedArrayQueue(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); - this.onTerminate = new AtomicReference(onTerminate); + this.queue = new SpscLinkedArrayQueue<>(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); + this.onTerminate = new AtomicReference<>(Objects.requireNonNull(onTerminate, "onTerminate")); this.delayError = delayError; - this.downstream = new AtomicReference>(); + this.downstream = new AtomicReference<>(); this.once = new AtomicBoolean(); this.wip = new UnicastQueueSubscription(); this.requested = new AtomicLong(); @@ -559,12 +539,14 @@ public void cancel() { } @Override + @CheckReturnValue public boolean hasSubscribers() { return downstream.get() != null; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { if (done) { return error; @@ -573,11 +555,13 @@ public Throwable getThrowable() { } @Override + @CheckReturnValue public boolean hasComplete() { return done && error == null; } @Override + @CheckReturnValue public boolean hasThrowable() { return done && error != null; } diff --git a/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java b/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java index d9247acee2..3e05665283 100644 --- a/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/schedulers/TestScheduler.java @@ -28,7 +28,7 @@ */ public final class TestScheduler extends Scheduler { /** The ordered queue for the runnable tasks. */ - final Queue queue = new PriorityBlockingQueue(11); + final Queue queue = new PriorityBlockingQueue<>(11); /** The per-scheduler global order counter. */ long counter; // Storing time in nanoseconds internally. diff --git a/src/main/java/io/reactivex/rxjava3/subjects/AsyncSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/AsyncSubject.java index 174c98c4f5..cfeafcd4a8 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/AsyncSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/AsyncSubject.java @@ -129,7 +129,7 @@ public final class AsyncSubject extends Subject { @CheckReturnValue @NonNull public static AsyncSubject create() { - return new AsyncSubject(); + return new AsyncSubject<>(); } /** @@ -138,7 +138,7 @@ public static AsyncSubject create() { */ @SuppressWarnings("unchecked") AsyncSubject() { - this.subscribers = new AtomicReference[]>(EMPTY); + this.subscribers = new AtomicReference<>(EMPTY); } @Override @@ -192,28 +192,32 @@ public void onComplete() { } @Override + @CheckReturnValue public boolean hasObservers() { return subscribers.get().length != 0; } @Override + @CheckReturnValue public boolean hasThrowable() { return subscribers.get() == TERMINATED && error != null; } @Override + @CheckReturnValue public boolean hasComplete() { return subscribers.get() == TERMINATED && error == null; } @Override + @CheckReturnValue public Throwable getThrowable() { return subscribers.get() == TERMINATED ? error : null; } @Override protected void subscribeActual(Observer observer) { - AsyncDisposable as = new AsyncDisposable(observer, this); + AsyncDisposable as = new AsyncDisposable<>(observer, this); observer.onSubscribe(as); if (add(as)) { if (as.isDisposed()) { @@ -304,6 +308,7 @@ void remove(AsyncDisposable ps) { *

The method is thread-safe. * @return true if the subject has any value */ + @CheckReturnValue public boolean hasValue() { return subscribers.get() == TERMINATED && value != null; } @@ -314,6 +319,7 @@ public boolean hasValue() { * @return a single value the Subject currently has or null if no such value exists */ @Nullable + @CheckReturnValue public T getValue() { return subscribers.get() == TERMINATED ? value : null; } diff --git a/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java index b3b212eea4..d7f5516e7d 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/BehaviorSubject.java @@ -176,7 +176,7 @@ public final class BehaviorSubject extends Subject { @CheckReturnValue @NonNull public static BehaviorSubject create() { - return new BehaviorSubject(); + return new BehaviorSubject<>(); } /** @@ -192,8 +192,8 @@ public static BehaviorSubject create() { */ @CheckReturnValue @NonNull - public static BehaviorSubject createDefault(T defaultValue) { - return new BehaviorSubject(defaultValue); + public static <@NonNull T> BehaviorSubject createDefault(T defaultValue) { + return new BehaviorSubject<>(defaultValue); } /** @@ -205,9 +205,9 @@ public static BehaviorSubject createDefault(T defaultValue) { this.lock = new ReentrantReadWriteLock(); this.readLock = lock.readLock(); this.writeLock = lock.writeLock(); - this.subscribers = new AtomicReference[]>(EMPTY); - this.value = new AtomicReference(); - this.terminalEvent = new AtomicReference(); + this.subscribers = new AtomicReference<>(EMPTY); + this.value = new AtomicReference<>(); + this.terminalEvent = new AtomicReference<>(); } /** @@ -223,7 +223,7 @@ public static BehaviorSubject createDefault(T defaultValue) { @Override protected void subscribeActual(Observer observer) { - BehaviorDisposable bs = new BehaviorDisposable(observer, this); + BehaviorDisposable bs = new BehaviorDisposable<>(observer, this); observer.onSubscribe(bs); if (add(bs)) { if (bs.cancelled) { @@ -287,16 +287,19 @@ public void onComplete() { } @Override + @CheckReturnValue public boolean hasObservers() { return subscribers.get().length != 0; } + @CheckReturnValue /* test support*/ int subscriberCount() { return subscribers.get().length; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { Object o = value.get(); if (NotificationLite.isError(o)) { @@ -311,6 +314,7 @@ public Throwable getThrowable() { * @return a single value the Subject currently has or null if no such value exists */ @Nullable + @CheckReturnValue public T getValue() { Object o = value.get(); if (NotificationLite.isComplete(o) || NotificationLite.isError(o)) { @@ -320,12 +324,14 @@ public T getValue() { } @Override + @CheckReturnValue public boolean hasComplete() { Object o = value.get(); return NotificationLite.isComplete(o); } @Override + @CheckReturnValue public boolean hasThrowable() { Object o = value.get(); return NotificationLite.isError(o); @@ -336,6 +342,7 @@ public boolean hasThrowable() { *

The method is thread-safe. * @return true if the subject has any value */ + @CheckReturnValue public boolean hasValue() { Object o = value.get(); return o != null && !NotificationLite.isComplete(o) && !NotificationLite.isError(o); @@ -493,7 +500,7 @@ void emitNext(Object value, long stateIndex) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(value); diff --git a/src/main/java/io/reactivex/rxjava3/subjects/CompletableSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/CompletableSubject.java index bbb4b24b0c..9753d276a1 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/CompletableSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/CompletableSubject.java @@ -107,7 +107,7 @@ public static CompletableSubject create() { CompletableSubject() { once = new AtomicBoolean(); - observers = new AtomicReference(EMPTY); + observers = new AtomicReference<>(EMPTY); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/subjects/MaybeSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/MaybeSubject.java index a2f62524f0..f06d8c1aae 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/MaybeSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/MaybeSubject.java @@ -131,13 +131,13 @@ public final class MaybeSubject extends Maybe implements MaybeObserver @CheckReturnValue @NonNull public static MaybeSubject create() { - return new MaybeSubject(); + return new MaybeSubject<>(); } @SuppressWarnings("unchecked") MaybeSubject() { once = new AtomicBoolean(); - observers = new AtomicReference[]>(EMPTY); + observers = new AtomicReference<>(EMPTY); } @Override @@ -185,7 +185,7 @@ public void onComplete() { @Override protected void subscribeActual(MaybeObserver observer) { - MaybeDisposable md = new MaybeDisposable(observer, this); + MaybeDisposable md = new MaybeDisposable<>(observer, this); observer.onSubscribe(md); if (add(md)) { if (md.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/subjects/PublishSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/PublishSubject.java index a81d6062fd..2c3a847832 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/PublishSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/PublishSubject.java @@ -115,7 +115,7 @@ public final class PublishSubject extends Subject { @CheckReturnValue @NonNull public static PublishSubject create() { - return new PublishSubject(); + return new PublishSubject<>(); } /** @@ -124,12 +124,12 @@ public static PublishSubject create() { */ @SuppressWarnings("unchecked") PublishSubject() { - subscribers = new AtomicReference[]>(EMPTY); + subscribers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(Observer t) { - PublishDisposable ps = new PublishDisposable(t, this); + PublishDisposable ps = new PublishDisposable<>(t, this); t.onSubscribe(ps); if (add(ps)) { // if cancellation happened while a successful add, the remove() didn't work @@ -254,12 +254,14 @@ public void onComplete() { } @Override + @CheckReturnValue public boolean hasObservers() { return subscribers.get().length != 0; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { if (subscribers.get() == TERMINATED) { return error; @@ -268,11 +270,13 @@ public Throwable getThrowable() { } @Override + @CheckReturnValue public boolean hasThrowable() { return subscribers.get() == TERMINATED && error != null; } @Override + @CheckReturnValue public boolean hasComplete() { return subscribers.get() == TERMINATED && error == null; } diff --git a/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java b/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java index aad2092429..9bb3d93b99 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java @@ -160,7 +160,7 @@ public final class ReplaySubject extends Subject { @CheckReturnValue @NonNull public static ReplaySubject create() { - return new ReplaySubject(new UnboundedReplayBuffer(16)); + return new ReplaySubject<>(new UnboundedReplayBuffer(16)); } /** @@ -181,7 +181,7 @@ public static ReplaySubject create() { @CheckReturnValue @NonNull public static ReplaySubject create(int capacityHint) { - return new ReplaySubject(new UnboundedReplayBuffer(capacityHint)); + return new ReplaySubject<>(new UnboundedReplayBuffer(capacityHint)); } /** @@ -207,7 +207,7 @@ public static ReplaySubject create(int capacityHint) { @CheckReturnValue @NonNull public static ReplaySubject createWithSize(int maxSize) { - return new ReplaySubject(new SizeBoundReplayBuffer(maxSize)); + return new ReplaySubject<>(new SizeBoundReplayBuffer(maxSize)); } /** @@ -224,7 +224,7 @@ public static ReplaySubject createWithSize(int maxSize) { * @return the created subject */ /* test */ static ReplaySubject createUnbounded() { - return new ReplaySubject(new SizeBoundReplayBuffer(Integer.MAX_VALUE)); + return new ReplaySubject<>(new SizeBoundReplayBuffer(Integer.MAX_VALUE)); } /** @@ -261,8 +261,8 @@ public static ReplaySubject createWithSize(int maxSize) { */ @CheckReturnValue @NonNull - public static ReplaySubject createWithTime(long maxAge, TimeUnit unit, Scheduler scheduler) { - return new ReplaySubject(new SizeAndTimeBoundReplayBuffer(Integer.MAX_VALUE, maxAge, unit, scheduler)); + public static ReplaySubject createWithTime(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { + return new ReplaySubject<>(new SizeAndTimeBoundReplayBuffer(Integer.MAX_VALUE, maxAge, unit, scheduler)); } /** @@ -301,8 +301,8 @@ public static ReplaySubject createWithTime(long maxAge, TimeUnit unit, Sc */ @CheckReturnValue @NonNull - public static ReplaySubject createWithTimeAndSize(long maxAge, TimeUnit unit, Scheduler scheduler, int maxSize) { - return new ReplaySubject(new SizeAndTimeBoundReplayBuffer(maxSize, maxAge, unit, scheduler)); + public static ReplaySubject createWithTimeAndSize(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int maxSize) { + return new ReplaySubject<>(new SizeAndTimeBoundReplayBuffer(maxSize, maxAge, unit, scheduler)); } /** @@ -312,12 +312,12 @@ public static ReplaySubject createWithTimeAndSize(long maxAge, TimeUnit u @SuppressWarnings("unchecked") ReplaySubject(ReplayBuffer buffer) { this.buffer = buffer; - this.observers = new AtomicReference[]>(EMPTY); + this.observers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(Observer observer) { - ReplayDisposable rs = new ReplayDisposable(observer, this); + ReplayDisposable rs = new ReplayDisposable<>(observer, this); observer.onSubscribe(rs); if (!rs.cancelled) { @@ -392,16 +392,19 @@ public void onComplete() { } @Override + @CheckReturnValue public boolean hasObservers() { return observers.get().length != 0; } + @CheckReturnValue /* test */ int observerCount() { return observers.get().length; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { Object o = buffer.get(); if (NotificationLite.isError(o)) { @@ -416,6 +419,7 @@ public Throwable getThrowable() { * @return a single value the Subject currently has or null if no such value exists */ @Nullable + @CheckReturnValue public T getValue() { return buffer.getValue(); } @@ -446,6 +450,7 @@ public void cleanupBuffer() { *

The method is thread-safe. * @return the array containing the snapshot of all values of the Subject */ + @CheckReturnValue public Object[] getValues() { @SuppressWarnings("unchecked") T[] a = (T[])EMPTY_ARRAY; @@ -465,17 +470,20 @@ public Object[] getValues() { * @param array the target array to copy values into if it fits * @return the given array if the values fit into it or a new array containing all values */ + @CheckReturnValue public T[] getValues(T[] array) { return buffer.getValues(array); } @Override + @CheckReturnValue public boolean hasComplete() { Object o = buffer.get(); return NotificationLite.isComplete(o); } @Override + @CheckReturnValue public boolean hasThrowable() { Object o = buffer.get(); return NotificationLite.isError(o); @@ -486,10 +494,12 @@ public boolean hasThrowable() { *

The method is thread-safe. * @return true if the subject has any value */ + @CheckReturnValue public boolean hasValue() { return buffer.size() != 0; // NOPMD } + @CheckReturnValue /* test*/ int size() { return buffer.size(); } @@ -636,7 +646,7 @@ static final class UnboundedReplayBuffer volatile int size; UnboundedReplayBuffer(int capacityHint) { - this.buffer = new ArrayList(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); + this.buffer = new ArrayList<>(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); } @Override @@ -839,7 +849,7 @@ static final class SizeBoundReplayBuffer SizeBoundReplayBuffer(int maxSize) { this.maxSize = ObjectHelper.verifyPositive(maxSize, "maxSize"); - Node h = new Node(null); + Node h = new Node<>(null); this.tail = h; this.head = h; } @@ -854,7 +864,7 @@ void trim() { @Override public void add(T value) { - Node n = new Node(value); + Node n = new Node<>(value); Node t = tail; tail = n; @@ -866,7 +876,7 @@ public void add(T value) { @Override public void addFinal(Object notificationLite) { - Node n = new Node(notificationLite); + Node n = new Node<>(notificationLite); Node t = tail; tail = n; @@ -885,7 +895,7 @@ public void addFinal(Object notificationLite) { public void trimHead() { Node h = head; if (h.value != null) { - Node n = new Node(null); + Node n = new Node<>(null); n.lazySet(h.get()); head = n; } @@ -1055,7 +1065,7 @@ static final class SizeAndTimeBoundReplayBuffer this.maxAge = ObjectHelper.verifyPositive(maxAge, "maxAge"); this.unit = Objects.requireNonNull(unit, "unit is null"); this.scheduler = Objects.requireNonNull(scheduler, "scheduler is null"); - TimedNode h = new TimedNode(null, 0L); + TimedNode h = new TimedNode<>(null, 0L); this.tail = h; this.head = h; } @@ -1101,7 +1111,7 @@ void trimFinal() { TimedNode next = h.get(); if (next.get() == null) { if (h.value != null) { - TimedNode lasth = new TimedNode(null, 0L); + TimedNode lasth = new TimedNode<>(null, 0L); lasth.lazySet(h.get()); head = lasth; } else { @@ -1112,7 +1122,7 @@ void trimFinal() { if (next.time > limit) { if (h.value != null) { - TimedNode lasth = new TimedNode(null, 0L); + TimedNode lasth = new TimedNode<>(null, 0L); lasth.lazySet(h.get()); head = lasth; } else { @@ -1127,7 +1137,7 @@ void trimFinal() { @Override public void add(T value) { - TimedNode n = new TimedNode(value, scheduler.now(unit)); + TimedNode n = new TimedNode<>(value, scheduler.now(unit)); TimedNode t = tail; tail = n; @@ -1139,7 +1149,7 @@ public void add(T value) { @Override public void addFinal(Object notificationLite) { - TimedNode n = new TimedNode(notificationLite, Long.MAX_VALUE); + TimedNode n = new TimedNode<>(notificationLite, Long.MAX_VALUE); TimedNode t = tail; tail = n; @@ -1158,7 +1168,7 @@ public void addFinal(Object notificationLite) { public void trimHead() { TimedNode h = head; if (h.value != null) { - TimedNode n = new TimedNode(null, 0); + TimedNode n = new TimedNode<>(null, 0); n.lazySet(h.get()); head = n; } diff --git a/src/main/java/io/reactivex/rxjava3/subjects/SerializedSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/SerializedSubject.java index d70cf2f15c..137cefc895 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/SerializedSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/SerializedSubject.java @@ -60,7 +60,7 @@ public void onSubscribe(Disposable d) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(NotificationLite.disposable(d)); @@ -93,7 +93,7 @@ public void onNext(T t) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(NotificationLite.next(t)); @@ -120,7 +120,7 @@ public void onError(Throwable t) { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.setFirst(NotificationLite.error(t)); @@ -150,7 +150,7 @@ public void onComplete() { if (emitting) { AppendOnlyLinkedArrayList q = queue; if (q == null) { - q = new AppendOnlyLinkedArrayList(4); + q = new AppendOnlyLinkedArrayList<>(4); queue = q; } q.add(NotificationLite.complete()); diff --git a/src/main/java/io/reactivex/rxjava3/subjects/SingleSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/SingleSubject.java index fdd1c806f6..6f8f8b6f35 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/SingleSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/SingleSubject.java @@ -115,13 +115,13 @@ public final class SingleSubject extends Single implements SingleObserver< @CheckReturnValue @NonNull public static SingleSubject create() { - return new SingleSubject(); + return new SingleSubject<>(); } @SuppressWarnings("unchecked") SingleSubject() { once = new AtomicBoolean(); - observers = new AtomicReference[]>(EMPTY); + observers = new AtomicReference<>(EMPTY); } @Override @@ -159,7 +159,7 @@ public void onError(@NonNull Throwable e) { @Override protected void subscribeActual(@NonNull SingleObserver observer) { - SingleDisposable md = new SingleDisposable(observer, this); + SingleDisposable md = new SingleDisposable<>(observer, this); observer.onSubscribe(md); if (add(md)) { if (md.isDisposed()) { diff --git a/src/main/java/io/reactivex/rxjava3/subjects/Subject.java b/src/main/java/io/reactivex/rxjava3/subjects/Subject.java index 6a3b6462be..35d26d49b5 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/Subject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/Subject.java @@ -32,6 +32,7 @@ public abstract class Subject extends Observable implements Observer { *

The method is thread-safe. * @return true if the subject has any Observers */ + @CheckReturnValue public abstract boolean hasObservers(); /** @@ -41,6 +42,7 @@ public abstract class Subject extends Observable implements Observer { * @see #getThrowable() * @see #hasComplete() */ + @CheckReturnValue public abstract boolean hasThrowable(); /** @@ -49,6 +51,7 @@ public abstract class Subject extends Observable implements Observer { * @return true if the subject has reached a terminal state through a complete event * @see #hasThrowable() */ + @CheckReturnValue public abstract boolean hasComplete(); /** @@ -59,6 +62,7 @@ public abstract class Subject extends Observable implements Observer { * hasn't terminated yet */ @Nullable + @CheckReturnValue public abstract Throwable getThrowable(); /** @@ -68,10 +72,11 @@ public abstract class Subject extends Observable implements Observer { * @return the wrapped and serialized subject */ @NonNull + @CheckReturnValue public final Subject toSerialized() { if (this instanceof SerializedSubject) { return this; } - return new SerializedSubject(this); + return new SerializedSubject<>(this); } } diff --git a/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java index c522343aed..8f495a9ca5 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java @@ -20,7 +20,7 @@ import io.reactivex.rxjava3.core.Observer; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; -import io.reactivex.rxjava3.internal.functions.ObjectHelper; +import io.reactivex.rxjava3.internal.functions.*; import io.reactivex.rxjava3.internal.fuseable.SimpleQueue; import io.reactivex.rxjava3.internal.observers.BasicIntQueueDisposable; import io.reactivex.rxjava3.internal.queue.SpscLinkedArrayQueue; @@ -180,7 +180,7 @@ public final class UnicastSubject extends Subject { @CheckReturnValue @NonNull public static UnicastSubject create() { - return new UnicastSubject(bufferSize(), true); + return create(bufferSize(), Functions.EMPTY_RUNNABLE, true); } /** @@ -192,12 +192,13 @@ public static UnicastSubject create() { @CheckReturnValue @NonNull public static UnicastSubject create(int capacityHint) { - return new UnicastSubject(capacityHint, true); + return create(capacityHint, Functions.EMPTY_RUNNABLE, true); } /** * Creates an UnicastSubject with the given internal buffer capacity hint and a callback for - * the case when the single Subscriber cancels its subscription. + * the case when the single Subscriber cancels its subscription + * or the subject is terminated. * *

The callback, if not null, is called exactly once and * non-overlapped with any active replay. @@ -209,13 +210,14 @@ public static UnicastSubject create(int capacityHint) { */ @CheckReturnValue @NonNull - public static UnicastSubject create(int capacityHint, Runnable onTerminate) { - return new UnicastSubject(capacityHint, onTerminate, true); + public static UnicastSubject create(int capacityHint, @NonNull Runnable onTerminate) { + return create(capacityHint, onTerminate, true); } /** * Creates an UnicastSubject with the given internal buffer capacity hint, delay error flag and - * a callback for the case when the single Subscriber cancels its subscription. + * a callback for the case when the single Observer disposes its {@link Disposable} + * or the subject is terminated. * *

The callback, if not null, is called exactly once and * non-overlapped with any active replay. @@ -229,8 +231,8 @@ public static UnicastSubject create(int capacityHint, Runnable onTerminat */ @CheckReturnValue @NonNull - public static UnicastSubject create(int capacityHint, Runnable onTerminate, boolean delayError) { - return new UnicastSubject(capacityHint, onTerminate, delayError); + public static UnicastSubject create(int capacityHint, @NonNull Runnable onTerminate, boolean delayError) { + return new UnicastSubject<>(capacityHint, onTerminate, delayError); } /** @@ -247,35 +249,7 @@ public static UnicastSubject create(int capacityHint, Runnable onTerminat @CheckReturnValue @NonNull public static UnicastSubject create(boolean delayError) { - return new UnicastSubject(bufferSize(), delayError); - } - - /** - * Creates an UnicastSubject with the given capacity hint and delay error flag. - *

History: 2.0.8 - experimental - * @param capacityHint the capacity hint for the internal, unbounded queue - * @param delayError deliver pending onNext events before onError - * @since 2.2 - */ - UnicastSubject(int capacityHint, boolean delayError) { - this.queue = new SpscLinkedArrayQueue(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); - this.onTerminate = new AtomicReference(); - this.delayError = delayError; - this.downstream = new AtomicReference>(); - this.once = new AtomicBoolean(); - this.wip = new UnicastQueueDisposable(); - } - - /** - * Creates an UnicastSubject with the given capacity hint and callback - * for when the Subject is terminated normally or its single Subscriber cancels. - * @param capacityHint the capacity hint for the internal, unbounded queue - * @param onTerminate the callback to run when the Subject is terminated or cancelled, null not allowed - * @since 2.0 - * - * */ - UnicastSubject(int capacityHint, Runnable onTerminate) { - this(capacityHint, onTerminate, true); + return create(bufferSize(), Functions.EMPTY_RUNNABLE, delayError); } /** @@ -288,10 +262,10 @@ public static UnicastSubject create(boolean delayError) { * @since 2.2 */ UnicastSubject(int capacityHint, Runnable onTerminate, boolean delayError) { - this.queue = new SpscLinkedArrayQueue(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); - this.onTerminate = new AtomicReference(Objects.requireNonNull(onTerminate, "onTerminate")); + this.queue = new SpscLinkedArrayQueue<>(ObjectHelper.verifyPositive(capacityHint, "capacityHint")); + this.onTerminate = new AtomicReference<>(Objects.requireNonNull(onTerminate, "onTerminate")); this.delayError = delayError; - this.downstream = new AtomicReference>(); + this.downstream = new AtomicReference<>(); this.once = new AtomicBoolean(); this.wip = new UnicastQueueDisposable(); } @@ -494,12 +468,14 @@ void drain() { } @Override + @CheckReturnValue public boolean hasObservers() { return downstream.get() != null; } @Override @Nullable + @CheckReturnValue public Throwable getThrowable() { if (done) { return error; @@ -508,11 +484,13 @@ public Throwable getThrowable() { } @Override + @CheckReturnValue public boolean hasThrowable() { return done && error != null; } @Override + @CheckReturnValue public boolean hasComplete() { return done && error == null; } diff --git a/src/test/java/io/reactivex/rxjava3/core/ConverterTest.java b/src/test/java/io/reactivex/rxjava3/core/ConverterTest.java index 7a39c85cfd..68006e3c64 100644 --- a/src/test/java/io/reactivex/rxjava3/core/ConverterTest.java +++ b/src/test/java/io/reactivex/rxjava3/core/ConverterTest.java @@ -175,7 +175,17 @@ public void compositeTest() { .assertValue(1); } + /** + * Two argument type. + * @param the input type + * @param the output type + */ interface A { } + + /** + * One argument type. + * @param the type + */ interface B { } private static ObservableConverter, B> testObservableConverterCreator() { diff --git a/src/test/java/io/reactivex/rxjava3/core/TransformerTest.java b/src/test/java/io/reactivex/rxjava3/core/TransformerTest.java index 8f9f102a04..20725c80fa 100644 --- a/src/test/java/io/reactivex/rxjava3/core/TransformerTest.java +++ b/src/test/java/io/reactivex/rxjava3/core/TransformerTest.java @@ -18,6 +18,7 @@ import org.junit.Test; import org.reactivestreams.Publisher; +import io.reactivex.rxjava3.core.ConverterTest.*; import io.reactivex.rxjava3.exceptions.TestException; public class TransformerTest extends RxJavaTest { @@ -127,9 +128,6 @@ public void flowableGenericsSignatureTest() { Flowable.just(a).compose(TransformerTest.testFlowableTransformerCreator()); } - interface A { } - interface B { } - private static ObservableTransformer, B> testObservableTransformerCreator() { return new ObservableTransformer, B>() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java b/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java index c5b75a9b39..a579a82ae7 100644 --- a/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java +++ b/src/test/java/io/reactivex/rxjava3/flowable/FlowableConversionTest.java @@ -59,7 +59,7 @@ public CylonDetectorObservable lift(FlowableOperator(operator)); } - public O x(Function, O> operator) { + public O x(Function, O> operator) { try { return operator.apply(onSubscribe); } catch (Throwable ex) { diff --git a/src/test/java/io/reactivex/rxjava3/internal/functions/FunctionsTest.java b/src/test/java/io/reactivex/rxjava3/internal/functions/FunctionsTest.java index 98488bcd49..72e188fe99 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/functions/FunctionsTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/functions/FunctionsTest.java @@ -173,62 +173,6 @@ public Integer apply(Integer t1, Integer t2, Integer t3, Integer t4, Integer t5, }).apply(new Object[20]); } - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void biFunctionFail() throws Exception { - BiFunction biFunction = null; - Functions.toFunction(biFunction); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void function3Fail() throws Exception { - Function3 function3 = null; - Functions.toFunction(function3); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void function4Fail() throws Exception { - Function4 function4 = null; - Functions.toFunction(function4); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void function5Fail() throws Exception { - Function5 function5 = null; - Functions.toFunction(function5); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void function6Fail() throws Exception { - Function6 function6 = null; - Functions.toFunction(function6); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void function7Fail() throws Exception { - Function7 function7 = null; - Functions.toFunction(function7); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void function8Fail() throws Exception { - Function8 function8 = null; - Functions.toFunction(function8); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test(expected = NullPointerException.class) - public void function9Fail() throws Exception { - Function9 function9 = null; - Functions.toFunction(function9); - } - @Test public void identityFunctionToString() { assertEquals("IdentityFunction", Functions.identity().toString()); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java index c8e041b951..6829c99986 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatestTest.java @@ -1046,7 +1046,7 @@ public void combineLatestNArguments() throws Exception { if (j < i) { assertEquals("source" + (j + 1) + " is null", ex.getCause().getMessage()); } else { - assertEquals("f is null", ex.getCause().getMessage()); + assertEquals("combiner is null", ex.getCause().getMessage()); } } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java index 4de1cfb8b0..607801bd19 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryTest.java @@ -73,7 +73,7 @@ public Flowable apply(Flowable attempts) { .map(new Function() { @Override public Tuple apply(Throwable n) { - return new Tuple(new Long(1), n); + return new Tuple(1L, n); }}) .scan(new BiFunction() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java index e4b41c8bea..614dc28aaa 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipTest.java @@ -1297,7 +1297,7 @@ public void zipNArguments() throws Exception { if (j < i) { assertEquals("source" + (j + 1) + " is null", ex.getCause().getMessage()); } else { - assertEquals("f is null", ex.getCause().getMessage()); + assertEquals("zipper is null", ex.getCause().getMessage()); } } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java index 6a88930b75..ab4ba0273f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEagerTest.java @@ -893,7 +893,6 @@ public void arrayDelayErrorMaxConcurrency() { PublishSubject ps2 = PublishSubject.create(); PublishSubject ps3 = PublishSubject.create(); - @SuppressWarnings("unchecked") TestObserver to = Observable.concatArrayEagerDelayError(2, 2, ps1, ps2, ps3) .test(); @@ -929,7 +928,6 @@ public void arrayDelayErrorMaxConcurrencyErrorDelayed() { PublishSubject ps2 = PublishSubject.create(); PublishSubject ps3 = PublishSubject.create(); - @SuppressWarnings("unchecked") TestObserver to = Observable.concatArrayEagerDelayError(2, 2, ps1, ps2, ps3) .test(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java index 05df845a44..b1c42a1175 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatTest.java @@ -780,7 +780,6 @@ public void concat4() { .assertResult(1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatArrayDelayError() { Observable.concatArrayDelayError(Observable.just(1), Observable.just(2), @@ -789,7 +788,6 @@ public void concatArrayDelayError() { .assertResult(1, 2, 3, 4); } - @SuppressWarnings("unchecked") @Test public void concatArrayDelayErrorWithError() { Observable.concatArrayDelayError(Observable.just(1), Observable.just(2), @@ -886,13 +884,11 @@ public Iterable apply(Integer v) throws Exception { .assertResult(1, 2, 3, 4, 5, 1, 2, 3, 4, 5); } - @SuppressWarnings("unchecked") @Test public void emptyArray() { assertSame(Observable.empty(), Observable.concatArrayDelayError()); } - @SuppressWarnings("unchecked") @Test public void singleElementArray() { assertSame(Observable.never(), Observable.concatArrayDelayError(Observable.never())); @@ -923,13 +919,11 @@ public ObservableSource apply(Object v) throws Exception { } - @SuppressWarnings("unchecked") @Test public void concatArrayEmpty() { assertSame(Observable.empty(), Observable.concatArray()); } - @SuppressWarnings("unchecked") @Test public void concatArraySingleElement() { assertSame(Observable.never(), Observable.concatArray(Observable.never())); @@ -960,7 +954,6 @@ public ObservableSource apply(Object v) throws Exception { } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscription() { final int[] calls = { 0 }; @@ -981,7 +974,6 @@ public void subscribe(ObservableEmitter s) throws Exception { assertEquals(1, calls[0]); } - @SuppressWarnings("unchecked") @Test public void noSubsequentSubscriptionDelayError() { final int[] calls = { 0 }; @@ -1071,7 +1063,6 @@ public void onComplete() { } @Test - @SuppressWarnings("unchecked") public void concatReportsDisposedOnCompleteDelayError() { final Disposable[] disposable = { null }; @@ -1128,7 +1119,6 @@ public void onComplete() { } @Test - @SuppressWarnings("unchecked") public void concatReportsDisposedOnErrorDelayError() { final Disposable[] disposable = { null }; @@ -1156,7 +1146,6 @@ public void onComplete() { assertTrue(disposable[0].isDisposed()); } - @SuppressWarnings("unchecked") @Test public void noCancelPreviousArray() { final AtomicInteger counter = new AtomicInteger(); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java index c4d24a69c7..fefce52867 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeDelayErrorTest.java @@ -506,7 +506,6 @@ public void mergeIterableDelayError() { .assertResult(1, 2); } - @SuppressWarnings("unchecked") @Test public void mergeArrayDelayError() { Observable.mergeArrayDelayError(Observable.just(1), Observable.just(2)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java index 92d30f8a85..d5b9a45a6f 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeTest.java @@ -1065,7 +1065,6 @@ public void onNext(Integer t) { } } - @SuppressWarnings("unchecked") @Test public void mergeArray2() { Observable.mergeArray(Observable.just(1), Observable.just(2)) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java index 381ee0a790..94009c5b86 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryTest.java @@ -74,7 +74,7 @@ public Observable apply(Observable attempts) { .map(new Function() { @Override public Tuple apply(Throwable n) { - return new Tuple(new Long(1), n); + return new Tuple(1L, n); }}) .scan(new BiFunction() { @Override diff --git a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java index d222f92392..b5a03ede3b 100644 --- a/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java +++ b/src/test/java/io/reactivex/rxjava3/maybe/MaybeTest.java @@ -2662,7 +2662,7 @@ public void mergeDelayError4() { @Test public void sequenceEqual() { - Maybe.sequenceEqual(Maybe.just(1), Maybe.just(new Integer(1))).test().assertResult(true); + Maybe.sequenceEqual(Maybe.just(1_000_000), Maybe.just(Integer.valueOf(1_000_000))).test().assertResult(true); Maybe.sequenceEqual(Maybe.just(1), Maybe.just(2)).test().assertResult(false); diff --git a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java index 074057a8d3..0fa7c22711 100644 --- a/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java +++ b/src/test/java/io/reactivex/rxjava3/observable/ObservableNullTests.java @@ -202,7 +202,6 @@ public void concatArrayNull() { Observable.concatArray((Observable[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void concatArrayOneIsNull() { Observable.concatArray(just1, null).blockingLast(); @@ -521,7 +520,6 @@ public void mergeArrayNull() { Observable.mergeArray(128, 128, (Observable[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeArrayOneIsNull() { Observable.mergeArray(128, 128, just1, null).blockingLast(); @@ -552,7 +550,6 @@ public void mergeDelayErrorArrayNull() { Observable.mergeArrayDelayError(128, 128, (Observable[])null); } - @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void mergeDelayErrorArrayOneIsNull() { Observable.mergeArrayDelayError(128, 128, just1, null).blockingLast(); diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java index 93ffe8de7d..1d1f97163f 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlowableTest.java @@ -306,7 +306,6 @@ public Iterable apply(List v) throws Exception { TestHelper.assertValueSet(ts, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); } - @SuppressWarnings("unchecked") @Test public void from() { TestSubscriberEx ts = new TestSubscriberEx<>(); diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceFullTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceFullTest.java index 56595583bb..25d72af909 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceFullTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelReduceFullTest.java @@ -73,7 +73,6 @@ public Integer apply(Integer a, Integer b) throws Exception { } } - @SuppressWarnings("unchecked") @Test public void error2() { List errors = TestHelper.trackPluginErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelSortedJoinTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelSortedJoinTest.java index d2cc8aaec0..e7b5cda9b0 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelSortedJoinTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelSortedJoinTest.java @@ -82,7 +82,6 @@ public void error3() { } } - @SuppressWarnings("unchecked") @Test public void error2() { List errors = TestHelper.trackPluginErrors(); diff --git a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java index 9c9301176c..17e4cb0a72 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java @@ -368,7 +368,7 @@ public void dispose() { public void run() { calls[0]++; } - }); + }, true); TestHelper.checkDisposed(us); diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index 9bfb21668e..eabbb6f96d 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -3257,10 +3257,23 @@ public static TestObserverEx assertValueSet(TestObserverEx to, T... va * @throws Exception on error */ public static File findSource(String baseClassName) throws Exception { + return findSource(baseClassName, "io.reactivex.rxjava3.core"); + } + + /** + * Given a base reactive type name, try to find its source in the current runtime + * path and return a file to it or null if not found. + * @param baseClassName the class name such as {@code Maybe} + * @param parentPackage the parent package such as {@code io.reactivex.rxjava3.core} + * @return the File pointing to the source + * @throws Exception on error + */ + public static File findSource(String baseClassName, String parentPackage) throws Exception { URL u = TestHelper.class.getResource(TestHelper.class.getSimpleName() + ".class"); String path = new File(u.toURI()).toString().replace('\\', '/'); + parentPackage = parentPackage.replace(".", "/"); // System.out.println(path); int i = path.toLowerCase().indexOf("/rxjava"); @@ -3272,16 +3285,18 @@ public static File findSource(String baseClassName) throws Exception { // find end of any potential postfix to /RxJava int j = path.indexOf("/", i + 6); - String p = path.substring(0, j + 1) + "src/main/java/io/reactivex/rxjava3/core/" + baseClassName + ".java"; + String basePackage = path.substring(0, j + 1) + "src/main/java"; + + String p = basePackage + "/" + parentPackage + "/" + baseClassName + ".java"; File f = new File(p); - if (!f.canRead()) { - System.out.println("Can't read " + p); - return null; + if (f.canRead()) { + return f; } - return f; + System.out.println("Can't read " + p); + return null; } /** diff --git a/src/test/java/io/reactivex/rxjava3/validators/BaseTypeAnnotations.java b/src/test/java/io/reactivex/rxjava3/validators/BaseTypeAnnotations.java index 36457ffd86..504232be2e 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/BaseTypeAnnotations.java +++ b/src/test/java/io/reactivex/rxjava3/validators/BaseTypeAnnotations.java @@ -22,6 +22,11 @@ import io.reactivex.rxjava3.annotations.*; import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.flowables.ConnectableFlowable; +import io.reactivex.rxjava3.observables.ConnectableObservable; +import io.reactivex.rxjava3.parallel.ParallelFlowable; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.subjects.*; /** * Verifies several properties. @@ -38,16 +43,14 @@ static void checkCheckReturnValueSupport(Class clazz) { StringBuilder b = new StringBuilder(); for (Method m : clazz.getMethods()) { - if (m.getName().equals("bufferSize")) { - continue; - } if (m.getDeclaringClass() == clazz) { boolean isSubscribeMethod = "subscribe".equals(m.getName()) && m.getParameterTypes().length == 0; + boolean isConnectMethod = "connect".equals(m.getName()) && m.getParameterTypes().length == 0; boolean isAnnotationPresent = m.isAnnotationPresent(CheckReturnValue.class); - if (isSubscribeMethod) { + if (isSubscribeMethod || isConnectMethod) { if (isAnnotationPresent) { - b.append("subscribe() method has @CheckReturnValue: ").append(m).append("\r\n"); + b.append(m.getName()).append(" method has @CheckReturnValue: ").append(m).append("\r\n"); } continue; } @@ -83,7 +86,8 @@ static void checkSchedulerSupport(Class clazz) { StringBuilder b = new StringBuilder(); for (Method m : clazz.getMethods()) { - if (m.getName().equals("bufferSize")) { + if (m.getName().equals("bufferSize") + || m.getName().equals("parallelism")) { continue; } if (m.getDeclaringClass() == clazz) { @@ -130,18 +134,24 @@ static void checkBackpressureSupport(Class clazz) { StringBuilder b = new StringBuilder(); for (Method m : clazz.getMethods()) { - if (m.getName().equals("bufferSize")) { + if (m.getName().equals("bufferSize") + || m.getName().equals("parallelism")) { continue; } if (m.getDeclaringClass() == clazz) { - if (clazz == Flowable.class) { + if (clazz == Flowable.class || clazz == ParallelFlowable.class) { if (!m.isAnnotationPresent(BackpressureSupport.class)) { - b.append("No @BackpressureSupport annotation (being Flowable): ").append(m).append("\r\n"); + b.append("No @BackpressureSupport annotation (being ") + .append(clazz.getSimpleName()) + .append("): ").append(m).append("\r\n"); } } else { - if (m.getReturnType() == Flowable.class) { + if (m.getReturnType() == Flowable.class + || m.getReturnType() == ParallelFlowable.class) { if (!m.isAnnotationPresent(BackpressureSupport.class)) { - b.append("No @BackpressureSupport annotation (having Flowable return): ").append(m).append("\r\n"); + b.append("No @BackpressureSupport annotation (having ") + .append(m.getReturnType().getSimpleName()) + .append(" return): ").append(m).append("\r\n"); } } else { boolean found = false; @@ -200,6 +210,86 @@ public void checkReturnValueMaybe() { checkCheckReturnValueSupport(Maybe.class); } + @Test + public void checkReturnValueConnectableObservable() { + checkCheckReturnValueSupport(ConnectableObservable.class); + } + + @Test + public void checkReturnValueConnectableFlowable() { + checkCheckReturnValueSupport(ConnectableFlowable.class); + } + + @Test + public void checkReturnValueParallelFlowable() { + checkCheckReturnValueSupport(ParallelFlowable.class); + } + + @Test + public void checkReturnValueAsyncSubject() { + checkCheckReturnValueSupport(AsyncSubject.class); + } + + @Test + public void checkReturnValueBehaviorSubject() { + checkCheckReturnValueSupport(BehaviorSubject.class); + } + + @Test + public void checkReturnValuePublishSubject() { + checkCheckReturnValueSupport(PublishSubject.class); + } + + @Test + public void checkReturnValueReplaySubject() { + checkCheckReturnValueSupport(ReplaySubject.class); + } + + @Test + public void checkReturnValueUnicastSubject() { + checkCheckReturnValueSupport(UnicastSubject.class); + } + + @Test + public void checkReturnValueAsyncProcessor() { + checkCheckReturnValueSupport(AsyncProcessor.class); + } + + @Test + public void checkReturnValueBehaviorProcessor() { + checkCheckReturnValueSupport(BehaviorProcessor.class); + } + + @Test + public void checkReturnValuePublishProcessor() { + checkCheckReturnValueSupport(PublishProcessor.class); + } + + @Test + public void checkReturnValueReplayProcessor() { + checkCheckReturnValueSupport(ReplayProcessor.class); + } + + @Test + public void checkReturnValueUnicastProcessor() { + checkCheckReturnValueSupport(UnicastProcessor.class); + } + + @Test + public void checkReturnValueMulticastProcessor() { + checkCheckReturnValueSupport(MulticastProcessor.class); + } + + @Test + public void checkReturnValueSubject() { + checkCheckReturnValueSupport(Subject.class); + } + + @Test + public void checkReturnValueFlowableProcessor() { + checkCheckReturnValueSupport(FlowableProcessor.class); + } + @Test public void schedulerSupportFlowable() { checkSchedulerSupport(Flowable.class); @@ -225,6 +315,21 @@ public void schedulerSupportMaybe() { checkSchedulerSupport(Maybe.class); } + @Test + public void schedulerSupportConnectableObservable() { + checkSchedulerSupport(ConnectableObservable.class); + } + + @Test + public void schedulerSupportConnectableFlowable() { + checkSchedulerSupport(ConnectableFlowable.class); + } + + @Test + public void schedulerSupportParallelFlowable() { + checkSchedulerSupport(ParallelFlowable.class); + } + @Test public void backpressureSupportFlowable() { checkBackpressureSupport(Flowable.class); @@ -249,4 +354,19 @@ public void backpressureSupportCompletable() { public void backpressureSupportMaybe() { checkBackpressureSupport(Maybe.class); } + + @Test + public void backpressureSupportConnectableFlowable() { + checkBackpressureSupport(ConnectableFlowable.class); + } + + @Test + public void backpressureSupportConnectableObservable() { + checkBackpressureSupport(ConnectableObservable.class); + } + + @Test + public void backpressureSupportParallelFlowable() { + checkBackpressureSupport(ParallelFlowable.class); + } } diff --git a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationNaming.java b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationNaming.java new file mode 100644 index 0000000000..6e66d4249e --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationNaming.java @@ -0,0 +1,235 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.validators; + +import java.io.File; +import java.nio.file.Files; +import java.util.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.flowables.ConnectableFlowable; +import io.reactivex.rxjava3.observables.ConnectableObservable; +import io.reactivex.rxjava3.parallel.ParallelFlowable; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.schedulers.Schedulers; +import io.reactivex.rxjava3.subjects.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +/** + * Check if the parameter name in Objects.requireNonNull + * and ObjectHelper.verifyPositive calls match the parameter + * name in the message. + */ +public class ParamValidationNaming { + + @Test + public void checkCompletable() throws Exception { + processFile(Completable.class); + } + + @Test + public void checkSingle() throws Exception { + processFile(Single.class); + } + + @Test + public void checkMaybe() throws Exception { + processFile(Maybe.class); + } + + @Test + public void checkObservable() throws Exception { + processFile(Observable.class); + } + + @Test + public void checkFlowable() throws Exception { + processFile(Flowable.class); + } + + @Test + public void checkParallelFlowable() throws Exception { + processFile(ParallelFlowable.class); + } + + @Test + public void checkConnectableObservable() throws Exception { + processFile(ConnectableObservable.class); + } + + @Test + public void checkConnectableFlowable() throws Exception { + processFile(ConnectableFlowable.class); + } + + @Test + public void checkSubject() throws Exception { + processFile(Subject.class); + } + + @Test + public void checkFlowableProcessor() throws Exception { + processFile(FlowableProcessor.class); + } + + @Test + public void checkDisposable() throws Exception { + processFile(Disposable.class); + } + + @Test + public void checkScheduler() throws Exception { + processFile(Scheduler.class); + } + + @Test + public void checkSchedulers() throws Exception { + processFile(Schedulers.class); + } + + @Test + public void checkAsyncSubject() throws Exception { + processFile(AsyncSubject.class); + } + + @Test + public void checkBehaviorSubject() throws Exception { + processFile(BehaviorSubject.class); + } + + @Test + public void checkPublishSubject() throws Exception { + processFile(PublishSubject.class); + } + + @Test + public void checkReplaySubject() throws Exception { + processFile(ReplaySubject.class); + } + + @Test + public void checkUnicastSubject() throws Exception { + processFile(UnicastSubject.class); + } + + @Test + public void checkSingleSubject() throws Exception { + processFile(SingleSubject.class); + } + + @Test + public void checkMaybeSubject() throws Exception { + processFile(MaybeSubject.class); + } + + @Test + public void checkCompletableSubject() throws Exception { + processFile(CompletableSubject.class); + } + + @Test + public void checkAsyncProcessor() throws Exception { + processFile(AsyncProcessor.class); + } + + @Test + public void checkBehaviorProcessor() throws Exception { + processFile(BehaviorProcessor.class); + } + + @Test + public void checkPublishProcessor() throws Exception { + processFile(PublishProcessor.class); + } + + @Test + public void checkReplayProcessor() throws Exception { + processFile(ReplayProcessor.class); + } + + @Test + public void checkUnicastProcessor() throws Exception { + processFile(UnicastProcessor.class); + } + + @Test + public void checkMulticastProcessor() throws Exception { + processFile(MulticastProcessor.class); + } + + static void processFile(Class clazz) throws Exception { + String baseClassName = clazz.getSimpleName(); + File f = TestHelper.findSource(baseClassName, clazz.getPackage().getName()); + if (f == null) { + return; + } + String fullClassName = clazz.getName(); + + int errorCount = 0; + StringBuilder errors = new StringBuilder(); + + List lines = Files.readAllLines(f.toPath()); + + for (int j = 0; j < lines.size(); j++) { + String line = lines.get(j).trim(); + + for (String validatorStr : VALIDATOR_STRINGS) + if (line.startsWith(validatorStr)) { + + int comma = line.indexOf(','); + + String paramName = line.substring(validatorStr.length(), comma); + + int quote = line.indexOf('"', comma); + + String message = line.substring(quote + 1, quote + 2 + paramName.length()); + + if (!line.contains("The RxJavaPlugins") + && !(message.startsWith(paramName) + && (message.endsWith(" ") || message.endsWith("\"")))) { + errorCount++; + errors.append("L") + .append(j) + .append(" : Wrong validator message parameter name\r\n ") + .append(line) + .append("\r\n") + .append(" ").append(paramName).append(" != ").append(message) + .append("\r\n at ") + .append(fullClassName) + .append(".method(") + .append(f.getName()) + .append(":") + .append(j + 1) + .append(")\r\n") + ; + } + } + } + + if (errorCount != 0) { + errors.insert(0, errorCount + " problems\r\n"); + errors.setLength(errors.length() - 2); + throw new AssertionError(errors.toString()); + } + } + + static final List VALIDATOR_STRINGS = Arrays.asList( + "Objects.requireNonNull(", + "ObjectHelper.requirePositive(" + ); +} diff --git a/src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java b/src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java index 8cde4fac7e..6fb50a141b 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java +++ b/src/test/java/io/reactivex/rxjava3/validators/SourceAnnotationCheck.java @@ -21,7 +21,13 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.flowables.ConnectableFlowable; +import io.reactivex.rxjava3.observables.ConnectableObservable; import io.reactivex.rxjava3.parallel.ParallelFlowable; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.schedulers.Schedulers; +import io.reactivex.rxjava3.subjects.*; import io.reactivex.rxjava3.testsupport.TestHelper; /** @@ -46,8 +52,7 @@ public void checkMaybe() throws Exception { processFile(Maybe.class); } - // TODO later - // @Test + @Test public void checkObservable() throws Exception { processFile(Observable.class); } @@ -57,15 +62,119 @@ public void checkFlowable() throws Exception { processFile(Flowable.class); } - // TODO later - // @Test + @Test public void checkParallelFlowable() throws Exception { processFile(ParallelFlowable.class); } + @Test + public void checkConnectableObservable() throws Exception { + processFile(ConnectableObservable.class); + } + + @Test + public void checkConnectableFlowable() throws Exception { + processFile(ConnectableFlowable.class); + } + + @Test + public void checkSubject() throws Exception { + processFile(Subject.class); + } + + @Test + public void checkFlowableProcessor() throws Exception { + processFile(FlowableProcessor.class); + } + + @Test + public void checkDisposable() throws Exception { + processFile(Disposable.class); + } + + @Test + public void checkScheduler() throws Exception { + processFile(Scheduler.class); + } + + @Test + public void checkSchedulers() throws Exception { + processFile(Schedulers.class); + } + + @Test + public void checkAsyncSubject() throws Exception { + processFile(AsyncSubject.class); + } + + @Test + public void checkBehaviorSubject() throws Exception { + processFile(BehaviorSubject.class); + } + + @Test + public void checkPublishSubject() throws Exception { + processFile(PublishSubject.class); + } + + @Test + public void checkReplaySubject() throws Exception { + processFile(ReplaySubject.class); + } + + @Test + public void checkUnicastSubject() throws Exception { + processFile(UnicastSubject.class); + } + + @Test + public void checkSingleSubject() throws Exception { + processFile(SingleSubject.class); + } + + @Test + public void checkMaybeSubject() throws Exception { + processFile(MaybeSubject.class); + } + + @Test + public void checkCompletableSubject() throws Exception { + processFile(CompletableSubject.class); + } + + @Test + public void checkAsyncProcessor() throws Exception { + processFile(AsyncProcessor.class); + } + + @Test + public void checkBehaviorProcessor() throws Exception { + processFile(BehaviorProcessor.class); + } + + @Test + public void checkPublishProcessor() throws Exception { + processFile(PublishProcessor.class); + } + + @Test + public void checkReplayProcessor() throws Exception { + processFile(ReplayProcessor.class); + } + + @Test + public void checkUnicastProcessor() throws Exception { + processFile(UnicastProcessor.class); + } + + @Test + public void checkMulticastProcessor() throws Exception { + processFile(MulticastProcessor.class); + } + static void processFile(Class clazz) throws Exception { String baseClassName = clazz.getSimpleName(); - File f = TestHelper.findSource(baseClassName); + File f = TestHelper.findSource(baseClassName, clazz.getPackage().getName()); if (f == null) { return; } @@ -79,7 +188,14 @@ static void processFile(Class clazz) throws Exception { for (int j = 0; j < lines.size(); j++) { String line = lines.get(j).trim(); - if (line.startsWith("public static") || line.startsWith("public final")) { + if (line.contains("class")) { + continue; + } + if (line.startsWith("public static") + || line.startsWith("public final") + || line.startsWith("protected final") + || line.startsWith("protected abstract") + || line.startsWith("public abstract")) { int methodArgStart = line.indexOf("("); int isBoolean = line.indexOf(" boolean "); @@ -168,10 +284,12 @@ else if (skippingDepth == 0) { String strippedArgumentsStr = strippedArguments.toString(); String[] args = strippedArgumentsStr.split("\\s*,\\s*"); - for (String typeName : CLASS_NAMES) { - String typeNameSpaced = typeName + " "; - for (int k = 0; k < args.length; k++) { - String typeDef = args[k]; + for (int k = 0; k < args.length; k++) { + String typeDef = args[k]; + + for (String typeName : CLASS_NAMES) { + String typeNameSpaced = typeName + " "; + if (typeDef.contains(typeNameSpaced) && !typeDef.contains("@NonNull") && !typeDef.contains("@Nullable")) { @@ -181,8 +299,9 @@ else if (skippingDepth == 0) { errorCount++; errors.append("L") .append(j) - .append(" - argument ").append(k + 1).append(" - ").append(typeDef) - .append(" : Missing argument type nullability annotation |\r\n ") + .append(" - argument ").append(k + 1) + .append(" : Missing argument type nullability annotation\r\n ") + .append(typeDef).append("\r\n ") .append(strippedArgumentsStr) .append("\r\n") .append(" at ") @@ -196,13 +315,55 @@ else if (skippingDepth == 0) { } } } + + if (typeDef.contains("final ")) { + errorCount++; + errors.append("L") + .append(j) + .append(" - argument ").append(k + 1) + .append(" : unnecessary final on argument\r\n ") + .append(typeDef).append("\r\n ") + .append(strippedArgumentsStr) + .append("\r\n") + .append(" at ") + .append(fullClassName) + .append(".method(") + .append(f.getName()) + .append(":") + .append(j + 1) + .append(")\r\n") + ; + } + if (typeDef.contains("@NonNull int") + || typeDef.contains("@NonNull long") + || typeDef.contains("@Nullable int") + || typeDef.contains("@Nullable long") + ) { + errorCount++; + errors.append("L") + .append(j) + .append(" - argument ").append(k + 1) + .append(" : unnecessary nullability annotation\r\n ") + .append(typeDef).append("\r\n ") + .append(strippedArgumentsStr) + .append("\r\n") + .append(" at ") + .append(fullClassName) + .append(".method(") + .append(f.getName()) + .append(":") + .append(j + 1) + .append(")\r\n") + ; + } + } if (strippedArgumentsStr.contains("...") && !hasSafeVarargsAnnotation) { errorCount++; errors.append("L") .append(j) - .append(" : Missing @SafeVarargs annotation |\r\n ") + .append(" : Missing @SafeVarargs annotation\r\n ") .append(strippedArgumentsStr) .append("\r\n") .append(" at ") @@ -218,7 +379,7 @@ else if (skippingDepth == 0) { } if (errorCount != 0) { - errors.insert(0, errorCount + " missing annotations\r\n"); + errors.insert(0, errorCount + " problems\r\n"); errors.setLength(errors.length() - 2); throw new AssertionError(errors.toString()); } @@ -249,6 +410,8 @@ else if (skippingDepth == 0) { "Throwable", "Optional", "CompletionStage", "BooleanSupplier", "LongConsumer", "Predicate", "BiPredicate", "Object", + "Iterable", "Stream", "Iterator", + "BackpressureOverflowStrategy", "BackpressureStrategy", "Subject", "Processor", "FlowableProcessor", From af17c6e268d0e2fe4675feb46c19a29227cb135b Mon Sep 17 00:00:00 2001 From: David Karnok Date: Thu, 26 Dec 2019 13:33:30 +0100 Subject: [PATCH 024/665] 3.x: [Java 8] Add Observable operators + cleanup (#6797) --- .../io/reactivex/rxjava3/core/Observable.java | 509 +++++++++++++++++- .../internal/jdk8/FlowableFromStream.java | 14 +- .../jdk8/ObservableCollectWithCollector.java | 151 ++++++ .../ObservableCollectWithCollectorSingle.java | 159 ++++++ .../jdk8/ObservableFirstStageObserver.java | 51 ++ .../jdk8/ObservableFlatMapStream.java | 162 ++++++ .../jdk8/ObservableFromCompletionStage.java | 92 ++++ .../internal/jdk8/ObservableFromStream.java | 219 ++++++++ .../jdk8/ObservableLastStageObserver.java | 54 ++ .../internal/jdk8/ObservableMapOptional.java | 96 ++++ .../jdk8/ObservableSingleStageObserver.java | 60 +++ .../jdk8/ObservableStageObserver.java | 75 +++ .../internal/jdk8/FlowableFromStreamTest.java | 42 ++ .../jdk8/ObservableBlockingStreamTest.java | 107 ++++ .../ObservableCollectWithCollectorTest.java | 444 +++++++++++++++ .../jdk8/ObservableFlatMapStreamTest.java | 466 ++++++++++++++++ .../ObservableFromCompletionStageTest.java | 65 +++ .../jdk8/ObservableFromOptionalTest.java | 38 ++ .../jdk8/ObservableFromStreamTest.java | 488 +++++++++++++++++ .../jdk8/ObservableMapOptionalTest.java | 394 ++++++++++++++ ...bservableStageSubscriberOrDefaultTest.java | 475 ++++++++++++++++ .../ObservableStageSubscriberOrErrorTest.java | 469 ++++++++++++++++ .../flowable/FlowableConcatMapEagerTest.java | 16 +- .../flowable/FlowableDistinctTest.java | 6 +- .../flowable/FlowableFilterTest.java | 6 +- .../operators/flowable/FlowableMapTest.java | 6 +- .../flowable/FlowableObserveOnTest.java | 38 +- .../ObservableDistinctUntilChangedTest.java | 6 +- .../observable/ObservableDoAfterNextTest.java | 12 +- .../observable/ObservableDoFinallyTest.java | 24 +- .../observable/ObservableDoOnEachTest.java | 18 +- .../observable/ObservableObserveOnTest.java | 8 +- .../processors/UnicastProcessorTest.java | 68 +-- .../rxjava3/subjects/UnicastSubjectTest.java | 6 +- .../testsupport/TestObserverExTest.java | 14 +- .../CheckLocalVariablesInTests.java | 24 +- .../rxjava3/validators/JavadocWording.java | 186 +++---- .../ParamValidationCheckerTest.java | 4 + 38 files changed, 4854 insertions(+), 218 deletions(-) create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollector.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorSingle.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFirstStageObserver.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStage.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStream.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableLastStageObserver.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptional.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableSingleStageObserver.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageObserver.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableBlockingStreamTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStreamTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStageTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromOptionalTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStreamTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptionalTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrDefaultTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrErrorTest.java diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index 16e1454af3..0afdd29a93 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -15,6 +15,7 @@ import java.util.*; import java.util.concurrent.*; +import java.util.stream.*; import org.reactivestreams.Publisher; @@ -24,6 +25,7 @@ import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.functions.*; import io.reactivex.rxjava3.internal.fuseable.ScalarSupplier; +import io.reactivex.rxjava3.internal.jdk8.*; import io.reactivex.rxjava3.internal.observers.*; import io.reactivex.rxjava3.internal.operators.flowable.*; import io.reactivex.rxjava3.internal.operators.mixed.*; @@ -1931,6 +1933,7 @@ public static Observable fromFuture(@NonNull Future future, * resulting ObservableSource * @return an Observable that emits each item in the source {@link Iterable} sequence * @see ReactiveX operators documentation: From + * @see #fromStream(Stream) */ @CheckReturnValue @NonNull @@ -5211,16 +5214,16 @@ public final Iterable blockingIterable() { *
{@code blockingIterable} does not operate by default on a particular {@link Scheduler}.
* * - * @param bufferSize the number of items to prefetch from the current Observable + * @param capacityHint the expected number of items to be buffered * @return an {@link Iterable} version of this {@code Observable} * @see ReactiveX documentation: To */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @NonNull - public final Iterable blockingIterable(int bufferSize) { - ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return new BlockingObservableIterable<>(this, bufferSize); + public final Iterable blockingIterable(int capacityHint) { + ObjectHelper.verifyPositive(capacityHint, "bufferSize"); + return new BlockingObservableIterable<>(this, capacityHint); } /** @@ -15910,4 +15913,502 @@ public final TestObserver test(boolean dispose) { // NoPMD subscribe(to); return to; } + + // ------------------------------------------------------------------------- + // JDK 8 Support + // ------------------------------------------------------------------------- + + /** + * Converts the existing value of the provided optional into a {@link #just(Object)} + * or an empty optional into an {@link #empty()} {@code Observable} instance. + *

+ * + *

+ * Note that the operator takes an already instantiated optional reference and does not + * by any means create this original optional. If the optional is to be created per + * consumer upon subscription, use {@link #defer(Supplier)} around {@code fromOptional}: + *


+     * Observable.defer(() -> Observable.fromOptional(createOptional()));
+     * 
+ *
+ *
Scheduler:
+ *
{@code fromOptional} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the optional value + * @param optional the optional value to convert into an {@code Observable} + * @return the new Observable instance + * @see #just(Object) + * @see #empty() + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Observable<@NonNull T> fromOptional(@NonNull Optional optional) { + Objects.requireNonNull(optional, "optional is null"); + return optional.map(Observable::just).orElseGet(Observable::empty); + } + + /** + * Signals the completion value or error of the given (hot) {@link CompletionStage}-based asynchronous calculation. + *

+ * + *

+ * Note that the operator takes an already instantiated, running or terminated {@code CompletionStage}. + * If the optional is to be created per consumer upon subscription, use {@link #defer(Supplier)} + * around {@code fromCompletionStage}: + *


+     * Observable.defer(() -> Observable.fromCompletionStage(createCompletionStage()));
+     * 
+ *

+ * If the {@code CompletionStage} completes with {@code null}, a {@link NullPointerException} is signaled. + *

+ * Canceling the flow can't cancel the execution of the {@code CompletionStage} because {@code CompletionStage} + * itself doesn't support cancellation. Instead, the operator detaches from the {@code CompletionStage}. + *

+ *
Scheduler:
+ *
{@code fromCompletionStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the CompletionStage + * @param stage the CompletionStage to convert to Observable and signal its terminal value or error + * @return the new Observable instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Observable<@NonNull T> fromCompletionStage(@NonNull CompletionStage stage) { + Objects.requireNonNull(stage, "stage is null"); + return RxJavaPlugins.onAssembly(new ObservableFromCompletionStage<>(stage)); + } + + /** + * Converts a {@link Stream} into a finite {@code Observable} and emits its items in the sequence. + *

+ * + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #fromIterable(Iterable)}: + *


+     * Stream<T> stream = ...
+     * Observable.fromIterable(stream::iterator);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * IntStream intStream = IntStream.rangeClosed(1, 10);
+     * Observable.fromStream(intStream.boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Scheduler:
+ *
{@code fromStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the source {@code Stream} + * @param stream the {@code Stream} of values to emit + * @return the new Observable instance + * @since 3.0.0 + * @see #fromIterable(Iterable) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public static Observable<@NonNull T> fromStream(@NonNull Stream stream) { + Objects.requireNonNull(stream, "stream is null"); + return RxJavaPlugins.onAssembly(new ObservableFromStream<>(stream)); + } + + /** + * Maps each upstream value into an {@link Optional} and emits the contained item if not empty. + *

+ * + * + *

+ *
Scheduler:
+ *
{@code mapOptional} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the non-null output type + * @param mapper the function that receives the upstream item and should return a non-empty {@code Optional} + * to emit as the output or an empty {@code Optional} to skip to the next upstream value + * @return the new Observable instance + * @since 3.0.0 + * @see #map(Function) + * @see #filter(Predicate) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Observable mapOptional(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new ObservableMapOptional<>(this, mapper)); + } + + /** + * Collects the finite upstream's values into a container via a Stream {@link Collector} callback set and emits + * it as the success result. + *

+ * + * + *

+ *
Scheduler:
+ *
{@code collect} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the non-null result type + * @param the intermediate container type used for the accumulation + * @param collector the interface defining the container supplier, accumulator and finisher functions; + * see {@link Collectors} for some standard implementations + * @return the new Single instance + * @since 3.0.0 + * @see Collectors + * @see #collect(Supplier, BiConsumer) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R, A> Single collect(@NonNull Collector collector) { + Objects.requireNonNull(collector, "collector is null"); + return RxJavaPlugins.onAssembly(new ObservableCollectWithCollectorSingle<>(this, collector)); + } + + /** + * Signals the first upstream item (or the default item if the upstream is empty) via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).firstStage(Optional.empty());
+     * 
+ *
+ *
Scheduler:
+ *
{@code firstStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + * @see #firstOrErrorStage() + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage firstStage(@Nullable T defaultItem) { + return subscribeWith(new ObservableFirstStageObserver<>(true, defaultItem)); + } + + /** + * Signals the only expected upstream item (or the default item if the upstream is empty) + * or signals {@link IllegalArgumentException} if the upstream has more than one item + * via a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).singleStage(Optional.empty());
+     * 
+ *
+ *
Scheduler:
+ *
{@code singleStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + * @see #singleOrErrorStage() + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage singleStage(@Nullable T defaultItem) { + return subscribeWith(new ObservableSingleStageObserver<>(true, defaultItem)); + } + + /** + * Signals the last upstream item (or the default item if the upstream is empty) via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: + *


+     * CompletionStage<Optional<T>> stage = source.map(Optional::of).lastStage(Optional.empty());
+     * 
+ *
+ *
Scheduler:
+ *
{@code lastStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param defaultItem the item to signal if the upstream is empty + * @return the new CompletionStage instance + * @since 3.0.0 + * @see #lastOrErrorStage() + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage lastStage(@Nullable T defaultItem) { + return subscribeWith(new ObservableLastStageObserver<>(true, defaultItem)); + } + + /** + * Signals the first upstream item or a {@link NoSuchElementException} if the upstream is empty via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ *
Scheduler:
+ *
{@code firstOrErrorStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + * @see #firstStage(Object) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage firstOrErrorStage() { + return subscribeWith(new ObservableFirstStageObserver<>(false, null)); + } + + /** + * Signals the only expected upstream item, a {@link NoSuchElementException} if the upstream is empty + * or signals {@link IllegalArgumentException} if the upstream has more than one item + * via a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ *
Scheduler:
+ *
{@code singleOrErrorStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + * @see #singleStage(Object) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage singleOrErrorStage() { + return subscribeWith(new ObservableSingleStageObserver<>(false, null)); + } + + /** + * Signals the last upstream item or a {@link NoSuchElementException} if the upstream is empty via + * a {@link CompletionStage}. + *

+ * + *

+ * The upstream can be canceled by converting the resulting {@code CompletionStage} into + * {@link CompletableFuture} via {@link CompletionStage#toCompletableFuture()} and + * calling {@link CompletableFuture#cancel(boolean)} on it. + * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and + * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. + *

+ *
Scheduler:
+ *
{@code lastOrErrorStage} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @return the new CompletionStage instance + * @since 3.0.0 + * @see #lastStage(Object) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final CompletionStage lastOrErrorStage() { + return subscribeWith(new ObservableLastStageObserver<>(false, null)); + } + + /** + * Creates a sequential {@link Stream} to consume or process this {@code Observable} in a blocking manner via + * the Java {@code Stream} API. + *

+ * + *

+ * Cancellation of the upstream is done via {@link Stream#close()}, therefore, it is strongly recommended the + * consumption is performed within a try-with-resources construct: + *


+     * Observable<Integer> source = Observable.range(1, 10)
+     *        .subscribeOn(Schedulers.computation());
+     *
+     * try (Stream<Integer> stream = source.blockingStream()) {
+     *     stream.limit(3).forEach(System.out::println);
+     * }
+     * 
+ *
+ *
Scheduler:
+ *
{@code blockingStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @return the new Stream instance + * @since 3.0.0 + * @see #blockingStream(int) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final Stream blockingStream() { + return blockingStream(bufferSize()); + } + + /** + * Creates a sequential {@link Stream} to consume or process this {@code Observable} in a blocking manner via + * the Java {@code Stream} API. + *

+ * + *

+ * Cancellation of the upstream is done via {@link Stream#close()}, therefore, it is strongly recommended the + * consumption is performed within a try-with-resources construct: + *


+     * Observable<Integer> source = Observable.range(1, 10)
+     *        .subscribeOn(Schedulers.computation());
+     *
+     * try (Stream<Integer> stream = source.blockingStream(4)) {
+     *     stream.limit(3).forEach(System.out::println);
+     * }
+     * 
+ *
Scheduler:
+ *
{@code blockingStream} does not operate by default on a particular {@link Scheduler}.
+ * + * + * @param capacityHint the expected number of items to be buffered + * @return the new Stream instance + * @since 3.0.0 + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final Stream blockingStream(int capacityHint) { + Iterator iterator = blockingIterable(capacityHint).iterator(); + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, 0), false) + .onClose(() -> ((Disposable)iterator).dispose()); + } + + /** + * Maps each upstream item into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #concatMapIterable(Function)}: + *


+     * source.concatMapIterable(v -> createStream(v)::iterator);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.concatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Scheduler:
+ *
{@code concatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @return the new Observable instance + * @see #concatMap(Function) + * @see #concatMapIterable(Function) + * @see #flatMapStream(Function) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Observable concatMapStream(@NonNull Function> mapper) { + return flatMapStream(mapper); + } + + /** + * Maps each upstream item into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flatMapIterable(Function)}: + *


+     * source.flatMapIterable(v -> createStream(v)::iterator);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Scheduler:
+ *
{@code flatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @return the new Observable instance + * @see #flatMap(Function) + * @see #flatMapIterable(Function) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> Observable flatMapStream(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new ObservableFlatMapStream<>(this, mapper)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java index ecc18fb154..55ec854ddf 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStream.java @@ -69,7 +69,7 @@ public static void subscribeStream(Subscriber s, Stream stream } if (s instanceof ConditionalSubscriber) { - s.onSubscribe(new StreamConditionalSubscription((ConditionalSubscriber)s, iterator, stream)); + s.onSubscribe(new StreamConditionalSubscription<>((ConditionalSubscriber)s, iterator, stream)); } else { s.onSubscribe(new StreamSubscription<>(s, iterator, stream)); } @@ -147,15 +147,23 @@ public T poll() { once = true; } else { if (!iterator.hasNext()) { + clear(); return null; } } - return Objects.requireNonNull(iterator.next(), "Iterator.next() returned a null value"); + return Objects.requireNonNull(iterator.next(), "The Stream's Iterator.next() returned a null value"); } @Override public boolean isEmpty() { - return iterator == null || !iterator.hasNext(); + Iterator it = iterator; + if (it != null) { + if (!once || it.hasNext()) { + return false; + } + clear(); + } + return true; } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollector.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollector.java new file mode 100644 index 0000000000..34132a01b3 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollector.java @@ -0,0 +1,151 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Objects; +import java.util.function.*; +import java.util.stream.Collector; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.internal.disposables.*; +import io.reactivex.rxjava3.internal.observers.DeferredScalarDisposable; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Collect items into a container defined by a Stream {@link Collector} callback set. + * + * @param the upstream value type + * @param
the intermediate accumulator type + * @param the result type + * @since 3.0.0 + */ +public final class ObservableCollectWithCollector extends Observable { + + final Observable source; + + final Collector collector; + + public ObservableCollectWithCollector(Observable source, Collector collector) { + this.source = source; + this.collector = collector; + } + + @Override + protected void subscribeActual(@NonNull Observer observer) { + A container; + BiConsumer accumulator; + Function finisher; + + try { + container = collector.supplier().get(); + accumulator = collector.accumulator(); + finisher = collector.finisher(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + EmptyDisposable.error(ex, observer); + return; + } + + source.subscribe(new CollectorObserver<>(observer, container, accumulator, finisher)); + } + + static final class CollectorObserver + extends DeferredScalarDisposable + implements Observer { + + private static final long serialVersionUID = -229544830565448758L; + + final BiConsumer accumulator; + + final Function finisher; + + Disposable upstream; + + boolean done; + + A container; + + CollectorObserver(Observer downstream, A container, BiConsumer accumulator, Function finisher) { + super(downstream); + this.container = container; + this.accumulator = accumulator; + this.finisher = finisher; + } + + @Override + public void onSubscribe(@NonNull Disposable d) { + if (DisposableHelper.validate(this.upstream, d)) { + this.upstream = d; + + downstream.onSubscribe(this); + } + } + + @Override + public void onNext(T t) { + if (done) { + return; + } + try { + accumulator.accept(container, t); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + upstream.dispose(); + onError(ex); + } + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + } else { + done = true; + upstream = DisposableHelper.DISPOSED; + this.container = null; + downstream.onError(t); + } + } + + @Override + public void onComplete() { + if (done) { + return; + } + + done = true; + upstream = DisposableHelper.DISPOSED; + A container = this.container; + this.container = null; + R result; + try { + result = Objects.requireNonNull(finisher.apply(container), "The finisher returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + + complete(result); + } + + @Override + public void dispose() { + super.dispose(); + upstream.dispose(); + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorSingle.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorSingle.java new file mode 100644 index 0000000000..a919b02ab9 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorSingle.java @@ -0,0 +1,159 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Objects; +import java.util.function.*; +import java.util.stream.Collector; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.internal.disposables.*; +import io.reactivex.rxjava3.internal.fuseable.FuseToObservable; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Collect items into a container defined by a Stream {@link Collector} callback set. + * + * @param the upstream value type + * @param the intermediate accumulator type + * @param the result type + * @since 3.0.0 + */ +public final class ObservableCollectWithCollectorSingle extends Single implements FuseToObservable { + + final Observable source; + + final Collector collector; + + public ObservableCollectWithCollectorSingle(Observable source, Collector collector) { + this.source = source; + this.collector = collector; + } + + @Override + public Observable fuseToObservable() { + return new ObservableCollectWithCollector<>(source, collector); + } + + @Override + protected void subscribeActual(@NonNull SingleObserver observer) { + A container; + BiConsumer accumulator; + Function finisher; + + try { + container = collector.supplier().get(); + accumulator = collector.accumulator(); + finisher = collector.finisher(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + EmptyDisposable.error(ex, observer); + return; + } + + source.subscribe(new CollectorSingleObserver<>(observer, container, accumulator, finisher)); + } + + static final class CollectorSingleObserver implements Observer, Disposable { + + final SingleObserver downstream; + + final BiConsumer accumulator; + + final Function finisher; + + Disposable upstream; + + boolean done; + + A container; + + CollectorSingleObserver(SingleObserver downstream, A container, BiConsumer accumulator, Function finisher) { + this.downstream = downstream; + this.container = container; + this.accumulator = accumulator; + this.finisher = finisher; + } + + @Override + public void onSubscribe(@NonNull Disposable d) { + if (DisposableHelper.validate(this.upstream, d)) { + this.upstream = d; + + downstream.onSubscribe(this); + } + } + + @Override + public void onNext(T t) { + if (done) { + return; + } + try { + accumulator.accept(container, t); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + upstream.dispose(); + onError(ex); + } + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + } else { + done = true; + upstream = DisposableHelper.DISPOSED; + this.container = null; + downstream.onError(t); + } + } + + @Override + public void onComplete() { + if (done) { + return; + } + + done = true; + upstream = DisposableHelper.DISPOSED; + A container = this.container; + this.container = null; + R result; + try { + result = Objects.requireNonNull(finisher.apply(container), "The finisher returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + + downstream.onSuccess(result); + } + + @Override + public void dispose() { + upstream.dispose(); + upstream = DisposableHelper.DISPOSED; + } + + @Override + public boolean isDisposed() { + return upstream == DisposableHelper.DISPOSED; + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFirstStageObserver.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFirstStageObserver.java new file mode 100644 index 0000000000..cda0073067 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFirstStageObserver.java @@ -0,0 +1,51 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.NoSuchElementException; + +/** + * Signals the first element of the source via the underlying CompletableFuture, + * signals the a default item if the upstream is empty or signals {@link NoSuchElementException}. + * + * @param the element type + * @since 3.0.0 + */ +public final class ObservableFirstStageObserver extends ObservableStageObserver { + + final boolean hasDefault; + + final T defaultItem; + + public ObservableFirstStageObserver(boolean hasDefault, T defaultItem) { + this.hasDefault = hasDefault; + this.defaultItem = defaultItem; + } + + @Override + public void onNext(T t) { + complete(t); + } + + @Override + public void onComplete() { + if (!isDone()) { + clear(); + if (hasDefault) { + complete(defaultItem); + } else { + completeExceptionally(new NoSuchElementException()); + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java new file mode 100644 index 0000000000..9b7b446e28 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java @@ -0,0 +1,162 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.core.Observer; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.functions.*; +import io.reactivex.rxjava3.internal.disposables.*; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Maps the upstream values onto {@link Stream}s and emits their items in order to the downstream. + * + * @param the upstream element type + * @param the inner {@code Stream} and result element type + * @since 3.0.0 + */ +public final class ObservableFlatMapStream extends Observable { + + final Observable source; + + final Function> mapper; + + public ObservableFlatMapStream(Observable source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(Observer observer) { + if (source instanceof Supplier) { + Stream stream = null; + try { + @SuppressWarnings("unchecked") + T t = ((Supplier)source).get(); + if (t != null) { + stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream"); + } + } catch (Throwable ex) { + EmptyDisposable.error(ex, observer); + return; + } + + if (stream != null) { + ObservableFromStream.subscribeStream(observer, stream); + } else { + EmptyDisposable.complete(observer); + } + } else { + source.subscribe(new FlatMapStreamObserver<>(observer, mapper)); + } + } + + static final class FlatMapStreamObserver extends AtomicInteger + implements Observer, Disposable { + + private static final long serialVersionUID = -5127032662980523968L; + + final Observer downstream; + + final Function> mapper; + + Disposable upstream; + + volatile boolean disposed; + + boolean done; + + FlatMapStreamObserver(Observer downstream, Function> mapper) { + this.downstream = downstream; + this.mapper = mapper; + } + + @Override + public void onSubscribe(@NonNull Disposable d) { + if (DisposableHelper.validate(this.upstream, d)) { + this.upstream = d; + + downstream.onSubscribe(this); + } + } + + @Override + public void onNext(@NonNull T t) { + if (done) { + return; + } + try { + try (Stream stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream")) { + Iterator it = stream.iterator(); + while (it.hasNext()) { + if (disposed) { + done = true; + break; + } + R value = Objects.requireNonNull(it.next(), "The Stream's Iterator.next retuned a null value"); + if (disposed) { + done = true; + break; + } + downstream.onNext(value); + if (disposed) { + done = true; + break; + } + } + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + upstream.dispose(); + onError(ex); + } + } + + @Override + public void onError(@NonNull Throwable e) { + if (done) { + RxJavaPlugins.onError(e); + } else { + done = true; + downstream.onError(e); + } + } + + @Override + public void onComplete() { + if (!done) { + done = true; + downstream.onComplete(); + } + } + + @Override + public void dispose() { + disposed = true; + upstream.dispose(); + } + + @Override + public boolean isDisposed() { + return disposed; + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStage.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStage.java new file mode 100644 index 0000000000..262da56026 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStage.java @@ -0,0 +1,92 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletionStage; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.internal.observers.DeferredScalarDisposable; + +/** + * Wrap a CompletionStage and signal its outcome. + * @param the element type + * @since 3.0.0 + */ +public final class ObservableFromCompletionStage extends Observable { + + final CompletionStage stage; + + public ObservableFromCompletionStage(CompletionStage stage) { + this.stage = stage; + } + + @Override + protected void subscribeActual(Observer observer) { + // We need an indirection because one can't detach from a whenComplete + // and cancellation should not hold onto the stage. + BiConsumerAtomicReference whenReference = new BiConsumerAtomicReference<>(); + CompletionStageHandler handler = new CompletionStageHandler<>(observer, whenReference); + whenReference.lazySet(handler); + + observer.onSubscribe(handler); + stage.whenComplete(whenReference); + } + + static final class CompletionStageHandler + extends DeferredScalarDisposable + implements BiConsumer { + + private static final long serialVersionUID = 4665335664328839859L; + + final BiConsumerAtomicReference whenReference; + + CompletionStageHandler(Observer downstream, BiConsumerAtomicReference whenReference) { + super(downstream); + this.whenReference = whenReference; + } + + @Override + public void accept(T item, Throwable error) { + if (error != null) { + downstream.onError(error); + } + else if (item != null) { + complete(item); + } else { + downstream.onError(new NullPointerException("The CompletionStage terminated with null.")); + } + } + + @Override + public void dispose() { + super.dispose(); + whenReference.set(null); + } + } + + static final class BiConsumerAtomicReference extends AtomicReference> + implements BiConsumer { + + private static final long serialVersionUID = 45838553147237545L; + + @Override + public void accept(T t, Throwable u) { + BiConsumer biConsumer = get(); + if (biConsumer != null) { + biConsumer.accept(t, u); + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStream.java new file mode 100644 index 0000000000..90d7681303 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStream.java @@ -0,0 +1,219 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; +import java.util.stream.Stream; + +import io.reactivex.rxjava3.annotations.*; +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.core.Observer; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.internal.disposables.EmptyDisposable; +import io.reactivex.rxjava3.internal.fuseable.QueueDisposable; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Wraps a {@link Stream} and emits its values as an {@link Observable} sequence. + * @param the element type of the Stream + * @since 3.0.0 + */ +public final class ObservableFromStream extends Observable { + + final Stream stream; + + public ObservableFromStream(Stream stream) { + this.stream = stream; + } + + @Override + protected void subscribeActual(Observer observer) { + subscribeStream(observer, stream); + } + + /** + * Subscribes to the Stream. + * @param the element type of the flow + * @param observer the observer to drive + * @param stream the sequence to consume + */ + public static void subscribeStream(Observer observer, Stream stream) { + Iterator iterator; + try { + iterator = stream.iterator(); + + if (!iterator.hasNext()) { + EmptyDisposable.complete(observer); + closeSafely(stream); + return; + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + EmptyDisposable.error(ex, observer); + closeSafely(stream); + return; + } + + StreamDisposable disposable = new StreamDisposable<>(observer, iterator, stream); + observer.onSubscribe(disposable); + disposable.run(); + } + + static void closeSafely(AutoCloseable c) { + try { + c.close(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + RxJavaPlugins.onError(ex); + } + } + + static final class StreamDisposable implements QueueDisposable { + + final Observer downstream; + + Iterator iterator; + + AutoCloseable closeable; + + volatile boolean disposed; + + boolean once; + + boolean outputFused; + + StreamDisposable(Observer downstream, Iterator iterator, AutoCloseable closeable) { + this.downstream = downstream; + this.iterator = iterator; + this.closeable = closeable; + } + + @Override + public void dispose() { + disposed = true; + run(); + } + + @Override + public boolean isDisposed() { + return disposed; + } + + @Override + public int requestFusion(int mode) { + if ((mode & SYNC) != 0) { + outputFused = true; + return SYNC; + } + return NONE; + } + + @Override + public boolean offer(@NonNull T value) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean offer(@NonNull T v1, @NonNull T v2) { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public T poll() { + if (iterator == null) { + return null; + } + if (!once) { + once = true; + } else { + if (!iterator.hasNext()) { + clear(); + return null; + } + } + return Objects.requireNonNull(iterator.next(), "The Stream's Iterator.next() returned a null value"); + } + + @Override + public boolean isEmpty() { + Iterator it = iterator; + if (it != null) { + if (!once || it.hasNext()) { + return false; + } + clear(); + } + return true; + } + + @Override + public void clear() { + iterator = null; + AutoCloseable c = closeable; + closeable = null; + if (c != null) { + closeSafely(c); + } + } + + public void run() { + if (outputFused) { + return; + } + Iterator iterator = this.iterator; + Observer downstream = this.downstream; + + for (;;) { + if (disposed) { + clear(); + break; + } + + T next; + try { + next = Objects.requireNonNull(iterator.next(), "The Stream's Iterator.next returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + disposed = true; + continue; + } + + if (disposed) { + continue; + } + + downstream.onNext(next); + + if (disposed) { + continue; + } + + try { + if (iterator.hasNext()) { + continue; + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + disposed = true; + continue; + } + + downstream.onComplete(); + disposed = true; + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableLastStageObserver.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableLastStageObserver.java new file mode 100644 index 0000000000..e2f9dc2225 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableLastStageObserver.java @@ -0,0 +1,54 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.NoSuchElementException; + +/** + * Signals the last element of the source via the underlying CompletableFuture, + * signals the a default item if the upstream is empty or signals {@link NoSuchElementException}. + * + * @param the element type + * @since 3.0.0 + */ +public final class ObservableLastStageObserver extends ObservableStageObserver { + + final boolean hasDefault; + + final T defaultItem; + + public ObservableLastStageObserver(boolean hasDefault, T defaultItem) { + this.hasDefault = hasDefault; + this.defaultItem = defaultItem; + } + + @Override + public void onNext(T t) { + value = t; + } + + @Override + public void onComplete() { + if (!isDone()) { + T v = value; + clear(); + if (v != null) { + complete(v); + } else if (hasDefault) { + complete(defaultItem); + } else { + completeExceptionally(new NoSuchElementException()); + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptional.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptional.java new file mode 100644 index 0000000000..eb134482be --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptional.java @@ -0,0 +1,96 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; + +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.core.Observer; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.observers.BasicFuseableObserver; + +/** + * Map the upstream values into an Optional and emit its value if any. + * @param the upstream element type + * @param the output element type + * @since 3.0.0 + */ +public final class ObservableMapOptional extends Observable { + + final Observable source; + + final Function> mapper; + + public ObservableMapOptional(Observable source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(Observer observer) { + source.subscribe(new MapOptionalObserver<>(observer, mapper)); + } + + static final class MapOptionalObserver extends BasicFuseableObserver { + + final Function> mapper; + + MapOptionalObserver(Observer downstream, Function> mapper) { + super(downstream); + this.mapper = mapper; + } + + @Override + public void onNext(T t) { + if (done) { + return; + } + + if (sourceMode != NONE) { + downstream.onNext(null); + return; + } + + Optional result; + try { + result = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Optional"); + } catch (Throwable ex) { + fail(ex); + return; + } + + if (result.isPresent()) { + downstream.onNext(result.get()); + } + } + + @Override + public int requestFusion(int mode) { + return transitiveBoundaryFusion(mode); + } + + @Override + public R poll() throws Throwable { + for (;;) { + T item = qd.poll(); + if (item == null) { + return null; + } + Optional result = Objects.requireNonNull(mapper.apply(item), "The mapper returned a null Optional"); + if (result.isPresent()) { + return result.get(); + } + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableSingleStageObserver.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableSingleStageObserver.java new file mode 100644 index 0000000000..fa8714397a --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableSingleStageObserver.java @@ -0,0 +1,60 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.NoSuchElementException; + +/** + * Signals the only element of the source via the underlying CompletableFuture, + * signals the a default item if the upstream is empty or signals {@link IllegalArgumentException} + * if the upstream has more than one item. + * + * @param the element type + * @since 3.0.0 + */ +public final class ObservableSingleStageObserver extends ObservableStageObserver { + + final boolean hasDefault; + + final T defaultItem; + + public ObservableSingleStageObserver(boolean hasDefault, T defaultItem) { + this.hasDefault = hasDefault; + this.defaultItem = defaultItem; + } + + @Override + public void onNext(T t) { + if (value != null) { + value = null; + completeExceptionally(new IllegalArgumentException("Sequence contains more than one element!")); + } else { + value = t; + } + } + + @Override + public void onComplete() { + if (!isDone()) { + T v = value; + clear(); + if (v != null) { + complete(v); + } else if (hasDefault) { + complete(defaultItem); + } else { + completeExceptionally(new NoSuchElementException()); + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageObserver.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageObserver.java new file mode 100644 index 0000000000..54acae086f --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageObserver.java @@ -0,0 +1,75 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.Observer; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.internal.disposables.DisposableHelper; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Base class that extends CompletableFuture and provides basic infrastructure + * to notify watchers upon upstream signals. + * @param the element type + * @since 3.0.0 + */ +abstract class ObservableStageObserver extends CompletableFuture implements Observer { + + final AtomicReference upstream = new AtomicReference<>(); + + T value; + + @Override + public final void onSubscribe(@NonNull Disposable d) { + DisposableHelper.setOnce(upstream, d); + } + + @Override + public final void onError(Throwable t) { + clear(); + if (!completeExceptionally(t)) { + RxJavaPlugins.onError(t); + } + } + + protected final void disposeUpstream() { + DisposableHelper.dispose(upstream); + } + + protected final void clear() { + value = null; + upstream.lazySet(DisposableHelper.DISPOSED); + } + + @Override + public final boolean cancel(boolean mayInterruptIfRunning) { + disposeUpstream(); + return super.cancel(mayInterruptIfRunning); + } + + @Override + public final boolean complete(T value) { + disposeUpstream(); + return super.complete(value); + } + + @Override + public final boolean completeExceptionally(Throwable ex) { + disposeUpstream(); + return super.completeExceptionally(ex); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java index 940ac95c4d..7d5f7a0a74 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/FlowableFromStreamTest.java @@ -184,6 +184,43 @@ public void onComplete() { assertTrue(q.isEmpty()); } + @Test + public void fusedPoll() throws Throwable { + AtomicReference> queue = new AtomicReference<>(); + AtomicInteger calls = new AtomicInteger(); + + Flowable.fromStream(Stream.of(1).onClose(() -> calls.getAndIncrement())) + .subscribe(new FlowableSubscriber() { + @Override + public void onSubscribe(@NonNull Subscription s) { + queue.set((SimpleQueue)s); + ((QueueSubscription)s).requestFusion(QueueFuseable.ANY); + } + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + }); + + SimpleQueue q = queue.get(); + + assertFalse(q.isEmpty()); + + assertEquals(1, q.poll()); + + assertTrue(q.isEmpty()); + + assertEquals(1, calls.get()); + } + @Test public void streamOfNull() { Flowable.fromStream(Stream.of((Integer)null)) @@ -512,4 +549,9 @@ public void closeCalledOnItemCrashConditional() { assertEquals(1, calls.get()); } + + @Test + public void badRequest() { + TestHelper.assertBadRequestReported(Flowable.fromStream(Stream.of(1))); + } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableBlockingStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableBlockingStreamTest.java new file mode 100644 index 0000000000..b62bedb6f8 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableBlockingStreamTest.java @@ -0,0 +1,107 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.List; +import java.util.stream.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.processors.UnicastProcessor; +import io.reactivex.rxjava3.schedulers.Schedulers; + +public class ObservableBlockingStreamTest extends RxJavaTest { + + @Test + public void empty() { + try (Stream stream = Observable.empty().blockingStream()) { + assertEquals(0, stream.toArray().length); + } + } + + @Test + public void just() { + try (Stream stream = Observable.just(1).blockingStream()) { + assertArrayEquals(new Integer[] { 1 }, stream.toArray(Integer[]::new)); + } + } + + @Test + public void range() { + try (Stream stream = Observable.range(1, 5).blockingStream()) { + assertArrayEquals(new Integer[] { 1, 2, 3, 4, 5 }, stream.toArray(Integer[]::new)); + } + } + + @Test + public void rangeBackpressured() { + try (Stream stream = Observable.range(1, 5).blockingStream(1)) { + assertArrayEquals(new Integer[] { 1, 2, 3, 4, 5 }, stream.toArray(Integer[]::new)); + } + } + + @Test + public void rangeAsyncBackpressured() { + try (Stream stream = Observable.range(1, 1000).subscribeOn(Schedulers.computation()).blockingStream()) { + List list = stream.collect(Collectors.toList()); + + assertEquals(1000, list.size()); + for (int i = 1; i <= 1000; i++) { + assertEquals(i, list.get(i - 1).intValue()); + } + } + } + + @Test + public void rangeAsyncBackpressured1() { + try (Stream stream = Observable.range(1, 1000).subscribeOn(Schedulers.computation()).blockingStream(1)) { + List list = stream.collect(Collectors.toList()); + + assertEquals(1000, list.size()); + for (int i = 1; i <= 1000; i++) { + assertEquals(i, list.get(i - 1).intValue()); + } + } + } + + @Test + public void error() { + try (Stream stream = Observable.error(new TestException()).blockingStream()) { + stream.toArray(Integer[]::new); + fail("Should have thrown!"); + } catch (TestException expected) { + // expected + } + } + + @Test + public void close() { + UnicastProcessor up = UnicastProcessor.create(); + up.onNext(1); + up.onNext(2); + up.onNext(3); + up.onNext(4); + up.onNext(5); + + try (Stream stream = up.blockingStream()) { + assertArrayEquals(new Integer[] { 1, 2, 3 }, stream.limit(3).toArray(Integer[]::new)); + } + + assertFalse(up.hasSubscribers()); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorTest.java new file mode 100644 index 0000000000..923a7dbaa3 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableCollectWithCollectorTest.java @@ -0,0 +1,444 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.assertFalse; + +import java.io.IOException; +import java.util.*; +import java.util.function.*; +import java.util.stream.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.core.Observer; +import io.reactivex.rxjava3.core.RxJavaTest; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.processors.*; +import io.reactivex.rxjava3.subjects.PublishSubject; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class ObservableCollectWithCollectorTest extends RxJavaTest { + + @Test + public void basic() { + Observable.range(1, 5) + .collect(Collectors.toList()) + .test() + .assertResult(Arrays.asList(1, 2, 3, 4, 5)); + } + + @Test + public void empty() { + Observable.empty() + .collect(Collectors.toList()) + .test() + .assertResult(Collections.emptyList()); + } + + @Test + public void error() { + Observable.error(new TestException()) + .collect(Collectors.toList()) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorSupplierCrash() { + Observable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + throw new TestException(); + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorCrash() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void collectorFinisherCrash() { + Observable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> { throw new TestException(); }; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorDropSignals() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Observable source = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onNext(2); + observer.onError(new IOException()); + observer.onComplete(); + } + }; + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + + TestHelper.assertUndeliverable(errors, 0, IOException.class); + }); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(PublishSubject.create() + .collect(Collectors.toList())); + } + + @Test + public void onSubscribe() { + TestHelper.checkDoubleOnSubscribeObservableToSingle(f -> f.collect(Collectors.toList())); + } + + @Test + public void basicToObservable() { + Observable.range(1, 5) + .collect(Collectors.toList()) + .toObservable() + .test() + .assertResult(Arrays.asList(1, 2, 3, 4, 5)); + } + + @Test + public void emptyToObservable() { + Observable.empty() + .collect(Collectors.toList()) + .toObservable() + .test() + .assertResult(Collections.emptyList()); + } + + @Test + public void errorToObservable() { + Observable.error(new TestException()) + .collect(Collectors.toList()) + .toObservable() + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorSupplierCrashToObservable() { + Observable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + throw new TestException(); + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toObservable() + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorCrashToObservable() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toObservable() + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void collectorFinisherCrashToObservable() { + Observable.range(1, 5) + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> { throw new TestException(); }; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toObservable() + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorDropSignalsToObservable() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Observable source = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onNext(2); + observer.onError(new IOException()); + observer.onComplete(); + } + }; + + source + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .toObservable() + .test() + .assertFailure(TestException.class); + + TestHelper.assertUndeliverable(errors, 0, IOException.class); + }); + } + + @Test + public void disposeToObservable() { + TestHelper.checkDisposed(PublishProcessor.create() + .collect(Collectors.toList()).toObservable()); + } + + @Test + public void onSubscribeToObservable() { + TestHelper.checkDoubleOnSubscribeObservable(f -> f.collect(Collectors.toList()).toObservable()); + } + + @Test + public void toObservableTake() { + Observable.range(1, 5) + .collect(Collectors.toList()) + .toObservable() + .take(1) + .test() + .assertResult(Arrays.asList(1, 2, 3, 4, 5)); + } + + @Test + public void disposeBeforeEnd() { + TestObserver> to = Observable.range(1, 5).concatWith(Observable.never()) + .collect(Collectors.toList()) + .test(); + + to.dispose(); + + to.assertEmpty(); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStreamTest.java new file mode 100644 index 0000000000..8341aa03dd --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStreamTest.java @@ -0,0 +1,466 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.subjects.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class ObservableFlatMapStreamTest extends RxJavaTest { + + @Test + public void empty() { + Observable.empty() + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertResult(); + } + + @Test + public void emptyHidden() { + Observable.empty() + .hide() + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertResult(); + } + + @Test + public void just() { + Observable.just(1) + .flatMapStream(v -> Stream.of(v + 1, v + 2, v + 3, v + 4, v + 5)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void justHidden() { + Observable.just(1).hide() + .flatMapStream(v -> Stream.of(v + 1, v + 2, v + 3, v + 4, v + 5)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void error() { + Observable.error(new TestException()) + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertFailure(TestException.class); + } + + @Test + public void supplierFusedError() { + Observable.fromCallable(() -> { throw new TestException(); }) + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertFailure(TestException.class); + } + + @Test + public void errorHidden() { + Observable.error(new TestException()) + .hide() + .flatMapStream(v -> Stream.of(1, 2, 3, 4, 5)) + .test() + .assertFailure(TestException.class); + } + + @Test + public void range() { + Observable.range(1, 5) + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31, 32, 33, 34, + 40, 41, 42, 43, 44, + 50, 51, 52, 53, 54 + ); + } + + @Test + public void rangeHidden() { + Observable.range(1, 5) + .hide() + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31, 32, 33, 34, + 40, 41, 42, 43, 44, + 50, 51, 52, 53, 54 + ); + } + + @Test + public void rangeToEmpty() { + Observable.range(1, 5) + .flatMapStream(v -> Stream.of()) + .test() + .assertResult(); + } + + @Test + public void rangeTake() { + Observable.range(1, 5) + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .take(12) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31 + ); + } + + @Test + public void rangeTakeHidden() { + Observable.range(1, 5) + .hide() + .flatMapStream(v -> IntStream.range(v * 10, v * 10 + 5).boxed()) + .take(12) + .test() + .assertResult( + 10, 11, 12, 13, 14, + 20, 21, 22, 23, 24, + 30, 31 + ); + } + + @Test + public void upstreamCancelled() { + PublishSubject ps = PublishSubject.create(); + + AtomicInteger calls = new AtomicInteger(); + + TestObserver to = ps + .flatMapStream(v -> Stream.of(v + 1, v + 2).onClose(() -> calls.getAndIncrement())) + .take(1) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to.assertResult(2); + + assertFalse(ps.hasObservers()); + + assertEquals(1, calls.get()); + } + + @Test + public void upstreamCancelledCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + PublishSubject ps = PublishSubject.create(); + + TestObserver to = ps + .flatMapStream(v -> Stream.of(v + 1, v + 2).onClose(() -> { throw new TestException(); })) + .take(1) + .test(); + + assertTrue(ps.hasObservers()); + + ps.onNext(1); + + to.assertResult(2); + + assertFalse(ps.hasObservers()); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void crossMap() { + Observable.range(1, 1000) + .flatMapStream(v -> IntStream.range(v * 1000, v * 1000 + 1000).boxed()) + .test() + .assertValueCount(1_000_000) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void crossMapHidden() { + Observable.range(1, 1000) + .hide() + .flatMapStream(v -> IntStream.range(v * 1000, v * 1000 + 1000).boxed()) + .test() + .assertValueCount(1_000_000) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void onSubscribe() { + TestHelper.checkDoubleOnSubscribeObservable(f -> f.flatMapStream(v -> Stream.of(1, 2))); + } + + @Test + public void mapperThrows() { + Observable.just(1).hide() + .concatMapStream(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void mapperNull() { + Observable.just(1).hide() + .concatMapStream(v -> null) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void streamNull() { + Observable.just(1).hide() + .concatMapStream(v -> Stream.of(1, null)) + .test() + .assertFailure(NullPointerException.class, 1); + } + + @Test + public void hasNextThrows() { + Observable.just(1).hide() + .concatMapStream(v -> Stream.generate(() -> { throw new TestException(); })) + .test() + .assertFailure(TestException.class); + } + + @Test + public void hasNextThrowsLater() { + AtomicInteger counter = new AtomicInteger(); + Observable.just(1).hide() + .concatMapStream(v -> Stream.generate(() -> { + if (counter.getAndIncrement() == 0) { + return 1; + } + throw new TestException(); + })) + .test() + .assertFailure(TestException.class, 1); + } + + @Test + public void mapperThrowsWhenUpstreamErrors() throws Throwable { + TestHelper.withErrorTracking(errors -> { + PublishSubject ps = PublishSubject.create(); + + AtomicInteger counter = new AtomicInteger(); + + TestObserver to = ps.hide() + .concatMapStream(v -> { + if (counter.getAndIncrement() == 0) { + return Stream.of(1, 2); + } + ps.onError(new IOException()); + throw new TestException(); + }) + .test(); + + ps.onNext(1); + ps.onNext(2); + + to + .assertFailure(IOException.class, 1, 2); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void cancelAfterIteratorNext() throws Exception { + TestObserver to = new TestObserver<>(); + + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + to.dispose(); + return 1; + } + }); + + Observable.just(1) + .hide() + .concatMapStream(v -> stream) + .subscribe(to); + + to.assertEmpty(); + } + + @Test + public void cancelAfterIteratorHasNext() throws Exception { + TestObserver to = new TestObserver<>(); + + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + to.dispose(); + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Observable.just(1) + .hide() + .concatMapStream(v -> stream) + .subscribe(to); + + to.assertEmpty(); + } + + @Test + public void asyncUpstreamFused() { + UnicastSubject us = UnicastSubject.create(); + + TestObserver to = us.flatMapStream(v -> Stream.of(1, 2)) + .test(); + + assertTrue(us.hasObservers()); + + us.onNext(1); + + to.assertValuesOnly(1, 2); + + us.onComplete(); + + to.assertResult(1, 2); + } + + @Test + public void asyncUpstreamFusionBoundary() { + UnicastSubject us = UnicastSubject.create(); + + TestObserver to = us + .map(v -> v + 1) + .flatMapStream(v -> Stream.of(1, 2)) + .test(); + + assertTrue(us.hasObservers()); + + us.onNext(1); + + to.assertValuesOnly(1, 2); + + us.onComplete(); + + to.assertResult(1, 2); + } + + @Test + public void fusedPollCrash() { + UnicastSubject us = UnicastSubject.create(); + + TestObserver to = us + .map(v -> { throw new TestException(); }) + .compose(TestHelper.observableStripBoundary()) + .flatMapStream(v -> Stream.of(1, 2)) + .test(); + + assertTrue(us.hasObservers()); + + us.onNext(1); + + assertFalse(us.hasObservers()); + + to.assertFailure(TestException.class); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(PublishSubject.create().flatMapStream(v -> Stream.of(1))); + } + + @Test + public void eventsIgnoredAfterCrash() { + AtomicInteger calls = new AtomicInteger(); + + new Observable() { + @Override + protected void subscribeActual(@NonNull Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onNext(2); + observer.onComplete(); + } + } + .flatMapStream(v -> { + calls.getAndIncrement(); + throw new TestException(); + }) + .take(1) + .test() + .assertFailure(TestException.class); + + assertEquals(1, calls.get()); + } + + @Test + public void eventsIgnoredAfterDispose() { + AtomicInteger calls = new AtomicInteger(); + + new Observable() { + @Override + protected void subscribeActual(@NonNull Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onNext(2); + observer.onComplete(); + } + } + .flatMapStream(v -> { + calls.getAndIncrement(); + return Stream.of(1); + }) + .take(1) + .test() + .assertResult(1); + + assertEquals(1, calls.get()); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStageTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStageTest.java new file mode 100644 index 0000000000..612ab0724b --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromCompletionStageTest.java @@ -0,0 +1,65 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.observers.TestObserver; + +public class ObservableFromCompletionStageTest extends RxJavaTest { + + @Test + public void syncSuccess() { + Observable.fromCompletionStage(CompletableFuture.completedFuture(1)) + .test() + .assertResult(1); + } + + @Test + public void syncFailure() { + CompletableFuture cf = new CompletableFuture<>(); + cf.completeExceptionally(new TestException()); + + Observable.fromCompletionStage(cf) + .test() + .assertFailure(TestException.class); + } + + @Test + public void syncNull() { + Observable.fromCompletionStage(CompletableFuture.completedFuture(null)) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void cancel() { + CompletableFuture cf = new CompletableFuture<>(); + + TestObserver to = Observable.fromCompletionStage(cf) + .test(); + + to.assertEmpty(); + + to.dispose(); + + cf.complete(1); + + to.assertEmpty(); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromOptionalTest.java new file mode 100644 index 0000000000..e2e4059e70 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromOptionalTest.java @@ -0,0 +1,38 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Optional; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; + +public class ObservableFromOptionalTest extends RxJavaTest { + + @Test + public void hasValue() { + Observable.fromOptional(Optional.of(1)) + .test() + .assertResult(1); + } + + @Test + public void empty() { + Observable.fromOptional(Optional.empty()) + .test() + .assertResult(); + } + +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStreamTest.java new file mode 100644 index 0000000000..6074e54689 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableFromStreamTest.java @@ -0,0 +1,488 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import java.util.Iterator; +import java.util.concurrent.atomic.*; +import java.util.stream.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.testsupport.*; + +public class ObservableFromStreamTest extends RxJavaTest { + + @Test + public void empty() { + Observable.fromStream(Stream.of()) + .test() + .assertResult(); + } + + @Test + public void just() { + Observable.fromStream(Stream.of(1)) + .test() + .assertResult(1); + } + + @Test + public void many() { + Observable.fromStream(Stream.of(1, 2, 3, 4, 5)) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void noReuse() { + Observable source = Observable.fromStream(Stream.of(1, 2, 3, 4, 5)); + + source + .test() + .assertResult(1, 2, 3, 4, 5); + + source + .test() + .assertFailure(IllegalStateException.class); + } + + @Test + public void take() { + Observable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .take(5) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void emptyConditional() { + Observable.fromStream(Stream.of()) + .filter(v -> true) + .test() + .assertResult(); + } + + @Test + public void justConditional() { + Observable.fromStream(Stream.of(1)) + .filter(v -> true) + .test() + .assertResult(1); + } + + @Test + public void manyConditional() { + Observable.fromStream(Stream.of(1, 2, 3, 4, 5)) + .filter(v -> true) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void manyConditionalSkip() { + Observable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .filter(v -> v % 2 == 0) + .test() + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void takeConditional() { + Observable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .filter(v -> true) + .take(5) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void noOfferNoCrashAfterClear() throws Throwable { + AtomicReference> queue = new AtomicReference<>(); + + Observable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .subscribe(new Observer() { + @Override + public void onSubscribe(@NonNull Disposable d) { + queue.set((SimpleQueue)d); + ((QueueDisposable)d).requestFusion(QueueFuseable.ANY); + } + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + }); + + SimpleQueue q = queue.get(); + TestHelper.assertNoOffer(q); + + assertFalse(q.isEmpty()); + + q.clear(); + + assertNull(q.poll()); + + assertTrue(q.isEmpty()); + + q.clear(); + + assertNull(q.poll()); + + assertTrue(q.isEmpty()); + } + + @Test + public void fusedPoll() throws Throwable { + AtomicReference> queue = new AtomicReference<>(); + AtomicInteger calls = new AtomicInteger(); + + Observable.fromStream(Stream.of(1).onClose(() -> calls.getAndIncrement())) + .subscribe(new Observer() { + @Override + public void onSubscribe(@NonNull Disposable d) { + queue.set((SimpleQueue)d); + ((QueueDisposable)d).requestFusion(QueueFuseable.ANY); + } + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + }); + + SimpleQueue q = queue.get(); + + assertFalse(q.isEmpty()); + + assertEquals(1, q.poll()); + + assertTrue(q.isEmpty()); + + assertEquals(1, calls.get()); + } + + @Test + public void fusedPoll2() throws Throwable { + AtomicReference> queue = new AtomicReference<>(); + AtomicInteger calls = new AtomicInteger(); + + Observable.fromStream(Stream.of(1, 2).onClose(() -> calls.getAndIncrement())) + .subscribe(new Observer() { + @Override + public void onSubscribe(@NonNull Disposable d) { + queue.set((SimpleQueue)d); + ((QueueDisposable)d).requestFusion(QueueFuseable.ANY); + } + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + }); + + SimpleQueue q = queue.get(); + + assertFalse(q.isEmpty()); + + assertEquals(1, q.poll()); + + assertFalse(q.isEmpty()); + + assertEquals(2, q.poll()); + + assertTrue(q.isEmpty()); + + assertEquals(1, calls.get()); + } + + @Test + public void streamOfNull() { + Observable.fromStream(Stream.of((Integer)null)) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void streamOfNullConditional() { + Observable.fromStream(Stream.of((Integer)null)) + .filter(v -> true) + .test() + .assertFailure(NullPointerException.class); + } + + @Test + public void syncFusionSupport() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + Observable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .subscribeWith(to) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + } + + @Test + public void asyncFusionNotSupported() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ASYNC); + + Observable.fromStream(IntStream.rangeClosed(1, 10).boxed()) + .subscribeWith(to) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + } + + @Test + public void runToEndCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Stream stream = Stream.of(1, 2, 3, 4, 5).onClose(() -> { throw new TestException(); }); + + Observable.fromStream(stream) + .test() + .assertResult(1, 2, 3, 4, 5); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void takeCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Stream stream = Stream.of(1, 2, 3, 4, 5).onClose(() -> { throw new TestException(); }); + + Observable.fromStream(stream) + .take(3) + .test() + .assertResult(1, 2, 3); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void hasNextCrash() { + AtomicInteger v = new AtomicInteger(); + Observable.fromStream(Stream.generate(() -> { + int value = v.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + })) + .test() + .assertFailure(TestException.class, 0); + } + + @Test + public void hasNextCrashConditional() { + AtomicInteger counter = new AtomicInteger(); + Observable.fromStream(Stream.generate(() -> { + int value = counter.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + })) + .filter(v -> true) + .test() + .assertFailure(TestException.class, 0); + } + + @Test + public void closeCalledOnEmpty() { + AtomicInteger calls = new AtomicInteger(); + + Observable.fromStream(Stream.of().onClose(() -> calls.getAndIncrement())) + .test() + .assertResult(); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledAfterItems() { + AtomicInteger calls = new AtomicInteger(); + + Observable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .test() + .assertResult(1, 2, 3, 4, 5); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnCancel() { + AtomicInteger calls = new AtomicInteger(); + + Observable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .take(3) + .test() + .assertResult(1, 2, 3); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnItemCrash() { + AtomicInteger calls = new AtomicInteger(); + AtomicInteger counter = new AtomicInteger(); + Observable.fromStream(Stream.generate(() -> { + int value = counter.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + }).onClose(() -> calls.getAndIncrement())) + .test() + .assertFailure(TestException.class, 0); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledAfterItemsConditional() { + AtomicInteger calls = new AtomicInteger(); + + Observable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .filter(v -> true) + .test() + .assertResult(1, 2, 3, 4, 5); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnCancelConditional() { + AtomicInteger calls = new AtomicInteger(); + + Observable.fromStream(Stream.of(1, 2, 3, 4, 5).onClose(() -> calls.getAndIncrement())) + .filter(v -> true) + .take(3) + .test() + .assertResult(1, 2, 3); + + assertEquals(1, calls.get()); + } + + @Test + public void closeCalledOnItemCrashConditional() { + AtomicInteger calls = new AtomicInteger(); + AtomicInteger counter = new AtomicInteger(); + Observable.fromStream(Stream.generate(() -> { + int value = counter.getAndIncrement(); + if (value == 1) { + throw new TestException(); + } + return value; + }).onClose(() -> calls.getAndIncrement())) + .filter(v -> true) + .test() + .assertFailure(TestException.class, 0); + + assertEquals(1, calls.get()); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(Observable.fromStream(Stream.of(1))); + } + + @Test + public void cancelAfterIteratorNext() throws Exception { + TestObserver to = new TestObserver<>(); + + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + to.dispose(); + return 1; + } + }); + + Observable.fromStream(stream) + .subscribe(to); + + to.assertEmpty(); + } + + @Test + public void cancelAfterIteratorHasNext() throws Exception { + TestObserver to = new TestObserver<>(); + + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + int calls; + + @Override + public boolean hasNext() { + if (++calls == 1) { + to.dispose(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Observable.fromStream(stream) + .subscribe(to); + + to.assertEmpty(); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptionalTest.java new file mode 100644 index 0000000000..ba38417b12 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableMapOptionalTest.java @@ -0,0 +1,394 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.assertFalse; + +import java.util.Optional; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.QueueFuseable; +import io.reactivex.rxjava3.subjects.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class ObservableMapOptionalTest extends RxJavaTest { + + static final Function> MODULO = v -> v % 2 == 0 ? Optional.of(v) : Optional.empty(); + + @Test + public void allPresent() { + Observable.range(1, 5) + .mapOptional(Optional::of) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void allEmpty() { + Observable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .test() + .assertResult(); + } + + @Test + public void mixed() { + Observable.range(1, 10) + .mapOptional(MODULO) + .test() + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void mapperChash() { + BehaviorSubject source = BehaviorSubject.createDefault(1); + + source + .mapOptional(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasObservers()); + } + + @Test + public void mapperNull() { + BehaviorSubject source = BehaviorSubject.createDefault(1); + + source + .mapOptional(v -> null) + .test() + .assertFailure(NullPointerException.class); + + assertFalse(source.hasObservers()); + } + + @Test + public void crashDropsOnNexts() { + Observable source = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onNext(2); + } + }; + + source + .mapOptional(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void syncFusedAll() { + Observable.range(1, 5) + .mapOptional(Optional::of) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void asyncFusedAll() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(Optional::of) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void boundaryFusedAll() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(Optional::of) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void syncFusedNone() { + Observable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(); + } + + @Test + public void asyncFusedNone() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(v -> Optional.empty()) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void boundaryFusedNone() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(v -> Optional.empty()) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(); + } + + @Test + public void syncFusedMixed() { + Observable.range(1, 10) + .mapOptional(MODULO) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void asyncFusedMixed() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + us + .mapOptional(MODULO) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void boundaryFusedMixed() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + us + .mapOptional(MODULO) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void allPresentConditional() { + Observable.range(1, 5) + .mapOptional(Optional::of) + .filter(v -> true) + .test() + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void allEmptyConditional() { + Observable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .test() + .assertResult(); + } + + @Test + public void mixedConditional() { + Observable.range(1, 10) + .mapOptional(MODULO) + .filter(v -> true) + .test() + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void mapperChashConditional() { + BehaviorSubject source = BehaviorSubject.createDefault(1); + + source + .mapOptional(v -> { throw new TestException(); }) + .filter(v -> true) + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasObservers()); + } + + @Test + public void mapperNullConditional() { + BehaviorSubject source = BehaviorSubject.createDefault(1); + + source + .mapOptional(v -> null) + .filter(v -> true) + .test() + .assertFailure(NullPointerException.class); + + assertFalse(source.hasObservers()); + } + + @Test + public void crashDropsOnNextsConditional() { + Observable source = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onNext(2); + } + }; + + source + .mapOptional(v -> { throw new TestException(); }) + .filter(v -> true) + .test() + .assertFailure(TestException.class); + } + + @Test + public void syncFusedAllConditional() { + Observable.range(1, 5) + .mapOptional(Optional::of) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void asyncFusedAllConditional() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(Optional::of) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void boundaryFusedAllConditiona() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(Optional::of) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void syncFusedNoneConditional() { + Observable.range(1, 5) + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(); + } + + @Test + public void asyncFusedNoneConditional() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void boundaryFusedNoneConditional() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); + + us + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(); + } + + @Test + public void syncFusedMixedConditional() { + Observable.range(1, 10) + .mapOptional(MODULO) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.SYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.SYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void asyncFusedMixedConditional() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + us + .mapOptional(MODULO) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC)) + .assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(2, 4, 6, 8, 10); + } + + @Test + public void boundaryFusedMixedConditional() { + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + us + .mapOptional(MODULO) + .filter(v -> true) + .to(TestHelper.testConsumer(false, QueueFuseable.ASYNC | QueueFuseable.BOUNDARY)) + .assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(2, 4, 6, 8, 10); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrDefaultTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrDefaultTest.java new file mode 100644 index 0000000000..9b02622c9b --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrDefaultTest.java @@ -0,0 +1,475 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.subjects.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class ObservableStageSubscriberOrDefaultTest extends RxJavaTest { + + @Test + public void firstJust() throws Exception { + Integer v = Observable.just(1) + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void firstEmpty() throws Exception { + Integer v = Observable.empty() + .firstStage(2) + .toCompletableFuture() + .get(); + + assertEquals((Integer)2, v); + } + + @Test + public void firstCancels() throws Exception { + BehaviorSubject source = BehaviorSubject.createDefault(1); + + Integer v = source + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + assertFalse(source.hasObservers()); + } + + @Test + public void firstCompletableFutureCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .firstStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void firstCompletableManualCompleteCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .firstStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void firstCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .firstStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstError() throws Exception { + CompletableFuture cf = Observable.error(new TestException()) + .firstStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void firstDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + } + } + .firstStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void singleJust() throws Exception { + Integer v = Observable.just(1) + .singleStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void singleEmpty() throws Exception { + Integer v = Observable.empty() + .singleStage(2) + .toCompletableFuture() + .get(); + + assertEquals((Integer)2, v); + } + + @Test + public void singleTooManyCancels() throws Exception { + ReplaySubject source = ReplaySubject.create(); + source.onNext(1); + source.onNext(2); + + TestHelper.assertError(source + .singleStage(null) + .toCompletableFuture(), IllegalArgumentException.class); + + assertFalse(source.hasObservers()); + } + + @Test + public void singleCompletableFutureCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .singleStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void singleCompletableManualCompleteCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .singleStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void singleCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .singleStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleError() throws Exception { + CompletableFuture cf = Observable.error(new TestException()) + .singleStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .singleStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void singleDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + } + } + .singleStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void lastJust() throws Exception { + Integer v = Observable.just(1) + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void lastRange() throws Exception { + Integer v = Observable.range(1, 5) + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)5, v); + } + + @Test + public void lastEmpty() throws Exception { + Integer v = Observable.empty() + .lastStage(2) + .toCompletableFuture() + .get(); + + assertEquals((Integer)2, v); + } + + @Test + public void lastCompletableFutureCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .lastStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void lastCompletableManualCompleteCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .lastStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void lastCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .lastStage(null) + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastError() throws Exception { + CompletableFuture cf = Observable.error(new TestException()) + .lastStage(null) + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void lastDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + } + } + .lastStage(null) + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrErrorTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrErrorTest.java new file mode 100644 index 0000000000..d8e6fbe918 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ObservableStageSubscriberOrErrorTest.java @@ -0,0 +1,469 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.subjects.*; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class ObservableStageSubscriberOrErrorTest extends RxJavaTest { + + @Test + public void firstJust() throws Exception { + Integer v = Observable.just(1) + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void firstEmpty() throws Exception { + TestHelper.assertError( + Observable.empty() + .firstOrErrorStage() + .toCompletableFuture(), NoSuchElementException.class); + } + + @Test + public void firstCancels() throws Exception { + BehaviorSubject source = BehaviorSubject.createDefault(1); + + Integer v = source + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + assertFalse(source.hasObservers()); + } + + @Test + public void firstCompletableFutureCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void firstCompletableManualCompleteCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void firstCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstError() throws Exception { + CompletableFuture cf = Observable.error(new TestException()) + .firstOrErrorStage() + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void firstSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void firstDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + } + } + .firstOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void singleJust() throws Exception { + Integer v = Observable.just(1) + .singleOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void singleEmpty() throws Exception { + TestHelper.assertError( + Observable.empty() + .singleOrErrorStage() + .toCompletableFuture(), NoSuchElementException.class); + } + + @Test + public void singleTooManyCancels() throws Exception { + ReplaySubject source = ReplaySubject.create(); + source.onNext(1); + source.onNext(2); + + TestHelper.assertError(source + .singleOrErrorStage() + .toCompletableFuture(), IllegalArgumentException.class); + + assertFalse(source.hasObservers()); + } + + @Test + public void singleCompletableFutureCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void singleCompletableManualCompleteCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void singleCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleError() throws Exception { + CompletableFuture cf = Observable.error(new TestException()) + .singleOrErrorStage() + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void singleSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .singleOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void singleDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + } + } + .singleOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } + + @Test + public void lastJust() throws Exception { + Integer v = Observable.just(1) + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + } + + @Test + public void lastRange() throws Exception { + Integer v = Observable.range(1, 5) + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)5, v); + } + + @Test + public void lastEmpty() throws Exception { + TestHelper.assertError(Observable.empty() + .lastOrErrorStage() + .toCompletableFuture(), NoSuchElementException.class); + } + + @Test + public void lastCompletableFutureCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.cancel(true); + + assertTrue(cf.isCancelled()); + + assertFalse(source.hasObservers()); + } + + @Test + public void lastCompletableManualCompleteCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.complete(1); + + assertTrue(cf.isDone()); + assertFalse(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + assertEquals((Integer)1, cf.get()); + } + + @Test + public void lastCompletableManualCompleteExceptionallyCancels() throws Exception { + PublishSubject source = PublishSubject.create(); + + CompletableFuture cf = source + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(source.hasObservers()); + + cf.completeExceptionally(new TestException()); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + assertFalse(source.hasObservers()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastError() throws Exception { + CompletableFuture cf = Observable.error(new TestException()) + .lastOrErrorStage() + .toCompletableFuture(); + + assertTrue(cf.isDone()); + assertTrue(cf.isCompletedExceptionally()); + assertFalse(cf.isCancelled()); + + TestHelper.assertError(cf, TestException.class); + } + + @Test + public void lastSourceIgnoresCancel() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + observer.onError(new TestException()); + observer.onComplete(); + } + } + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void lastDoubleOnSubscribe() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Integer v = new Observable() { + @Override + protected void subscribeActual(Observer observer) { + observer.onSubscribe(Disposable.empty()); + observer.onSubscribe(Disposable.empty()); + observer.onNext(1); + observer.onComplete(); + } + } + .lastOrErrorStage() + .toCompletableFuture() + .get(); + + assertEquals((Integer)1, v); + + TestHelper.assertError(errors, 0, ProtocolViolationException.class); + }); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java index e7625da6f9..0adfd655b0 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerTest.java @@ -882,14 +882,14 @@ public Integer call() throws Exception { @Test public void innerErrorAfterPoll() { - final UnicastProcessor us = UnicastProcessor.create(); - us.onNext(1); + final UnicastProcessor up = UnicastProcessor.create(); + up.onNext(1); TestSubscriber ts = new TestSubscriber() { @Override public void onNext(Integer t) { super.onNext(t); - us.onError(new TestException()); + up.onError(new TestException()); } }; @@ -897,7 +897,7 @@ public void onNext(Integer t) { .concatMapEager(new Function>() { @Override public Flowable apply(Integer v) throws Exception { - return us; + return up; } }, 1, 128) .subscribe(ts); @@ -973,12 +973,12 @@ public Integer apply(Integer v) throws Exception { @Test public void fuseAndTake() { - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - us.onNext(1); - us.onComplete(); + up.onNext(1); + up.onComplete(); - us.concatMapEager(new Function>() { + up.concatMapEager(new Function>() { @Override public Flowable apply(Integer v) throws Exception { return Flowable.just(1); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctTest.java index 70682d9e03..87d5cda476 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctTest.java @@ -128,13 +128,13 @@ public void fusedSync() { public void fusedAsync() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - us + up .distinct() .subscribe(ts); - TestHelper.emit(us, 1, 1, 2, 1, 3, 2, 4, 5, 4); + TestHelper.emit(up, 1, 1, 2, 1, 3, 2, 4, 5, 4); ts.assertFusionMode(QueueFuseable.ASYNC) .assertResult(1, 2, 3, 4, 5); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java index 86ad5aa874..586a7b56c2 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilterTest.java @@ -557,9 +557,9 @@ public boolean test(Integer v) throws Exception { public void fusedAsync() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - us + up .filter(new Predicate() { @Override public boolean test(Integer v) throws Exception { @@ -568,7 +568,7 @@ public boolean test(Integer v) throws Exception { }) .subscribe(ts); - TestHelper.emit(us, 1, 2, 3, 4, 5); + TestHelper.emit(up, 1, 2, 3, 4, 5); ts.assertFusionMode(QueueFuseable.ASYNC) .assertResult(2, 4); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java index c12aafa21d..1c0e6c58a3 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapTest.java @@ -583,13 +583,13 @@ public void fusedSync() { public void fusedAsync() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - us + up .map(Functions.identity()) .subscribe(ts); - TestHelper.emit(us, 1, 2, 3, 4, 5); + TestHelper.emit(up, 1, 2, 3, 4, 5); ts.assertFusionMode(QueueFuseable.ASYNC) .assertResult(1, 2, 3, 4, 5); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java index 73efdfc426..7fbf19fa60 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOnTest.java @@ -1193,11 +1193,11 @@ public void inputSyncFused() { @Test public void inputAsyncFused() { - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - TestSubscriber ts = us.observeOn(Schedulers.single()).test(); + TestSubscriber ts = up.observeOn(Schedulers.single()).test(); - TestHelper.emit(us, 1, 2, 3, 4, 5); + TestHelper.emit(up, 1, 2, 3, 4, 5); ts .awaitDone(5, TimeUnit.SECONDS) @@ -1206,11 +1206,11 @@ public void inputAsyncFused() { @Test public void inputAsyncFusedError() { - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - TestSubscriber ts = us.observeOn(Schedulers.single()).test(); + TestSubscriber ts = up.observeOn(Schedulers.single()).test(); - us.onError(new TestException()); + up.onError(new TestException()); ts .awaitDone(5, TimeUnit.SECONDS) @@ -1219,11 +1219,11 @@ public void inputAsyncFusedError() { @Test public void inputAsyncFusedErrorDelayed() { - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - TestSubscriber ts = us.observeOn(Schedulers.single(), true).test(); + TestSubscriber ts = up.observeOn(Schedulers.single(), true).test(); - us.onError(new TestException()); + up.onError(new TestException()); ts .awaitDone(5, TimeUnit.SECONDS) @@ -1260,12 +1260,12 @@ public void outputFusedReject() { public void inputOutputAsyncFusedError() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - us.observeOn(Schedulers.single()) + up.observeOn(Schedulers.single()) .subscribe(ts); - us.onError(new TestException()); + up.onError(new TestException()); ts .awaitDone(5, TimeUnit.SECONDS) @@ -1280,12 +1280,12 @@ public void inputOutputAsyncFusedError() { public void inputOutputAsyncFusedErrorDelayed() { TestSubscriberEx ts = new TestSubscriberEx().setInitialFusionMode(QueueFuseable.ANY); - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - us.observeOn(Schedulers.single(), true) + up.observeOn(Schedulers.single(), true) .subscribe(ts); - us.onError(new TestException()); + up.onError(new TestException()); ts .awaitDone(5, TimeUnit.SECONDS) @@ -1298,11 +1298,11 @@ public void inputOutputAsyncFusedErrorDelayed() { @Test public void outputFusedCancelReentrant() throws Exception { - final UnicastProcessor us = UnicastProcessor.create(); + final UnicastProcessor up = UnicastProcessor.create(); final CountDownLatch cdl = new CountDownLatch(1); - us.observeOn(Schedulers.single()) + up.observeOn(Schedulers.single()) .subscribe(new FlowableSubscriber() { Subscription upstream; int count; @@ -1315,7 +1315,7 @@ public void onSubscribe(Subscription s) { @Override public void onNext(Integer value) { if (++count == 1) { - us.onNext(2); + up.onNext(2); upstream.cancel(); cdl.countDown(); } @@ -1332,7 +1332,7 @@ public void onComplete() { } }); - us.onNext(1); + up.onNext(1); cdl.await(); } diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java index da1d67e819..190506aed2 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChangedTest.java @@ -169,9 +169,9 @@ public boolean test(Integer a, Integer b) throws Exception { public void fusedAsync() { TestObserverEx to = new TestObserverEx<>(QueueFuseable.ANY); - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - up + us .distinctUntilChanged(new BiPredicate() { @Override public boolean test(Integer a, Integer b) throws Exception { @@ -180,7 +180,7 @@ public boolean test(Integer a, Integer b) throws Exception { }) .subscribe(to); - TestHelper.emit(up, 1, 2, 2, 3, 3, 4, 5); + TestHelper.emit(us, 1, 2, 2, 3, 3, 4, 5); to.assertFuseable() .assertFusionMode(QueueFuseable.ASYNC) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java index 5b6cd18987..411989a0f1 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNextTest.java @@ -131,11 +131,11 @@ public void asyncFusedRejected() { public void asyncFused() { TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.ASYNC); - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - TestHelper.emit(up, 1, 2, 3, 4, 5); + TestHelper.emit(us, 1, 2, 3, 4, 5); - up + us .doAfterNext(afterNext) .subscribe(to0); @@ -228,11 +228,11 @@ public void asyncFusedRejectedConditional() { public void asyncFusedConditional() { TestObserverEx to0 = new TestObserverEx<>(QueueFuseable.ASYNC); - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - TestHelper.emit(up, 1, 2, 3, 4, 5); + TestHelper.emit(us, 1, 2, 3, 4, 5); - up + us .doAfterNext(afterNext) .filter(Functions.alwaysTrue()) .subscribe(to0); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java index a0eeccb405..0cc21b8f13 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinallyTest.java @@ -129,10 +129,10 @@ public void syncFusedBoundary() { public void asyncFused() { TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); - UnicastSubject up = UnicastSubject.create(); - TestHelper.emit(up, 1, 2, 3, 4, 5); + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); - up + us .doFinally(this) .subscribe(to); @@ -146,10 +146,10 @@ public void asyncFused() { public void asyncFusedBoundary() { TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC | QueueFuseable.BOUNDARY); - UnicastSubject up = UnicastSubject.create(); - TestHelper.emit(up, 1, 2, 3, 4, 5); + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); - up + us .doFinally(this) .subscribe(to); @@ -267,10 +267,10 @@ public void syncFusedBoundaryConditional() { public void asyncFusedConditional() { TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC); - UnicastSubject up = UnicastSubject.create(); - TestHelper.emit(up, 1, 2, 3, 4, 5); + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); - up + us .doFinally(this) .filter(Functions.alwaysTrue()) .subscribe(to); @@ -285,10 +285,10 @@ public void asyncFusedConditional() { public void asyncFusedBoundaryConditional() { TestObserverEx to = new TestObserverEx<>(QueueFuseable.ASYNC | QueueFuseable.BOUNDARY); - UnicastSubject up = UnicastSubject.create(); - TestHelper.emit(up, 1, 2, 3, 4, 5); + UnicastSubject us = UnicastSubject.create(); + TestHelper.emit(us, 1, 2, 3, 4, 5); - up + us .doFinally(this) .filter(Functions.alwaysTrue()) .subscribe(to); diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java index 0482d23906..6b8627b02b 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEachTest.java @@ -596,9 +596,9 @@ public void fusedAsync() { final int[] call = { 0, 0 }; - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - up + us .doOnNext(new Consumer() { @Override public void accept(Integer v) throws Exception { @@ -613,7 +613,7 @@ public void run() throws Exception { }) .subscribe(to); - TestHelper.emit(up, 1, 2, 3, 4, 5); + TestHelper.emit(us, 1, 2, 3, 4, 5); to.assertFuseable() .assertFusionMode(QueueFuseable.ASYNC) @@ -630,9 +630,9 @@ public void fusedAsyncConditional() { final int[] call = { 0, 0 }; - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - up + us .doOnNext(new Consumer() { @Override public void accept(Integer v) throws Exception { @@ -648,7 +648,7 @@ public void run() throws Exception { .filter(Functions.alwaysTrue()) .subscribe(to); - TestHelper.emit(up, 1, 2, 3, 4, 5); + TestHelper.emit(us, 1, 2, 3, 4, 5); to.assertFuseable() .assertFusionMode(QueueFuseable.ASYNC) @@ -665,9 +665,9 @@ public void fusedAsyncConditional2() { final int[] call = { 0, 0 }; - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - up.hide() + us.hide() .doOnNext(new Consumer() { @Override public void accept(Integer v) throws Exception { @@ -683,7 +683,7 @@ public void run() throws Exception { .filter(Functions.alwaysTrue()) .subscribe(to); - TestHelper.emit(up, 1, 2, 3, 4, 5); + TestHelper.emit(us, 1, 2, 3, 4, 5); to.assertFuseable() .assertFusionMode(QueueFuseable.NONE) diff --git a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java index 579b713313..2b8c2e0274 100644 --- a/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java +++ b/src/test/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOnTest.java @@ -775,12 +775,12 @@ public void workerNotDisposedPrematurelySyncInNormalOut() { public void workerNotDisposedPrematurelyAsyncInNormalOut() { DisposeTrackingScheduler s = new DisposeTrackingScheduler(); - UnicastSubject up = UnicastSubject.create(); - up.onNext(1); - up.onComplete(); + UnicastSubject us = UnicastSubject.create(); + us.onNext(1); + us.onComplete(); Observable.concat( - up.observeOn(s), + us.observeOn(s), Observable.just(2) ) .test() diff --git a/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java index 53b7062028..002d6a17d3 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/UnicastProcessorTest.java @@ -131,14 +131,14 @@ public void run() { public void onTerminateCalledWhenOnError() { final AtomicBoolean didRunOnTerminate = new AtomicBoolean(); - UnicastProcessor us = UnicastProcessor.create(Observable.bufferSize(), new Runnable() { + UnicastProcessor up = UnicastProcessor.create(Observable.bufferSize(), new Runnable() { @Override public void run() { didRunOnTerminate.set(true); } }); assertFalse(didRunOnTerminate.get()); - us.onError(new RuntimeException("some error")); + up.onError(new RuntimeException("some error")); assertTrue(didRunOnTerminate.get()); } @@ -146,14 +146,14 @@ public void onTerminateCalledWhenOnError() { public void onTerminateCalledWhenOnComplete() { final AtomicBoolean didRunOnTerminate = new AtomicBoolean(); - UnicastProcessor us = UnicastProcessor.create(Observable.bufferSize(), new Runnable() { + UnicastProcessor up = UnicastProcessor.create(Observable.bufferSize(), new Runnable() { @Override public void run() { didRunOnTerminate.set(true); } }); assertFalse(didRunOnTerminate.get()); - us.onComplete(); + up.onComplete(); assertTrue(didRunOnTerminate.get()); } @@ -161,13 +161,13 @@ public void onTerminateCalledWhenOnComplete() { public void onTerminateCalledWhenCanceled() { final AtomicBoolean didRunOnTerminate = new AtomicBoolean(); - UnicastProcessor us = UnicastProcessor.create(Observable.bufferSize(), new Runnable() { + UnicastProcessor up = UnicastProcessor.create(Observable.bufferSize(), new Runnable() { @Override public void run() { didRunOnTerminate.set(true); } }); - final Disposable subscribe = us.subscribe(); + final Disposable subscribe = up.subscribe(); assertFalse(didRunOnTerminate.get()); subscribe.dispose(); @@ -327,7 +327,7 @@ public void run() { @Test public void subscribeRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { - final UnicastProcessor us = UnicastProcessor.create(); + final UnicastProcessor up = UnicastProcessor.create(); final TestSubscriberEx ts1 = new TestSubscriberEx<>(); final TestSubscriberEx ts2 = new TestSubscriberEx<>(); @@ -335,14 +335,14 @@ public void subscribeRace() { Runnable r1 = new Runnable() { @Override public void run() { - us.subscribe(ts1); + up.subscribe(ts1); } }; Runnable r2 = new Runnable() { @Override public void run() { - us.subscribe(ts2); + up.subscribe(ts2); } }; @@ -361,67 +361,67 @@ public void run() { @Test public void hasObservers() { - UnicastProcessor us = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); - assertFalse(us.hasSubscribers()); + assertFalse(up.hasSubscribers()); - TestSubscriber ts = us.test(); + TestSubscriber ts = up.test(); - assertTrue(us.hasSubscribers()); + assertTrue(up.hasSubscribers()); ts.cancel(); - assertFalse(us.hasSubscribers()); + assertFalse(up.hasSubscribers()); } @Test public void drainFusedFailFast() { - UnicastProcessor us = UnicastProcessor.create(false); + UnicastProcessor up = UnicastProcessor.create(false); - TestSubscriberEx ts = us.to(TestHelper.testSubscriber(1, QueueFuseable.ANY, false)); + TestSubscriberEx ts = up.to(TestHelper.testSubscriber(1, QueueFuseable.ANY, false)); - us.done = true; - us.drainFused(ts); + up.done = true; + up.drainFused(ts); ts.assertResult(); } @Test public void drainFusedFailFastEmpty() { - UnicastProcessor us = UnicastProcessor.create(false); + UnicastProcessor up = UnicastProcessor.create(false); - TestSubscriberEx ts = us.to(TestHelper.testSubscriber(1, QueueFuseable.ANY, false)); + TestSubscriberEx ts = up.to(TestHelper.testSubscriber(1, QueueFuseable.ANY, false)); - us.drainFused(ts); + up.drainFused(ts); ts.assertEmpty(); } @Test public void checkTerminatedFailFastEmpty() { - UnicastProcessor us = UnicastProcessor.create(false); + UnicastProcessor up = UnicastProcessor.create(false); - TestSubscriberEx ts = us.to(TestHelper.testSubscriber(1, QueueFuseable.ANY, false)); + TestSubscriberEx ts = up.to(TestHelper.testSubscriber(1, QueueFuseable.ANY, false)); - us.checkTerminated(true, true, false, ts, us.queue); + up.checkTerminated(true, true, false, ts, up.queue); ts.assertEmpty(); } @Test public void alreadyCancelled() { - UnicastProcessor us = UnicastProcessor.create(false); + UnicastProcessor up = UnicastProcessor.create(false); - us.test().cancel(); + up.test().cancel(); BooleanSubscription bs = new BooleanSubscription(); - us.onSubscribe(bs); + up.onSubscribe(bs); assertTrue(bs.isCancelled()); List errors = TestHelper.trackPluginErrors(); try { - us.onError(new TestException()); + up.onError(new TestException()); TestHelper.assertUndeliverable(errors, 0, TestException.class); } finally { @@ -431,9 +431,9 @@ public void alreadyCancelled() { @Test public void unicastSubscriptionBadRequest() { - UnicastProcessor us = UnicastProcessor.create(false); + UnicastProcessor up = UnicastProcessor.create(false); - UnicastProcessor.UnicastQueueSubscription usc = (UnicastProcessor.UnicastQueueSubscription)us.wip; + UnicastProcessor.UnicastQueueSubscription usc = (UnicastProcessor.UnicastQueueSubscription)up.wip; List errors = TestHelper.trackPluginErrors(); try { @@ -449,17 +449,17 @@ public void fusedNoConcurrentCleanDueToCancel() { for (int j = 0; j < TestHelper.RACE_LONG_LOOPS; j++) { List errors = TestHelper.trackPluginErrors(); try { - final UnicastProcessor us = UnicastProcessor.create(); + final UnicastProcessor up = UnicastProcessor.create(); - TestObserver to = us + TestObserver to = up .observeOn(Schedulers.io()) .map(Functions.identity()) .observeOn(Schedulers.single()) .firstOrError() .test(); - for (int i = 0; us.hasSubscribers(); i++) { - us.onNext(i); + for (int i = 0; up.hasSubscribers(); i++) { + up.onNext(i); } to diff --git a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java index 17e4cb0a72..ac000e4228 100644 --- a/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java +++ b/src/test/java/io/reactivex/rxjava3/subjects/UnicastSubjectTest.java @@ -228,14 +228,14 @@ public void zeroCapacityHint() { public void completeCancelRace() { for (int i = 0; i < TestHelper.RACE_DEFAULT_LOOPS; i++) { final int[] calls = { 0 }; - final UnicastSubject up = UnicastSubject.create(100, new Runnable() { + final UnicastSubject us = UnicastSubject.create(100, new Runnable() { @Override public void run() { calls[0]++; } }); - final TestObserver to = up.test(); + final TestObserver to = us.test(); Runnable r1 = new Runnable() { @Override @@ -247,7 +247,7 @@ public void run() { Runnable r2 = new Runnable() { @Override public void run() { - up.onComplete(); + us.onComplete(); } }; diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java index 9ee8a1cfc5..0d6edd6c3d 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestObserverExTest.java @@ -1094,16 +1094,16 @@ public void asyncQueueThrows() { TestObserverEx to = new TestObserverEx<>(); to.setInitialFusionMode(QueueFuseable.ANY); - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - up + us .map(new Function() { @Override public Object apply(Integer v) throws Exception { throw new TestException(); } }) .subscribe(to); - up.onNext(1); + us.onNext(1); to.assertSubscribed() .assertFuseable() @@ -1132,13 +1132,13 @@ public void asyncFusion() { TestObserverEx to = new TestObserverEx<>(); to.setInitialFusionMode(QueueFuseable.ANY); - UnicastSubject up = UnicastSubject.create(); + UnicastSubject us = UnicastSubject.create(); - up + us .subscribe(to); - up.onNext(1); - up.onComplete(); + us.onNext(1); + us.onComplete(); to.assertSubscribed() .assertFuseable() diff --git a/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java b/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java index 40a976a165..df6f0a378a 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java +++ b/src/test/java/io/reactivex/rxjava3/validators/CheckLocalVariablesInTests.java @@ -94,7 +94,15 @@ static void findPattern(String pattern, boolean checkMain) throws Exception { .append(fname) .append("#L").append(lineNum) .append(" ").append(line) - .append("\n"); + .append("\n") + .append(" at ") + .append(fname.replace(".java", "")) + .append(".method(") + .append(fname) + .append(":") + .append(lineNum) + .append(")\n"); + total++; } } @@ -111,9 +119,7 @@ static void findPattern(String pattern, boolean checkMain) throws Exception { } } if (total != 0) { - fail.append("Found ") - .append(total) - .append(" instances"); + fail.insert(0, "Found " + total + " instances"); System.out.println(fail); throw new AssertionError(fail.toString()); } @@ -149,6 +155,16 @@ public void publishProcessorAsPs() throws Exception { findPattern("PublishProcessor<.*>\\s+ps"); } + @Test + public void unicastSubjectAsUp() throws Exception { + findPattern("UnicastSubject<.*>\\s+up"); + } + + @Test + public void unicastProcessorAsUs() throws Exception { + findPattern("UnicastProcessor<.*>\\s+us"); + } + @Test public void behaviorProcessorAsBs() throws Exception { findPattern("BehaviorProcessor<.*>\\s+bs"); diff --git a/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java b/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java index e8219df148..4a4baaf8ae 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java +++ b/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java @@ -57,8 +57,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { && !m.signature.contains("Flowable") && !m.signature.contains("Observable") && !m.signature.contains("ObservableSource")) { - e.append("java.lang.RuntimeException: Maybe doc mentions onNext but no Flowable/Observable in signature\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions onNext but no Flowable/Observable in signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -74,8 +74,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { && !m.signature.contains("Flowable") && !m.signature.contains("TestSubscriber") ) { - e.append("java.lang.RuntimeException: Maybe doc mentions Subscriber but not using Flowable\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions Subscriber but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -90,8 +90,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { if (!m.signature.contains("Publisher") && !m.signature.contains("Flowable") ) { - e.append("java.lang.RuntimeException: Maybe doc mentions Subscription but not using Flowable\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions Subscription but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -108,8 +108,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { && !m.signature.contains("TestObserver")) { if (idx < 5 || !m.javadoc.substring(idx - 5, idx + 8).equals("MaybeObserver")) { - e.append("java.lang.RuntimeException: Maybe doc mentions Observer but not using Observable\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions Observer but not using Observable\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -124,8 +124,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { if (idx >= 0) { if (!m.signature.contains("Publisher")) { if (idx == 0 || !m.javadoc.substring(idx - 1, idx + 9).equals("(Publisher")) { - e.append("java.lang.RuntimeException: Maybe doc mentions Publisher but not in the signature\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions Publisher but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -139,8 +139,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { int idx = m.javadoc.indexOf("Flowable", jdx); if (idx >= 0) { if (!m.signature.contains("Flowable")) { - e.append("java.lang.RuntimeException: Maybe doc mentions Flowable but not in the signature\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions Flowable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -154,7 +154,7 @@ public void maybeDocRefersToMaybeTypes() throws Exception { int j = m.javadoc.indexOf("#toSingle", jdx); int k = m.javadoc.indexOf("{@code Single", jdx); if (!m.signature.contains("Single") && (j + 3 != idx && k + 7 != idx)) { - e.append("java.lang.RuntimeException: Maybe doc mentions Single but not in the signature\r\n at io.reactivex.") + e.append("java.lang.RuntimeException: Maybe doc mentions Single but not in the signature\r\n at io.reactivex.rxjava3.core.") .append("Maybe(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -167,8 +167,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { int idx = m.javadoc.indexOf("SingleSource", jdx); if (idx >= 0) { if (!m.signature.contains("SingleSource")) { - e.append("java.lang.RuntimeException: Maybe doc mentions SingleSource but not in the signature\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions SingleSource but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -180,8 +180,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { int idx = m.javadoc.indexOf("Observable", jdx); if (idx >= 0) { if (!m.signature.contains("Observable")) { - e.append("java.lang.RuntimeException: Maybe doc mentions Observable but not in the signature\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions Observable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -193,8 +193,8 @@ public void maybeDocRefersToMaybeTypes() throws Exception { int idx = m.javadoc.indexOf("ObservableSource", jdx); if (idx >= 0) { if (!m.signature.contains("ObservableSource")) { - e.append("java.lang.RuntimeException: Maybe doc mentions ObservableSource but not in the signature\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Maybe doc mentions ObservableSource but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -233,8 +233,8 @@ public void flowableDocRefersToFlowableTypes() throws Exception { && !m.signature.contains("MaybeSource") && !m.signature.contains("Single") && !m.signature.contains("SingleSource")) { - e.append("java.lang.RuntimeException: Flowable doc mentions onSuccess\r\n at io.reactivex.") - .append("Flowable (Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Flowable doc mentions onSuccess\r\n at io.reactivex.rxjava3.core.") + .append("Flowable.method(Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -248,8 +248,8 @@ public void flowableDocRefersToFlowableTypes() throws Exception { if (idx >= 0) { if (!m.signature.contains("ObservableSource") && !m.signature.contains("Observable")) { - e.append("java.lang.RuntimeException: Flowable doc mentions Observer but not using Flowable\r\n at io.reactivex.") - .append("Flowable (Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Flowable doc mentions Observer but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Flowable.method(Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -273,8 +273,8 @@ public void flowableDocRefersToFlowableTypes() throws Exception { ) { CharSequence subSequence = m.javadoc.subSequence(idx - 6, idx + 11); if (idx < 6 || !subSequence.equals("{@link Disposable")) { - e.append("java.lang.RuntimeException: Flowable doc mentions Disposable but not using Flowable\r\n at io.reactivex.") - .append("Flowable (Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Flowable doc mentions Disposable but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Flowable.method(Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -288,8 +288,8 @@ public void flowableDocRefersToFlowableTypes() throws Exception { int idx = m.javadoc.indexOf("Observable", jdx); if (idx >= 0) { if (!m.signature.contains("Observable")) { - e.append("java.lang.RuntimeException: Flowable doc mentions Observable but not in the signature\r\n at io.reactivex.") - .append("Flowable (Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Flowable doc mentions Observable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Flowable.method(Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -302,8 +302,8 @@ public void flowableDocRefersToFlowableTypes() throws Exception { int idx = m.javadoc.indexOf("ObservableSource", jdx); if (idx >= 0) { if (!m.signature.contains("ObservableSource")) { - e.append("java.lang.RuntimeException: Flowable doc mentions ObservableSource but not in the signature\r\n at io.reactivex.") - .append("Flowable (Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Flowable doc mentions ObservableSource but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Flowable.method(Flowable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -342,8 +342,8 @@ public void observableDocRefersToObservableTypes() throws Exception { && !m.signature.contains("MaybeSource") && !m.signature.contains("Single") && !m.signature.contains("SingleSource")) { - e.append("java.lang.RuntimeException: Observable doc mentions onSuccess\r\n at io.reactivex.") - .append("Observable (Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Observable doc mentions onSuccess\r\n at io.reactivex.rxjava3.core.") + .append("Observable.method(Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -358,8 +358,8 @@ public void observableDocRefersToObservableTypes() throws Exception { if (!m.signature.contains("Flowable") && !m.signature.contains("Publisher") ) { - e.append("java.lang.RuntimeException: Observable doc mentions Subscription but not using Flowable\r\n at io.reactivex.") - .append("Observable (Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Observable doc mentions Subscription but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Observable.method(Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -373,8 +373,8 @@ public void observableDocRefersToObservableTypes() throws Exception { if (idx >= 0) { if (!m.signature.contains("Flowable")) { if (idx < 6 || !m.javadoc.substring(idx - 6, idx + 8).equals("@link Flowable")) { - e.append("java.lang.RuntimeException: Observable doc mentions Flowable but not in the signature\r\n at io.reactivex.") - .append("Observable (Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Observable doc mentions Flowable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Observable.method(Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -388,8 +388,8 @@ public void observableDocRefersToObservableTypes() throws Exception { int idx = m.javadoc.indexOf("Publisher", jdx); if (idx >= 0) { if (!m.signature.contains("Publisher")) { - e.append("java.lang.RuntimeException: Observable doc mentions Publisher but not in the signature\r\n at io.reactivex.") - .append("Observable (Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Observable doc mentions Publisher but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Observable.method(Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -402,8 +402,8 @@ public void observableDocRefersToObservableTypes() throws Exception { if (idx >= 0) { if (!m.signature.contains("Publisher") && !m.signature.contains("Flowable")) { - e.append("java.lang.RuntimeException: Observable doc mentions Subscriber but not using Flowable\r\n at io.reactivex.") - .append("Observable (Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Observable doc mentions Subscriber but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Observable.method(Observable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -443,8 +443,8 @@ public void singleDocRefersToSingleTypes() throws Exception { && !m.signature.contains("Flowable") && !m.signature.contains("Observable") && !m.signature.contains("ObservableSource")) { - e.append("java.lang.RuntimeException: Single doc mentions onNext but no Flowable/Observable in signature\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions onNext but no Flowable/Observable in signature\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -459,8 +459,8 @@ public void singleDocRefersToSingleTypes() throws Exception { if (!m.signature.contains("Publisher") && !m.signature.contains("Flowable") && !m.signature.contains("TestSubscriber")) { - e.append("java.lang.RuntimeException: Single doc mentions Subscriber but not using Flowable\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions Subscriber but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -475,8 +475,8 @@ public void singleDocRefersToSingleTypes() throws Exception { if (!m.signature.contains("Flowable") && !m.signature.contains("Publisher") ) { - e.append("java.lang.RuntimeException: Single doc mentions Subscription but not using Flowable\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions Subscription but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -493,8 +493,8 @@ public void singleDocRefersToSingleTypes() throws Exception { && !m.signature.contains("TestObserver")) { if (idx < 6 || !m.javadoc.substring(idx - 6, idx + 8).equals("SingleObserver")) { - e.append("java.lang.RuntimeException: Single doc mentions Observer but not using Observable\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions Observer but not using Observable\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -509,8 +509,8 @@ public void singleDocRefersToSingleTypes() throws Exception { if (idx >= 0) { if (!m.signature.contains("Publisher")) { if (idx == 0 || !m.javadoc.substring(idx - 1, idx + 9).equals("(Publisher")) { - e.append("java.lang.RuntimeException: Single doc mentions Publisher but not in the signature\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions Publisher but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -524,8 +524,8 @@ public void singleDocRefersToSingleTypes() throws Exception { int idx = m.javadoc.indexOf(" Flowable", jdx); if (idx >= 0) { if (!m.signature.contains("Flowable")) { - e.append("java.lang.RuntimeException: Single doc mentions Flowable but not in the signature\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions Flowable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -537,8 +537,8 @@ public void singleDocRefersToSingleTypes() throws Exception { int idx = m.javadoc.indexOf(" Maybe", jdx); if (idx >= 0) { if (!m.signature.contains("Maybe")) { - e.append("java.lang.RuntimeException: Single doc mentions Maybe but not in the signature\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions Maybe but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -550,8 +550,8 @@ public void singleDocRefersToSingleTypes() throws Exception { int idx = m.javadoc.indexOf(" MaybeSource", jdx); if (idx >= 0) { if (!m.signature.contains("MaybeSource")) { - e.append("java.lang.RuntimeException: Single doc mentions SingleSource but not in the signature\r\n at io.reactivex.") - .append("Maybe (Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions SingleSource but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -563,8 +563,8 @@ public void singleDocRefersToSingleTypes() throws Exception { int idx = m.javadoc.indexOf(" Observable", jdx); if (idx >= 0) { if (!m.signature.contains("Observable")) { - e.append("java.lang.RuntimeException: Single doc mentions Observable but not in the signature\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions Observable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -576,8 +576,8 @@ public void singleDocRefersToSingleTypes() throws Exception { int idx = m.javadoc.indexOf(" ObservableSource", jdx); if (idx >= 0) { if (!m.signature.contains("ObservableSource")) { - e.append("java.lang.RuntimeException: Single doc mentions ObservableSource but not in the signature\r\n at io.reactivex.") - .append("Single (Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Single doc mentions ObservableSource but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Single.method(Single.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; } else { @@ -617,8 +617,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { && !m.signature.contains("Flowable") && !m.signature.contains("Observable") && !m.signature.contains("ObservableSource")) { - e.append("java.lang.RuntimeException: Completable doc mentions onNext but no Flowable/Observable in signature\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions onNext but no Flowable/Observable in signature\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -633,8 +633,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (!m.signature.contains("Publisher") && !m.signature.contains("Flowable") && !m.signature.contains("TestSubscriber")) { - e.append("java.lang.RuntimeException: Completable doc mentions Subscriber but not using Flowable\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions Subscriber but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -649,8 +649,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (!m.signature.contains("Flowable") && !m.signature.contains("Publisher") ) { - e.append("java.lang.RuntimeException: Completable doc mentions Subscription but not using Flowable\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions Subscription but not using Flowable\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } jdx = idx + 6; @@ -667,8 +667,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { && !m.signature.contains("TestObserver")) { if (idx < 11 || !m.javadoc.substring(idx - 11, idx + 8).equals("CompletableObserver")) { - e.append("java.lang.RuntimeException: Completable doc mentions Observer but not using Observable\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions Observer but not using Observable\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -683,8 +683,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (idx >= 0) { if (!m.signature.contains("Publisher")) { if (idx == 0 || !m.javadoc.substring(idx - 1, idx + 9).equals("(Publisher")) { - e.append("java.lang.RuntimeException: Completable doc mentions Publisher but not in the signature\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions Publisher but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } @@ -700,8 +700,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (!m.signature.contains("Flowable")) { Pattern p = Pattern.compile("@see\\s+#[A-Za-z0-9 _.,()]*Flowable"); if (!p.matcher(m.javadoc).find()) { - e.append("java.lang.RuntimeException: Completable doc mentions Flowable but not in the signature\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions Flowable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } jdx = idx + 6; @@ -716,8 +716,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (!m.signature.contains("Single")) { Pattern p = Pattern.compile("@see\\s+#[A-Za-z0-9 _.,()]*Single"); if (!p.matcher(m.javadoc).find()) { - e.append("java.lang.RuntimeException: Completable doc mentions Single but not in the signature\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions Single but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } jdx = idx + 6; @@ -732,8 +732,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (!m.signature.contains("SingleSource")) { Pattern p = Pattern.compile("@see\\s+#[A-Za-z0-9 _.,()]*SingleSource"); if (!p.matcher(m.javadoc).find()) { - e.append("java.lang.RuntimeException: Completable doc mentions SingleSource but not in the signature\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions SingleSource but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } jdx = idx + 6; @@ -748,8 +748,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (!m.signature.contains("Observable")) { Pattern p = Pattern.compile("@see\\s+#[A-Za-z0-9 _.,()]*Observable"); if (!p.matcher(m.javadoc).find()) { - e.append("java.lang.RuntimeException: Completable doc mentions Observable but not in the signature\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions Observable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } jdx = idx + 6; @@ -764,8 +764,8 @@ public void completableDocRefersToCompletableTypes() throws Exception { if (!m.signature.contains("ObservableSource")) { Pattern p = Pattern.compile("@see\\s+#[A-Za-z0-9 _.,()]*ObservableSource"); if (!p.matcher(m.javadoc).find()) { - e.append("java.lang.RuntimeException: Completable doc mentions ObservableSource but not in the signature\r\n at io.reactivex.") - .append("Completable (Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + e.append("java.lang.RuntimeException: Completable doc mentions ObservableSource but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Completable.method(Completable.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); } } jdx = idx + 6; @@ -807,9 +807,9 @@ static void aOrAn(StringBuilder e, RxMethod m, String wrongPre, String word, Str if (idx >= 0) { e.append("java.lang.RuntimeException: a/an typo ") .append(word) - .append("\r\n at io.reactivex.") + .append("\r\n at io.reactivex.rxjava3.core.") .append(baseTypeName) - .append(" (") + .append(".method(") .append(baseTypeName) .append(".java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); jdx = idx + 6; @@ -824,9 +824,9 @@ static void aOrAn(StringBuilder e, RxMethod m, String wrongPre, String word, Str if (idx >= 0) { e.append("java.lang.RuntimeException: a/an typo ") .append(word) - .append("\r\n at io.reactivex.") + .append("\r\n at io.reactivex.rxjava3.core.") .append(baseTypeName) - .append(" (") + .append(".method(") .append(baseTypeName) .append(".java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); jdx = idx + 6; @@ -841,9 +841,9 @@ static void aOrAn(StringBuilder e, RxMethod m, String wrongPre, String word, Str if (idx >= 0) { e.append("java.lang.RuntimeException: a/an typo ") .append(word) - .append("\r\n at io.reactivex.") + .append("\r\n at io.reactivex.rxjava3.core.") .append(baseTypeName) - .append(" (") + .append(".method(") .append(baseTypeName) .append(".java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); jdx = idx + 6; @@ -858,9 +858,9 @@ static void aOrAn(StringBuilder e, RxMethod m, String wrongPre, String word, Str if (idx >= 0) { e.append("java.lang.RuntimeException: a/an typo ") .append(word) - .append("\r\n at io.reactivex.") + .append("\r\n at io.reactivex.rxjava3.core.") .append(baseTypeName) - .append(" (") + .append(".method(") .append(baseTypeName) .append(".java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); jdx = idx + 6; @@ -895,9 +895,9 @@ static void aOrAn(StringBuilder e, RxMethod m, String wrongPre, String word, Str if (idx >= 0) { e.append("java.lang.RuntimeException: a/an typo ") .append(word) - .append("\r\n at io.reactivex.") + .append("\r\n at io.reactivex.rxjava3.core.") .append(baseTypeName) - .append(" (") + .append(".method(") .append(baseTypeName) .append(".java:").append(m.javadocLine).append(")\r\n\r\n"); jdx = idx + wrongPre.length() + 1 + word.length(); @@ -923,9 +923,9 @@ static void missingClosingDD(StringBuilder e, RxMethod m, String baseTypeName) { jdx = idx2 + 5; } else { e.append("java.lang.RuntimeException: unbalanced
") - .append("\r\n at io.reactivex.") + .append("\r\n at io.reactivex.rxjava3.core.") .append(baseTypeName) - .append(" (") + .append(".method(") .append(baseTypeName) .append(".java:").append(m.javadocLine + lineNumber(m.javadoc, idx1) - 1).append(")\r\n\r\n"); break; @@ -936,9 +936,9 @@ static void missingClosingDD(StringBuilder e, RxMethod m, String baseTypeName) { static void backpressureMentionedWithoutAnnotation(StringBuilder e, RxMethod m, String baseTypeName) { if (m.backpressureDocLine > 0 && m.backpressureKind == null) { e.append("java.lang.RuntimeException: backpressure documented but not annotated ") - .append("\r\n at io.reactivex.") + .append("\r\n at io.reactivex.rxjava3.core.") .append(baseTypeName) - .append(" (") + .append(".method(") .append(baseTypeName) .append(".java:").append(m.backpressureDocLine).append(")\r\n\r\n"); } diff --git a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java index 5bc4306492..d8c3e32636 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java +++ b/src/test/java/io/reactivex/rxjava3/validators/ParamValidationCheckerTest.java @@ -509,6 +509,10 @@ public void checkParallelFlowable() { addOverride(new ParamOverride(Flowable.class, 0, ParamMode.ANY, "singleStage", Object.class)); addOverride(new ParamOverride(Flowable.class, 0, ParamMode.ANY, "lastStage", Object.class)); + addOverride(new ParamOverride(Observable.class, 0, ParamMode.ANY, "firstStage", Object.class)); + addOverride(new ParamOverride(Observable.class, 0, ParamMode.ANY, "singleStage", Object.class)); + addOverride(new ParamOverride(Observable.class, 0, ParamMode.ANY, "lastStage", Object.class)); + addOverride(new ParamOverride(Maybe.class, 0, ParamMode.ANY, "toCompletionStage", Object.class)); addOverride(new ParamOverride(Completable.class, 0, ParamMode.ANY, "toCompletionStage", Object.class)); From 3d00eb188bad1d49aa058e896ad9d1b5026eba7d Mon Sep 17 00:00:00 2001 From: David Karnok Date: Thu, 26 Dec 2019 22:37:39 +0100 Subject: [PATCH 025/665] 3.x: [Java 8] Add ParallelFlowable operators + cleanup (#6798) --- .../io/reactivex/rxjava3/core/Flowable.java | 6 +- .../java/io/reactivex/rxjava3/core/Maybe.java | 2 +- .../io/reactivex/rxjava3/core/Observable.java | 4 +- .../internal/jdk8/FlowableFlatMapStream.java | 15 +- .../internal/jdk8/ParallelCollector.java | 270 +++++++++++++ .../internal/jdk8/ParallelFlatMapStream.java | 69 ++++ .../internal/jdk8/ParallelMapOptional.java | 236 +++++++++++ .../internal/jdk8/ParallelMapTryOptional.java | 306 ++++++++++++++ .../flowable/FlowableFlattenIterable.java | 20 +- .../operators/parallel/ParallelCollect.java | 2 +- .../parallel/ParallelDoOnNextTry.java | 8 +- .../operators/parallel/ParallelFilter.java | 4 +- .../operators/parallel/ParallelFilterTry.java | 8 +- .../parallel/ParallelFlatMapIterable.java | 68 ++++ .../parallel/ParallelFromPublisher.java | 4 +- .../operators/parallel/ParallelJoin.java | 8 +- .../operators/parallel/ParallelMapTry.java | 4 +- .../operators/parallel/ParallelPeek.java | 2 +- .../operators/parallel/ParallelReduce.java | 2 +- .../parallel/ParallelReduceFull.java | 11 +- .../operators/parallel/ParallelRunOn.java | 6 +- .../parallel/ParallelSortedJoin.java | 6 +- .../rxjava3/parallel/ParallelFlowable.java | 310 +++++++++++++- .../internal/jdk8/ParallelCollectorTest.java | 305 ++++++++++++++ .../jdk8/ParallelFlatMapStreamTest.java | 73 ++++ .../jdk8/ParallelMapOptionalTest.java | 228 +++++++++++ .../jdk8/ParallelMapTryOptionalTest.java | 378 ++++++++++++++++++ .../parallel/ParallelFlatMapIterableTest.java | 73 ++++ .../rxjava3/parallel/ParallelMapTest.java | 22 +- .../rxjava3/parallel/ParallelMapTryTest.java | 22 +- .../rxjava3/testsupport/TestHelper.java | 65 +++ .../validators/TooManyEmptyNewLines.java | 13 +- 32 files changed, 2497 insertions(+), 53 deletions(-) create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStream.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptional.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptional.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFlatMapIterable.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollectorTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStreamTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptionalTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptionalTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/parallel/ParallelFlatMapIterableTest.java diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 15b1864d8b..57efac10e0 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -18944,9 +18944,9 @@ public final TestSubscriber test(long initialRequest, boolean cancel) { // No * @param the element type of the optional value * @param optional the optional value to convert into a {@code Flowable} * @return the new Flowable instance + * @since 3.0.0 * @see #just(Object) * @see #empty() - * @since 3.0.0 */ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @@ -19409,6 +19409,7 @@ public final Stream blockingStream(int prefetch) { * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements * will be emitted to the downstream * @return the new Flowable instance + * @since 3.0.0 * @see #concatMap(Function) * @see #concatMapIterable(Function) * @see #concatMapStream(Function, int) @@ -19461,6 +19462,7 @@ public final Stream blockingStream(int prefetch) { * will be emitted to the downstream * @param prefetch the number of upstream items to request upfront, then 75% of this amount after each 75% upstream items received * @return the new Flowable instance + * @since 3.0.0 * @see #concatMap(Function, int) * @see #concatMapIterable(Function, int) * @see #flatMapStream(Function, int) @@ -19515,6 +19517,7 @@ public final Stream blockingStream(int prefetch) { * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements * will be emitted to the downstream * @return the new Flowable instance + * @since 3.0.0 * @see #flatMap(Function) * @see #flatMapIterable(Function) * @see #flatMapStream(Function, int) @@ -19567,6 +19570,7 @@ public final Stream blockingStream(int prefetch) { * will be emitted to the downstream * @param prefetch the number of upstream items to request upfront, then 75% of this amount after each 75% upstream items received * @return the new Flowable instance + * @since 3.0.0 * @see #flatMap(Function, int) * @see #flatMapIterable(Function, int) * @see #concatMapStream(Function, int) diff --git a/src/main/java/io/reactivex/rxjava3/core/Maybe.java b/src/main/java/io/reactivex/rxjava3/core/Maybe.java index d5336f96ce..f26e584c16 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Maybe.java +++ b/src/main/java/io/reactivex/rxjava3/core/Maybe.java @@ -4870,9 +4870,9 @@ public final TestObserver test(boolean dispose) { * @param the element type of the optional value * @param optional the optional value to convert into a {@code Maybe} * @return the new Maybe instance + * @since 3.0.0 * @see #just(Object) * @see #empty() - * @since 3.0.0 */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index 0afdd29a93..eb33792f34 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -15937,9 +15937,9 @@ public final TestObserver test(boolean dispose) { // NoPMD * @param the element type of the optional value * @param optional the optional value to convert into an {@code Observable} * @return the new Observable instance + * @since 3.0.0 * @see #just(Object) * @see #empty() - * @since 3.0.0 */ @CheckReturnValue @SchedulerSupport(SchedulerSupport.NONE) @@ -16355,6 +16355,7 @@ public final Stream blockingStream(int capacityHint) { * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements * will be emitted to the downstream * @return the new Observable instance + * @since 3.0.0 * @see #concatMap(Function) * @see #concatMapIterable(Function) * @see #flatMapStream(Function) @@ -16401,6 +16402,7 @@ public final Stream blockingStream(int capacityHint) { * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements * will be emitted to the downstream * @return the new Observable instance + * @since 3.0.0 * @see #flatMap(Function) * @see #flatMapIterable(Function) */ diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java index ad1b3ad140..c967a011e5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java @@ -71,10 +71,23 @@ protected void subscribeActual(Subscriber s) { EmptySubscription.complete(s); } } else { - source.subscribe(new FlatMapStreamSubscriber<>(s, mapper, prefetch)); + source.subscribe(subscribe(s, mapper, prefetch)); } } + /** + * Create a {@link Subscriber} with the given parameters. + * @param the upstream value type + * @param the {@link Stream} and output value type + * @param downstream the downstream {@code Subscriber} to wrap + * @param mapper the mapper function + * @param prefetch the number of items to prefetch + * @return the new {@code Subscriber} + */ + public static Subscriber subscribe(Subscriber downstream, Function> mapper, int prefetch) { + return new FlatMapStreamSubscriber<>(downstream, mapper, prefetch); + } + static final class FlatMapStreamSubscriber extends AtomicInteger implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java new file mode 100644 index 0000000000..e94f50cb67 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java @@ -0,0 +1,270 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.Objects; +import java.util.concurrent.atomic.*; +import java.util.function.*; +import java.util.stream.Collector; + +import org.reactivestreams.*; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.internal.subscriptions.*; +import io.reactivex.rxjava3.internal.util.AtomicThrowable; +import io.reactivex.rxjava3.parallel.ParallelFlowable; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Reduces all 'rails' into a single via a Java 8 {@link Collector} callback set. + * + * @param the value type + * @param the accumulator type + * @param the result type + * @since 3.0.0 + */ +public final class ParallelCollector extends Flowable { + + final ParallelFlowable source; + + final Collector collector; + + public ParallelCollector(ParallelFlowable source, Collector collector) { + this.source = source; + this.collector = collector; + } + + @Override + protected void subscribeActual(Subscriber s) { + ParallelCollectorSubscriber parent; + try { + parent = new ParallelCollectorSubscriber<>(s, source.parallelism(), collector); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + EmptySubscription.error(ex, s); + return; + } + s.onSubscribe(parent); + + source.subscribe(parent.subscribers); + } + + static final class ParallelCollectorSubscriber extends DeferredScalarSubscription { + + private static final long serialVersionUID = -5370107872170712765L; + + final ParallelCollectorInnerSubscriber[] subscribers; + + final AtomicReference> current = new AtomicReference<>(); + + final AtomicInteger remaining = new AtomicInteger(); + + final AtomicThrowable error = new AtomicThrowable(); + + final Function finisher; + + ParallelCollectorSubscriber(Subscriber subscriber, int n, Collector collector) { + super(subscriber); + this.finisher = collector.finisher(); + @SuppressWarnings("unchecked") + ParallelCollectorInnerSubscriber[] a = new ParallelCollectorInnerSubscriber[n]; + for (int i = 0; i < n; i++) { + a[i] = new ParallelCollectorInnerSubscriber<>(this, collector.supplier().get(), collector.accumulator(), collector.combiner()); + } + this.subscribers = a; + remaining.lazySet(n); + } + + SlotPair addValue(A value) { + for (;;) { + SlotPair curr = current.get(); + + if (curr == null) { + curr = new SlotPair<>(); + if (!current.compareAndSet(null, curr)) { + continue; + } + } + + int c = curr.tryAcquireSlot(); + if (c < 0) { + current.compareAndSet(curr, null); + continue; + } + if (c == 0) { + curr.first = value; + } else { + curr.second = value; + } + + if (curr.releaseSlot()) { + current.compareAndSet(curr, null); + return curr; + } + return null; + } + } + + @Override + public void cancel() { + for (ParallelCollectorInnerSubscriber inner : subscribers) { + inner.cancel(); + } + } + + void innerError(Throwable ex) { + if (error.compareAndSet(null, ex)) { + cancel(); + downstream.onError(ex); + } else { + if (ex != error.get()) { + RxJavaPlugins.onError(ex); + } + } + } + + void innerComplete(A value, BinaryOperator combiner) { + for (;;) { + SlotPair sp = addValue(value); + + if (sp != null) { + + try { + value = combiner.apply(sp.first, sp.second); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + innerError(ex); + return; + } + + } else { + break; + } + } + + if (remaining.decrementAndGet() == 0) { + SlotPair sp = current.get(); + current.lazySet(null); + + R result; + try { + result = Objects.requireNonNull(finisher.apply(sp.first), "The finisher returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + innerError(ex); + return; + } + + complete(result); + } + } + } + + static final class ParallelCollectorInnerSubscriber + extends AtomicReference + implements FlowableSubscriber { + + private static final long serialVersionUID = -7954444275102466525L; + + final ParallelCollectorSubscriber parent; + + final BiConsumer accumulator; + + final BinaryOperator combiner; + + A container; + + boolean done; + + ParallelCollectorInnerSubscriber(ParallelCollectorSubscriber parent, A container, BiConsumer accumulator, BinaryOperator combiner) { + this.parent = parent; + this.accumulator = accumulator; + this.combiner = combiner; + this.container = container; + } + + @Override + public void onSubscribe(Subscription s) { + SubscriptionHelper.setOnce(this, s, Long.MAX_VALUE); + } + + @Override + public void onNext(T t) { + if (!done) { + try { + accumulator.accept(container, t); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + get().cancel(); + onError(ex); + return; + } + } + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + return; + } + container = null; + done = true; + parent.innerError(t); + } + + @Override + public void onComplete() { + if (!done) { + A v = container; + container = null; + done = true; + parent.innerComplete(v, combiner); + } + } + + void cancel() { + SubscriptionHelper.cancel(this); + } + } + + static final class SlotPair extends AtomicInteger { + + private static final long serialVersionUID = 473971317683868662L; + + T first; + + T second; + + final AtomicInteger releaseIndex = new AtomicInteger(); + + int tryAcquireSlot() { + for (;;) { + int acquired = get(); + if (acquired >= 2) { + return -1; + } + + if (compareAndSet(acquired, acquired + 1)) { + return acquired; + } + } + } + + boolean releaseSlot() { + return releaseIndex.incrementAndGet() == 2; + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStream.java new file mode 100644 index 0000000000..3f38d1fec3 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStream.java @@ -0,0 +1,69 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.Stream; + +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.parallel.ParallelFlowable; + +/** + * Flattens the generated {@link Stream}s on each rail. + * + * @param the input value type + * @param the output value type + * @since 3.0.0 + */ +public final class ParallelFlatMapStream extends ParallelFlowable { + + final ParallelFlowable source; + + final Function> mapper; + + final int prefetch; + + public ParallelFlatMapStream( + ParallelFlowable source, + Function> mapper, + int prefetch) { + this.source = source; + this.mapper = mapper; + this.prefetch = prefetch; + } + + @Override + public int parallelism() { + return source.parallelism(); + } + + @Override + public void subscribe(Subscriber[] subscribers) { + if (!validate(subscribers)) { + return; + } + + int n = subscribers.length; + + @SuppressWarnings("unchecked") + final Subscriber[] parents = new Subscriber[n]; + + for (int i = 0; i < n; i++) { + parents[i] = FlowableFlatMapStream.subscribe(subscribers[i], mapper, prefetch); + } + + source.subscribe(parents); + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptional.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptional.java new file mode 100644 index 0000000000..350f59f2b2 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptional.java @@ -0,0 +1,236 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; + +import org.reactivestreams.*; + +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; +import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; +import io.reactivex.rxjava3.parallel.ParallelFlowable; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Maps each 'rail' of the source ParallelFlowable with a mapper function. + * + * @param the input value type + * @param the output value type + * @since 3.0.0 + */ +public final class ParallelMapOptional extends ParallelFlowable { + + final ParallelFlowable source; + + final Function> mapper; + + public ParallelMapOptional(ParallelFlowable source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + public void subscribe(Subscriber[] subscribers) { + if (!validate(subscribers)) { + return; + } + + int n = subscribers.length; + @SuppressWarnings("unchecked") + Subscriber[] parents = new Subscriber[n]; + + for (int i = 0; i < n; i++) { + Subscriber a = subscribers[i]; + if (a instanceof ConditionalSubscriber) { + parents[i] = new ParallelMapConditionalSubscriber<>((ConditionalSubscriber)a, mapper); + } else { + parents[i] = new ParallelMapSubscriber<>(a, mapper); + } + } + + source.subscribe(parents); + } + + @Override + public int parallelism() { + return source.parallelism(); + } + + static final class ParallelMapSubscriber implements ConditionalSubscriber, Subscription { + + final Subscriber downstream; + + final Function> mapper; + + Subscription upstream; + + boolean done; + + ParallelMapSubscriber(Subscriber actual, Function> mapper) { + this.downstream = actual; + this.mapper = mapper; + } + + @Override + public void request(long n) { + upstream.request(n); + } + + @Override + public void cancel() { + upstream.cancel(); + } + + @Override + public void onSubscribe(Subscription s) { + if (SubscriptionHelper.validate(this.upstream, s)) { + this.upstream = s; + + downstream.onSubscribe(this); + } + } + + @Override + public void onNext(T t) { + if (!tryOnNext(t)) { + upstream.request(1); + } + } + + @Override + public boolean tryOnNext(T t) { + if (done) { + return true; + } + Optional v; + + try { + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Optional"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + cancel(); + onError(ex); + return true; + } + + if (v.isPresent()) { + downstream.onNext(v.get()); + return true; + } + return false; + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + return; + } + done = true; + downstream.onError(t); + } + + @Override + public void onComplete() { + if (done) { + return; + } + done = true; + downstream.onComplete(); + } + + } + static final class ParallelMapConditionalSubscriber implements ConditionalSubscriber, Subscription { + + final ConditionalSubscriber downstream; + + final Function> mapper; + + Subscription upstream; + + boolean done; + + ParallelMapConditionalSubscriber(ConditionalSubscriber actual, Function> mapper) { + this.downstream = actual; + this.mapper = mapper; + } + + @Override + public void request(long n) { + upstream.request(n); + } + + @Override + public void cancel() { + upstream.cancel(); + } + + @Override + public void onSubscribe(Subscription s) { + if (SubscriptionHelper.validate(this.upstream, s)) { + this.upstream = s; + + downstream.onSubscribe(this); + } + } + + @Override + public void onNext(T t) { + if (!tryOnNext(t)) { + upstream.request(1); + } + } + + @Override + public boolean tryOnNext(T t) { + if (done) { + return false; + } + Optional v; + + try { + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null value"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + cancel(); + onError(ex); + return false; + } + + return v.isPresent() && downstream.tryOnNext(v.get()); + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + return; + } + done = true; + downstream.onError(t); + } + + @Override + public void onComplete() { + if (done) { + return; + } + done = true; + downstream.onComplete(); + } + + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptional.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptional.java new file mode 100644 index 0000000000..a2d454c994 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptional.java @@ -0,0 +1,306 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; + +import org.reactivestreams.*; + +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.functions.*; +import io.reactivex.rxjava3.internal.fuseable.ConditionalSubscriber; +import io.reactivex.rxjava3.internal.subscriptions.SubscriptionHelper; +import io.reactivex.rxjava3.parallel.*; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Maps each 'rail' of the source ParallelFlowable with a mapper function + * and handle any failure based on a handler function. + * @param the input value type + * @param the output value type + * @since 3.0.0 + */ +public final class ParallelMapTryOptional extends ParallelFlowable { + + final ParallelFlowable source; + + final Function> mapper; + + final BiFunction errorHandler; + + public ParallelMapTryOptional( + ParallelFlowable source, + Function> mapper, + BiFunction errorHandler) { + this.source = source; + this.mapper = mapper; + this.errorHandler = errorHandler; + } + + @Override + public void subscribe(Subscriber[] subscribers) { + if (!validate(subscribers)) { + return; + } + + int n = subscribers.length; + @SuppressWarnings("unchecked") + Subscriber[] parents = new Subscriber[n]; + + for (int i = 0; i < n; i++) { + Subscriber a = subscribers[i]; + if (a instanceof ConditionalSubscriber) { + parents[i] = new ParallelMapTryConditionalSubscriber<>((ConditionalSubscriber)a, mapper, errorHandler); + } else { + parents[i] = new ParallelMapTrySubscriber<>(a, mapper, errorHandler); + } + } + + source.subscribe(parents); + } + + @Override + public int parallelism() { + return source.parallelism(); + } + + static final class ParallelMapTrySubscriber implements ConditionalSubscriber, Subscription { + + final Subscriber downstream; + + final Function> mapper; + + final BiFunction errorHandler; + + Subscription upstream; + + boolean done; + + ParallelMapTrySubscriber(Subscriber actual, + Function> mapper, + BiFunction errorHandler) { + this.downstream = actual; + this.mapper = mapper; + this.errorHandler = errorHandler; + } + + @Override + public void request(long n) { + upstream.request(n); + } + + @Override + public void cancel() { + upstream.cancel(); + } + + @Override + public void onSubscribe(Subscription s) { + if (SubscriptionHelper.validate(this.upstream, s)) { + this.upstream = s; + + downstream.onSubscribe(this); + } + } + + @Override + public void onNext(T t) { + if (!tryOnNext(t) && !done) { + upstream.request(1); + } + } + + @Override + public boolean tryOnNext(T t) { + if (done) { + return false; + } + long retries = 0; + + for (;;) { + Optional v; + + try { + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Optional"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + + ParallelFailureHandling h; + + try { + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); + } catch (Throwable exc) { + Exceptions.throwIfFatal(exc); + cancel(); + onError(new CompositeException(ex, exc)); + return false; + } + + switch (h) { + case RETRY: + continue; + case SKIP: + return false; + case STOP: + cancel(); + onComplete(); + return false; + default: + cancel(); + onError(ex); + return false; + } + } + + if (v.isPresent()) { + downstream.onNext(v.get()); + return true; + } + return false; + } + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + return; + } + done = true; + downstream.onError(t); + } + + @Override + public void onComplete() { + if (done) { + return; + } + done = true; + downstream.onComplete(); + } + + } + static final class ParallelMapTryConditionalSubscriber implements ConditionalSubscriber, Subscription { + + final ConditionalSubscriber downstream; + + final Function> mapper; + + final BiFunction errorHandler; + + Subscription upstream; + + boolean done; + + ParallelMapTryConditionalSubscriber(ConditionalSubscriber actual, + Function> mapper, + BiFunction errorHandler) { + this.downstream = actual; + this.mapper = mapper; + this.errorHandler = errorHandler; + } + + @Override + public void request(long n) { + upstream.request(n); + } + + @Override + public void cancel() { + upstream.cancel(); + } + + @Override + public void onSubscribe(Subscription s) { + if (SubscriptionHelper.validate(this.upstream, s)) { + this.upstream = s; + + downstream.onSubscribe(this); + } + } + + @Override + public void onNext(T t) { + if (!tryOnNext(t) && !done) { + upstream.request(1); + } + } + + @Override + public boolean tryOnNext(T t) { + if (done) { + return false; + } + long retries = 0; + + for (;;) { + Optional v; + + try { + v = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Optional"); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + + ParallelFailureHandling h; + + try { + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); + } catch (Throwable exc) { + Exceptions.throwIfFatal(exc); + cancel(); + onError(new CompositeException(ex, exc)); + return false; + } + + switch (h) { + case RETRY: + continue; + case SKIP: + return false; + case STOP: + cancel(); + onComplete(); + return false; + default: + cancel(); + onError(ex); + return false; + } + } + + return v.isPresent() && downstream.tryOnNext(v.get()); + } + } + + @Override + public void onError(Throwable t) { + if (done) { + RxJavaPlugins.onError(t); + return; + } + done = true; + downstream.onError(t); + } + + @Override + public void onComplete() { + if (done) { + return; + } + done = true; + downstream.onComplete(); + } + + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java index de7221cb38..b275337841 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java @@ -13,8 +13,7 @@ package io.reactivex.rxjava3.internal.operators.flowable; -import java.util.Iterator; -import java.util.Objects; +import java.util.*; import java.util.concurrent.atomic.*; import org.reactivestreams.*; @@ -80,6 +79,19 @@ public void subscribeActual(Subscriber s) { source.subscribe(new FlattenIterableSubscriber(s, mapper, prefetch)); } + /** + * Create a {@link Subscriber} with the given parameters. + * @param the upstream value type + * @param the {@link Iterable} and output value type + * @param downstream the downstream {@code Subscriber} to wrap + * @param mapper the mapper function + * @param prefetch the number of items to prefetch + * @return the new {@code Subscriber} + */ + public static Subscriber subscribe(Subscriber downstream, Function> mapper, int prefetch) { + return new FlattenIterableSubscriber<>(downstream, mapper, prefetch); + } + static final class FlattenIterableSubscriber extends BasicIntQueueSubscription implements FlowableSubscriber { @@ -118,7 +130,7 @@ static final class FlattenIterableSubscriber this.mapper = mapper; this.prefetch = prefetch; this.limit = prefetch - (prefetch >> 2); - this.error = new AtomicReference(); + this.error = new AtomicReference<>(); this.requested = new AtomicLong(); } @@ -153,7 +165,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); downstream.onSubscribe(this); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java index e1da1b76b7..29f9d91674 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelCollect.java @@ -67,7 +67,7 @@ public void subscribe(Subscriber[] subscribers) { return; } - parents[i] = new ParallelCollectSubscriber(subscribers[i], initialValue, collector); + parents[i] = new ParallelCollectSubscriber<>(subscribers[i], initialValue, collector); } source.subscribe(parents); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java index 557d5282c4..6b3f69ff97 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelDoOnNextTry.java @@ -59,9 +59,9 @@ public void subscribe(Subscriber[] subscribers) { for (int i = 0; i < n; i++) { Subscriber a = subscribers[i]; if (a instanceof ConditionalSubscriber) { - parents[i] = new ParallelDoOnNextConditionalSubscriber((ConditionalSubscriber)a, onNext, errorHandler); + parents[i] = new ParallelDoOnNextConditionalSubscriber<>((ConditionalSubscriber)a, onNext, errorHandler); } else { - parents[i] = new ParallelDoOnNextSubscriber(a, onNext, errorHandler); + parents[i] = new ParallelDoOnNextSubscriber<>(a, onNext, errorHandler); } } @@ -134,7 +134,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); @@ -245,7 +245,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilter.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilter.java index 6d1fc2fb96..14f414dce3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilter.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilter.java @@ -51,9 +51,9 @@ public void subscribe(Subscriber[] subscribers) { for (int i = 0; i < n; i++) { Subscriber a = subscribers[i]; if (a instanceof ConditionalSubscriber) { - parents[i] = new ParallelFilterConditionalSubscriber((ConditionalSubscriber)a, predicate); + parents[i] = new ParallelFilterConditionalSubscriber<>((ConditionalSubscriber)a, predicate); } else { - parents[i] = new ParallelFilterSubscriber(a, predicate); + parents[i] = new ParallelFilterSubscriber<>(a, predicate); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java index f6eff78249..bfd22ec5a8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFilterTry.java @@ -57,9 +57,9 @@ public void subscribe(Subscriber[] subscribers) { for (int i = 0; i < n; i++) { Subscriber a = subscribers[i]; if (a instanceof ConditionalSubscriber) { - parents[i] = new ParallelFilterConditionalSubscriber((ConditionalSubscriber)a, predicate, errorHandler); + parents[i] = new ParallelFilterConditionalSubscriber<>((ConditionalSubscriber)a, predicate, errorHandler); } else { - parents[i] = new ParallelFilterSubscriber(a, predicate, errorHandler); + parents[i] = new ParallelFilterSubscriber<>(a, predicate, errorHandler); } } @@ -137,7 +137,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); @@ -226,7 +226,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFlatMapIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFlatMapIterable.java new file mode 100644 index 0000000000..1bb127233f --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFlatMapIterable.java @@ -0,0 +1,68 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.operators.parallel; + +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.operators.flowable.FlowableFlattenIterable; +import io.reactivex.rxjava3.parallel.ParallelFlowable; + +/** + * Flattens the generated {@link Iterable}s on each rail. + * + * @param the input value type + * @param the output value type + * @since 3.0.0 + */ +public final class ParallelFlatMapIterable extends ParallelFlowable { + + final ParallelFlowable source; + + final Function> mapper; + + final int prefetch; + + public ParallelFlatMapIterable( + ParallelFlowable source, + Function> mapper, + int prefetch) { + this.source = source; + this.mapper = mapper; + this.prefetch = prefetch; + } + + @Override + public int parallelism() { + return source.parallelism(); + } + + @Override + public void subscribe(Subscriber[] subscribers) { + if (!validate(subscribers)) { + return; + } + + int n = subscribers.length; + + @SuppressWarnings("unchecked") + final Subscriber[] parents = new Subscriber[n]; + + for (int i = 0; i < n; i++) { + parents[i] = FlowableFlattenIterable.subscribe(subscribers[i], mapper, prefetch); + } + + source.subscribe(parents); + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFromPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFromPublisher.java index 126f7cdf13..0d4fb9a4c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFromPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelFromPublisher.java @@ -55,7 +55,7 @@ public void subscribe(Subscriber[] subscribers) { return; } - source.subscribe(new ParallelDispatcher(subscribers, prefetch)); + source.subscribe(new ParallelDispatcher<>(subscribers, prefetch)); } static final class ParallelDispatcher @@ -137,7 +137,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); setupSubscribers(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelJoin.java index b15fa8dfd5..4545b133f5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelJoin.java @@ -50,9 +50,9 @@ public ParallelJoin(ParallelFlowable source, int prefetch, boolean protected void subscribeActual(Subscriber s) { JoinSubscriptionBase parent; if (delayErrors) { - parent = new JoinSubscriptionDelayError(s, source.parallelism(), prefetch); + parent = new JoinSubscriptionDelayError<>(s, source.parallelism(), prefetch); } else { - parent = new JoinSubscription(s, source.parallelism(), prefetch); + parent = new JoinSubscription<>(s, source.parallelism(), prefetch); } s.onSubscribe(parent); source.subscribe(parent.subscribers); @@ -81,7 +81,7 @@ abstract static class JoinSubscriptionBase extends AtomicInteger JoinInnerSubscriber[] a = new JoinInnerSubscriber[n]; for (int i = 0; i < n; i++) { - a[i] = new JoinInnerSubscriber(this, prefetch); + a[i] = new JoinInnerSubscriber<>(this, prefetch); } this.subscribers = a; @@ -550,7 +550,7 @@ public boolean cancel() { SimplePlainQueue getQueue() { SimplePlainQueue q = queue; if (q == null) { - q = new SpscArrayQueue(prefetch); + q = new SpscArrayQueue<>(prefetch); this.queue = q; } return q; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java index ff8da40c62..30250f2e60 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelMapTry.java @@ -137,7 +137,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); @@ -250,7 +250,7 @@ public boolean tryOnNext(T t) { ParallelFailureHandling h; try { - h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null item"); + h = Objects.requireNonNull(errorHandler.apply(++retries, ex), "The errorHandler returned a null ParallelFailureHandling"); } catch (Throwable exc) { Exceptions.throwIfFatal(exc); cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java index 50f04d46ec..206a2dd472 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelPeek.java @@ -75,7 +75,7 @@ public void subscribe(Subscriber[] subscribers) { Subscriber[] parents = new Subscriber[n]; for (int i = 0; i < n; i++) { - parents[i] = new ParallelPeekSubscriber(subscribers[i], this); + parents[i] = new ParallelPeekSubscriber<>(subscribers[i], this); } source.subscribe(parents); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java index 87c02c8922..53fa2b3899 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduce.java @@ -66,7 +66,7 @@ public void subscribe(Subscriber[] subscribers) { return; } - parents[i] = new ParallelReduceSubscriber(subscribers[i], initialValue, reducer); + parents[i] = new ParallelReduceSubscriber<>(subscribers[i], initialValue, reducer); } source.subscribe(parents); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java index e376c352cb..a47bc5b898 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelReduceFull.java @@ -22,6 +22,7 @@ import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.BiFunction; import io.reactivex.rxjava3.internal.subscriptions.*; +import io.reactivex.rxjava3.internal.util.AtomicThrowable; import io.reactivex.rxjava3.parallel.ParallelFlowable; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -44,7 +45,7 @@ public ParallelReduceFull(ParallelFlowable source, BiFunction s) { - ParallelReduceFullMainSubscriber parent = new ParallelReduceFullMainSubscriber(s, source.parallelism(), reducer); + ParallelReduceFullMainSubscriber parent = new ParallelReduceFullMainSubscriber<>(s, source.parallelism(), reducer); s.onSubscribe(parent); source.subscribe(parent.subscribers); @@ -58,18 +59,18 @@ static final class ParallelReduceFullMainSubscriber extends DeferredScalarSub final BiFunction reducer; - final AtomicReference> current = new AtomicReference>(); + final AtomicReference> current = new AtomicReference<>(); final AtomicInteger remaining = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicThrowable error = new AtomicThrowable(); ParallelReduceFullMainSubscriber(Subscriber subscriber, int n, BiFunction reducer) { super(subscriber); @SuppressWarnings("unchecked") ParallelReduceFullInnerSubscriber[] a = new ParallelReduceFullInnerSubscriber[n]; for (int i = 0; i < n; i++) { - a[i] = new ParallelReduceFullInnerSubscriber(this, reducer); + a[i] = new ParallelReduceFullInnerSubscriber<>(this, reducer); } this.subscribers = a; this.reducer = reducer; @@ -81,7 +82,7 @@ SlotPair addValue(T value) { SlotPair curr = current.get(); if (curr == null) { - curr = new SlotPair(); + curr = new SlotPair<>(); if (!current.compareAndSet(null, curr)) { continue; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelRunOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelRunOn.java index bb6c1fd582..ceb2a7665a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelRunOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelRunOn.java @@ -75,12 +75,12 @@ void createSubscriber(int i, Subscriber[] subscribers, Subscriber a = subscribers[i]; - SpscArrayQueue q = new SpscArrayQueue(prefetch); + SpscArrayQueue q = new SpscArrayQueue<>(prefetch); if (a instanceof ConditionalSubscriber) { - parents[i] = new RunOnConditionalSubscriber((ConditionalSubscriber)a, prefetch, q, worker); + parents[i] = new RunOnConditionalSubscriber<>((ConditionalSubscriber)a, prefetch, q, worker); } else { - parents[i] = new RunOnSubscriber(a, prefetch, q, worker); + parents[i] = new RunOnSubscriber<>(a, prefetch, q, worker); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelSortedJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelSortedJoin.java index 0e20b6f2c9..7994abf697 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelSortedJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/parallel/ParallelSortedJoin.java @@ -46,7 +46,7 @@ public ParallelSortedJoin(ParallelFlowable> source, Comparator s) { - SortedJoinSubscription parent = new SortedJoinSubscription(s, source.parallelism(), comparator); + SortedJoinSubscription parent = new SortedJoinSubscription<>(s, source.parallelism(), comparator); s.onSubscribe(parent); source.subscribe(parent.subscribers); @@ -74,7 +74,7 @@ static final class SortedJoinSubscription final AtomicInteger remaining = new AtomicInteger(); - final AtomicReference error = new AtomicReference(); + final AtomicReference error = new AtomicReference<>(); @SuppressWarnings("unchecked") SortedJoinSubscription(Subscriber actual, int n, Comparator comparator) { @@ -84,7 +84,7 @@ static final class SortedJoinSubscription SortedJoinInnerSubscriber[] s = new SortedJoinInnerSubscriber[n]; for (int i = 0; i < n; i++) { - s[i] = new SortedJoinInnerSubscriber(this, i); + s[i] = new SortedJoinInnerSubscriber<>(this, i); } this.subscribers = s; this.lists = new List[n]; diff --git a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java index d4dff1cbb0..c8fbf203ef 100644 --- a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java @@ -14,6 +14,7 @@ package io.reactivex.rxjava3.parallel; import java.util.*; +import java.util.stream.*; import org.reactivestreams.*; @@ -21,17 +22,19 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.functions.*; +import io.reactivex.rxjava3.internal.jdk8.*; import io.reactivex.rxjava3.internal.operators.parallel.*; +import io.reactivex.rxjava3.internal.operators.parallel.ParallelReduceFull; import io.reactivex.rxjava3.internal.subscriptions.EmptySubscription; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** - * Abstract base class for Parallel publishers that take an array of Subscribers. + * Abstract base class for parallel publishing of events signaled to an array of {@link Subscriber}s. *

- * Use {@code from()} to start processing a regular Publisher in 'rails'. - * Use {@code runOn()} to introduce where each 'rail' should run on thread-vise. - * Use {@code sequential()} to merge the sources back into a single Flowable. + * Use {@link #from(Publisher)} to start processing a regular {@link Publisher} in 'rails'. + * Use {@link #runOn(Scheduler)} to introduce where each 'rail' should run on thread-vise. + * Use {@link #sequential()} to merge the sources back into a single {@link Flowable}. * *

History: 2.0.5 - experimental; 2.1 - beta * @param the value type @@ -173,7 +176,7 @@ public static ParallelFlowable from(@NonNull Publisher sourc *

{@code map} does not operate by default on a particular {@link Scheduler}.
* * @param the output value type - * @param mapper the mapper function turning Ts into Us. + * @param mapper the mapper function turning Ts into Rs. * @return the new ParallelFlowable instance */ @CheckReturnValue @@ -199,7 +202,7 @@ public final ParallelFlowable map(@NonNull Function *

History: 2.0.8 - experimental * @param the output value type - * @param mapper the mapper function turning Ts into Us. + * @param mapper the mapper function turning Ts into Rs. * @param errorHandler the enumeration that defines how to handle errors thrown * from the mapper function * @return the new ParallelFlowable instance @@ -229,7 +232,7 @@ public final ParallelFlowable map(@NonNull Function *

History: 2.0.8 - experimental * @param the output value type - * @param mapper the mapper function turning Ts into Us. + * @param mapper the mapper function turning Ts into Rs. * @param errorHandler the function called with the current repeat count and * failure Throwable and should return one of the {@link ParallelFailureHandling} * enumeration values to indicate how to proceed. @@ -1331,4 +1334,297 @@ public final ParallelFlowable concatMapDelayError( return RxJavaPlugins.onAssembly(new ParallelConcatMap<>( this, mapper, prefetch, tillTheEnd ? ErrorMode.END : ErrorMode.BOUNDARY)); } + + /** + * Returns a {@code ParallelFlowable} that merges each item emitted by the source on each rail with the values in an + * {@link Iterable} corresponding to that item that is generated by a selector. + *

+ * + *

+ *
Backpressure:
+ *
The operator honors backpressure from each downstream rail. The source {@code ParallelFlowable}s is + * expected to honor backpressure as well. If the source {@code ParallelFlowable} violates the rule, the operator will + * signal a {@code MissingBackpressureException}.
+ *
Scheduler:
+ *
{@code flatMapIterable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param + * the type of item emitted by the resulting {@code Iterable} + * @param mapper + * a function that returns an {@code Iterable} sequence of values for when given an item emitted by the + * source {@code ParallelFlowable} + * @return the new ParallelFlowable instance + * @see
ReactiveX operators documentation: FlatMap + * @see #flatMapStream(Function) + * @since 3.0.0 + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final ParallelFlowable flatMapIterable(@NonNull Function> mapper) { + return flatMapIterable(mapper, Flowable.bufferSize()); + } + + /** + * Returns a {@code ParallelFlowable} that merges each item emitted by the source {@code ParallelFlowable} with the values in an + * Iterable corresponding to that item that is generated by a selector. + *

+ * + *

+ *
Backpressure:
+ *
The operator honors backpressure from each downstream rail. The source {@code ParallelFlowable}s is + * expected to honor backpressure as well. If the source {@code ParallelFlowable} violates the rule, the operator will + * signal a {@code MissingBackpressureException}.
+ *
Scheduler:
+ *
{@code flatMapIterable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param + * the type of item emitted by the resulting {@code Iterable} + * @param mapper + * a function that returns an {@code Iterable} sequence of values for when given an item emitted by the + * source {@code ParallelFlowable} + * @param bufferSize + * the number of elements to prefetch from each upstream rail + * @return the new {@code ParallelFlowable} instance + * @see ReactiveX operators documentation: FlatMap + * @see #flatMapStream(Function, int) + * @since 3.0.0 + */ + @CheckReturnValue + @NonNull + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + public final ParallelFlowable flatMapIterable(@NonNull Function> mapper, int bufferSize) { + Objects.requireNonNull(mapper, "mapper is null"); + ObjectHelper.verifyPositive(bufferSize, "bufferSize"); + return RxJavaPlugins.onAssembly(new ParallelFlatMapIterable<>(this, mapper, bufferSize)); + } + + // ------------------------------------------------------------------------- + // JDK 8 Support + // ------------------------------------------------------------------------- + + /** + * Maps the source values on each 'rail' to an optional and emits its value if any. + *

+ * Note that the same mapper function may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the output value type + * @param mapper the mapper function turning Ts into optional of Rs. + * @return the new ParallelFlowable instance + * @since 3.0.0 + */ + @CheckReturnValue + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + public final ParallelFlowable mapOptional(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper"); + return RxJavaPlugins.onAssembly(new ParallelMapOptional<>(this, mapper)); + } + + /** + * Maps the source values on each 'rail' to an optional and emits its value if any and + * handles errors based on the given {@link ParallelFailureHandling} enumeration value. + *

+ * Note that the same mapper function may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
+ *

History: 2.0.8 - experimental + * @param the output value type + * @param mapper the mapper function turning Ts into optional of Rs. + * @param errorHandler the enumeration that defines how to handle errors thrown + * from the mapper function + * @return the new ParallelFlowable instance + * @since 3.0.0 + */ + @CheckReturnValue + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + public final ParallelFlowable mapOptional(@NonNull Function> mapper, @NonNull ParallelFailureHandling errorHandler) { + Objects.requireNonNull(mapper, "mapper"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); + return RxJavaPlugins.onAssembly(new ParallelMapTryOptional<>(this, mapper, errorHandler)); + } + + /** + * Maps the source values on each 'rail' to an optional and emits its value if any and + * handles errors based on the returned value by the handler function. + *

+ * Note that the same mapper function may be called from multiple threads concurrently. + *

+ *
Backpressure:
+ *
The operator is a pass-through for backpressure and the behavior + * is determined by the upstream and downstream rail behaviors.
+ *
Scheduler:
+ *
{@code map} does not operate by default on a particular {@link Scheduler}.
+ *
+ *

History: 2.0.8 - experimental + * @param the output value type + * @param mapper the mapper function turning Ts into optional of Rs. + * @param errorHandler the function called with the current repeat count and + * failure Throwable and should return one of the {@link ParallelFailureHandling} + * enumeration values to indicate how to proceed. + * @return the new ParallelFlowable instance + * @since 3.0.0 + */ + @CheckReturnValue + @NonNull + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.PASS_THROUGH) + public final ParallelFlowable mapOptional(@NonNull Function> mapper, @NonNull BiFunction errorHandler) { + Objects.requireNonNull(mapper, "mapper"); + Objects.requireNonNull(errorHandler, "errorHandler is null"); + return RxJavaPlugins.onAssembly(new ParallelMapTryOptional<>(this, mapper, errorHandler)); + } + + /** + * Maps each upstream item on each rail into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flatMapIterable(Function)}: + *


+     * source.flatMapIterable(v -> createStream(v)::iterator);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors the downstream backpressure and consumes the inner stream only on demand. The operator + * prefetches {@link Flowable#bufferSize()} items of the upstream (then 75% of it after the 75% received) + * and caches them until they are ready to be mapped into {@code Stream}s + * after the current {@code Stream} has been consumed.
+ *
Scheduler:
+ *
{@code flatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @return the new Flowable instance + * @see #flatMap(Function) + * @see #flatMapIterable(Function) + * @see #flatMapStream(Function, int) + * @since 3.0.0 + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> ParallelFlowable flatMapStream(@NonNull Function> mapper) { + return flatMapStream(mapper, Flowable.bufferSize()); + } + + /** + * Maps each upstream item of each rail into a {@link Stream} and emits the {@code Stream}'s items to the downstream in a sequential fashion. + *

+ * + *

+ * Due to the blocking and sequential nature of Java {@link Stream}s, the streams are mapped and consumed in a sequential fashion + * without interleaving (unlike a more general {@link #flatMap(Function)}). Therefore, {@code flatMapStream} and + * {@code concatMapStream} are identical operators and are provided as aliases. + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flatMapIterable(Function, int)}: + *


+     * source.flatMapIterable(v -> createStream(v)::iterator, 32);
+     * 
+ *

+ * Note that {@code Stream}s can be consumed only once; any subsequent attempt to consume a {@code Stream} + * will result in an {@link IllegalStateException}. + *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flatMapStream(v -> IntStream.rangeClosed(v + 1, v + 10).boxed(), 32);
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors the downstream backpressure and consumes the inner stream only on demand. The operator + * prefetches the given amount of upstream items and caches them until they are ready to be mapped into {@code Stream}s + * after the current {@code Stream} has been consumed.
+ *
Scheduler:
+ *
{@code flatMapStream} does not operate by default on a particular {@link Scheduler}.
+ *
+ * + * @param the element type of the {@code Stream}s and the result + * @param mapper the function that receives an upstream item and should return a {@code Stream} whose elements + * will be emitted to the downstream + * @param prefetch the number of upstream items to request upfront, then 75% of this amount after each 75% upstream items received + * @return the new Flowable instance + * @see #flatMap(Function, boolean, int) + * @see #flatMapIterable(Function, int) + * @since 3.0.0 + */ + @CheckReturnValue + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final <@NonNull R> ParallelFlowable flatMapStream(@NonNull Function> mapper, int prefetch) { + Objects.requireNonNull(mapper, "mapper is null"); + ObjectHelper.verifyPositive(prefetch, "prefetch"); + return RxJavaPlugins.onAssembly(new ParallelFlatMapStream<>(this, mapper, prefetch)); + } + + /** + * Reduces all values within a 'rail' and across 'rails' with a callbacks + * of the given {@link Collector} into a single sequential value. + *

+ * Each parallel rail receives its own {@link Collector#accumulator()} and + * {@link Collector#combiner()}. + *

+ *
Backpressure:
+ *
The operator honors backpressure from the downstream and consumes + * the upstream rails in an unbounded manner (requesting {@link Long#MAX_VALUE}).
+ *
Scheduler:
+ *
{@code collect} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the accumulator type + * @param the output value type + * @param collector the {@code Collector} instance + * @return the new Flowable instance emitting the collected value. + * @since 3.0.0 + */ + @CheckReturnValue + @NonNull + @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @SchedulerSupport(SchedulerSupport.NONE) + public final Flowable collect(@NonNull Collector collector) { + Objects.requireNonNull(collector, "collector is null"); + return RxJavaPlugins.onAssembly(new ParallelCollector<>(this, collector)); + } } diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollectorTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollectorTest.java new file mode 100644 index 0000000000..8e558a0048 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollectorTest.java @@ -0,0 +1,305 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.function.*; +import java.util.stream.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.parallel.ParallelInvalid; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import io.reactivex.rxjava3.processors.BehaviorProcessor; +import io.reactivex.rxjava3.schedulers.Schedulers; +import io.reactivex.rxjava3.subscribers.TestSubscriber; +import io.reactivex.rxjava3.testsupport.*; + +public class ParallelCollectorTest extends RxJavaTest { + + static Set set(int count) { + return IntStream.rangeClosed(1, count) + .boxed() + .collect(Collectors.toSet()); + } + + @Test + public void basic() { + TestSubscriberEx> ts = Flowable.range(1, 5) + .parallel() + .collect(Collectors.toList()) + .subscribeWith(new TestSubscriberEx<>()); + + ts + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + assertEquals(5, ts.values().get(0).size()); + assertTrue(ts.values().get(0).containsAll(set(5))); + } + + @Test + public void empty() { + Flowable.empty() + .parallel() + .collect(Collectors.toList()) + .test() + .assertResult(Collections.emptyList()); + } + + @Test + public void error() { + Flowable.error(new TestException()) + .parallel() + .collect(Collectors.toList()) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorSupplierCrash() { + Flowable.range(1, 5) + .parallel() + .collect(new Collector() { + + @Override + public Supplier supplier() { + throw new TestException(); + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorAccumulatorCrash() { + BehaviorProcessor source = BehaviorProcessor.createDefault(1); + + source + .parallel() + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + + assertFalse(source.hasSubscribers()); + } + + @Test + public void collectorCombinerCrash() { + Flowable.range(1, 5) + .parallel() + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> { throw new TestException(); }; + } + + @Override + public Function finisher() { + return a -> a; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void collectorFinisherCrash() { + Flowable.range(1, 5) + .parallel() + .collect(new Collector() { + + @Override + public Supplier supplier() { + return () -> 1; + } + + @Override + public BiConsumer accumulator() { + return (a, b) -> { }; + } + + @Override + public BinaryOperator combiner() { + return (a, b) -> a + b; + } + + @Override + public Function finisher() { + return a -> { throw new TestException(); }; + } + + @Override + public Set characteristics() { + return Collections.emptySet(); + } + }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void async() { + for (int i = 1; i < 32; i++) { + TestSubscriber> ts = Flowable.range(1, 1000) + .parallel(i) + .runOn(Schedulers.computation()) + .collect(Collectors.toList()) + .test() + .withTag("Parallelism: " + i) + .awaitDone(5, TimeUnit.SECONDS) + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + assertEquals(1000, ts.values().get(0).size()); + + assertTrue(ts.values().get(0).containsAll(set(1000))); + } + } + + @Test + public void asyncHidden() { + for (int i = 1; i < 32; i++) { + TestSubscriber> ts = Flowable.range(1, 1000) + .hide() + .parallel(i) + .runOn(Schedulers.computation()) + .collect(Collectors.toList()) + .test() + .withTag("Parallelism: " + i) + .awaitDone(5, TimeUnit.SECONDS) + .assertValueCount(1) + .assertNoErrors() + .assertComplete(); + + assertEquals(1000, ts.values().get(0).size()); + + assertTrue(ts.values().get(0).containsAll(set(1000))); + } + } + + @Test + public void doubleError() { + List errors = TestHelper.trackPluginErrors(); + try { + new ParallelInvalid() + .collect(Collectors.toList()) + .test() + .assertFailure(TestException.class); + + assertFalse(errors.isEmpty()); + for (Throwable ex : errors) { + assertTrue(ex.toString(), ex.getCause() instanceof TestException); + } + } finally { + RxJavaPlugins.reset(); + } + } + + @Test + public void asyncSum() { + long n = 1_000; + for (int i = 1; i < 32; i++) { + Flowable.rangeLong(1, n) + .parallel(i) + .runOn(Schedulers.computation()) + .collect(Collectors.summingLong(v -> v)) + .test() + .withTag("Parallelism: " + i) + .awaitDone(5, TimeUnit.SECONDS) + .assertResult(n * (n + 1) / 2); + } + } + + @Test + public void asyncSumLong() { + long n = 1_000_000; + Flowable.rangeLong(1, n) + .parallel() + .runOn(Schedulers.computation()) + .collect(Collectors.summingLong(v -> v)) + .test() + .awaitDone(5, TimeUnit.SECONDS) + .assertResult(n * (n + 1) / 2); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStreamTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStreamTest.java new file mode 100644 index 0000000000..0f39d4c842 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelFlatMapStreamTest.java @@ -0,0 +1,73 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.Stream; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.parallel.ParallelFlowableTest; + +public class ParallelFlatMapStreamTest extends RxJavaTest { + + @Test + public void subscriberCount() { + ParallelFlowableTest.checkSubscriberCount(Flowable.range(1, 5).parallel() + .flatMapStream(v -> Stream.of(1, 2, 3))); + } + + @Test + public void normal() { + for (int i = 1; i < 32; i++) { + Flowable.range(1, 1000) + .parallel(i) + .flatMapStream(v -> Stream.of(v, v + 1)) + .sequential() + .test() + .withTag("Parallelism: " + i) + .assertValueCount(2000) + .assertNoErrors() + .assertComplete(); + } + } + + @Test + public void none() { + for (int i = 1; i < 32; i++) { + Flowable.range(1, 1000) + .parallel(i) + .flatMapStream(v -> Stream.of()) + .sequential() + .test() + .withTag("Parallelism: " + i) + .assertResult(); + } + } + + @Test + public void mixed() { + for (int i = 1; i < 32; i++) { + Flowable.range(1, 1000) + .parallel(i) + .flatMapStream(v -> v % 2 == 0 ? Stream.of(v) : Stream.of()) + .sequential() + .test() + .withTag("Parallelism: " + i) + .assertValueCount(500) + .assertNoErrors() + .assertComplete(); + } + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptionalTest.java new file mode 100644 index 0000000000..67cab4087e --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapOptionalTest.java @@ -0,0 +1,228 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; + +import java.util.*; +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.*; +import io.reactivex.rxjava3.internal.functions.Functions; +import io.reactivex.rxjava3.parallel.*; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import io.reactivex.rxjava3.schedulers.Schedulers; +import io.reactivex.rxjava3.testsupport.TestHelper; + +public class ParallelMapOptionalTest extends RxJavaTest { + + @Test + public void doubleFilter() { + Flowable.range(1, 10) + .parallel() + .mapOptional(Optional::of) + .filter(new Predicate() { + @Override + public boolean test(Integer v) throws Exception { + return v % 2 == 0; + } + }) + .filter(new Predicate() { + @Override + public boolean test(Integer v) throws Exception { + return v % 3 == 0; + } + }) + .sequential() + .test() + .assertResult(6); + } + + @Test + public void doubleFilterAsync() { + Flowable.range(1, 10) + .parallel() + .runOn(Schedulers.computation()) + .mapOptional(Optional::of) + .filter(new Predicate() { + @Override + public boolean test(Integer v) throws Exception { + return v % 2 == 0; + } + }) + .filter(new Predicate() { + @Override + public boolean test(Integer v) throws Exception { + return v % 3 == 0; + } + }) + .sequential() + .test() + .awaitDone(5, TimeUnit.SECONDS) + .assertResult(6); + } + + @Test + public void doubleError() { + List errors = TestHelper.trackPluginErrors(); + try { + new ParallelInvalid() + .mapOptional(Optional::of) + .sequential() + .test() + .assertFailure(TestException.class); + + assertFalse(errors.isEmpty()); + for (Throwable ex : errors) { + assertTrue(ex.toString(), ex.getCause() instanceof TestException); + } + } finally { + RxJavaPlugins.reset(); + } + } + + @Test + public void doubleError2() { + List errors = TestHelper.trackPluginErrors(); + try { + new ParallelInvalid() + .mapOptional(Optional::of) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertFailure(TestException.class); + + assertFalse(errors.isEmpty()); + for (Throwable ex : errors) { + assertTrue(ex.toString(), ex.getCause() instanceof TestException); + } + } finally { + RxJavaPlugins.reset(); + } + } + + @Test + public void error() { + Flowable.error(new TestException()) + .parallel() + .mapOptional(Optional::of) + .sequential() + .test() + .assertFailure(TestException.class); + } + + @Test + public void mapCrash() { + Flowable.just(1) + .parallel() + .mapOptional(v -> { throw new TestException(); }) + .sequential() + .test() + .assertFailure(TestException.class); + } + + @Test + public void mapCrashConditional() { + Flowable.just(1) + .parallel() + .mapOptional(v -> { throw new TestException(); }) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertFailure(TestException.class); + } + + @Test + public void mapCrashConditional2() { + Flowable.just(1) + .parallel() + .runOn(Schedulers.computation()) + .mapOptional(v -> { throw new TestException(); }) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .awaitDone(5, TimeUnit.SECONDS) + .assertFailure(TestException.class); + } + + @Test + public void allNone() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> Optional.empty()) + .sequential() + .test() + .assertResult(); + } + + @Test + public void allNoneConditional() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> Optional.empty()) + .filter(v -> true) + .sequential() + .test() + .assertResult(); + } + + @Test + public void mixed() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> v % 2 == 0 ? Optional.of(v) : Optional.empty()) + .sequential() + .test() + .assertValueCount(500) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void mixedConditional() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> v % 2 == 0 ? Optional.of(v) : Optional.empty()) + .filter(v -> true) + .sequential() + .test() + .assertValueCount(500) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void invalidSubscriberCount() { + TestHelper.checkInvalidParallelSubscribers( + Flowable.range(1, 10).parallel() + .mapOptional(Optional::of) + ); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.mapOptional(Optional::of) + ); + + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.mapOptional(Optional::of) + .filter(v -> true) + ); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptionalTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptionalTest.java new file mode 100644 index 0000000000..b7cae91568 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/ParallelMapTryOptionalTest.java @@ -0,0 +1,378 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.*; +import io.reactivex.rxjava3.functions.*; +import io.reactivex.rxjava3.internal.functions.Functions; +import io.reactivex.rxjava3.parallel.*; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import io.reactivex.rxjava3.testsupport.*; + +public class ParallelMapTryOptionalTest extends RxJavaTest implements Consumer { + + volatile int calls; + + @Override + public void accept(Object t) throws Exception { + calls++; + } + + @Test + public void mapNoError() { + for (ParallelFailureHandling e : ParallelFailureHandling.values()) { + Flowable.just(1) + .parallel(1) + .mapOptional(Optional::of, e) + .sequential() + .test() + .assertResult(1); + } + } + + @Test + public void mapErrorNoError() { + for (ParallelFailureHandling e : ParallelFailureHandling.values()) { + Flowable.error(new TestException()) + .parallel(1) + .mapOptional(Optional::of, e) + .sequential() + .test() + .assertFailure(TestException.class); + } + } + + @Test + public void mapConditionalNoError() { + for (ParallelFailureHandling e : ParallelFailureHandling.values()) { + Flowable.just(1) + .parallel(1) + .mapOptional(Optional::of, e) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertResult(1); + } + } + + @Test + public void mapErrorConditionalNoError() { + for (ParallelFailureHandling e : ParallelFailureHandling.values()) { + Flowable.error(new TestException()) + .parallel(1) + .mapOptional(Optional::of, e) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertFailure(TestException.class); + } + } + + @Test + public void mapFailWithError() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), ParallelFailureHandling.ERROR) + .sequential() + .test() + .assertFailure(ArithmeticException.class); + } + + @Test + public void mapFailWithStop() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), ParallelFailureHandling.STOP) + .sequential() + .test() + .assertResult(); + } + + @Test + public void mapFailWithRetry() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(new Function>() { + int count; + @Override + public Optional apply(Integer v) throws Exception { + if (count++ == 1) { + return Optional.of(-1); + } + return Optional.of(1 / v); + } + }, ParallelFailureHandling.RETRY) + .sequential() + .test() + .assertResult(-1, 1); + } + + @Test + public void mapFailWithRetryLimited() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), new BiFunction() { + @Override + public ParallelFailureHandling apply(Long n, Throwable e) throws Exception { + return n < 5 ? ParallelFailureHandling.RETRY : ParallelFailureHandling.SKIP; + } + }) + .sequential() + .test() + .assertResult(1); + } + + @Test + public void mapFailWithSkip() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), ParallelFailureHandling.SKIP) + .sequential() + .test() + .assertResult(1); + } + + @Test + public void mapFailHandlerThrows() { + TestSubscriberEx ts = Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), new BiFunction() { + @Override + public ParallelFailureHandling apply(Long n, Throwable e) throws Exception { + throw new TestException(); + } + }) + .sequential() + .to(TestHelper.testConsumer()) + .assertFailure(CompositeException.class); + + TestHelper.assertCompositeExceptions(ts, ArithmeticException.class, TestException.class); + } + + @Test + public void mapInvalidSource() { + List errors = TestHelper.trackPluginErrors(); + try { + new ParallelInvalid() + .mapOptional(Optional::of, ParallelFailureHandling.ERROR) + .sequential() + .test(); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + } finally { + RxJavaPlugins.reset(); + } + } + + @Test + public void mapFailWithErrorConditional() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), ParallelFailureHandling.ERROR) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertFailure(ArithmeticException.class); + } + + @Test + public void mapFailWithStopConditional() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), ParallelFailureHandling.STOP) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertResult(); + } + + @Test + public void mapFailWithRetryConditional() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(new Function>() { + int count; + @Override + public Optional apply(Integer v) throws Exception { + if (count++ == 1) { + return Optional.of(-1); + } + return Optional.of(1 / v); + } + }, ParallelFailureHandling.RETRY) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertResult(-1, 1); + } + + @Test + public void mapFailWithRetryLimitedConditional() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), new BiFunction() { + @Override + public ParallelFailureHandling apply(Long n, Throwable e) throws Exception { + return n < 5 ? ParallelFailureHandling.RETRY : ParallelFailureHandling.SKIP; + } + }) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertResult(1); + } + + @Test + public void mapFailWithSkipConditional() { + Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), ParallelFailureHandling.SKIP) + .filter(Functions.alwaysTrue()) + .sequential() + .test() + .assertResult(1); + } + + @Test + public void mapFailHandlerThrowsConditional() { + TestSubscriberEx ts = Flowable.range(0, 2) + .parallel(1) + .mapOptional(v -> Optional.of(1 / v), new BiFunction() { + @Override + public ParallelFailureHandling apply(Long n, Throwable e) throws Exception { + throw new TestException(); + } + }) + .filter(Functions.alwaysTrue()) + .sequential() + .to(TestHelper.testConsumer()) + .assertFailure(CompositeException.class); + + TestHelper.assertCompositeExceptions(ts, ArithmeticException.class, TestException.class); + } + + @Test + public void mapWrongParallelismConditional() { + TestHelper.checkInvalidParallelSubscribers( + Flowable.just(1).parallel(1) + .mapOptional(Optional::of, ParallelFailureHandling.ERROR) + .filter(Functions.alwaysTrue()) + ); + } + + @Test + public void mapInvalidSourceConditional() { + List errors = TestHelper.trackPluginErrors(); + try { + new ParallelInvalid() + .mapOptional(Optional::of, ParallelFailureHandling.ERROR) + .filter(Functions.alwaysTrue()) + .sequential() + .test(); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + } finally { + RxJavaPlugins.reset(); + } + } + + @Test + public void failureHandlingEnum() { + TestHelper.checkEnum(ParallelFailureHandling.class); + } + + @Test + public void allNone() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> Optional.empty(), ParallelFailureHandling.SKIP) + .sequential() + .test() + .assertResult(); + } + + @Test + public void allNoneConditional() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> Optional.empty(), ParallelFailureHandling.SKIP) + .filter(v -> true) + .sequential() + .test() + .assertResult(); + } + + @Test + public void mixed() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> v % 2 == 0 ? Optional.of(v) : Optional.empty(), ParallelFailureHandling.SKIP) + .sequential() + .test() + .assertValueCount(500) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void mixedConditional() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> v % 2 == 0 ? Optional.of(v) : Optional.empty(), ParallelFailureHandling.SKIP) + .filter(v -> true) + .sequential() + .test() + .assertValueCount(500) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void mixedConditional2() { + Flowable.range(1, 1000) + .parallel() + .mapOptional(v -> v % 2 == 0 ? Optional.of(v) : Optional.empty(), ParallelFailureHandling.SKIP) + .filter(v -> v % 4 == 0) + .sequential() + .test() + .assertValueCount(250) + .assertNoErrors() + .assertComplete(); + } + + @Test + public void invalidSubscriberCount() { + TestHelper.checkInvalidParallelSubscribers( + Flowable.range(1, 10).parallel() + .mapOptional(Optional::of, ParallelFailureHandling.SKIP) + ); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.mapOptional(Optional::of, ParallelFailureHandling.ERROR) + ); + + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.mapOptional(Optional::of, ParallelFailureHandling.ERROR) + .filter(v -> true) + ); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlatMapIterableTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlatMapIterableTest.java new file mode 100644 index 0000000000..916d3651b9 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelFlatMapIterableTest.java @@ -0,0 +1,73 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.parallel; + +import java.util.Arrays; +import java.util.stream.Stream; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; + +public class ParallelFlatMapIterableTest extends RxJavaTest { + + @Test + public void subscriberCount() { + ParallelFlowableTest.checkSubscriberCount(Flowable.range(1, 5).parallel() + .flatMapStream(v -> Stream.of(1, 2, 3))); + } + + @Test + public void normal() { + for (int i = 1; i < 32; i++) { + Flowable.range(1, 1000) + .parallel(i) + .flatMapIterable(v -> Arrays.asList(v, v + 1)) + .sequential() + .test() + .withTag("Parallelism: " + i) + .assertValueCount(2000) + .assertNoErrors() + .assertComplete(); + } + } + + @Test + public void none() { + for (int i = 1; i < 32; i++) { + Flowable.range(1, 1000) + .parallel(i) + .flatMapIterable(v -> Arrays.asList()) + .sequential() + .test() + .withTag("Parallelism: " + i) + .assertResult(); + } + } + + @Test + public void mixed() { + for (int i = 1; i < 32; i++) { + Flowable.range(1, 1000) + .parallel(i) + .flatMapIterable(v -> v % 2 == 0 ? Arrays.asList(v) : Arrays.asList()) + .sequential() + .test() + .withTag("Parallelism: " + i) + .assertValueCount(500) + .assertNoErrors() + .assertComplete(); + } + } +} diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTest.java index c8e850daf1..e45034d416 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTest.java @@ -15,7 +15,7 @@ import static org.junit.Assert.*; -import java.util.List; +import java.util.*; import java.util.concurrent.TimeUnit; import org.junit.Test; @@ -179,4 +179,24 @@ public Object apply(Integer v) throws Exception { .awaitDone(5, TimeUnit.SECONDS) .assertFailure(TestException.class); } + + @Test + public void invalidSubscriberCount() { + TestHelper.checkInvalidParallelSubscribers( + Flowable.range(1, 10).parallel() + .map(v -> v) + ); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.map(v -> v) + ); + + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.map(v -> v) + .filter(v -> true) + ); + } } diff --git a/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java b/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java index b036b2e17f..d56669d755 100644 --- a/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java +++ b/src/test/java/io/reactivex/rxjava3/parallel/ParallelMapTryTest.java @@ -13,7 +13,7 @@ package io.reactivex.rxjava3.parallel; -import java.util.List; +import java.util.*; import org.junit.Test; @@ -353,4 +353,24 @@ public void mapInvalidSourceConditional() { public void failureHandlingEnum() { TestHelper.checkEnum(ParallelFailureHandling.class); } + + @Test + public void invalidSubscriberCount() { + TestHelper.checkInvalidParallelSubscribers( + Flowable.range(1, 10).parallel() + .map(v -> v, ParallelFailureHandling.SKIP) + ); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.map(v -> v, ParallelFailureHandling.ERROR) + ); + + TestHelper.checkDoubleOnSubscribeParallel( + p -> p.map(v -> v, ParallelFailureHandling.ERROR) + .filter(v -> true) + ); + } } diff --git a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java index eabbb6f96d..f50354b6a5 100644 --- a/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java +++ b/src/test/java/io/reactivex/rxjava3/testsupport/TestHelper.java @@ -1421,6 +1421,71 @@ protected void subscribeActual(Subscriber subscriber) { } } + /** + * Check if the given transformed reactive type reports multiple onSubscribe calls to + * RxJavaPlugins. + * @param the input value type + * @param transform the transform to drive an operator + */ + @SuppressWarnings("unchecked") + public static void checkDoubleOnSubscribeParallel(Function, ? extends ParallelFlowable> transform) { + List errors = trackPluginErrors(); + try { + final Boolean[] b = { null, null, null, null }; + final CountDownLatch cdl = new CountDownLatch(2); + + ParallelFlowable source = new ParallelFlowable() { + @Override + public void subscribe(Subscriber[] subscribers) { + for (int i = 0; i < subscribers.length; i++) { + try { + BooleanSubscription bs1 = new BooleanSubscription(); + + subscribers[i].onSubscribe(bs1); + + BooleanSubscription bs2 = new BooleanSubscription(); + + subscribers[i].onSubscribe(bs2); + + b[i * 2 + 0] = bs1.isCancelled(); + b[i * 2 + 1] = bs2.isCancelled(); + } finally { + cdl.countDown(); + } + } + } + + @Override + public int parallelism() { + return 2; + } + }; + + ParallelFlowable out = transform.apply(source); + + out.subscribe(new Subscriber[] { NoOpConsumer.INSTANCE, NoOpConsumer.INSTANCE }); + + try { + assertTrue("Timed out", cdl.await(5, TimeUnit.SECONDS)); + } catch (InterruptedException ex) { + throw ExceptionHelper.wrapOrThrow(ex); + } + + assertEquals("Rail 1 First disposed?", false, b[0]); + assertEquals("Rail 1 Second not disposed?", true, b[1]); + + assertEquals("Rail 2 First disposed?", false, b[2]); + assertEquals("Rail 2 Second not disposed?", true, b[3]); + + assertError(errors, 0, IllegalStateException.class, "Subscription already set!"); + assertError(errors, 1, IllegalStateException.class, "Subscription already set!"); + } catch (Throwable ex) { + throw ExceptionHelper.wrapOrThrow(ex); + } finally { + RxJavaPlugins.reset(); + } + } + /** * Check if the given transformed reactive type reports multiple onSubscribe calls to * RxJavaPlugins. diff --git a/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java b/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java index c87f19f10e..581e78301b 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java +++ b/src/test/java/io/reactivex/rxjava3/validators/TooManyEmptyNewLines.java @@ -112,7 +112,14 @@ static void findPattern(int newLines) throws Exception { fail .append(fname) .append("#L").append(i + 1) - .append("\n"); + .append("\n") + .append(" at ") + .append(fname.replace(".java", "")) + .append(".method(") + .append(fname) + .append(":").append(i + 1) + .append(")\n") + ; total++; i += c; } @@ -124,9 +131,7 @@ static void findPattern(int newLines) throws Exception { } } if (total != 0) { - fail.append("Found ") - .append(total) - .append(" instances"); + fail.insert(0, "Found " + total + " instances\n"); System.out.println(fail); throw new AssertionError(fail.toString()); } From 38bcd5e292c9e44c542ff4bbcf8e23736b4a7748 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Thu, 26 Dec 2019 23:43:56 +0100 Subject: [PATCH 026/665] 3.x: XProcessor.offer to throw NPE immediately (#6799) --- .../rxjava3/processors/BehaviorProcessor.java | 19 +++++++++---------- .../processors/MulticastProcessor.java | 19 +++++++++++-------- .../rxjava3/processors/PublishProcessor.java | 17 ++++++++--------- .../processors/BehaviorProcessorTest.java | 12 +++++++----- .../processors/MulticastProcessorTest.java | 7 ++++++- .../processors/PublishProcessorTest.java | 12 +++++++----- 6 files changed, 48 insertions(+), 38 deletions(-) diff --git a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java index ab4c517e65..ea93669ca3 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/BehaviorProcessor.java @@ -234,11 +234,11 @@ public static BehaviorProcessor create() { */ BehaviorProcessor(T defaultValue) { this(); - this.value.lazySet(Objects.requireNonNull(defaultValue, "defaultValue is null")); + this.value.lazySet(defaultValue); } @Override - protected void subscribeActual(Subscriber s) { + protected void subscribeActual(@NonNull Subscriber s) { BehaviorSubscription bs = new BehaviorSubscription<>(s, this); s.onSubscribe(bs); if (add(bs)) { @@ -258,7 +258,7 @@ protected void subscribeActual(Subscriber s) { } @Override - public void onSubscribe(Subscription s) { + public void onSubscribe(@NonNull Subscription s) { if (terminalEvent.get() != null) { s.cancel(); return; @@ -267,7 +267,7 @@ public void onSubscribe(Subscription s) { } @Override - public void onNext(T t) { + public void onNext(@NonNull T t) { ExceptionHelper.nullCheck(t, "onNext called with a null value."); if (terminalEvent.get() != null) { @@ -281,7 +281,7 @@ public void onNext(T t) { } @Override - public void onError(Throwable t) { + public void onError(@NonNull Throwable t) { ExceptionHelper.nullCheck(t, "onError called with a null Throwable."); if (!terminalEvent.compareAndSet(null, t)) { RxJavaPlugins.onError(t); @@ -316,14 +316,13 @@ public void onComplete() { *

History: 2.0.8 - experimental * @param t the item to emit, not null * @return true if the item was emitted to all Subscribers + * @throws NullPointerException if {@code t} is {@code null} * @since 2.2 */ @CheckReturnValue - public boolean offer(T t) { - if (t == null) { - onError(ExceptionHelper.createNullPointerException("offer called with a null value.")); - return true; - } + public boolean offer(@NonNull T t) { + ExceptionHelper.nullCheck(t, "offer called with a null value."); + BehaviorSubscription[] array = subscribers.get(); for (BehaviorSubscription s : array) { diff --git a/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java index d258ba1718..ea300afe34 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java @@ -258,7 +258,7 @@ public void startUnbounded() { } @Override - public void onSubscribe(Subscription s) { + public void onSubscribe(@NonNull Subscription s) { if (SubscriptionHelper.setOnce(upstream, s)) { if (s instanceof QueueSubscription) { @SuppressWarnings("unchecked") @@ -288,7 +288,7 @@ public void onSubscribe(Subscription s) { } @Override - public void onNext(T t) { + public void onNext(@NonNull T t) { if (once.get()) { return; } @@ -306,26 +306,29 @@ public void onNext(T t) { /** * Tries to offer an item into the internal queue and returns false * if the queue is full. - * @param t the item to offer, not null + * @param t the item to offer, not {@code null} * @return true if successful, false if the queue is full + * @throws NullPointerException if {@code t} is {@code null} + * @throws IllegalStateException if the processor is in fusion mode */ @CheckReturnValue - public boolean offer(T t) { + public boolean offer(@NonNull T t) { + ExceptionHelper.nullCheck(t, "offer called with a null value."); if (once.get()) { return false; } - ExceptionHelper.nullCheck(t, "offer called with a null value."); if (fusionMode == QueueSubscription.NONE) { if (queue.offer(t)) { drain(); return true; } + return false; } - return false; + throw new IllegalStateException("offer() should not be called in fusion mode!"); } @Override - public void onError(Throwable t) { + public void onError(@NonNull Throwable t) { ExceptionHelper.nullCheck(t, "onError called with a null Throwable."); if (once.compareAndSet(false, true)) { error = t; @@ -369,7 +372,7 @@ public Throwable getThrowable() { } @Override - protected void subscribeActual(Subscriber s) { + protected void subscribeActual(@NonNull Subscriber s) { MulticastSubscription ms = new MulticastSubscription<>(s, this); s.onSubscribe(ms); if (add(ms)) { diff --git a/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java index 36bc4eff34..0a73cf1bf8 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/PublishProcessor.java @@ -141,7 +141,7 @@ public static PublishProcessor create() { } @Override - protected void subscribeActual(Subscriber t) { + protected void subscribeActual(@NonNull Subscriber t) { PublishSubscription ps = new PublishSubscription<>(t, this); t.onSubscribe(ps); if (add(ps)) { @@ -226,7 +226,7 @@ void remove(PublishSubscription ps) { } @Override - public void onSubscribe(Subscription s) { + public void onSubscribe(@NonNull Subscription s) { if (subscribers.get() == TERMINATED) { s.cancel(); return; @@ -236,7 +236,7 @@ public void onSubscribe(Subscription s) { } @Override - public void onNext(T t) { + public void onNext(@NonNull T t) { ExceptionHelper.nullCheck(t, "onNext called with a null value."); for (PublishSubscription s : subscribers.get()) { s.onNext(t); @@ -245,7 +245,7 @@ public void onNext(T t) { @SuppressWarnings("unchecked") @Override - public void onError(Throwable t) { + public void onError(@NonNull Throwable t) { ExceptionHelper.nullCheck(t, "onError called with a null Throwable."); if (subscribers.get() == TERMINATED) { RxJavaPlugins.onError(t); @@ -281,14 +281,13 @@ public void onComplete() { *

History: 2.0.8 - experimental * @param t the item to emit, not null * @return true if the item was emitted to all Subscribers + * @throws NullPointerException if {@code t} is {@code null} * @since 2.2 */ @CheckReturnValue - public boolean offer(T t) { - if (t == null) { - onError(ExceptionHelper.createNullPointerException("offer called with a null value.")); - return true; - } + public boolean offer(@NonNull T t) { + ExceptionHelper.nullCheck(t, "offer called with a null value."); + PublishSubscription[] array = subscribers.get(); for (PublishSubscription s : array) { diff --git a/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java index 0f04c4baea..d49ba5f565 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/BehaviorProcessorTest.java @@ -673,12 +673,14 @@ public void offer() { ts = pp.test(1); - assertTrue(pp.offer(null)); - - ts.assertFailure(NullPointerException.class, 2); + try { + pp.offer(null); + fail("Should have thrown NPE!"); + } catch (NullPointerException expected) { + // expected + } - assertTrue(pp.hasThrowable()); - assertTrue(pp.getThrowable().toString(), pp.getThrowable() instanceof NullPointerException); + ts.assertValuesOnly(2); } @Test diff --git a/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java index 4e71919b09..afa14b8deb 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/MulticastProcessorTest.java @@ -466,7 +466,12 @@ public void asyncFused() { up.onNext(i); } - assertFalse(mp.offer(10)); + try { + mp.offer(10); + fail("Should have thrown IllegalStateException"); + } catch (IllegalStateException expected) { + // expected + } up.onComplete(); diff --git a/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java b/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java index 277ac40c78..7c413d9dc0 100644 --- a/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java +++ b/src/test/java/io/reactivex/rxjava3/processors/PublishProcessorTest.java @@ -597,12 +597,14 @@ public void offer() { ts = pp.test(0); - assertTrue(pp.offer(null)); - - ts.assertFailure(NullPointerException.class); + try { + pp.offer(null); + fail("Should have thrown NPE!"); + } catch (NullPointerException expected) { + // expected + } - assertTrue(pp.hasThrowable()); - assertTrue(pp.getThrowable().toString(), pp.getThrowable() instanceof NullPointerException); + ts.assertEmpty(); } @Test From d1cbf57c4357569094e3b2727f9c57a9d56f2e22 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Fri, 27 Dec 2019 00:33:42 +0100 Subject: [PATCH 027/665] 3.x: Add blockingForEach(Consumer, int) overload (#6800) --- .../io/reactivex/rxjava3/core/Flowable.java | 53 +++++++++++++++++-- .../io/reactivex/rxjava3/core/Observable.java | 41 +++++++++++++- 2 files changed, 88 insertions(+), 6 deletions(-) diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 57efac10e0..df3a8a270a 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -5663,8 +5663,8 @@ public final T blockingFirst(@NonNull T defaultItem) { * sequence. *

*
Backpressure:
- *
The operator consumes the source {@code Flowable} in an unbounded manner - * (i.e., no backpressure applied to it).
+ *
The operator requests {@link Flowable#bufferSize()} upfront, then 75% of this + * amount when 75% is received.
*
Scheduler:
*
{@code blockingForEach} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
@@ -5676,14 +5676,57 @@ public final T blockingFirst(@NonNull T defaultItem) { * @param onNext * the {@link Consumer} to invoke for each item emitted by the {@code Flowable} * @throws RuntimeException - * if an error occurs + * if an error occurs; {@link Error}s and {@link RuntimeException}s are rethrown + * as they are, checked {@link Exception}s are wrapped into {@code RuntimeException}s * @see ReactiveX documentation: Subscribe * @see #subscribe(Consumer) + * @see #blockingForEach(Consumer, int) */ - @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) + @BackpressureSupport(BackpressureKind.FULL) @SchedulerSupport(SchedulerSupport.NONE) public final void blockingForEach(@NonNull Consumer onNext) { - Iterator it = blockingIterable().iterator(); + blockingForEach(onNext, bufferSize()); + } + + /** + * Consumes the upstream {@code Flowable} in a blocking fashion and invokes the given + * {@code Consumer} with each upstream item on the current thread until the + * upstream terminates. + *

+ * + *

+ * Note: the method will only return if the upstream terminates or the current + * thread is interrupted. + *

+ * This method executes the {@code Consumer} on the current thread while + * {@link #subscribe(Consumer)} executes the consumer on the original caller thread of the + * sequence. + *

+ *
Backpressure:
+ *
The operator requests the given {@code prefetch} amount upfront, then 75% of this + * amount when 75% is received.
+ *
Scheduler:
+ *
{@code blockingForEach} does not operate by default on a particular {@link Scheduler}.
+ *
Error handling:
+ *
If the source signals an error, the operator wraps a checked {@link Exception} + * into {@link RuntimeException} and throws that. Otherwise, {@code RuntimeException}s and + * {@link Error}s are rethrown as they are.
+ *
+ * + * @param onNext + * the {@link Consumer} to invoke for each item emitted by the {@code Flowable} + * @param bufferSize + * the number of items to prefetch upfront, then 75% of it after 75% received + * @throws RuntimeException + * if an error occurs; {@link Error}s and {@link RuntimeException}s are rethrown + * as they are, checked {@link Exception}s are wrapped into {@code RuntimeException}s + * @see ReactiveX documentation: Subscribe + * @see #subscribe(Consumer) + */ + @BackpressureSupport(BackpressureKind.FULL) + @SchedulerSupport(SchedulerSupport.NONE) + public final void blockingForEach(@NonNull Consumer onNext, int bufferSize) { + Iterator it = blockingIterable(bufferSize).iterator(); while (it.hasNext()) { try { onNext.accept(it.next()); diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index eb33792f34..1933a6db0b 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -5170,11 +5170,50 @@ public final T blockingFirst(@NonNull T defaultItem) { * if an error occurs * @see ReactiveX documentation: Subscribe * @see #subscribe(Consumer) + * @see #blockingForEach(Consumer, int) */ @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final void blockingForEach(@NonNull Consumer onNext) { - Iterator it = blockingIterable().iterator(); + blockingForEach(onNext, bufferSize()); + } + + /** + * Consumes the upstream {@code Observable} in a blocking fashion and invokes the given + * {@code Consumer} with each upstream item on the current thread until the + * upstream terminates. + *

+ * + *

+ * Note: the method will only return if the upstream terminates or the current + * thread is interrupted. + *

+ * This method executes the {@code Consumer} on the current thread while + * {@link #subscribe(Consumer)} executes the consumer on the original caller thread of the + * sequence. + *

+ *
Scheduler:
+ *
{@code blockingForEach} does not operate by default on a particular {@link Scheduler}.
+ *
Error handling:
+ *
If the source signals an error, the operator wraps a checked {@link Exception} + * into {@link RuntimeException} and throws that. Otherwise, {@code RuntimeException}s and + * {@link Error}s are rethrown as they are.
+ *
+ * + * @param onNext + * the {@link Consumer} to invoke for each item emitted by the {@code Observable} + * @param capacityHint + * the number of items expected to be buffered (allows reducing buffer reallocations) + * @throws RuntimeException + * if an error occurs; {@link Error}s and {@link RuntimeException}s are rethrown + * as they are, checked {@link Exception}s are wrapped into {@code RuntimeException}s + * @see ReactiveX documentation: Subscribe + * @see #subscribe(Consumer) + */ + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final void blockingForEach(@NonNull Consumer onNext, int capacityHint) { + Iterator it = blockingIterable(capacityHint).iterator(); while (it.hasNext()) { try { onNext.accept(it.next()); From a97d871ee7161fc9f4684d95cae3e94340cd0ccf Mon Sep 17 00:00:00 2001 From: David Karnok Date: Fri, 27 Dec 2019 00:55:36 +0100 Subject: [PATCH 028/665] 3.x: Add missing throwIfFatal calls (#6801) --- .../disposables/AutoCloseableDisposable.java | 4 +- .../internal/jdk8/FlowableFlatMapStream.java | 8 +- .../jdk8/ObservableFlatMapStream.java | 1 + .../operators/flowable/FlowableCollect.java | 1 + .../flowable/FlowableCollectSingle.java | 1 + .../operators/flowable/FlowableDoOnEach.java | 4 + .../operators/flowable/FlowableReplay.java | 1 + .../flowable/FlowableScalarXMap.java | 1 + .../FlowableWindowBoundarySelector.java | 1 + .../flowable/FlowableZipIterable.java | 8 +- .../operators/maybe/MaybeFromFuture.java | 1 + .../observable/ObservableBuffer.java | 1 + .../observable/ObservableCollect.java | 3 + .../observable/ObservableCollectSingle.java | 3 + .../observable/ObservableReplay.java | 1 + .../observable/ObservableScalarXMap.java | 1 + .../ObservableWindowBoundarySelector.java | 1 + .../schedulers/InstantPeriodicTask.java | 2 + .../ScheduledDirectPeriodicTask.java | 2 + .../schedulers/SchedulerPoolFactory.java | 3 + .../rxjava3/disposables/DisposableTest.java | 11 ++- .../validators/CatchThrowIfFatalCheck.java | 96 +++++++++++++++++++ 22 files changed, 144 insertions(+), 11 deletions(-) create mode 100644 src/test/java/io/reactivex/rxjava3/validators/CatchThrowIfFatalCheck.java diff --git a/src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java b/src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java index 8c0f2be2dc..08cbe9cd99 100644 --- a/src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/disposables/AutoCloseableDisposable.java @@ -14,7 +14,7 @@ package io.reactivex.rxjava3.disposables; import io.reactivex.rxjava3.annotations.NonNull; -import io.reactivex.rxjava3.plugins.RxJavaPlugins; +import io.reactivex.rxjava3.internal.util.ExceptionHelper; /** * A disposable container that manages an {@link AutoCloseable} instance. @@ -33,7 +33,7 @@ protected void onDisposed(@NonNull AutoCloseable value) { try { value.close(); } catch (Throwable ex) { - RxJavaPlugins.onError(ex); + throw ExceptionHelper.wrapOrThrow(ex); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java index c967a011e5..b01b24748a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/FlowableFlatMapStream.java @@ -21,7 +21,7 @@ import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.*; -import io.reactivex.rxjava3.exceptions.MissingBackpressureException; +import io.reactivex.rxjava3.exceptions.*; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.fuseable.*; import io.reactivex.rxjava3.internal.queue.SpscArrayQueue; @@ -61,6 +61,7 @@ protected void subscribeActual(Subscriber s) { stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream"); } } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); EmptySubscription.error(ex, s); return; } @@ -243,6 +244,7 @@ void drain() { try { t = queue.poll(); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); trySignalError(downstream, ex); continue; } @@ -271,6 +273,7 @@ else if (!isEmpty) { iterator = null; } } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); trySignalError(downstream, ex); } continue; @@ -282,6 +285,7 @@ else if (!isEmpty) { try { item = Objects.requireNonNull(iterator.next(), "The Stream.Iterator returned a null value"); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); trySignalError(downstream, ex); continue; } @@ -297,6 +301,7 @@ else if (!isEmpty) { clearCurrentRethrowCloseError(); } } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); trySignalError(downstream, ex); } } @@ -328,6 +333,7 @@ void clearCurrentSuppressCloseError() { try { clearCurrentRethrowCloseError(); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); RxJavaPlugins.onError(ex); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java index 9b7b446e28..968e581625 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java @@ -55,6 +55,7 @@ protected void subscribeActual(Observer observer) { stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream"); } } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); EmptyDisposable.error(ex, observer); return; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java index 80345f2f3b..c2560cf6e3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java @@ -39,6 +39,7 @@ protected void subscribeActual(Subscriber s) { try { u = Objects.requireNonNull(initialSupplier.get(), "The initial value supplied is null"); } catch (Throwable e) { + Exceptions.throwIfFatal(e); EmptySubscription.error(e, s); return; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java index fcb5590b49..edf17d95ef 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java @@ -44,6 +44,7 @@ protected void subscribeActual(SingleObserver observer) { try { u = Objects.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); } catch (Throwable e) { + Exceptions.throwIfFatal(e); EmptyDisposable.error(e, observer); return; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java index b6900b077e..74d74f41cf 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java @@ -159,6 +159,7 @@ public T poll() throws Throwable { try { onError.accept(ex); } catch (Throwable exc) { + Exceptions.throwIfFatal(exc); throw new CompositeException(ex, exc); } throw ExceptionHelper.throwIfThrowable(ex); @@ -173,6 +174,7 @@ public T poll() throws Throwable { try { onError.accept(ex); } catch (Throwable exc) { + Exceptions.throwIfFatal(exc); throw new CompositeException(ex, exc); } throw ExceptionHelper.throwIfThrowable(ex); @@ -314,6 +316,7 @@ public T poll() throws Throwable { try { onError.accept(ex); } catch (Throwable exc) { + Exceptions.throwIfFatal(exc); throw new CompositeException(ex, exc); } throw ExceptionHelper.throwIfThrowable(ex); @@ -328,6 +331,7 @@ public T poll() throws Throwable { try { onError.accept(ex); } catch (Throwable exc) { + Exceptions.throwIfFatal(exc); throw new CompositeException(ex, exc); } throw ExceptionHelper.throwIfThrowable(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java index 8d9e2e8460..6a352e2fc4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java @@ -209,6 +209,7 @@ public void connect(Consumer connection) { try { connection.accept(ps); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); if (doConnect) { ps.shouldConnect.compareAndSet(true, false); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java index 2ef89c0023..d78e83a3d2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java @@ -136,6 +136,7 @@ public void subscribeActual(Subscriber s) { try { other = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null Publisher"); } catch (Throwable e) { + Exceptions.throwIfFatal(e); EmptySubscription.error(e, s); return; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java index 4e38351ef7..d55198010a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java @@ -245,6 +245,7 @@ void drain() { try { endSource = Objects.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null Publisher"); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); upstream.cancel(); startSubscriber.cancel(); resources.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java index a41c591fe2..1fb9777a50 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZipIterable.java @@ -101,7 +101,7 @@ public void onNext(T t) { try { u = Objects.requireNonNull(iterator.next(), "The iterator returned a null value"); } catch (Throwable e) { - error(e); + fail(e); return; } @@ -109,7 +109,7 @@ public void onNext(T t) { try { v = Objects.requireNonNull(zipper.apply(t, u), "The zipper function returned a null value"); } catch (Throwable e) { - error(e); + fail(e); return; } @@ -120,7 +120,7 @@ public void onNext(T t) { try { b = iterator.hasNext(); } catch (Throwable e) { - error(e); + fail(e); return; } @@ -131,7 +131,7 @@ public void onNext(T t) { } } - void error(Throwable e) { + void fail(Throwable e) { Exceptions.throwIfFatal(e); done = true; upstream.cancel(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java index 5d4dbc0f0b..acc93b3ea6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromFuture.java @@ -52,6 +52,7 @@ protected void subscribeActual(MaybeObserver observer) { v = future.get(timeout, unit); } } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); if (ex instanceof ExecutionException) { ex = ex.getCause(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java index 2b09748a61..bd7f4b1baa 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java @@ -186,6 +186,7 @@ public void onNext(T t) { try { b = ExceptionHelper.nullCheck(bufferSupplier.get(), "The bufferSupplier returned a null Collection."); } catch (Throwable e) { + Exceptions.throwIfFatal(e); buffers.clear(); upstream.dispose(); downstream.onError(e); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java index 6471d7538c..76b03eddb6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java @@ -14,6 +14,7 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -37,6 +38,7 @@ protected void subscribeActual(Observer t) { try { u = Objects.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); } catch (Throwable e) { + Exceptions.throwIfFatal(e); EmptyDisposable.error(e, t); return; } @@ -86,6 +88,7 @@ public void onNext(T t) { try { collector.accept(u, t); } catch (Throwable e) { + Exceptions.throwIfFatal(e); upstream.dispose(); onError(e); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java index 09c86cb24a..07db8d5bf3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java @@ -14,6 +14,7 @@ import io.reactivex.rxjava3.core.*; import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; import io.reactivex.rxjava3.internal.disposables.*; import io.reactivex.rxjava3.internal.fuseable.FuseToObservable; @@ -41,6 +42,7 @@ protected void subscribeActual(SingleObserver t) { try { u = Objects.requireNonNull(initialSupplier.get(), "The initialSupplier returned a null value"); } catch (Throwable e) { + Exceptions.throwIfFatal(e); EmptyDisposable.error(e, t); return; } @@ -94,6 +96,7 @@ public void onNext(T t) { try { collector.accept(u, t); } catch (Throwable e) { + Exceptions.throwIfFatal(e); upstream.dispose(); onError(e); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java index ba0811b85c..b6a945e8b7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java @@ -205,6 +205,7 @@ public void connect(Consumer connection) { try { connection.accept(ps); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); if (doConnect) { ps.shouldConnect.compareAndSet(true, false); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java index 128d327e2b..f4a1bce25b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java @@ -140,6 +140,7 @@ public void subscribeActual(Observer observer) { try { other = Objects.requireNonNull(mapper.apply(value), "The mapper returned a null ObservableSource"); } catch (Throwable e) { + Exceptions.throwIfFatal(e); EmptyDisposable.error(e, observer); return; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java index 5ea42da118..8722d6df0c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java @@ -237,6 +237,7 @@ void drain() { try { endSource = Objects.requireNonNull(closingIndicator.apply(startItem), "The closingIndicator returned a null ObservableSource"); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); upstream.dispose(); startObserver.dispose(); resources.dispose(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java index 678c82e794..5df8793ebb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java @@ -20,6 +20,7 @@ import java.util.concurrent.atomic.AtomicReference; import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.internal.functions.Functions; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -56,6 +57,7 @@ public Void call() throws Exception { setRest(executor.submit(this)); runner = null; } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); runner = null; RxJavaPlugins.onError(ex); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectPeriodicTask.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectPeriodicTask.java index 695760f2ca..ac30532e68 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectPeriodicTask.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectPeriodicTask.java @@ -16,6 +16,7 @@ package io.reactivex.rxjava3.internal.schedulers; +import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.plugins.RxJavaPlugins; /** @@ -38,6 +39,7 @@ public void run() { runnable.run(); runner = null; } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); runner = null; lazySet(FINISHED); RxJavaPlugins.onError(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java index cc82349cfc..392a3c3051 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java @@ -20,6 +20,7 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; +import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.Function; /** @@ -108,6 +109,7 @@ static int getIntProperty(boolean enabled, String key, int defaultNotFound, int } return Integer.parseInt(value); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); return defaultNotFound; } } @@ -123,6 +125,7 @@ static boolean getBooleanProperty(boolean enabled, String key, boolean defaultNo } return "true".equals(value); } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); return defaultNotFound; } } diff --git a/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java b/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java index 66abc6a280..b966a1db0f 100644 --- a/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java +++ b/src/test/java/io/reactivex/rxjava3/disposables/DisposableTest.java @@ -213,17 +213,20 @@ public void fromAutoCloseableThrows() throws Throwable { assertTrue(errors.isEmpty()); - d.dispose(); + try { + d.dispose(); + fail("Should have thrown!"); + } catch (TestException expected) { + // expected + } assertTrue(d.isDisposed()); - assertEquals(1, errors.size()); d.dispose(); assertTrue(d.isDisposed()); - assertEquals(1, errors.size()); - TestHelper.assertUndeliverable(errors, 0, TestException.class); + assertTrue(errors.isEmpty()); }); } diff --git a/src/test/java/io/reactivex/rxjava3/validators/CatchThrowIfFatalCheck.java b/src/test/java/io/reactivex/rxjava3/validators/CatchThrowIfFatalCheck.java new file mode 100644 index 0000000000..5b78a2da0a --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/validators/CatchThrowIfFatalCheck.java @@ -0,0 +1,96 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.validators; + +import java.io.File; +import java.nio.file.Files; +import java.util.*; + +import org.junit.Test; + +import io.reactivex.rxjava3.testsupport.TestHelper; + +/** + * Check if a {@code catch(Throwable} is followed by a + * {@code Exceptions.throwIfFatal}, {@code Exceptions.wrapOrThrow} + * or {@code fail} call. + * @since 3.0.0 + */ +public class CatchThrowIfFatalCheck { + + @Test + public void check() throws Exception { + File f = TestHelper.findSource("Flowable"); + if (f == null) { + System.out.println("Unable to find sources of RxJava"); + return; + } + + Queue dirs = new ArrayDeque<>(); + + StringBuilder fail = new StringBuilder(); + int errors = 0; + + File parent = f.getParentFile().getParentFile(); + + dirs.offer(new File(parent.getAbsolutePath().replace('\\', '/'))); + + while (!dirs.isEmpty()) { + f = dirs.poll(); + + File[] list = f.listFiles(); + if (list != null && list.length != 0) { + + for (File u : list) { + if (u.isDirectory()) { + dirs.offer(u); + } else { + List lines = Files.readAllLines(u.toPath()); + + for (int i = 0; i < lines.size(); i++) { + String line = lines.get(i).trim(); + + if (line.startsWith("} catch (Throwable ")) { + String next = lines.get(i + 1).trim(); + boolean throwIfFatal = next.contains("Exceptions.throwIfFatal"); + boolean wrapOrThrow = next.contains("ExceptionHelper.wrapOrThrow"); + boolean failCall = next.startsWith("fail("); + + if (!(throwIfFatal || wrapOrThrow || failCall)) { + errors++; + fail.append("Missing Exceptions.throwIfFatal\n ") + .append(next) + .append("\n at ") + .append(u.getName().replace(".java", "")) + .append(".method(") + .append(u.getName()) + .append(":") + .append(i + 1) + .append(")\n") + ; + } + } + } + } + } + } + } + + if (errors != 0) { + fail.insert(0, "Found " + errors + " cases\n"); + System.out.println(fail); + throw new AssertionError(fail.toString()); + } + } +} From e6eb21bc9df0777105cd1f469754a0bed658e966 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Fri, 27 Dec 2019 09:49:14 +0100 Subject: [PATCH 029/665] Bump guava from 28.1-jre to 28.2-jre (#6802) Bumps [guava](https://github.com/google/guava) from 28.1-jre to 28.2-jre. - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) Signed-off-by: dependabot-preview[bot] --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 2ae17aa25a..f21e95925a 100644 --- a/build.gradle +++ b/build.gradle @@ -9,7 +9,7 @@ buildscript { ext.mockitoVersion = "3.2.4" ext.jmhLibVersion = "1.21" ext.jmhGradleVersion = "0.5.0" - ext.guavaVersion = "28.1-jre" + ext.guavaVersion = "28.2-jre" ext.jacocoVersion = "0.8.4" ext.animalSnifferVersion = "1.5.0" ext.licenseVersion = "0.15.0" From 79f8e6dde6446b1aa33c146eaedbb958086daf56 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Fri, 27 Dec 2019 20:01:34 +0100 Subject: [PATCH 030/665] 3.x: Fix diamonds, spelling, unnecessary code (#6804) --- docs/Backpressure.md | 30 ++-- docs/Combining-Observables.md | 8 +- docs/How-To-Use-RxJava.md | 4 +- docs/Writing-operators-for-2.0.md | 2 +- .../rxjava3/core/BinaryFlatMapPerf.java | 36 ++--- .../rxjava3/core/CallableAsyncPerf.java | 30 ++-- .../rxjava3/core/EachTypeFlatMapPerf.java | 4 +- .../rxjava3/core/FlatMapJustPerf.java | 4 +- .../rxjava3/core/FlattenCrossMapPerf.java | 4 +- .../rxjava3/core/FlattenJustPerf.java | 4 +- .../rxjava3/core/FlattenRangePerf.java | 4 +- .../FlowableFlatMapCompletableAsyncPerf.java | 2 +- .../core/InputWithIncrementingInteger.java | 4 +- .../reactivex/rxjava3/core/JustAsyncPerf.java | 28 ++-- .../io/reactivex/rxjava3/core/MemoryPerf.java | 110 ++++++------- .../rxjava3/core/OperatorFlatMapPerf.java | 4 +- .../rxjava3/core/PerfSubscriber.java | 2 +- .../io/reactivex/rxjava3/core/ReducePerf.java | 2 +- .../reactivex/rxjava3/core/TakeUntilPerf.java | 10 +- .../rxjava3/core/ToFlowablePerf.java | 6 +- .../io/reactivex/rxjava3/core/XMapYPerf.java | 76 ++++----- .../rxjava3/parallel/ParallelPerf.java | 8 +- .../FlowableConcatMapCompletablePerf.java | 9 +- .../FlowableConcatMapMaybeEmptyPerf.java | 9 +- .../xmapz/FlowableConcatMapMaybePerf.java | 9 +- .../xmapz/FlowableConcatMapSinglePerf.java | 9 +- .../xmapz/FlowableFlatMapCompletablePerf.java | 9 +- .../xmapz/FlowableFlatMapMaybeEmptyPerf.java | 9 +- .../xmapz/FlowableFlatMapMaybePerf.java | 9 +- .../xmapz/FlowableFlatMapSinglePerf.java | 9 +- .../FlowableSwitchMapCompletablePerf.java | 9 +- .../FlowableSwitchMapMaybeEmptyPerf.java | 9 +- .../xmapz/FlowableSwitchMapMaybePerf.java | 9 +- .../xmapz/FlowableSwitchMapSinglePerf.java | 9 +- .../ObservableConcatMapCompletablePerf.java | 9 +- .../ObservableConcatMapMaybeEmptyPerf.java | 9 +- .../xmapz/ObservableConcatMapMaybePerf.java | 9 +- .../xmapz/ObservableConcatMapSinglePerf.java | 9 +- .../ObservableFlatMapCompletablePerf.java | 9 +- .../ObservableFlatMapMaybeEmptyPerf.java | 9 +- .../xmapz/ObservableFlatMapMaybePerf.java | 9 +- .../xmapz/ObservableFlatMapSinglePerf.java | 9 +- .../ObservableSwitchMapCompletablePerf.java | 9 +- .../ObservableSwitchMapMaybeEmptyPerf.java | 9 +- .../xmapz/ObservableSwitchMapMaybePerf.java | 9 +- .../xmapz/ObservableSwitchMapSinglePerf.java | 9 +- .../reactivex/rxjava3/core/Completable.java | 14 +- .../io/reactivex/rxjava3/core/Flowable.java | 148 +++++++++--------- .../java/io/reactivex/rxjava3/core/Maybe.java | 36 ++--- .../io/reactivex/rxjava3/core/Observable.java | 131 ++++++++-------- .../io/reactivex/rxjava3/core/Single.java | 43 +++-- .../exceptions/CompositeException.java | 3 +- .../rxjava3/exceptions/package-info.java | 2 +- .../internal/disposables/EmptyDisposable.java | 2 +- .../disposables/ListCompositeDisposable.java | 10 +- .../rxjava3/internal/functions/Functions.java | 88 +++++------ .../jdk8/ObservableFlatMapStream.java | 2 +- .../internal/jdk8/ParallelCollector.java | 1 - .../observers/DeferredScalarDisposable.java | 2 +- .../internal/observers/FutureObserver.java | 5 +- .../observers/FutureSingleObserver.java | 5 +- .../completable/CompletableCache.java | 2 +- .../completable/CompletableConcat.java | 4 +- .../CompletableFromObservable.java | 2 +- .../completable/CompletableFromPublisher.java | 2 +- .../completable/CompletableFromSingle.java | 2 +- .../completable/CompletableMaterialize.java | 2 +- .../completable/CompletableToFlowable.java | 2 +- .../completable/CompletableToObservable.java | 2 +- .../completable/CompletableUsing.java | 2 +- .../flowable/BlockingFlowableIterable.java | 4 +- .../flowable/BlockingFlowableLatest.java | 4 +- .../flowable/BlockingFlowableMostRecent.java | 2 +- .../flowable/BlockingFlowableNext.java | 6 +- .../operators/flowable/FlowableAll.java | 2 +- .../operators/flowable/FlowableAllSingle.java | 4 +- .../operators/flowable/FlowableAmb.java | 4 +- .../operators/flowable/FlowableAny.java | 2 +- .../operators/flowable/FlowableAnySingle.java | 4 +- .../flowable/FlowableBlockingSubscribe.java | 6 +- .../operators/flowable/FlowableBuffer.java | 8 +- .../flowable/FlowableBufferBoundary.java | 12 +- .../flowable/FlowableBufferExactBoundary.java | 6 +- .../flowable/FlowableBufferTimed.java | 24 +-- .../operators/flowable/FlowableCache.java | 8 +- .../operators/flowable/FlowableCollect.java | 2 +- .../flowable/FlowableCollectSingle.java | 4 +- .../flowable/FlowableCombineLatest.java | 10 +- .../flowable/FlowableConcatArray.java | 6 +- .../operators/flowable/FlowableConcatMap.java | 14 +- .../flowable/FlowableConcatMapEager.java | 6 +- .../FlowableConcatMapEagerPublisher.java | 2 +- .../flowable/FlowableConcatMapScheduler.java | 14 +- .../FlowableConcatWithCompletable.java | 2 +- .../flowable/FlowableConcatWithMaybe.java | 4 +- .../flowable/FlowableConcatWithSingle.java | 4 +- .../flowable/FlowableCountSingle.java | 2 +- .../operators/flowable/FlowableCreate.java | 18 +-- .../operators/flowable/FlowableDebounce.java | 6 +- .../flowable/FlowableDebounceTimed.java | 6 +- .../operators/flowable/FlowableDelay.java | 4 +- .../FlowableDelaySubscriptionOther.java | 4 +- .../flowable/FlowableDematerialize.java | 2 +- .../operators/flowable/FlowableDetach.java | 2 +- .../operators/flowable/FlowableDistinct.java | 2 +- .../FlowableDistinctUntilChanged.java | 4 +- .../flowable/FlowableDoAfterNext.java | 4 +- .../operators/flowable/FlowableDoFinally.java | 4 +- .../operators/flowable/FlowableDoOnEach.java | 6 +- .../flowable/FlowableDoOnLifecycle.java | 2 +- .../operators/flowable/FlowableElementAt.java | 2 +- .../flowable/FlowableElementAtMaybe.java | 4 +- .../flowable/FlowableElementAtSingle.java | 4 +- .../operators/flowable/FlowableFilter.java | 6 +- .../operators/flowable/FlowableFlatMap.java | 14 +- .../flowable/FlowableFlatMapCompletable.java | 4 +- ...FlowableFlatMapCompletableCompletable.java | 4 +- .../flowable/FlowableFlatMapMaybe.java | 6 +- .../flowable/FlowableFlatMapSingle.java | 6 +- .../flowable/FlowableFlattenIterable.java | 2 +- .../operators/flowable/FlowableFromArray.java | 4 +- .../flowable/FlowableFromCallable.java | 2 +- .../flowable/FlowableFromFuture.java | 2 +- .../flowable/FlowableFromObservable.java | 2 +- .../flowable/FlowableFromSupplier.java | 2 +- .../operators/flowable/FlowableGenerate.java | 2 +- .../operators/flowable/FlowableGroupBy.java | 16 +- .../operators/flowable/FlowableGroupJoin.java | 12 +- .../operators/flowable/FlowableHide.java | 2 +- .../flowable/FlowableIgnoreElements.java | 2 +- .../FlowableIgnoreElementsCompletable.java | 4 +- .../flowable/FlowableInternalHelper.java | 38 ++--- .../operators/flowable/FlowableInterval.java | 2 +- .../flowable/FlowableIntervalRange.java | 2 +- .../operators/flowable/FlowableJoin.java | 10 +- .../operators/flowable/FlowableJust.java | 2 +- .../operators/flowable/FlowableLastMaybe.java | 2 +- .../flowable/FlowableLastSingle.java | 2 +- .../flowable/FlowableMapNotification.java | 2 +- .../flowable/FlowableMaterialize.java | 6 +- .../FlowableMergeWithCompletable.java | 4 +- .../flowable/FlowableMergeWithMaybe.java | 8 +- .../flowable/FlowableMergeWithSingle.java | 8 +- .../operators/flowable/FlowableObserveOn.java | 8 +- .../FlowableOnBackpressureBuffer.java | 8 +- .../FlowableOnBackpressureBufferStrategy.java | 4 +- .../flowable/FlowableOnBackpressureDrop.java | 2 +- .../flowable/FlowableOnBackpressureError.java | 2 +- .../FlowableOnBackpressureLatest.java | 4 +- .../flowable/FlowableOnErrorNext.java | 2 +- .../flowable/FlowableOnErrorReturn.java | 2 +- .../operators/flowable/FlowablePublish.java | 20 +-- .../flowable/FlowablePublishMulticast.java | 10 +- .../operators/flowable/FlowableReduce.java | 2 +- .../flowable/FlowableReduceMaybe.java | 4 +- .../flowable/FlowableReduceSeedSingle.java | 2 +- .../flowable/FlowableReduceWithSingle.java | 2 +- .../operators/flowable/FlowableRefCount.java | 4 +- .../operators/flowable/FlowableRepeat.java | 2 +- .../flowable/FlowableRepeatUntil.java | 2 +- .../flowable/FlowableRepeatWhen.java | 10 +- .../operators/flowable/FlowableReplay.java | 30 ++-- .../flowable/FlowableRetryBiPredicate.java | 2 +- .../flowable/FlowableRetryPredicate.java | 2 +- .../operators/flowable/FlowableRetryWhen.java | 6 +- .../flowable/FlowableSamplePublisher.java | 10 +- .../flowable/FlowableSampleTimed.java | 6 +- .../flowable/FlowableScalarXMap.java | 6 +- .../operators/flowable/FlowableScan.java | 2 +- .../operators/flowable/FlowableScanSeed.java | 4 +- .../flowable/FlowableSequenceEqual.java | 8 +- .../flowable/FlowableSequenceEqualSingle.java | 8 +- .../flowable/FlowableSerialized.java | 2 +- .../operators/flowable/FlowableSingle.java | 2 +- .../flowable/FlowableSingleMaybe.java | 4 +- .../flowable/FlowableSingleSingle.java | 4 +- .../operators/flowable/FlowableSkip.java | 2 +- .../operators/flowable/FlowableSkipLast.java | 2 +- .../flowable/FlowableSkipLastTimed.java | 4 +- .../operators/flowable/FlowableSkipUntil.java | 4 +- .../operators/flowable/FlowableSkipWhile.java | 2 +- .../flowable/FlowableSubscribeOn.java | 4 +- .../flowable/FlowableSwitchIfEmpty.java | 2 +- .../operators/flowable/FlowableSwitchMap.java | 10 +- .../operators/flowable/FlowableTake.java | 9 +- .../operators/flowable/FlowableTakeLast.java | 2 +- .../flowable/FlowableTakeLastOne.java | 2 +- .../flowable/FlowableTakeLastTimed.java | 4 +- .../flowable/FlowableTakePublisher.java | 2 +- .../operators/flowable/FlowableTakeUntil.java | 4 +- .../flowable/FlowableTakeUntilPredicate.java | 2 +- .../operators/flowable/FlowableTakeWhile.java | 2 +- .../flowable/FlowableThrottleFirstTimed.java | 4 +- .../flowable/FlowableThrottleLatest.java | 4 +- .../flowable/FlowableTimeInterval.java | 4 +- .../operators/flowable/FlowableTimeout.java | 8 +- .../flowable/FlowableTimeoutTimed.java | 8 +- .../operators/flowable/FlowableToList.java | 2 +- .../flowable/FlowableToListSingle.java | 4 +- .../flowable/FlowableUnsubscribeOn.java | 2 +- .../operators/flowable/FlowableUsing.java | 2 +- .../operators/flowable/FlowableWindow.java | 24 +-- .../flowable/FlowableWindowBoundary.java | 10 +- .../FlowableWindowBoundarySelector.java | 14 +- .../flowable/FlowableWindowTimed.java | 22 +-- .../flowable/FlowableWithLatestFrom.java | 8 +- .../flowable/FlowableWithLatestFromMany.java | 8 +- .../operators/flowable/FlowableZip.java | 6 +- .../internal/operators/maybe/MaybeCache.java | 6 +- .../operators/maybe/MaybeConcatArray.java | 4 +- .../maybe/MaybeConcatArrayDelayError.java | 4 +- .../operators/maybe/MaybeConcatIterable.java | 4 +- .../internal/operators/maybe/MaybeCreate.java | 2 +- .../internal/operators/maybe/MaybeDelay.java | 2 +- .../maybe/MaybeDelayOtherPublisher.java | 4 +- .../MaybeDelaySubscriptionOtherPublisher.java | 4 +- .../maybe/MaybeDelayWithCompletable.java | 4 +- .../internal/operators/maybe/MaybeDetach.java | 2 +- .../operators/maybe/MaybeDoAfterSuccess.java | 2 +- .../operators/maybe/MaybeDoFinally.java | 2 +- .../operators/maybe/MaybeDoOnEvent.java | 2 +- .../operators/maybe/MaybeEqualSingle.java | 6 +- .../internal/operators/maybe/MaybeFilter.java | 2 +- .../operators/maybe/MaybeFilterSingle.java | 2 +- .../maybe/MaybeFlatMapBiSelector.java | 2 +- .../maybe/MaybeFlatMapCompletable.java | 2 +- .../maybe/MaybeFlatMapIterableFlowable.java | 4 +- .../maybe/MaybeFlatMapIterableObservable.java | 4 +- .../maybe/MaybeFlatMapNotification.java | 2 +- .../operators/maybe/MaybeFlatMapSingle.java | 2 +- .../maybe/MaybeFlatMapSingleElement.java | 2 +- .../operators/maybe/MaybeFlatten.java | 2 +- .../operators/maybe/MaybeFromRunnable.java | 2 +- .../operators/maybe/MaybeFromSingle.java | 2 +- .../internal/operators/maybe/MaybeHide.java | 2 +- .../operators/maybe/MaybeIgnoreElement.java | 2 +- .../maybe/MaybeIgnoreElementCompletable.java | 4 +- .../operators/maybe/MaybeIsEmpty.java | 2 +- .../operators/maybe/MaybeIsEmptySingle.java | 4 +- .../operators/maybe/MaybeMaterialize.java | 2 +- .../operators/maybe/MaybeMergeArray.java | 8 +- .../operators/maybe/MaybeObserveOn.java | 2 +- .../operators/maybe/MaybeOnErrorComplete.java | 2 +- .../operators/maybe/MaybeOnErrorNext.java | 2 +- .../operators/maybe/MaybeOnErrorReturn.java | 2 +- .../internal/operators/maybe/MaybePeek.java | 2 +- .../operators/maybe/MaybeSubscribeOn.java | 4 +- .../operators/maybe/MaybeSwitchIfEmpty.java | 2 +- .../maybe/MaybeSwitchIfEmptySingle.java | 2 +- .../operators/maybe/MaybeTakeUntilMaybe.java | 4 +- .../maybe/MaybeTakeUntilPublisher.java | 4 +- .../operators/maybe/MaybeTimeoutMaybe.java | 6 +- .../maybe/MaybeTimeoutPublisher.java | 6 +- .../operators/maybe/MaybeToFlowable.java | 2 +- .../operators/maybe/MaybeToObservable.java | 2 +- .../operators/maybe/MaybeToPublisher.java | 4 +- .../operators/maybe/MaybeToSingle.java | 2 +- .../operators/maybe/MaybeUnsubscribeOn.java | 2 +- .../operators/maybe/MaybeZipArray.java | 6 +- .../operators/maybe/MaybeZipIterable.java | 4 +- .../mixed/CompletableAndThenObservable.java | 2 +- .../mixed/FlowableConcatMapCompletable.java | 4 +- .../mixed/FlowableConcatMapMaybe.java | 6 +- .../mixed/FlowableConcatMapSingle.java | 6 +- .../mixed/FlowableSwitchMapCompletable.java | 4 +- .../mixed/FlowableSwitchMapMaybe.java | 8 +- .../mixed/FlowableSwitchMapSingle.java | 8 +- .../mixed/MaterializeSingleObserver.java | 6 +- .../mixed/MaybeFlatMapObservable.java | 2 +- .../mixed/MaybeFlatMapPublisher.java | 2 +- .../mixed/ObservableConcatMapCompletable.java | 4 +- .../mixed/ObservableConcatMapMaybe.java | 6 +- .../mixed/ObservableConcatMapSingle.java | 6 +- .../mixed/ObservableSwitchMapCompletable.java | 4 +- .../mixed/ObservableSwitchMapMaybe.java | 8 +- .../mixed/ObservableSwitchMapSingle.java | 8 +- .../mixed/SingleFlatMapObservable.java | 2 +- .../BlockingObservableIterable.java | 4 +- .../observable/BlockingObservableLatest.java | 4 +- .../BlockingObservableMostRecent.java | 2 +- .../observable/BlockingObservableNext.java | 8 +- .../operators/observable/ObservableAll.java | 2 +- .../observable/ObservableAllSingle.java | 4 +- .../operators/observable/ObservableAmb.java | 4 +- .../operators/observable/ObservableAny.java | 2 +- .../observable/ObservableAnySingle.java | 4 +- .../ObservableBlockingSubscribe.java | 6 +- .../observable/ObservableBuffer.java | 6 +- .../observable/ObservableBufferBoundary.java | 12 +- .../ObservableBufferExactBoundary.java | 6 +- .../observable/ObservableBufferTimed.java | 24 +-- .../operators/observable/ObservableCache.java | 8 +- .../observable/ObservableCollect.java | 2 +- .../observable/ObservableCollectSingle.java | 4 +- .../observable/ObservableCombineLatest.java | 6 +- .../observable/ObservableConcatMap.java | 14 +- .../observable/ObservableConcatMapEager.java | 8 +- .../ObservableConcatMapScheduler.java | 14 +- .../ObservableConcatWithCompletable.java | 2 +- .../observable/ObservableConcatWithMaybe.java | 2 +- .../ObservableConcatWithSingle.java | 2 +- .../observable/ObservableCountSingle.java | 2 +- .../observable/ObservableCreate.java | 6 +- .../observable/ObservableDebounce.java | 6 +- .../observable/ObservableDebounceTimed.java | 6 +- .../operators/observable/ObservableDelay.java | 4 +- .../observable/ObservableDematerialize.java | 2 +- .../observable/ObservableDetach.java | 2 +- .../observable/ObservableDistinct.java | 2 +- .../ObservableDistinctUntilChanged.java | 2 +- .../observable/ObservableDoAfterNext.java | 2 +- .../observable/ObservableDoFinally.java | 2 +- .../observable/ObservableDoOnEach.java | 2 +- .../observable/ObservableDoOnLifecycle.java | 2 +- .../observable/ObservableElementAt.java | 2 +- .../observable/ObservableElementAtMaybe.java | 4 +- .../observable/ObservableElementAtSingle.java | 4 +- .../observable/ObservableFilter.java | 2 +- .../observable/ObservableFlatMap.java | 14 +- .../ObservableFlatMapCompletable.java | 4 +- ...servableFlatMapCompletableCompletable.java | 4 +- .../observable/ObservableFlatMapMaybe.java | 6 +- .../observable/ObservableFlatMapSingle.java | 6 +- .../observable/ObservableFlattenIterable.java | 2 +- .../observable/ObservableFromArray.java | 2 +- .../observable/ObservableFromCallable.java | 2 +- .../observable/ObservableFromFuture.java | 2 +- .../observable/ObservableFromIterable.java | 2 +- .../observable/ObservableFromSupplier.java | 2 +- .../observable/ObservableGenerate.java | 2 +- .../observable/ObservableGroupBy.java | 16 +- .../observable/ObservableGroupJoin.java | 10 +- .../operators/observable/ObservableHide.java | 2 +- .../observable/ObservableIgnoreElements.java | 2 +- .../ObservableIgnoreElementsCompletable.java | 4 +- .../observable/ObservableInternalHelper.java | 38 ++--- .../operators/observable/ObservableJoin.java | 10 +- .../operators/observable/ObservableJust.java | 2 +- .../observable/ObservableLastMaybe.java | 2 +- .../observable/ObservableLastSingle.java | 2 +- .../observable/ObservableMapNotification.java | 2 +- .../observable/ObservableMaterialize.java | 2 +- .../ObservableMergeWithCompletable.java | 4 +- .../observable/ObservableMergeWithMaybe.java | 8 +- .../observable/ObservableMergeWithSingle.java | 8 +- .../observable/ObservableObserveOn.java | 4 +- .../observable/ObservableOnErrorNext.java | 2 +- .../observable/ObservableOnErrorReturn.java | 2 +- .../observable/ObservablePublish.java | 10 +- .../observable/ObservablePublishSelector.java | 8 +- .../operators/observable/ObservableRange.java | 2 +- .../observable/ObservableRangeLong.java | 2 +- .../observable/ObservableReduceMaybe.java | 2 +- .../ObservableReduceSeedSingle.java | 2 +- .../ObservableReduceWithSingle.java | 2 +- .../observable/ObservableRefCount.java | 4 +- .../observable/ObservableRepeat.java | 2 +- .../observable/ObservableRepeatUntil.java | 2 +- .../observable/ObservableRepeatWhen.java | 4 +- .../observable/ObservableReplay.java | 32 ++-- .../ObservableRetryBiPredicate.java | 2 +- .../observable/ObservableRetryPredicate.java | 2 +- .../observable/ObservableRetryWhen.java | 4 +- .../observable/ObservableSampleTimed.java | 8 +- .../ObservableSampleWithObservable.java | 10 +- .../observable/ObservableScalarXMap.java | 8 +- .../operators/observable/ObservableScan.java | 2 +- .../observable/ObservableScanSeed.java | 2 +- .../observable/ObservableSequenceEqual.java | 8 +- .../ObservableSequenceEqualSingle.java | 10 +- .../observable/ObservableSerialized.java | 2 +- .../observable/ObservableSingleMaybe.java | 2 +- .../observable/ObservableSingleSingle.java | 2 +- .../operators/observable/ObservableSkip.java | 2 +- .../observable/ObservableSkipLast.java | 2 +- .../observable/ObservableSkipLastTimed.java | 4 +- .../observable/ObservableSkipUntil.java | 4 +- .../observable/ObservableSkipWhile.java | 2 +- .../observable/ObservableSubscribeOn.java | 4 +- .../observable/ObservableSwitchIfEmpty.java | 2 +- .../observable/ObservableSwitchMap.java | 10 +- .../operators/observable/ObservableTake.java | 2 +- .../observable/ObservableTakeLast.java | 2 +- .../observable/ObservableTakeLastOne.java | 2 +- .../observable/ObservableTakeLastTimed.java | 4 +- .../observable/ObservableTakeUntil.java | 4 +- .../ObservableTakeUntilPredicate.java | 2 +- .../observable/ObservableTakeWhile.java | 2 +- .../ObservableThrottleFirstTimed.java | 4 +- .../observable/ObservableThrottleLatest.java | 4 +- .../observable/ObservableTimeInterval.java | 4 +- .../observable/ObservableTimeout.java | 8 +- .../observable/ObservableTimeoutTimed.java | 8 +- .../observable/ObservableToList.java | 2 +- .../observable/ObservableToListSingle.java | 4 +- .../observable/ObservableUnsubscribeOn.java | 2 +- .../operators/observable/ObservableUsing.java | 2 +- .../observable/ObservableWindow.java | 10 +- .../observable/ObservableWindowBoundary.java | 10 +- .../ObservableWindowBoundarySelector.java | 14 +- .../observable/ObservableWindowTimed.java | 22 +-- .../observable/ObservableWithLatestFrom.java | 8 +- .../ObservableWithLatestFromMany.java | 8 +- .../operators/observable/ObservableZip.java | 8 +- .../observable/ObserverResourceWrapper.java | 2 +- .../operators/single/SingleCache.java | 4 +- .../operators/single/SingleCreate.java | 2 +- .../single/SingleDelayWithCompletable.java | 4 +- .../single/SingleDelayWithObservable.java | 4 +- .../single/SingleDelayWithPublisher.java | 4 +- .../single/SingleDelayWithSingle.java | 4 +- .../operators/single/SingleDematerialize.java | 2 +- .../operators/single/SingleDetach.java | 2 +- .../single/SingleDoAfterSuccess.java | 2 +- .../single/SingleDoAfterTerminate.java | 2 +- .../operators/single/SingleDoFinally.java | 2 +- .../operators/single/SingleDoOnDispose.java | 2 +- .../operators/single/SingleDoOnSubscribe.java | 2 +- .../single/SingleFlatMapCompletable.java | 2 +- .../single/SingleFlatMapIterableFlowable.java | 4 +- .../SingleFlatMapIterableObservable.java | 4 +- .../single/SingleFlatMapPublisher.java | 4 +- .../single/SingleInternalHelper.java | 12 +- .../operators/single/SingleMaterialize.java | 2 +- .../operators/single/SingleObserveOn.java | 2 +- .../operators/single/SingleResumeNext.java | 2 +- .../operators/single/SingleSubscribeOn.java | 2 +- .../operators/single/SingleTakeUntil.java | 2 +- .../operators/single/SingleTimeout.java | 6 +- .../operators/single/SingleToObservable.java | 2 +- .../operators/single/SingleUnsubscribeOn.java | 2 +- .../operators/single/SingleZipArray.java | 6 +- .../operators/single/SingleZipIterable.java | 4 +- .../internal/queue/MpscLinkedQueue.java | 8 +- .../internal/queue/SpscLinkedArrayQueue.java | 8 +- .../schedulers/AbstractDirectTask.java | 4 +- .../schedulers/ComputationScheduler.java | 2 +- .../internal/schedulers/DisposeOnCancel.java | 6 +- .../schedulers/ExecutorScheduler.java | 2 +- .../schedulers/InstantPeriodicTask.java | 8 +- .../internal/schedulers/IoScheduler.java | 4 +- .../internal/schedulers/RxThreadFactory.java | 4 +- .../schedulers/ScheduledDirectTask.java | 2 +- .../schedulers/SchedulerPoolFactory.java | 8 +- .../internal/schedulers/SingleScheduler.java | 2 +- .../schedulers/TrampolineScheduler.java | 2 +- .../subscribers/FutureSubscriber.java | 5 +- .../subscribers/StrictSubscriber.java | 2 +- .../SubscriberResourceWrapper.java | 2 +- .../subscriptions/AsyncSubscription.java | 4 +- .../subscriptions/SubscriptionArbiter.java | 2 +- .../internal/util/ArrayListSupplier.java | 4 +- .../internal/util/ConnectConsumer.java | 2 +- .../internal/util/ExceptionHelper.java | 8 +- .../internal/util/HashMapSupplier.java | 2 +- .../internal/util/LinkedArrayList.java | 2 +- .../internal/util/ListAddBiConsumer.java | 2 +- .../internal/util/MergerBiFunction.java | 6 +- .../internal/util/QueueDrainHelper.java | 4 +- .../rxjava3/internal/util/SorterFunction.java | 2 +- .../internal/util/VolatileSizeArrayList.java | 18 ++- .../rxjava3/parallel/ParallelFlowable.java | 21 ++- .../processors/MulticastProcessor.java | 2 +- .../rxjava3/processors/ReplayProcessor.java | 14 +- .../rxjava3/schedulers/Schedulers.java | 20 +-- .../rxjava3/subjects/ReplaySubject.java | 12 +- .../rxjava3/subjects/UnicastSubject.java | 8 +- 467 files changed, 1513 insertions(+), 1600 deletions(-) diff --git a/docs/Backpressure.md b/docs/Backpressure.md index 8feec0d487..bfe90330bb 100644 --- a/docs/Backpressure.md +++ b/docs/Backpressure.md @@ -34,25 +34,25 @@ The following diagrams show how you could use each of these operators on the bur The `sample` operator periodically "dips" into the sequence and emits only the most recently emitted item during each dip: ​ -````groovy +```java Observable burstySampled = bursty.sample(500, TimeUnit.MILLISECONDS); -```` +``` ### throttleFirst The `throttleFirst` operator is similar, but emits not the most recently emitted item, but the first item that was emitted after the previous "dip": ​ -````groovy +```java Observable burstyThrottled = bursty.throttleFirst(500, TimeUnit.MILLISECONDS); -```` +``` ### debounce (or throttleWithTimeout) The `debounce` operator emits only those items from the source Observable that are not followed by another item within a specified duration: ​ -````groovy +```java Observable burstyDebounced = bursty.debounce(10, TimeUnit.MILLISECONDS); -```` +``` ## Buffers and windows @@ -65,14 +65,14 @@ The following diagrams show how you could use each of these operators on the bur You could, for example, close and emit a buffer of items from the bursty Observable periodically, at a regular interval of time: ​ -````groovy +```java Observable> burstyBuffered = bursty.buffer(500, TimeUnit.MILLISECONDS); -```` +``` Or you could get fancy, and collect items in buffers during the bursty periods and emit them at the end of each burst, by using the `debounce` operator to emit a buffer closing indicator to the `buffer` operator: ​ -````groovy +```java // we have to multicast the original bursty Observable so we can use it // both as our source and as the source for our buffer closing selector: Observable burstyMulticast = bursty.publish().refCount(); @@ -87,16 +87,16 @@ Observable> burstyBuffered = burstyMulticast.buffer(burstyDebounce `window` is similar to `buffer`. One variant of `window` allows you to periodically emit Observable windows of items at a regular interval of time: ​ -````groovy +```java Observable> burstyWindowed = bursty.window(500, TimeUnit.MILLISECONDS); ```` You could also choose to emit a new window each time you have collected a particular number of items from the source Observable: ​ -````groovy +```java Observable> burstyWindowed = bursty.window(5); -```` +``` # Callstack blocking as a flow-control alternative to backpressure @@ -110,8 +110,8 @@ When you subscribe to an `Observable` with a `Subscriber`, you can request react Then, after handling this item (or these items) in `onNext()`, you can call `request()` again to instruct the `Observable` to emit another item (or items). Here is an example of a `Subscriber` that requests one item at a time from `someObservable`: -````java -someObservable.subscribe(new Subscriber() { +```java +someObservable.subscribe(new Subscriber() { @Override public void onStart() { request(1); @@ -134,7 +134,7 @@ someObservable.subscribe(new Subscriber() { request(1); } }); -```` +``` You can pass a magic number to `request`, `request(Long.MAX_VALUE)`, to disable reactive pull backpressure and to ask the Observable to emit items at its own pace. `request(0)` is a legal call, but has no effect. Passing values less than zero to `request` will cause an exception to be thrown. diff --git a/docs/Combining-Observables.md b/docs/Combining-Observables.md index bcc19f6b0f..fcde91b76b 100644 --- a/docs/Combining-Observables.md +++ b/docs/Combining-Observables.md @@ -2,13 +2,13 @@ This section explains operators you can use to combine multiple Observables. # Outline -- [`combineLatest`](#combineLatest) +- [`combineLatest`](#combinelatest) - [`join` and `groupJoin`](#joins) - [`merge`](#merge) -- [`mergeDelayError`](#mergeDelayError) +- [`mergeDelayError`](#mergedelayerror) - [`rxjava-joins`](#rxjava-joins) -- [`startWith`](#startWith) -- [`switchOnNext`](#switchOnNext) +- [`startWith`](#startwith) +- [`switchOnNext`](#switchonnext) - [`zip`](#zip) ## startWith diff --git a/docs/How-To-Use-RxJava.md b/docs/How-To-Use-RxJava.md index 16d156caa9..ad309496b9 100644 --- a/docs/How-To-Use-RxJava.md +++ b/docs/How-To-Use-RxJava.md @@ -96,9 +96,9 @@ You use the Observable [`just( )`](http://reactivex.io/documentation/operators Observable o = Observable.from("a", "b", "c"); def list = [5, 6, 7, 8] -Observable o = Observable.from(list); +Observable o2 = Observable.from(list); -Observable o = Observable.just("one object"); +Observable o3 = Observable.just("one object"); ``` These converted Observables will synchronously invoke the [`onNext( )`](Observable#onnext-oncompleted-and-onerror) method of any subscriber that subscribes to them, for each item to be emitted by the Observable, and will then invoke the subscriber’s [`onCompleted( )`](Observable#onnext-oncompleted-and-onerror) method. diff --git a/docs/Writing-operators-for-2.0.md b/docs/Writing-operators-for-2.0.md index 7b6e57666e..e8486564b1 100644 --- a/docs/Writing-operators-for-2.0.md +++ b/docs/Writing-operators-for-2.0.md @@ -1,7 +1,7 @@ ##### Table of contents - [Introduction](#introduction) - - [Warning on internal components](warning-on-internal-components) + - [Warning on internal components](#warning-on-internal-components) - [Atomics, serialization, deferred actions](#atomics-serialization-deferred-actions) - [Field updaters and Android](#field-updaters-and-android) - [Request accounting](#request-accounting) diff --git a/src/jmh/java/io/reactivex/rxjava3/core/BinaryFlatMapPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/BinaryFlatMapPerf.java index 6d5bb44b23..836957192e 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/BinaryFlatMapPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/BinaryFlatMapPerf.java @@ -86,48 +86,42 @@ public void setup() { singleFlatMapPublisher = Single.just(1).flatMapPublisher(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return arrayFlowable; } }); singleFlatMapHidePublisher = Single.just(1).flatMapPublisher(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return arrayFlowableHide; } }); singleFlattenAsPublisher = Single.just(1).flattenAsFlowable(new Function>() { @Override - public Iterable apply(Integer v) - throws Exception { + public Iterable apply(Integer v) { return list; } }); maybeFlatMapPublisher = Maybe.just(1).flatMapPublisher(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return arrayFlowable; } }); maybeFlatMapHidePublisher = Maybe.just(1).flatMapPublisher(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return arrayFlowableHide; } }); maybeFlattenAsPublisher = Maybe.just(1).flattenAsFlowable(new Function>() { @Override - public Iterable apply(Integer v) - throws Exception { + public Iterable apply(Integer v) { return list; } }); @@ -140,48 +134,42 @@ public Iterable apply(Integer v) singleFlatMapObservable = Single.just(1).flatMapObservable(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return arrayObservable; } }); singleFlatMapHideObservable = Single.just(1).flatMapObservable(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return arrayObservableHide; } }); singleFlattenAsObservable = Single.just(1).flattenAsObservable(new Function>() { @Override - public Iterable apply(Integer v) - throws Exception { + public Iterable apply(Integer v) { return list; } }); maybeFlatMapObservable = Maybe.just(1).flatMapObservable(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return arrayObservable; } }); maybeFlatMapHideObservable = Maybe.just(1).flatMapObservable(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return arrayObservableHide; } }); maybeFlattenAsObservable = Maybe.just(1).flattenAsObservable(new Function>() { @Override - public Iterable apply(Integer v) - throws Exception { + public Iterable apply(Integer v) { return list; } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/CallableAsyncPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/CallableAsyncPerf.java index 4afa54ecc8..8098b1aaf4 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/CallableAsyncPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/CallableAsyncPerf.java @@ -68,7 +68,7 @@ public void setup() { Callable c = new Callable() { @Override - public Integer call() throws Exception { + public Integer call() { return 1; } }; @@ -120,71 +120,71 @@ public void subscribeOnFlowable(Blackhole bh) { @Benchmark public void observeOnFlowable(Blackhole bh) { observeOnFlowable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineFlowable(Blackhole bh) { pipelineFlowable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnObservable(Blackhole bh) { subscribeOnObservable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnObservable(Blackhole bh) { observeOnObservable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineObservable(Blackhole bh) { pipelineObservable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnSingle(Blackhole bh) { observeOnSingle.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnSingle(Blackhole bh) { subscribeOnSingle.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineSingle(Blackhole bh) { pipelineSingle.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnCompletable(Blackhole bh) { observeOnCompletable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnCompletable(Blackhole bh) { subscribeOnCompletable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineCompletable(Blackhole bh) { pipelineCompletable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnMaybe(Blackhole bh) { observeOnMaybe.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnMaybe(Blackhole bh) { subscribeOnMaybe.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineMaybe(Blackhole bh) { pipelineMaybe.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } } diff --git a/src/jmh/java/io/reactivex/rxjava3/core/EachTypeFlatMapPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/EachTypeFlatMapPerf.java index e05921c531..37a4f196e8 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/EachTypeFlatMapPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/EachTypeFlatMapPerf.java @@ -114,11 +114,11 @@ public void nbpRangeMapRange(Blackhole bh) { @Benchmark public void singleJust(Blackhole bh) { - singleJust.subscribe(new LatchedSingleObserver(bh)); + singleJust.subscribe(new LatchedSingleObserver<>(bh)); } @Benchmark public void singleJustMapJust(Blackhole bh) { - singleJustMapJust.subscribe(new LatchedSingleObserver(bh)); + singleJustMapJust.subscribe(new LatchedSingleObserver<>(bh)); } } diff --git a/src/jmh/java/io/reactivex/rxjava3/core/FlatMapJustPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/FlatMapJustPerf.java index 8541f92834..2a1fd7539c 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/FlatMapJustPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/FlatMapJustPerf.java @@ -41,14 +41,14 @@ public void setup() { flowable = Flowable.fromArray(array).flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); observable = Observable.fromArray(array).flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/FlattenCrossMapPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/FlattenCrossMapPerf.java index cfc5a1af97..866fe5de77 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/FlattenCrossMapPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/FlattenCrossMapPerf.java @@ -47,14 +47,14 @@ public void setup() { flowable = Flowable.fromArray(array).flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return list; } }); observable = Observable.fromArray(array).flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return list; } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/FlattenJustPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/FlattenJustPerf.java index bb2394a053..d2ac9aa5b7 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/FlattenJustPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/FlattenJustPerf.java @@ -44,14 +44,14 @@ public void setup() { flowable = Flowable.fromArray(array).flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return singletonList; } }); observable = Observable.fromArray(array).flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return singletonList; } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/FlattenRangePerf.java b/src/jmh/java/io/reactivex/rxjava3/core/FlattenRangePerf.java index 663fd8cdde..a3195b760d 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/FlattenRangePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/FlattenRangePerf.java @@ -44,14 +44,14 @@ public void setup() { flowable = Flowable.fromArray(array).flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return list; } }); observable = Observable.fromArray(array).flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return list; } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/FlowableFlatMapCompletableAsyncPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/FlowableFlatMapCompletableAsyncPerf.java index 83f2d4c4f4..e9494e97df 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/FlowableFlatMapCompletableAsyncPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/FlowableFlatMapCompletableAsyncPerf.java @@ -45,7 +45,7 @@ public class FlowableFlatMapCompletableAsyncPerf implements Action { Flowable flatMap; @Override - public void run() throws Exception { + public void run() { Blackhole.consumeCPU(work); } diff --git a/src/jmh/java/io/reactivex/rxjava3/core/InputWithIncrementingInteger.java b/src/jmh/java/io/reactivex/rxjava3/core/InputWithIncrementingInteger.java index 24c4e016d3..772d202dca 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/InputWithIncrementingInteger.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/InputWithIncrementingInteger.java @@ -43,7 +43,7 @@ public void onNext(Integer t) { } } - final class IncrementingIterable implements Iterable { + static final class IncrementingIterable implements Iterable { final class IncrementingIterator implements Iterator { int i; @@ -77,7 +77,7 @@ public Iterator iterator() { } } - final class IncrementingPublisher implements Publisher { + static final class IncrementingPublisher implements Publisher { final int size; diff --git a/src/jmh/java/io/reactivex/rxjava3/core/JustAsyncPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/JustAsyncPerf.java index d22c1699e5..f60204c88b 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/JustAsyncPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/JustAsyncPerf.java @@ -113,71 +113,71 @@ public void subscribeOnFlowable(Blackhole bh) { @Benchmark public void observeOnFlowable(Blackhole bh) { observeOnFlowable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineFlowable(Blackhole bh) { pipelineFlowable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnObservable(Blackhole bh) { subscribeOnObservable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnObservable(Blackhole bh) { observeOnObservable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineObservable(Blackhole bh) { pipelineObservable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnSingle(Blackhole bh) { observeOnSingle.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnSingle(Blackhole bh) { subscribeOnSingle.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineSingle(Blackhole bh) { pipelineSingle.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnCompletable(Blackhole bh) { observeOnCompletable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnCompletable(Blackhole bh) { subscribeOnCompletable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineCompletable(Blackhole bh) { pipelineCompletable.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void observeOnMaybe(Blackhole bh) { observeOnMaybe.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void subscribeOnMaybe(Blackhole bh) { subscribeOnMaybe.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } @Benchmark public void pipelineMaybe(Blackhole bh) { pipelineMaybe.subscribeWith(new PerfAsyncConsumer(bh)).await(1); - }; + } } diff --git a/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java index b7ed7e4ae1..c7c407de15 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/MemoryPerf.java @@ -143,31 +143,31 @@ public static void main(String[] args) throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.just(1); } }, "just", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.range(1, 10); } }, "range", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.empty(); } }, "empty", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.fromCallable(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return 1; } }); @@ -176,38 +176,38 @@ public Object call() throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return new MyRx2Observer(); } }, "consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return new io.reactivex.rxjava3.observers.TestObserver<>(); } }, "test-consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.just(1).subscribeWith(new MyRx2Observer()); } }, "just+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.range(1, 10).subscribeWith(new MyRx2Observer()); } }, "range+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.range(1, 10).map(new Function() { @Override - public Object apply(Integer v) throws Exception { + public Object apply(Integer v) { return v; } }).subscribeWith(new MyRx2Observer()); @@ -216,15 +216,15 @@ public Object apply(Integer v) throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.range(1, 10).map(new Function() { @Override - public Object apply(Integer v) throws Exception { + public Object apply(Integer v) { return v; } }).filter(new Predicate() { @Override - public boolean test(Object v) throws Exception { + public boolean test(Object v) { return true; } }).subscribeWith(new MyRx2Observer()); @@ -233,91 +233,91 @@ public boolean test(Object v) throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.range(1, 10).subscribeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).subscribeWith(new MyRx2Observer()); } }, "range+subscribeOn+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.range(1, 10).observeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).subscribeWith(new MyRx2Observer()); } }, "range+observeOn+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Observable.range(1, 10).subscribeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).observeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).subscribeWith(new MyRx2Observer()); } }, "range+subscribeOn+observeOn+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.AsyncSubject.create(); } }, "Async", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.PublishSubject.create(); } }, "Publish", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.ReplaySubject.create(); } }, "Replay", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.BehaviorSubject.create(); } }, "Behavior", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.UnicastSubject.create(); } }, "Unicast", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.AsyncSubject.create().subscribeWith(new MyRx2Observer()); } }, "Async+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.PublishSubject.create().subscribeWith(new MyRx2Observer()); } }, "Publish+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.ReplaySubject.create().subscribeWith(new MyRx2Observer()); } }, "Replay+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.BehaviorSubject.create().subscribeWith(new MyRx2Observer()); } }, "Behavior+consumer", "Rx2Observable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.subjects.UnicastSubject.create().subscribeWith(new MyRx2Observer()); } }, "Unicast+consumer", "Rx2Observable"); @@ -326,38 +326,38 @@ public Object call() throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.just(1); } }, "just", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.range(1, 10); } }, "range", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.empty(); } }, "empty", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.empty(); } }, "empty", "Rx2Flowable", 10000000); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.fromCallable(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return 1; } }); @@ -366,38 +366,38 @@ public Object call() throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return new MyRx2Subscriber(); } }, "consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return new io.reactivex.rxjava3.observers.TestObserver<>(); } }, "test-consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.just(1).subscribeWith(new MyRx2Subscriber()); } }, "just+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.range(1, 10).subscribeWith(new MyRx2Subscriber()); } }, "range+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.range(1, 10).map(new Function() { @Override - public Object apply(Integer v) throws Exception { + public Object apply(Integer v) { return v; } }).subscribeWith(new MyRx2Subscriber()); @@ -406,15 +406,15 @@ public Object apply(Integer v) throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.range(1, 10).map(new Function() { @Override - public Object apply(Integer v) throws Exception { + public Object apply(Integer v) { return v; } }).filter(new Predicate() { @Override - public boolean test(Object v) throws Exception { + public boolean test(Object v) { return true; } }).subscribeWith(new MyRx2Subscriber()); @@ -423,91 +423,91 @@ public boolean test(Object v) throws Exception { checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.range(1, 10).subscribeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).subscribeWith(new MyRx2Subscriber()); } }, "range+subscribeOn+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.range(1, 10).observeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).subscribeWith(new MyRx2Subscriber()); } }, "range+observeOn+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.core.Flowable.range(1, 10).subscribeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).observeOn(io.reactivex.rxjava3.schedulers.Schedulers.computation()).subscribeWith(new MyRx2Subscriber()); } }, "range+subscribeOn+observeOn+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.AsyncProcessor.create(); } }, "Async", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.PublishProcessor.create(); } }, "Publish", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.ReplayProcessor.create(); } }, "Replay", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.BehaviorProcessor.create(); } }, "Behavior", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.UnicastProcessor.create(); } }, "Unicast", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.AsyncProcessor.create().subscribeWith(new MyRx2Subscriber()); } }, "Async+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.PublishProcessor.create().subscribeWith(new MyRx2Subscriber()); } }, "Publish+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.ReplayProcessor.create().subscribeWith(new MyRx2Subscriber()); } }, "Replay+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.BehaviorProcessor.create().subscribeWith(new MyRx2Subscriber()); } }, "Behavior+consumer", "Rx2Flowable"); checkMemory(new Callable() { @Override - public Object call() throws Exception { + public Object call() { return io.reactivex.rxjava3.processors.UnicastProcessor.create().subscribeWith(new MyRx2Subscriber()); } }, "Unicast+consumer", "Rx2Flowable"); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/OperatorFlatMapPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/OperatorFlatMapPerf.java index 621894bd74..0eb7f03aa9 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/OperatorFlatMapPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/OperatorFlatMapPerf.java @@ -40,7 +40,7 @@ public int getSize() { } @Benchmark - public void flatMapIntPassthruSync(Input input) throws InterruptedException { + public void flatMapIntPassthruSync(Input input) { input.flowable.flatMap(new Function>() { @Override public Publisher apply(Integer v) { @@ -66,7 +66,7 @@ public Publisher apply(Integer i) { } @Benchmark - public void flatMapTwoNestedSync(final Input input) throws InterruptedException { + public void flatMapTwoNestedSync(final Input input) { Flowable.range(1, 2).flatMap(new Function>() { @Override public Publisher apply(Integer i) { diff --git a/src/jmh/java/io/reactivex/rxjava3/core/PerfSubscriber.java b/src/jmh/java/io/reactivex/rxjava3/core/PerfSubscriber.java index e4c52b5040..51da5a5c53 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/PerfSubscriber.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/PerfSubscriber.java @@ -20,7 +20,7 @@ public class PerfSubscriber implements FlowableSubscriber { - public CountDownLatch latch = new CountDownLatch(1); + public final CountDownLatch latch = new CountDownLatch(1); private final Blackhole bh; public PerfSubscriber(Blackhole bh) { diff --git a/src/jmh/java/io/reactivex/rxjava3/core/ReducePerf.java b/src/jmh/java/io/reactivex/rxjava3/core/ReducePerf.java index 0282aa31e5..faf0423760 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/ReducePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/ReducePerf.java @@ -40,7 +40,7 @@ public class ReducePerf implements BiFunction { Maybe flowMaybe; @Override - public Integer apply(Integer t1, Integer t2) throws Exception { + public Integer apply(Integer t1, Integer t2) { return t1 + t2; } diff --git a/src/jmh/java/io/reactivex/rxjava3/core/TakeUntilPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/TakeUntilPerf.java index 71161b32d3..b2f89a39ca 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/TakeUntilPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/TakeUntilPerf.java @@ -39,7 +39,7 @@ public class TakeUntilPerf implements Consumer { Observable observable; @Override - public void accept(Integer t) throws Exception { + public void accept(Integer t) { items++; } @@ -48,7 +48,7 @@ public void setup() { flowable = Flowable.range(1, 1000 * 1000).takeUntil(Flowable.fromCallable(new Callable() { @Override - public Object call() throws Exception { + public Object call() { int c = count; while (items < c) { } return 1; @@ -57,7 +57,7 @@ public Object call() throws Exception { observable = Observable.range(1, 1000 * 1000).takeUntil(Observable.fromCallable(new Callable() { @Override - public Object call() throws Exception { + public Object call() { int c = count; while (items < c) { } return 1; @@ -71,7 +71,7 @@ public void flowable() { flowable.subscribe(this, Functions.emptyConsumer(), new Action() { @Override - public void run() throws Exception { + public void run() { cdl.countDown(); } }); @@ -85,7 +85,7 @@ public void observable() { observable.subscribe(this, Functions.emptyConsumer(), new Action() { @Override - public void run() throws Exception { + public void run() { cdl.countDown(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/ToFlowablePerf.java b/src/jmh/java/io/reactivex/rxjava3/core/ToFlowablePerf.java index 28b1d050fc..9c86e83678 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/ToFlowablePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/ToFlowablePerf.java @@ -48,7 +48,7 @@ public void setup() { final BiFunction second = new BiFunction() { @Override - public Integer apply(Integer a, Integer b) throws Exception { + public Integer apply(Integer a, Integer b) { return b; } }; @@ -57,7 +57,7 @@ public Integer apply(Integer a, Integer b) throws Exception { flowableInner = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Flowable.range(1, 50).reduce(second).toFlowable(); } }); @@ -68,7 +68,7 @@ public Publisher apply(Integer v) throws Exception { observableInner = sourceObs.concatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Observable.range(1, 50).reduce(second).toObservable(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/core/XMapYPerf.java b/src/jmh/java/io/reactivex/rxjava3/core/XMapYPerf.java index 582d9da8dc..6205efa25d 100644 --- a/src/jmh/java/io/reactivex/rxjava3/core/XMapYPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/core/XMapYPerf.java @@ -103,84 +103,84 @@ public void setup() { flowFlatMapFlowable1 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); flowFlatMapFlowable0 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Flowable.empty(); } }); flowFlatMapSingle1 = fsource.flatMapSingle(new Function>() { @Override - public SingleSource apply(Integer v) throws Exception { + public SingleSource apply(Integer v) { return Single.just(v); } }); flowFlatMapMaybe1 = fsource.flatMapMaybe(new Function>() { @Override - public MaybeSource apply(Integer v) throws Exception { + public MaybeSource apply(Integer v) { return Maybe.just(v); } }); flowFlatMapMaybe0 = fsource.flatMapMaybe(new Function>() { @Override - public MaybeSource apply(Integer v) throws Exception { + public MaybeSource apply(Integer v) { return Maybe.empty(); } }); flowFlatMapCompletable0 = fsource.flatMapCompletable(new Function() { @Override - public CompletableSource apply(Integer v) throws Exception { + public CompletableSource apply(Integer v) { return Completable.complete(); } }); flowFlatMapIterable1 = fsource.flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return Collections.singletonList(v); } }); flowFlatMapIterable0 = fsource.flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { - return Collections.emptyList(); + public Iterable apply(Integer v) { + return Collections.emptyList(); } }); flowFlatMapSingle1 = fsource.flatMapSingle(new Function>() { @Override - public SingleSource apply(Integer v) throws Exception { + public SingleSource apply(Integer v) { return Single.just(v); } }); flowFlatMapMaybe1 = fsource.flatMapMaybe(new Function>() { @Override - public MaybeSource apply(Integer v) throws Exception { + public MaybeSource apply(Integer v) { return Maybe.just(v); } }); flowFlatMapMaybe0 = fsource.flatMapMaybe(new Function>() { @Override - public MaybeSource apply(Integer v) throws Exception { + public MaybeSource apply(Integer v) { return Maybe.empty(); } }); flowFlatMapCompletable0 = fsource.flatMapCompletable(new Function() { @Override - public CompletableSource apply(Integer v) throws Exception { + public CompletableSource apply(Integer v) { return Completable.complete(); } }); @@ -189,43 +189,43 @@ public CompletableSource apply(Integer v) throws Exception { flowFlatMapSingleAsFlow1 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Single.just(v).toFlowable(); } }); flowFlatMapMaybeAsFlow1 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Maybe.just(v).toFlowable(); } }); flowFlatMapMaybeAsFlow0 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Maybe.empty().toFlowable(); } }); flowFlatMapCompletableAsFlow0 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { - return Completable.complete().toFlowable(); + public Publisher apply(Integer v) { + return Completable.complete().toFlowable(); } }); flowFlatMapIterableAsFlow1 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Flowable.fromIterable(Collections.singletonList(v)); } }); flowFlatMapIterableAsFlow0 = fsource.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { - return Flowable.fromIterable(Collections.emptyList()); + public Publisher apply(Integer v) { + return Flowable.fromIterable(Collections.emptyList()); } }); @@ -235,57 +235,57 @@ public Publisher apply(Integer v) throws Exception { obsFlatMapObservable1 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); obsFlatMapObservable0 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Observable.empty(); } }); obsFlatMapSingle1 = osource.flatMapSingle(new Function>() { @Override - public SingleSource apply(Integer v) throws Exception { + public SingleSource apply(Integer v) { return Single.just(v); } }); obsFlatMapMaybe1 = osource.flatMapMaybe(new Function>() { @Override - public MaybeSource apply(Integer v) throws Exception { + public MaybeSource apply(Integer v) { return Maybe.just(v); } }); obsFlatMapMaybe0 = osource.flatMapMaybe(new Function>() { @Override - public MaybeSource apply(Integer v) throws Exception { + public MaybeSource apply(Integer v) { return Maybe.empty(); } }); obsFlatMapCompletable0 = osource.flatMapCompletable(new Function() { @Override - public CompletableSource apply(Integer v) throws Exception { + public CompletableSource apply(Integer v) { return Completable.complete(); } }); obsFlatMapIterable1 = osource.flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { + public Iterable apply(Integer v) { return Collections.singletonList(v); } }); obsFlatMapIterable0 = osource.flatMapIterable(new Function>() { @Override - public Iterable apply(Integer v) throws Exception { - return Collections.emptyList(); + public Iterable apply(Integer v) { + return Collections.emptyList(); } }); @@ -293,43 +293,43 @@ public Iterable apply(Integer v) throws Exception { obsFlatMapSingleAsObs1 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Single.just(v).toObservable(); } }); obsFlatMapMaybeAsObs1 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Maybe.just(v).toObservable(); } }); obsFlatMapMaybeAsObs0 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Maybe.empty().toObservable(); } }); obsFlatMapCompletableAsObs0 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { - return Completable.complete().toObservable(); + public Observable apply(Integer v) { + return Completable.complete().toObservable(); } }); obsFlatMapIterableAsObs1 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { + public Observable apply(Integer v) { return Observable.fromIterable(Collections.singletonList(v)); } }); obsFlatMapIterableAsObs0 = osource.flatMap(new Function>() { @Override - public Observable apply(Integer v) throws Exception { - return Observable.fromIterable(Collections.emptyList()); + public Observable apply(Integer v) { + return Observable.fromIterable(Collections.emptyList()); } }); } diff --git a/src/jmh/java/io/reactivex/rxjava3/parallel/ParallelPerf.java b/src/jmh/java/io/reactivex/rxjava3/parallel/ParallelPerf.java index 31d0ecc8c5..4eaac7b0da 100644 --- a/src/jmh/java/io/reactivex/rxjava3/parallel/ParallelPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/parallel/ParallelPerf.java @@ -49,7 +49,7 @@ public class ParallelPerf implements Function { Flowable parallel; @Override - public Integer apply(Integer t) throws Exception { + public Integer apply(Integer t) { Blackhole.consumeCPU(compute); return t; } @@ -66,7 +66,7 @@ public void setup() { flatMap = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v).subscribeOn(Schedulers.computation()) .map(ParallelPerf.this); } @@ -75,13 +75,13 @@ public Publisher apply(Integer v) throws Exception { groupBy = source.groupBy(new Function() { int i; @Override - public Integer apply(Integer v) throws Exception { + public Integer apply(Integer v) { return (i++) % cpu; } }) .flatMap(new Function, Publisher>() { @Override - public Publisher apply(GroupedFlowable g) throws Exception { + public Publisher apply(GroupedFlowable g) { return g.observeOn(Schedulers.computation()).map(ParallelPerf.this); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapCompletablePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapCompletablePerf.java index cb58173206..9e1096ab74 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapCompletablePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapCompletablePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.empty(); } }); flowableConvert = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Completable.complete().toFlowable(); } }); flowableDedicated = source.concatMapCompletable(new Function() { @Override - public Completable apply(Integer v) - throws Exception { + public Completable apply(Integer v) { return Completable.complete(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybeEmptyPerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybeEmptyPerf.java index 1aff189e56..a211fbe829 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybeEmptyPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybeEmptyPerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.empty(); } }); concatMapToFlowableEmpty = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Maybe.empty().toFlowable(); } }); flowableDedicated = source.concatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.empty(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybePerf.java index 38ca515171..059a0b162b 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapMaybePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); flowableConvert = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Maybe.just(v).toFlowable(); } }); flowableDedicated = source.concatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapSinglePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapSinglePerf.java index 04148d7a47..e812708a15 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapSinglePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableConcatMapSinglePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); flowableConvert = source.concatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Single.just(v).toFlowable(); } }); flowableDedicated = source.concatMapSingle(new Function>() { @Override - public Single apply(Integer v) - throws Exception { + public Single apply(Integer v) { return Single.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapCompletablePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapCompletablePerf.java index e605629a8d..1960735cfa 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapCompletablePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapCompletablePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.empty(); } }); flowableConvert = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Completable.complete().toFlowable(); } }); flowableDedicated = source.flatMapCompletable(new Function() { @Override - public Completable apply(Integer v) - throws Exception { + public Completable apply(Integer v) { return Completable.complete(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybeEmptyPerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybeEmptyPerf.java index 23bd80fe6b..96e6fcf672 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybeEmptyPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybeEmptyPerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.empty(); } }); flowableConvert = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Maybe.empty().toFlowable(); } }); flowableDedicated = source.flatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.empty(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybePerf.java index bbbc96fc03..2f71d5793e 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapMaybePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); flowableConvert = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Maybe.just(v).toFlowable(); } }); flowableDedicated = source.flatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapSinglePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapSinglePerf.java index 8407a60e6a..2bbdf9bf95 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapSinglePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableFlatMapSinglePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); flowableConvert = source.flatMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Single.just(v).toFlowable(); } }); flowableDedicated = source.flatMapSingle(new Function>() { @Override - public Single apply(Integer v) - throws Exception { + public Single apply(Integer v) { return Single.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapCompletablePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapCompletablePerf.java index 25b4e34529..033d35c429 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapCompletablePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapCompletablePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.empty(); } }); flowableConvert = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Completable.complete().toFlowable(); } }); flowableDedicated = source.switchMapCompletable(new Function() { @Override - public Completable apply(Integer v) - throws Exception { + public Completable apply(Integer v) { return Completable.complete(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybeEmptyPerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybeEmptyPerf.java index ff6d5999ba..adc6060189 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybeEmptyPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybeEmptyPerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.empty(); } }); flowableConvert = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Maybe.empty().toFlowable(); } }); flowableDedicated = source.switchMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.empty(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybePerf.java index 87071d1108..d5d8cb9443 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapMaybePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); flowableConvert = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Maybe.just(v).toFlowable(); } }); flowableDedicated = source.switchMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapSinglePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapSinglePerf.java index a633856881..a997f27b53 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapSinglePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/FlowableSwitchMapSinglePerf.java @@ -48,24 +48,21 @@ public void setup() { flowablePlain = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Flowable.just(v); } }); flowableConvert = source.switchMap(new Function>() { @Override - public Publisher apply(Integer v) - throws Exception { + public Publisher apply(Integer v) { return Single.just(v).toFlowable(); } }); flowableDedicated = source.switchMapSingle(new Function>() { @Override - public Single apply(Integer v) - throws Exception { + public Single apply(Integer v) { return Single.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapCompletablePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapCompletablePerf.java index 5b2b9247fc..ca6174a006 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapCompletablePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapCompletablePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.empty(); } }); observableConvert = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Completable.complete().toObservable(); } }); observableDedicated = source.concatMapCompletable(new Function() { @Override - public Completable apply(Integer v) - throws Exception { + public Completable apply(Integer v) { return Completable.complete(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybeEmptyPerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybeEmptyPerf.java index 0ffbfcd321..b194580fa8 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybeEmptyPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybeEmptyPerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.empty(); } }); concatMapToObservableEmpty = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Maybe.empty().toObservable(); } }); observableDedicated = source.concatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.empty(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybePerf.java index 2ec317234b..5c7d6ad7a0 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapMaybePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); observableConvert = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Maybe.just(v).toObservable(); } }); observableDedicated = source.concatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapSinglePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapSinglePerf.java index e67138c816..1785b59135 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapSinglePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableConcatMapSinglePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); observableConvert = source.concatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Single.just(v).toObservable(); } }); observableDedicated = source.concatMapSingle(new Function>() { @Override - public Single apply(Integer v) - throws Exception { + public Single apply(Integer v) { return Single.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapCompletablePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapCompletablePerf.java index 6fc1ed9b07..64247ac219 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapCompletablePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapCompletablePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.empty(); } }); observableConvert = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Completable.complete().toObservable(); } }); observableDedicated = source.flatMapCompletable(new Function() { @Override - public Completable apply(Integer v) - throws Exception { + public Completable apply(Integer v) { return Completable.complete(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybeEmptyPerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybeEmptyPerf.java index d1b8dea70a..5e38b30061 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybeEmptyPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybeEmptyPerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.empty(); } }); observableConvert = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Maybe.empty().toObservable(); } }); observableDedicated = source.flatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.empty(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybePerf.java index 37bf095ce5..ffc5b15e56 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapMaybePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); observableConvert = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Maybe.just(v).toObservable(); } }); observableDedicated = source.flatMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapSinglePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapSinglePerf.java index 6139e6e080..6356ad8ace 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapSinglePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableFlatMapSinglePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); observableConvert = source.flatMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Single.just(v).toObservable(); } }); observableDedicated = source.flatMapSingle(new Function>() { @Override - public Single apply(Integer v) - throws Exception { + public Single apply(Integer v) { return Single.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapCompletablePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapCompletablePerf.java index 7f91a83adf..1fc6eef9b3 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapCompletablePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapCompletablePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.empty(); } }); observableConvert = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Completable.complete().toObservable(); } }); observableDedicated = source.switchMapCompletable(new Function() { @Override - public Completable apply(Integer v) - throws Exception { + public Completable apply(Integer v) { return Completable.complete(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybeEmptyPerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybeEmptyPerf.java index 967a244b07..4737493420 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybeEmptyPerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybeEmptyPerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.empty(); } }); observableConvert = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Maybe.empty().toObservable(); } }); observableDedicated = source.switchMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.empty(); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybePerf.java index 123e88b921..b8dca2d95b 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapMaybePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); observableConvert = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Maybe.just(v).toObservable(); } }); observableDedicated = source.switchMapMaybe(new Function>() { @Override - public Maybe apply(Integer v) - throws Exception { + public Maybe apply(Integer v) { return Maybe.just(v); } }); diff --git a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapSinglePerf.java b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapSinglePerf.java index abe4ea45b3..682e2712f0 100644 --- a/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapSinglePerf.java +++ b/src/jmh/java/io/reactivex/rxjava3/xmapz/ObservableSwitchMapSinglePerf.java @@ -47,24 +47,21 @@ public void setup() { observablePlain = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Observable.just(v); } }); observableConvert = source.switchMap(new Function>() { @Override - public Observable apply(Integer v) - throws Exception { + public Observable apply(Integer v) { return Single.just(v).toObservable(); } }); observableDedicated = source.switchMapSingle(new Function>() { @Override - public Single apply(Integer v) - throws Exception { + public Single apply(Integer v) { return Single.just(v); } }); diff --git a/src/main/java/io/reactivex/rxjava3/core/Completable.java b/src/main/java/io/reactivex/rxjava3/core/Completable.java index 00591742e4..9ce0ee1df8 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Completable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Completable.java @@ -1844,7 +1844,7 @@ public final Completable lift(@NonNull CompletableOperator onLift) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Single> materialize() { - return RxJavaPlugins.onAssembly(new CompletableMaterialize(this)); + return RxJavaPlugins.onAssembly(new CompletableMaterialize<>(this)); } /** @@ -2239,7 +2239,7 @@ public final Completable startWith(@NonNull CompletableSource other) { @SchedulerSupport(SchedulerSupport.NONE) public final Observable startWith(@NonNull ObservableSource other) { Objects.requireNonNull(other, "other is null"); - return Observable.wrap(other).concatWith(this.toObservable()); + return Observable.wrap(other).concatWith(this.toObservable()); } /** * Returns a Flowable which first delivers the events @@ -2637,7 +2637,7 @@ public final Flowable toFlowable() { if (this instanceof FuseToFlowable) { return ((FuseToFlowable)this).fuseToFlowable(); } - return RxJavaPlugins.onAssembly(new CompletableToFlowable(this)); + return RxJavaPlugins.onAssembly(new CompletableToFlowable<>(this)); } /** @@ -2661,7 +2661,7 @@ public final Maybe toMaybe() { if (this instanceof FuseToMaybe) { return ((FuseToMaybe)this).fuseToMaybe(); } - return RxJavaPlugins.onAssembly(new MaybeFromCompletable(this)); + return RxJavaPlugins.onAssembly(new MaybeFromCompletable<>(this)); } /** @@ -2684,7 +2684,7 @@ public final Observable toObservable() { if (this instanceof FuseToObservable) { return ((FuseToObservable)this).fuseToObservable(); } - return RxJavaPlugins.onAssembly(new CompletableToObservable(this)); + return RxJavaPlugins.onAssembly(new CompletableToObservable<>(this)); } /** @@ -2706,7 +2706,7 @@ public final Observable toObservable() { @SchedulerSupport(SchedulerSupport.NONE) public final <@NonNull T> Single toSingle(@NonNull Supplier completionValueSupplier) { Objects.requireNonNull(completionValueSupplier, "completionValueSupplier is null"); - return RxJavaPlugins.onAssembly(new CompletableToSingle(this, completionValueSupplier, null)); + return RxJavaPlugins.onAssembly(new CompletableToSingle<>(this, completionValueSupplier, null)); } /** @@ -2848,7 +2848,7 @@ public static Completable fromCompletionStage(@NonNull CompletionStage stage) * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. *

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).toCompletionStage(Optional.empty());
diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java
index df3a8a270a..97785c1f4c 100644
--- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java
+++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java
@@ -332,7 +332,7 @@ public static  Flowable combineLatestArray(@NonNull Publisher(sources, combiner, bufferSize, false));
+        return RxJavaPlugins.onAssembly(new FlowableCombineLatest<>(sources, combiner, bufferSize, false));
     }
 
     /**
@@ -429,7 +429,7 @@ public static  Flowable combineLatest(@NonNull Iterable(sources, combiner, bufferSize, false));
+        return RxJavaPlugins.onAssembly(new FlowableCombineLatest<>(sources, combiner, bufferSize, false));
     }
 
     /**
@@ -530,7 +530,7 @@ public static  Flowable combineLatest(@NonNull Iterable(sources, combiner, bufferSize, true));
+        return RxJavaPlugins.onAssembly(new FlowableCombineLatest<>(sources, combiner, bufferSize, true));
     }
 
     /**
@@ -629,7 +629,7 @@ public static  Flowable combineLatestDelayError(@NonNull Iterable(sources, combiner, bufferSize, true));
+        return RxJavaPlugins.onAssembly(new FlowableCombineLatest<>(sources, combiner, bufferSize, true));
     }
 
     /**
@@ -1892,7 +1892,7 @@ public static  Flowable empty() {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Flowable error(@NonNull Supplier supplier) {
         Objects.requireNonNull(supplier, "supplier is null");
-        return RxJavaPlugins.onAssembly(new FlowableError(supplier));
+        return RxJavaPlugins.onAssembly(new FlowableError<>(supplier));
     }
 
     /**
@@ -1997,7 +1997,7 @@ public static  Flowable error(@NonNull Throwable throwable) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Flowable fromCallable(@NonNull Callable supplier) {
         Objects.requireNonNull(supplier, "supplier is null");
-        return RxJavaPlugins.onAssembly(new FlowableFromCallable(supplier));
+        return RxJavaPlugins.onAssembly(new FlowableFromCallable<>(supplier));
     }
 
     /**
@@ -2039,7 +2039,7 @@ public static  Flowable error(@NonNull Throwable throwable) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Flowable fromFuture(@NonNull Future future) {
         Objects.requireNonNull(future, "future is null");
-        return RxJavaPlugins.onAssembly(new FlowableFromFuture(future, 0L, null));
+        return RxJavaPlugins.onAssembly(new FlowableFromFuture<>(future, 0L, null));
     }
 
     /**
@@ -2086,7 +2086,7 @@ public static  Flowable error(@NonNull Throwable throwable) {
     public static <@NonNull T> Flowable fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) {
         Objects.requireNonNull(future, "future is null");
         Objects.requireNonNull(unit, "unit is null");
-        return RxJavaPlugins.onAssembly(new FlowableFromFuture(future, timeout, unit));
+        return RxJavaPlugins.onAssembly(new FlowableFromFuture<>(future, timeout, unit));
     }
 
     /**
@@ -2205,7 +2205,7 @@ public static  Flowable error(@NonNull Throwable throwable) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Flowable fromIterable(@NonNull Iterable source) {
         Objects.requireNonNull(source, "source is null");
-        return RxJavaPlugins.onAssembly(new FlowableFromIterable(source));
+        return RxJavaPlugins.onAssembly(new FlowableFromIterable<>(source));
     }
 
     /**
@@ -2246,7 +2246,7 @@ public static  Flowable fromPublisher(@NonNull Publisher sour
         }
         Objects.requireNonNull(source, "source is null");
 
-        return RxJavaPlugins.onAssembly(new FlowableFromPublisher(source));
+        return RxJavaPlugins.onAssembly(new FlowableFromPublisher<>(source));
     }
 
     /**
@@ -2287,7 +2287,7 @@ public static  Flowable fromPublisher(@NonNull Publisher sour
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Flowable fromSupplier(@NonNull Supplier supplier) {
         Objects.requireNonNull(supplier, "supplier is null");
-        return RxJavaPlugins.onAssembly(new FlowableFromSupplier(supplier));
+        return RxJavaPlugins.onAssembly(new FlowableFromSupplier<>(supplier));
     }
 
     /**
@@ -2318,7 +2318,7 @@ public static  Flowable fromPublisher(@NonNull Publisher sour
     public static  Flowable generate(@NonNull Consumer<@NonNull Emitter> generator) {
         Objects.requireNonNull(generator, "generator is null");
         return generate(Functions.nullSupplier(),
-                FlowableInternalHelper.simpleGenerator(generator),
+                FlowableInternalHelper.simpleGenerator(generator),
                 Functions.emptyConsumer());
     }
 
@@ -2351,7 +2351,7 @@ public static  Flowable generate(@NonNull Consumer<@NonNull Emitter> ge
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Flowable generate(@NonNull Supplier initialState, @NonNull BiConsumer> generator) {
         Objects.requireNonNull(generator, "generator is null");
-        return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator),
+        return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator),
                 Functions.emptyConsumer());
     }
 
@@ -2387,7 +2387,7 @@ public static  Flowable generate(@NonNull Supplier initialState, @No
     public static  Flowable generate(@NonNull Supplier initialState, @NonNull BiConsumer> generator,
             @NonNull Consumer disposeState) {
         Objects.requireNonNull(generator, "generator is null");
-        return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator), disposeState);
+        return generate(initialState, FlowableInternalHelper.simpleBiGenerator(generator), disposeState);
     }
 
     /**
@@ -4622,7 +4622,7 @@ public static  Flowable using(
     public static  Flowable zip(@NonNull Iterable> sources, @NonNull Function zipper) {
         Objects.requireNonNull(zipper, "zipper is null");
         Objects.requireNonNull(sources, "sources is null");
-        return RxJavaPlugins.onAssembly(new FlowableZip(null, sources, zipper, bufferSize(), false));
+        return RxJavaPlugins.onAssembly(new FlowableZip<>(null, sources, zipper, bufferSize(), false));
     }
 
     /**
@@ -4684,7 +4684,7 @@ public static  Flowable zip(@NonNull Iterable(null, sources, zipper, bufferSize, delayError));
+        return RxJavaPlugins.onAssembly(new FlowableZip<>(null, sources, zipper, bufferSize, delayError));
     }
 
     /**
@@ -5486,7 +5486,7 @@ public static  Flowable zipArray(@NonNull Function(sources, null, zipper, bufferSize, delayError));
+        return RxJavaPlugins.onAssembly(new FlowableZip<>(sources, null, zipper, bufferSize, delayError));
     }
 
     // ***************************************************************************************************
@@ -6025,7 +6025,7 @@ public final T blockingSingle(@NonNull T defaultItem) {
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public final Future toFuture() {
-        return subscribeWith(new FutureSubscriber());
+        return subscribeWith(new FutureSubscriber<>());
     }
 
     /**
@@ -6303,7 +6303,7 @@ public final Flowable> buffer(int count) {
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public final Flowable> buffer(int count, int skip) {
-        return buffer(count, skip, ArrayListSupplier.asSupplier());
+        return buffer(count, skip, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -6415,7 +6415,7 @@ public final > Flowable buffer(int count, @No
     @SchedulerSupport(SchedulerSupport.COMPUTATION)
     @NonNull
     public final Flowable> buffer(long timespan, long timeskip, @NonNull TimeUnit unit) {
-        return buffer(timespan, timeskip, unit, Schedulers.computation(), ArrayListSupplier.asSupplier());
+        return buffer(timespan, timeskip, unit, Schedulers.computation(), ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -6453,7 +6453,7 @@ public final Flowable> buffer(long timespan, long timeskip, @NonNull Tim
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public final Flowable> buffer(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) {
-        return buffer(timespan, timeskip, unit, scheduler, ArrayListSupplier.asSupplier());
+        return buffer(timespan, timeskip, unit, scheduler, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -6610,7 +6610,7 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, int
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int count) {
-        return buffer(timespan, unit, scheduler, count, ArrayListSupplier.asSupplier(), false);
+        return buffer(timespan, unit, scheduler, count, ArrayListSupplier.asSupplier(), false);
     }
 
     /**
@@ -6700,7 +6700,7 @@ public final > Flowable buffer(
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) {
-        return buffer(timespan, unit, scheduler, Integer.MAX_VALUE, ArrayListSupplier.asSupplier(), false);
+        return buffer(timespan, unit, scheduler, Integer.MAX_VALUE, ArrayListSupplier.asSupplier(), false);
     }
 
     /**
@@ -6737,7 +6737,7 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @No
     public final  Flowable> buffer(
             @NonNull Flowable openingIndicator,
             @NonNull Function> closingIndicator) {
-        return buffer(openingIndicator, closingIndicator, ArrayListSupplier.asSupplier());
+        return buffer(openingIndicator, closingIndicator, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -6817,7 +6817,7 @@ public final > Flowable b
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public final  Flowable> buffer(@NonNull Publisher boundaryIndicator) {
-        return buffer(boundaryIndicator, ArrayListSupplier.asSupplier());
+        return buffer(boundaryIndicator, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -6855,7 +6855,7 @@ public final  Flowable> buffer(@NonNull Publisher boundaryIndicato
     @NonNull
     public final  Flowable> buffer(@NonNull Publisher boundaryIndicator, int initialCapacity) {
         ObjectHelper.verifyPositive(initialCapacity, "initialCapacity");
-        return buffer(boundaryIndicator, Functions.createArrayList(initialCapacity));
+        return buffer(boundaryIndicator, Functions.createArrayList(initialCapacity));
     }
 
     /**
@@ -7089,7 +7089,7 @@ public final  Flowable cast(@NonNull Class clazz) {
     public final  Single collect(@NonNull Supplier initialItemSupplier, @NonNull BiConsumer collector) {
         Objects.requireNonNull(initialItemSupplier, "initialItemSupplier is null");
         Objects.requireNonNull(collector, "collector is null");
-        return RxJavaPlugins.onAssembly(new FlowableCollectSingle(this, initialItemSupplier, collector));
+        return RxJavaPlugins.onAssembly(new FlowableCollectSingle<>(this, initialItemSupplier, collector));
     }
 
     /**
@@ -8896,7 +8896,7 @@ public final Flowable distinct() {
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public final  Flowable distinct(@NonNull Function keySelector) {
-        return distinct(keySelector, Functions.createHashSet());
+        return distinct(keySelector, Functions.createHashSet());
     }
 
     /**
@@ -9063,7 +9063,7 @@ public final  Flowable distinctUntilChanged(@NonNull Function distinctUntilChanged(@NonNull BiPredicate comparer) {
         Objects.requireNonNull(comparer, "comparer is null");
-        return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged<>(this, Functions.identity(), comparer));
+        return RxJavaPlugins.onAssembly(new FlowableDistinctUntilChanged<>(this, Functions.identity(), comparer));
     }
 
     /**
@@ -10632,7 +10632,7 @@ public final Disposable forEachWhile(@NonNull Predicate onNext, @NonN
      * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in
      * so-called group abandonment where a group will only contain one element and the group will be
      * re-created over and over as new upstream items trigger a new group. The behavior is
-     * a tradeoff between no-dataloss, upstream cancellation and excessive group creation.
+     * a trade-off between no-dataloss, upstream cancellation and excessive group creation.
      *
      * 
*
Backpressure:
@@ -10664,7 +10664,7 @@ public final Disposable forEachWhile(@NonNull Predicate onNext, @NonN @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Flowable> groupBy(@NonNull Function keySelector) { - return groupBy(keySelector, Functions.identity(), false, bufferSize()); + return groupBy(keySelector, Functions.identity(), false, bufferSize()); } /** @@ -10691,7 +10691,7 @@ public final Flowable> groupBy(@NonNull Function *
Backpressure:
@@ -10724,7 +10724,7 @@ public final Flowable> groupBy(@NonNull Function Flowable> groupBy(@NonNull Function keySelector, boolean delayError) { - return groupBy(keySelector, Functions.identity(), delayError, bufferSize()); + return groupBy(keySelector, Functions.identity(), delayError, bufferSize()); } /** @@ -10751,7 +10751,7 @@ public final Flowable> groupBy(@NonNull Function *
Backpressure:
@@ -10816,7 +10816,7 @@ public final Flowable> groupBy(@NonNull Function *
Backpressure:
@@ -10882,7 +10882,7 @@ public final Flowable> groupBy(@NonNull Function *
Backpressure:
@@ -10927,7 +10927,7 @@ public final Flowable> groupBy(@NonNull Function(this, keySelector, valueSelector, bufferSize, delayError, null)); + return RxJavaPlugins.onAssembly(new FlowableGroupBy<>(this, keySelector, valueSelector, bufferSize, delayError, null)); } /** @@ -10995,7 +10995,7 @@ public final Flowable> groupBy(@NonNull Function *
Backpressure:
@@ -11050,7 +11050,7 @@ public final Flowable> groupBy(@NonNull Function(this, keySelector, valueSelector, bufferSize, delayError, evictingMapFactory)); + return RxJavaPlugins.onAssembly(new FlowableGroupBy<>(this, keySelector, valueSelector, bufferSize, delayError, evictingMapFactory)); } /** @@ -11100,7 +11100,7 @@ public final Flowable groupJoin( Objects.requireNonNull(leftEnd, "leftEnd is null"); Objects.requireNonNull(rightEnd, "rightEnd is null"); Objects.requireNonNull(resultSelector, "resultSelector is null"); - return RxJavaPlugins.onAssembly(new FlowableGroupJoin( + return RxJavaPlugins.onAssembly(new FlowableGroupJoin<>( this, other, leftEnd, rightEnd, resultSelector)); } @@ -11456,7 +11456,7 @@ public final Single lastOrError() { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable lift(@NonNull FlowableOperator lifter) { Objects.requireNonNull(lifter, "lifter is null"); - return RxJavaPlugins.onAssembly(new FlowableLift(this, lifter)); + return RxJavaPlugins.onAssembly(new FlowableLift<>(this, lifter)); } /** @@ -11486,7 +11486,7 @@ public final Flowable lift(@NonNull FlowableOperator Flowable map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new FlowableMap(this, mapper)); + return RxJavaPlugins.onAssembly(new FlowableMap<>(this, mapper)); } /** @@ -13894,7 +13894,7 @@ public final void safeSubscribe(@NonNull Subscriber s) { if (s instanceof SafeSubscriber) { subscribe((SafeSubscriber)s); } else { - subscribe(new SafeSubscriber(s)); + subscribe(new SafeSubscriber<>(s)); } } @@ -14768,7 +14768,7 @@ public final Flowable skipWhile(@NonNull Predicate predicate) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Flowable sorted() { - return toList().toFlowable().map(Functions.listSorter(Functions.naturalComparator())).flatMapIterable(Functions.>identity()); + return toList().toFlowable().map(Functions.listSorter(Functions.naturalComparator())).flatMapIterable(Functions.identity()); } /** @@ -14797,7 +14797,7 @@ public final Flowable sorted() { @SchedulerSupport(SchedulerSupport.NONE) public final Flowable sorted(@NonNull Comparator<@NonNull ? super T> sortFunction) { Objects.requireNonNull(sortFunction, "sortFunction"); - return toList().toFlowable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.>identity()); + return toList().toFlowable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.identity()); } /** @@ -15065,7 +15065,7 @@ public final void subscribe(@NonNull Subscriber s) { subscribe((FlowableSubscriber)s); } else { Objects.requireNonNull(s, "s is null"); - subscribe(new StrictSubscriber(s)); + subscribe(new StrictSubscriber<>(s)); } } @@ -17119,7 +17119,7 @@ public final Flowable> timestamp(@NonNull TimeUnit unit) { public final Flowable> timestamp(@NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return map(Functions.timestampWith(unit, scheduler)); + return map(Functions.timestampWith(unit, scheduler)); } /** @@ -17178,7 +17178,7 @@ public final R to(@NonNull FlowableConverter converter) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Single> toList() { - return RxJavaPlugins.onAssembly(new FlowableToListSingle>(this)); + return RxJavaPlugins.onAssembly(new FlowableToListSingle<>(this)); } /** @@ -17216,7 +17216,7 @@ public final Single> toList() { @NonNull public final Single> toList(int capacityHint) { ObjectHelper.verifyPositive(capacityHint, "capacityHint"); - return RxJavaPlugins.onAssembly(new FlowableToListSingle<>(this, Functions.createArrayList(capacityHint))); + return RxJavaPlugins.onAssembly(new FlowableToListSingle<>(this, Functions.createArrayList(capacityHint))); } /** @@ -17290,7 +17290,7 @@ public final > Single toList(@NonNull Supplie @SchedulerSupport(SchedulerSupport.NONE) public final Single> toMap(@NonNull Function keySelector) { Objects.requireNonNull(keySelector, "keySelector is null"); - return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); + return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); } /** @@ -17330,7 +17330,7 @@ public final Single> toMap(@NonNull Function Single> toMap(@NonNull Function keySelector, @NonNull Function valueSelector) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); - return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); + return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); } /** @@ -17533,7 +17533,7 @@ public final Single>> toMultimap( @NonNull Function valueSelector, @NonNull Supplier>> mapSupplier ) { - return toMultimap(keySelector, valueSelector, mapSupplier, ArrayListSupplier.asFunction()); + return toMultimap(keySelector, valueSelector, mapSupplier, ArrayListSupplier.asFunction()); } /** @@ -17731,7 +17731,7 @@ public final Flowable unsubscribeOn(@NonNull Scheduler scheduler) { *

* Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window will only contain one element. The behavior is - * a tradeoff between no-dataloss and ensuring upstream cancellation can happen. + * a trade-off between no-dataloss and ensuring upstream cancellation can happen. *

*
Backpressure:
*
The operator honors backpressure of its inner and outer subscribers, however, the inner Publisher uses an @@ -17766,7 +17766,7 @@ public final Flowable> window(long count) { * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff between no-dataloss and ensuring upstream cancellation can happen under some race conditions. + * a trade-off between no-dataloss and ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator honors backpressure of its inner and outer subscribers, however, the inner Publisher uses an @@ -17804,7 +17804,7 @@ public final Flowable> window(long count, long skip) { * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff between no-dataloss and ensuring upstream cancellation can happen under some race conditions. + * a trade-off between no-dataloss and ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator honors backpressure of its inner and outer subscribers, however, the inner Publisher uses an @@ -17848,7 +17848,7 @@ public final Flowable> window(long count, long skip, int bufferSize) * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -17889,7 +17889,7 @@ public final Flowable> window(long timespan, long timeskip, @NonNull * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -17932,7 +17932,7 @@ public final Flowable> window(long timespan, long timeskip, @NonNull * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -17981,7 +17981,7 @@ public final Flowable> window(long timespan, long timeskip, @NonNull * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -18021,7 +18021,7 @@ public final Flowable> window(long timespan, @NonNull TimeUnit unit) * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -18065,7 +18065,7 @@ public final Flowable> window(long timespan, @NonNull TimeUnit unit, * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -18110,7 +18110,7 @@ public final Flowable> window(long timespan, @NonNull TimeUnit unit, * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -18154,7 +18154,7 @@ public final Flowable> window(long timespan, @NonNull TimeUnit unit, * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -18200,7 +18200,7 @@ public final Flowable> window(long timespan, @NonNull TimeUnit unit, * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -18248,7 +18248,7 @@ public final Flowable> window(long timespan, @NonNull TimeUnit unit, * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner. @@ -18301,7 +18301,7 @@ public final Flowable> window( * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The outer Publisher of this operator does not support backpressure as it uses a {@code boundary} Publisher to control data @@ -18337,7 +18337,7 @@ public final Flowable> window(@NonNull Publisher boundaryIndi * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The outer Publisher of this operator does not support backpressure as it uses a {@code boundary} Publisher to control data @@ -18378,7 +18378,7 @@ public final Flowable> window(@NonNull Publisher boundaryIndi * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The outer Publisher of this operator doesn't support backpressure because the emission of new @@ -18421,7 +18421,7 @@ public final Flowable> window( * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Backpressure:
*
The outer Publisher of this operator doesn't support backpressure because the emission of new @@ -18731,7 +18731,7 @@ public final Flowable withLatestFrom(@NonNull Iterable Flowable zipWith(@NonNull Iterable other, @NonNull BiFunction zipper) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(zipper, "zipper is null"); - return RxJavaPlugins.onAssembly(new FlowableZipIterable(this, other, zipper)); + return RxJavaPlugins.onAssembly(new FlowableZipIterable<>(this, other, zipper)); } /** @@ -19154,7 +19154,7 @@ public final TestSubscriber test(long initialRequest, boolean cancel) { // No * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. *

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).firstStage(Optional.empty());
@@ -19191,7 +19191,7 @@ public final CompletionStage firstStage(@Nullable T defaultItem) {
      * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and
      * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}.
      * 

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).singleStage(Optional.empty());
@@ -19227,7 +19227,7 @@ public final CompletionStage singleStage(@Nullable T defaultItem) {
      * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and
      * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}.
      * 

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).lastStage(Optional.empty());
@@ -19409,7 +19409,7 @@ public final Stream blockingStream() {
     public final Stream blockingStream(int prefetch) {
         Iterator iterator = blockingIterable(prefetch).iterator();
         return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, 0), false)
-                .onClose(() -> ((Disposable)iterator).dispose());
+                .onClose(((Disposable) iterator)::dispose);
     }
 
     /**
diff --git a/src/main/java/io/reactivex/rxjava3/core/Maybe.java b/src/main/java/io/reactivex/rxjava3/core/Maybe.java
index f26e584c16..dfe8c4f0b4 100644
--- a/src/main/java/io/reactivex/rxjava3/core/Maybe.java
+++ b/src/main/java/io/reactivex/rxjava3/core/Maybe.java
@@ -653,7 +653,7 @@ public static  Maybe empty() {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Maybe error(@NonNull Throwable exception) {
         Objects.requireNonNull(exception, "exception is null");
-        return RxJavaPlugins.onAssembly(new MaybeError(exception));
+        return RxJavaPlugins.onAssembly(new MaybeError<>(exception));
     }
 
     /**
@@ -679,7 +679,7 @@ public static  Maybe error(@NonNull Throwable exception) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Maybe error(@NonNull Supplier supplier) {
         Objects.requireNonNull(supplier, "supplier is null");
-        return RxJavaPlugins.onAssembly(new MaybeErrorCallable(supplier));
+        return RxJavaPlugins.onAssembly(new MaybeErrorCallable<>(supplier));
     }
 
     /**
@@ -706,7 +706,7 @@ public static  Maybe error(@NonNull Supplier supplier
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Maybe fromAction(@NonNull Action run) {
         Objects.requireNonNull(run, "run is null");
-        return RxJavaPlugins.onAssembly(new MaybeFromAction(run));
+        return RxJavaPlugins.onAssembly(new MaybeFromAction<>(run));
     }
 
     /**
@@ -726,7 +726,7 @@ public static  Maybe fromAction(@NonNull Action run) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Maybe fromCompletable(@NonNull CompletableSource completableSource) {
         Objects.requireNonNull(completableSource, "completableSource is null");
-        return RxJavaPlugins.onAssembly(new MaybeFromCompletable(completableSource));
+        return RxJavaPlugins.onAssembly(new MaybeFromCompletable<>(completableSource));
     }
 
     /**
@@ -790,7 +790,7 @@ public static  Maybe fromSingle(@NonNull SingleSource singleSource) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Maybe fromCallable(@NonNull Callable callable) {
         Objects.requireNonNull(callable, "callable is null");
-        return RxJavaPlugins.onAssembly(new MaybeFromCallable(callable));
+        return RxJavaPlugins.onAssembly(new MaybeFromCallable<>(callable));
     }
 
     /**
@@ -824,7 +824,7 @@ public static  Maybe fromSingle(@NonNull SingleSource singleSource) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Maybe fromFuture(@NonNull Future future) {
         Objects.requireNonNull(future, "future is null");
-        return RxJavaPlugins.onAssembly(new MaybeFromFuture(future, 0L, null));
+        return RxJavaPlugins.onAssembly(new MaybeFromFuture<>(future, 0L, null));
     }
 
     /**
@@ -863,7 +863,7 @@ public static  Maybe fromSingle(@NonNull SingleSource singleSource) {
     public static <@NonNull T> Maybe fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) {
         Objects.requireNonNull(future, "future is null");
         Objects.requireNonNull(unit, "unit is null");
-        return RxJavaPlugins.onAssembly(new MaybeFromFuture(future, timeout, unit));
+        return RxJavaPlugins.onAssembly(new MaybeFromFuture<>(future, timeout, unit));
     }
 
     /**
@@ -883,7 +883,7 @@ public static  Maybe fromSingle(@NonNull SingleSource singleSource) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Maybe fromRunnable(@NonNull Runnable run) {
         Objects.requireNonNull(run, "run is null");
-        return RxJavaPlugins.onAssembly(new MaybeFromRunnable(run));
+        return RxJavaPlugins.onAssembly(new MaybeFromRunnable<>(run));
     }
 
     /**
@@ -930,7 +930,7 @@ public static  Maybe fromRunnable(@NonNull Runnable run) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Maybe fromSupplier(@NonNull Supplier supplier) {
         Objects.requireNonNull(supplier, "supplier is null");
-        return RxJavaPlugins.onAssembly(new MaybeFromSupplier(supplier));
+        return RxJavaPlugins.onAssembly(new MaybeFromSupplier<>(supplier));
     }
 
     /**
@@ -1865,7 +1865,7 @@ public static  Maybe wrap(@NonNull MaybeSource source) {
     public static  Maybe zip(@NonNull Iterable> sources, @NonNull Function zipper) {
         Objects.requireNonNull(zipper, "zipper is null");
         Objects.requireNonNull(sources, "sources is null");
-        return RxJavaPlugins.onAssembly(new MaybeZipIterable(sources, zipper));
+        return RxJavaPlugins.onAssembly(new MaybeZipIterable<>(sources, zipper));
     }
 
     /**
@@ -2331,7 +2331,7 @@ public static  Maybe zipArray(@NonNull Function(sources, zipper));
+        return RxJavaPlugins.onAssembly(new MaybeZipArray<>(sources, zipper));
     }
 
     // ------------------------------------------------------------------
@@ -3530,7 +3530,7 @@ public final Single isEmpty() {
     @SchedulerSupport(SchedulerSupport.NONE)
     public final  Maybe lift(@NonNull MaybeOperator lift) {
         Objects.requireNonNull(lift, "lift is null");
-        return RxJavaPlugins.onAssembly(new MaybeLift(this, lift));
+        return RxJavaPlugins.onAssembly(new MaybeLift<>(this, lift));
     }
 
     /**
@@ -3554,7 +3554,7 @@ public final  Maybe lift(@NonNull MaybeOperator li
     @SchedulerSupport(SchedulerSupport.NONE)
     public final  Maybe map(@NonNull Function mapper) {
         Objects.requireNonNull(mapper, "mapper is null");
-        return RxJavaPlugins.onAssembly(new MaybeMap(this, mapper));
+        return RxJavaPlugins.onAssembly(new MaybeMap<>(this, mapper));
     }
 
     /**
@@ -4349,7 +4349,7 @@ public final Disposable subscribe(@NonNull Consumer onSuccess, @NonNu
         Objects.requireNonNull(onSuccess, "onSuccess is null");
         Objects.requireNonNull(onError, "onError is null");
         Objects.requireNonNull(onComplete, "onComplete is null");
-        return subscribeWith(new MaybeCallbackObserver(onSuccess, onError, onComplete));
+        return subscribeWith(new MaybeCallbackObserver<>(onSuccess, onError, onComplete));
     }
 
     @SchedulerSupport(SchedulerSupport.NONE)
@@ -4925,7 +4925,7 @@ public final TestObserver test(boolean dispose) {
      *  
{@code mapOptional} does not operate by default on a particular {@link Scheduler}.
*
* @param the non-null output type - * @param mapper the function that receives the upstream success iteem and should return a non-empty {@code Optional} + * @param mapper the function that receives the upstream success item and should return a non-empty {@code Optional} * to emit as the success output or an empty {@code Optional} to complete the {@code Maybe} * @return the new Maybe instance * @since 3.0.0 @@ -4952,8 +4952,8 @@ public final TestObserver test(boolean dispose) { * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. *

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use - * {@link #toCompletionStage(Object)} with {@code null} or turn the upstrea into a sequence of {@link Optional}s and + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use + * {@link #toCompletionStage(Object)} with {@code null} or turn the upstream into a sequence of {@link Optional}s and * default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).toCompletionStage(Optional.empty());
@@ -4985,7 +4985,7 @@ public final CompletionStage toCompletionStage() {
      * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and
      * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}.
      * 

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).toCompletionStage(Optional.empty());
diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java
index 1933a6db0b..373a576888 100644
--- a/src/main/java/io/reactivex/rxjava3/core/Observable.java
+++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java
@@ -265,7 +265,7 @@ public static  Observable combineLatest(
 
         // the queue holds a pair of values so we need to double the capacity
         int s = bufferSize << 1;
-        return RxJavaPlugins.onAssembly(new ObservableCombineLatest(null, sources, combiner, s, false));
+        return RxJavaPlugins.onAssembly(new ObservableCombineLatest<>(null, sources, combiner, s, false));
     }
 
     /**
@@ -364,7 +364,7 @@ public static  Observable combineLatestArray(
 
         // the queue holds a pair of values so we need to double the capacity
         int s = bufferSize << 1;
-        return RxJavaPlugins.onAssembly(new ObservableCombineLatest(sources, null, combiner, s, false));
+        return RxJavaPlugins.onAssembly(new ObservableCombineLatest<>(sources, null, combiner, s, false));
     }
 
     /**
@@ -917,7 +917,7 @@ public static  Observable combineLatestDelayError(@NonNull ObservableSo
         }
         // the queue holds a pair of values so we need to double the capacity
         int s = bufferSize << 1;
-        return RxJavaPlugins.onAssembly(new ObservableCombineLatest(sources, null, combiner, s, true));
+        return RxJavaPlugins.onAssembly(new ObservableCombineLatest<>(sources, null, combiner, s, true));
     }
 
     /**
@@ -1013,7 +1013,7 @@ public static  Observable combineLatestDelayError(@NonNull Iterable(null, sources, combiner, s, true));
+        return RxJavaPlugins.onAssembly(new ObservableCombineLatest<>(null, sources, combiner, s, true));
     }
 
     /**
@@ -1669,7 +1669,7 @@ public static  Observable empty() {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Observable error(@NonNull Supplier errorSupplier) {
         Objects.requireNonNull(errorSupplier, "errorSupplier is null");
-        return RxJavaPlugins.onAssembly(new ObservableError(errorSupplier));
+        return RxJavaPlugins.onAssembly(new ObservableError<>(errorSupplier));
     }
 
     /**
@@ -1763,7 +1763,7 @@ public static  Observable fromArray(@NonNull T... items) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Observable fromCallable(@NonNull Callable supplier) {
         Objects.requireNonNull(supplier, "supplier is null");
-        return RxJavaPlugins.onAssembly(new ObservableFromCallable(supplier));
+        return RxJavaPlugins.onAssembly(new ObservableFromCallable<>(supplier));
     }
 
     /**
@@ -1797,7 +1797,7 @@ public static  Observable fromCallable(@NonNull Callable supp
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Observable fromFuture(@NonNull Future future) {
         Objects.requireNonNull(future, "future is null");
-        return RxJavaPlugins.onAssembly(new ObservableFromFuture(future, 0L, null));
+        return RxJavaPlugins.onAssembly(new ObservableFromFuture<>(future, 0L, null));
     }
 
     /**
@@ -1836,7 +1836,7 @@ public static  Observable fromFuture(@NonNull Future future)
     public static  Observable fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) {
         Objects.requireNonNull(future, "future is null");
         Objects.requireNonNull(unit, "unit is null");
-        return RxJavaPlugins.onAssembly(new ObservableFromFuture(future, timeout, unit));
+        return RxJavaPlugins.onAssembly(new ObservableFromFuture<>(future, timeout, unit));
     }
 
     /**
@@ -1940,7 +1940,7 @@ public static  Observable fromFuture(@NonNull Future future,
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Observable fromIterable(@NonNull Iterable source) {
         Objects.requireNonNull(source, "source is null");
-        return RxJavaPlugins.onAssembly(new ObservableFromIterable(source));
+        return RxJavaPlugins.onAssembly(new ObservableFromIterable<>(source));
     }
 
     /**
@@ -1977,7 +1977,7 @@ public static  Observable fromIterable(@NonNull Iterable sour
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Observable fromPublisher(@NonNull Publisher publisher) {
         Objects.requireNonNull(publisher, "publisher is null");
-        return RxJavaPlugins.onAssembly(new ObservableFromPublisher(publisher));
+        return RxJavaPlugins.onAssembly(new ObservableFromPublisher<>(publisher));
     }
 
     /**
@@ -2014,7 +2014,7 @@ public static  Observable fromPublisher(@NonNull Publisher pu
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Observable fromSupplier(@NonNull Supplier supplier) {
         Objects.requireNonNull(supplier, "supplier is null");
-        return RxJavaPlugins.onAssembly(new ObservableFromSupplier(supplier));
+        return RxJavaPlugins.onAssembly(new ObservableFromSupplier<>(supplier));
     }
 
     /**
@@ -2043,8 +2043,8 @@ public static  Observable fromSupplier(@NonNull Supplier supp
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Observable generate(@NonNull Consumer> generator) {
         Objects.requireNonNull(generator, "generator is null");
-        return generate(Functions.nullSupplier(),
-                ObservableInternalHelper.simpleGenerator(generator), Functions.emptyConsumer());
+        return generate(Functions.nullSupplier(),
+                ObservableInternalHelper.simpleGenerator(generator), Functions.emptyConsumer());
     }
 
     /**
@@ -4188,7 +4188,7 @@ public static  Observable wrap(@NonNull ObservableSource source) {
     public static  Observable zip(@NonNull Iterable> sources, @NonNull Function zipper) {
         Objects.requireNonNull(zipper, "zipper is null");
         Objects.requireNonNull(sources, "sources is null");
-        return RxJavaPlugins.onAssembly(new ObservableZip(null, sources, zipper, bufferSize(), false));
+        return RxJavaPlugins.onAssembly(new ObservableZip<>(null, sources, zipper, bufferSize(), false));
     }
 
     /**
@@ -4250,7 +4250,7 @@ public static  Observable zip(@NonNull Iterable(null, sources, zipper, bufferSize, delayError));
+        return RxJavaPlugins.onAssembly(new ObservableZip<>(null, sources, zipper, bufferSize, delayError));
     }
 
     /**
@@ -5004,7 +5004,7 @@ public static  Observable zipArray(
         }
         Objects.requireNonNull(zipper, "zipper is null");
         ObjectHelper.verifyPositive(bufferSize, "bufferSize");
-        return RxJavaPlugins.onAssembly(new ObservableZip(sources, null, zipper, bufferSize, delayError));
+        return RxJavaPlugins.onAssembly(new ObservableZip<>(sources, null, zipper, bufferSize, delayError));
     }
 
     // ***************************************************************************************************
@@ -5477,7 +5477,7 @@ public final T blockingSingle(@NonNull T defaultItem) {
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public final Future toFuture() {
-        return subscribeWith(new FutureObserver());
+        return subscribeWith(new FutureObserver<>());
     }
 
     /**
@@ -5648,7 +5648,7 @@ public final void blockingSubscribe(@NonNull Observer observer) {
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public final Observable<@NonNull List> buffer(int count, int skip) {
-        return buffer(count, skip, ArrayListSupplier.asSupplier());
+        return buffer(count, skip, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -5746,7 +5746,7 @@ public final > Observable buffer(int count, i
     @SchedulerSupport(SchedulerSupport.COMPUTATION)
     @NonNull
     public final Observable<@NonNull List> buffer(long timespan, long timeskip, @NonNull TimeUnit unit) {
-        return buffer(timespan, timeskip, unit, Schedulers.computation(), ArrayListSupplier.asSupplier());
+        return buffer(timespan, timeskip, unit, Schedulers.computation(), ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -5780,7 +5780,7 @@ public final > Observable buffer(int count, i
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public final Observable<@NonNull List> buffer(long timespan, long timeskip, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) {
-        return buffer(timespan, timeskip, unit, scheduler, ArrayListSupplier.asSupplier());
+        return buffer(timespan, timeskip, unit, scheduler, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -5921,7 +5921,7 @@ public final > Observable buffer(int count, i
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public final Observable<@NonNull List> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int count) {
-        return buffer(timespan, unit, scheduler, count, ArrayListSupplier.asSupplier(), false);
+        return buffer(timespan, unit, scheduler, count, ArrayListSupplier.asSupplier(), false);
     }
 
     /**
@@ -6003,7 +6003,7 @@ public final > Observable buffer(int count, i
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public final Observable<@NonNull List> buffer(long timespan, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) {
-        return buffer(timespan, unit, scheduler, Integer.MAX_VALUE, ArrayListSupplier.asSupplier(), false);
+        return buffer(timespan, unit, scheduler, Integer.MAX_VALUE, ArrayListSupplier.asSupplier(), false);
     }
 
     /**
@@ -6036,7 +6036,7 @@ public final > Observable buffer(int count, i
     public final  Observable<@NonNull List> buffer(
             @NonNull ObservableSource openingIndicator,
             @NonNull Function> closingIndicator) {
-        return buffer(openingIndicator, closingIndicator, ArrayListSupplier.asSupplier());
+        return buffer(openingIndicator, closingIndicator, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -6108,7 +6108,7 @@ public final > Observable buffer(int count, i
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public final  Observable<@NonNull List> buffer(@NonNull ObservableSource boundary) {
-        return buffer(boundary, ArrayListSupplier.asSupplier());
+        return buffer(boundary, ArrayListSupplier.asSupplier());
     }
 
     /**
@@ -6142,7 +6142,7 @@ public final > Observable buffer(int count, i
     @NonNull
     public final  Observable<@NonNull List> buffer(@NonNull ObservableSource boundary, int initialCapacity) {
         ObjectHelper.verifyPositive(initialCapacity, "initialCapacity");
-        return buffer(boundary, Functions.createArrayList(initialCapacity));
+        return buffer(boundary, Functions.createArrayList(initialCapacity));
     }
 
     /**
@@ -6355,7 +6355,7 @@ public final  Observable cast(@NonNull Class clazz) {
     public final  Single collect(@NonNull Supplier initialValueSupplier, @NonNull BiConsumer collector) {
         Objects.requireNonNull(initialValueSupplier, "initialValueSupplier is null");
         Objects.requireNonNull(collector, "collector is null");
-        return RxJavaPlugins.onAssembly(new ObservableCollectSingle(this, initialValueSupplier, collector));
+        return RxJavaPlugins.onAssembly(new ObservableCollectSingle<>(this, initialValueSupplier, collector));
     }
 
     /**
@@ -8088,7 +8088,7 @@ public final  Observable distinctUntilChanged(@NonNull Function distinctUntilChanged(@NonNull BiPredicate comparer) {
         Objects.requireNonNull(comparer, "comparer is null");
-        return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged<>(this, Functions.identity(), comparer));
+        return RxJavaPlugins.onAssembly(new ObservableDistinctUntilChanged<>(this, Functions.identity(), comparer));
     }
 
     /**
@@ -9351,7 +9351,7 @@ public final Disposable forEachWhile(@NonNull Predicate onNext, @NonN
      * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in
      * so-called group abandonment where a group will only contain one element and the group will be
      * re-created over and over as new upstream items trigger a new group. The behavior is
-     * a tradeoff between no-dataloss, upstream cancellation and excessive group creation.
+     * a trade-off between no-dataloss, upstream cancellation and excessive group creation.
      *
      * 
*
Scheduler:
@@ -9392,7 +9392,7 @@ public final Observable> groupBy(@NonNull Function *
Scheduler:
@@ -9436,7 +9436,7 @@ public final Observable> groupBy(@NonNull Function *
Scheduler:
@@ -9481,7 +9481,7 @@ public final Observable> groupBy(@NonNull Functio * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be * re-created over and over as new upstream items trigger a new group. The behavior is - * a tradeoff between no-dataloss, upstream cancellation and excessive group creation. + * a trade-off between no-dataloss, upstream cancellation and excessive group creation. * *
*
Scheduler:
@@ -9529,7 +9529,7 @@ public final Observable> groupBy(@NonNull Functio * Note also that ignoring groups or subscribing later (i.e., on another thread) will result in * so-called group abandonment where a group will only contain one element and the group will be * re-created over and over as new upstream items trigger a new group. The behavior is - * a tradeoff between no-dataloss, upstream cancellation and excessive group creation. + * a trade-off between no-dataloss, upstream cancellation and excessive group creation. * *
*
Scheduler:
@@ -9564,7 +9564,7 @@ public final Observable> groupBy(@NonNull Functio Objects.requireNonNull(valueSelector, "valueSelector is null"); ObjectHelper.verifyPositive(bufferSize, "bufferSize"); - return RxJavaPlugins.onAssembly(new ObservableGroupBy(this, keySelector, valueSelector, bufferSize, delayError)); + return RxJavaPlugins.onAssembly(new ObservableGroupBy<>(this, keySelector, valueSelector, bufferSize, delayError)); } /** @@ -9611,7 +9611,7 @@ public final Observable groupJoin( Objects.requireNonNull(leftEnd, "leftEnd is null"); Objects.requireNonNull(rightEnd, "rightEnd is null"); Objects.requireNonNull(resultSelector, "resultSelector is null"); - return RxJavaPlugins.onAssembly(new ObservableGroupJoin( + return RxJavaPlugins.onAssembly(new ObservableGroupJoin<>( this, other, leftEnd, rightEnd, resultSelector)); } @@ -9939,7 +9939,7 @@ public final Single lastOrError() { @NonNull public final Observable lift(@NonNull ObservableOperator lifter) { Objects.requireNonNull(lifter, "lifter is null"); - return RxJavaPlugins.onAssembly(new ObservableLift(this, lifter)); + return RxJavaPlugins.onAssembly(new ObservableLift<>(this, lifter)); } /** @@ -9964,7 +9964,7 @@ public final Observable lift(@NonNull ObservableOperator Observable map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new ObservableMap(this, mapper)); + return RxJavaPlugins.onAssembly(new ObservableMap<>(this, mapper)); } /** @@ -11584,7 +11584,7 @@ public final void safeSubscribe(@NonNull Observer observer) { if (observer instanceof SafeObserver) { subscribe(observer); } else { - subscribe(new SafeObserver(observer)); + subscribe(new SafeObserver<>(observer)); } } @@ -12352,7 +12352,7 @@ public final Observable skipWhile(@NonNull Predicate predicate) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Observable sorted() { - return toList().toObservable().map(Functions.listSorter(Functions.naturalComparator())).flatMapIterable(Functions.>identity()); + return toList().toObservable().map(Functions.listSorter(Functions.naturalComparator())).flatMapIterable(Functions.identity()); } /** @@ -12377,7 +12377,7 @@ public final Observable sorted() { @NonNull public final Observable sorted(@NonNull Comparator sortFunction) { Objects.requireNonNull(sortFunction, "sortFunction is null"); - return toList().toObservable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.>identity()); + return toList().toObservable().map(Functions.listSorter(sortFunction)).flatMapIterable(Functions.identity()); } /** @@ -14309,7 +14309,7 @@ public final Observable> timestamp(@NonNull TimeUnit unit) { public final Observable> timestamp(@NonNull TimeUnit unit, @NonNull Scheduler scheduler) { Objects.requireNonNull(unit, "unit is null"); Objects.requireNonNull(scheduler, "scheduler is null"); - return map(Functions.timestampWith(unit, scheduler)); + return map(Functions.timestampWith(unit, scheduler)); } /** @@ -14396,7 +14396,7 @@ public final R to(@NonNull ObservableConverter converter) { @NonNull public final Single<@NonNull List> toList(int capacityHint) { ObjectHelper.verifyPositive(capacityHint, "capacityHint"); - return RxJavaPlugins.onAssembly(new ObservableToListSingle>(this, capacityHint)); + return RxJavaPlugins.onAssembly(new ObservableToListSingle<>(this, capacityHint)); } /** @@ -14463,7 +14463,7 @@ public final R to(@NonNull ObservableConverter converter) { @NonNull public final Single<@NonNull Map> toMap(@NonNull Function keySelector) { Objects.requireNonNull(keySelector, "keySelector is null"); - return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); + return collect(HashMapSupplier.asSupplier(), Functions.toMapKeySelector(keySelector)); } /** @@ -14501,7 +14501,7 @@ public final Single> toMap( @NonNull Function valueSelector) { Objects.requireNonNull(keySelector, "keySelector is null"); Objects.requireNonNull(valueSelector, "valueSelector is null"); - return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); + return collect(HashMapSupplier.asSupplier(), Functions.toMapKeyValueSelector(keySelector, valueSelector)); } /** @@ -14568,8 +14568,7 @@ public final Single> toMap( @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Single<@NonNull Map>> toMultimap(@NonNull Function keySelector) { - @SuppressWarnings({ "rawtypes", "unchecked" }) - Function valueSelector = (Function)Functions.identity(); + Function valueSelector = Functions.identity(); Supplier>> mapSupplier = HashMapSupplier.asSupplier(); Function> collectionFactory = ArrayListSupplier.asFunction(); return toMultimap(keySelector, valueSelector, mapSupplier, collectionFactory); @@ -14684,7 +14683,7 @@ public final Single> toMap( @NonNull Function valueSelector, @NonNull Supplier>> mapSupplier ) { - return toMultimap(keySelector, valueSelector, mapSupplier, ArrayListSupplier.asFunction()); + return toMultimap(keySelector, valueSelector, mapSupplier, ArrayListSupplier.asFunction()); } /** @@ -14864,7 +14863,7 @@ public final Flowable toFlowable(@NonNull BackpressureStrategy strategy) { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Single<@NonNull List> toSortedList(int capacityHint) { - return toSortedList(Functions.naturalOrder(), capacityHint); + return toSortedList(Functions.naturalOrder(), capacityHint); } /** @@ -14992,7 +14991,7 @@ public final Observable> window(long count, long skip, int bufferS * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -15026,7 +15025,7 @@ public final Observable> window(long timespan, long timeskip, @Non * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -15062,7 +15061,7 @@ public final Observable> window(long timespan, long timeskip, @Non * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -15104,7 +15103,7 @@ public final Observable> window(long timespan, long timeskip, @Non * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -15138,7 +15137,7 @@ public final Observable> window(long timespan, @NonNull TimeUnit u * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -15176,7 +15175,7 @@ public final Observable> window(long timespan, @NonNull TimeUnit u * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} operates by default on the {@code computation} {@link Scheduler}.
@@ -15215,7 +15214,7 @@ public final Observable> window(long timespan, @NonNull TimeUnit u * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -15252,7 +15251,7 @@ public final Observable> window(long timespan, @NonNull TimeUnit u * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -15292,7 +15291,7 @@ public final Observable> window(long timespan, @NonNull TimeUnit u * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -15334,7 +15333,7 @@ public final Observable> window(long timespan, @NonNull TimeUnit u * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
You specify which {@link Scheduler} this operator will use.
@@ -15381,7 +15380,7 @@ public final Observable> window( * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
@@ -15413,7 +15412,7 @@ public final Observable> window(@NonNull ObservableSource b * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
@@ -15450,7 +15449,7 @@ public final Observable> window(@NonNull ObservableSource b * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
@@ -15487,7 +15486,7 @@ public final Observable> window( * Note that ignoring windows or subscribing later (i.e., on another thread) will result in * so-called window abandonment where a window may not contain any elements. In this case, subsequent * elements will be dropped until the condition for the next window boundary is satisfied. The behavior is - * a tradeoff for ensuring upstream cancellation can happen under some race conditions. + * a trade-off for ensuring upstream cancellation can happen under some race conditions. *
*
Scheduler:
*
This version of {@code window} does not operate by default on a particular {@link Scheduler}.
@@ -15766,7 +15765,7 @@ public final Observable withLatestFrom(@NonNull Iterable Observable zipWith(@NonNull Iterable other, @NonNull BiFunction zipper) { Objects.requireNonNull(other, "other is null"); Objects.requireNonNull(zipper, "zipper is null"); - return RxJavaPlugins.onAssembly(new ObservableZipIterable(this, other, zipper)); + return RxJavaPlugins.onAssembly(new ObservableZipIterable<>(this, other, zipper)); } /** @@ -16127,7 +16126,7 @@ public final TestObserver test(boolean dispose) { // NoPMD * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}. *

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).firstStage(Optional.empty());
@@ -16161,7 +16160,7 @@ public final CompletionStage firstStage(@Nullable T defaultItem) {
      * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and
      * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}.
      * 

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).singleStage(Optional.empty());
@@ -16194,7 +16193,7 @@ public final CompletionStage singleStage(@Nullable T defaultItem) {
      * The upstream will be also cancelled if the resulting {@code CompletionStage} is converted to and
      * completed manually by {@link CompletableFuture#complete(Object)} or {@link CompletableFuture#completeExceptionally(Throwable)}.
      * 

- * {@code CompletionStage}s don't have a notion of emptyness and allow {@code null}s, therefore, one can either use + * {@code CompletionStage}s don't have a notion of emptiness and allow {@code null}s, therefore, one can either use * a {@code defaultItem} of {@code null} or turn the flow into a sequence of {@link Optional}s and default to {@link Optional#empty()}: *


      * CompletionStage<Optional<T>> stage = source.map(Optional::of).lastStage(Optional.empty());
@@ -16356,7 +16355,7 @@ public final Stream blockingStream() {
     public final Stream blockingStream(int capacityHint) {
         Iterator iterator = blockingIterable(capacityHint).iterator();
         return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, 0), false)
-                .onClose(() -> ((Disposable)iterator).dispose());
+                .onClose(((Disposable) iterator)::dispose);
     }
 
     /**
diff --git a/src/main/java/io/reactivex/rxjava3/core/Single.java b/src/main/java/io/reactivex/rxjava3/core/Single.java
index 8bd0f1db4e..f0022aef17 100644
--- a/src/main/java/io/reactivex/rxjava3/core/Single.java
+++ b/src/main/java/io/reactivex/rxjava3/core/Single.java
@@ -158,7 +158,7 @@ public static  Single amb(@NonNull Iterable Single ambArray(@NonNull SingleSource... sources) {
         if (sources.length == 0) {
-            return error(SingleInternalHelper.emptyThrower());
+            return error(SingleInternalHelper.emptyThrower());
         }
         if (sources.length == 1) {
             @SuppressWarnings("unchecked")
@@ -423,7 +423,7 @@ public static  Flowable concatArray(@NonNull SingleSource...
     @SchedulerSupport(SchedulerSupport.NONE)
     @SafeVarargs
     public static  Flowable concatArrayEager(@NonNull SingleSource... sources) {
-        return Flowable.fromArray(sources).concatMapEager(SingleInternalHelper.toFlowable());
+        return Flowable.fromArray(sources).concatMapEager(SingleInternalHelper.toFlowable());
     }
 
     /**
@@ -451,7 +451,7 @@ public static  Flowable concatArrayEager(@NonNull SingleSource Flowable concatEager(@NonNull Publisher> sources) {
-        return Flowable.fromPublisher(sources).concatMapEager(SingleInternalHelper.toFlowable());
+        return Flowable.fromPublisher(sources).concatMapEager(SingleInternalHelper.toFlowable());
     }
 
     /**
@@ -477,7 +477,7 @@ public static  Flowable concatEager(@NonNull Publisher Flowable concatEager(@NonNull Iterable> sources) {
-        return Flowable.fromIterable(sources).concatMapEager(SingleInternalHelper.toFlowable());
+        return Flowable.fromIterable(sources).concatMapEager(SingleInternalHelper.toFlowable());
     }
 
     /**
@@ -570,7 +570,7 @@ public static  Single defer(@NonNull Supplier Single error(@NonNull Supplier errorSupplier) {
         Objects.requireNonNull(errorSupplier, "errorSupplier is null");
-        return RxJavaPlugins.onAssembly(new SingleError(errorSupplier));
+        return RxJavaPlugins.onAssembly(new SingleError<>(errorSupplier));
     }
 
     /**
@@ -632,7 +632,7 @@ public static  Single error(@NonNull Throwable exception) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static <@NonNull T> Single fromCallable(@NonNull Callable callable) {
         Objects.requireNonNull(callable, "callable is null");
-        return RxJavaPlugins.onAssembly(new SingleFromCallable(callable));
+        return RxJavaPlugins.onAssembly(new SingleFromCallable<>(callable));
     }
 
     /**
@@ -662,7 +662,7 @@ public static  Single error(@NonNull Throwable exception) {
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public static <@NonNull T> Single fromFuture(@NonNull Future future) {
-        return toSingle(Flowable.fromFuture(future));
+        return toSingle(Flowable.fromFuture(future));
     }
 
     /**
@@ -696,7 +696,7 @@ public static  Single error(@NonNull Throwable exception) {
     @SchedulerSupport(SchedulerSupport.NONE)
     @NonNull
     public static <@NonNull T> Single fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit) {
-        return toSingle(Flowable.fromFuture(future, timeout, unit));
+        return toSingle(Flowable.fromFuture(future, timeout, unit));
     }
 
     /**
@@ -732,7 +732,7 @@ public static  Single error(@NonNull Throwable exception) {
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public static <@NonNull T> Single fromFuture(@NonNull Future future, long timeout, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) {
-        return toSingle(Flowable.fromFuture(future, timeout, unit, scheduler));
+        return toSingle(Flowable.fromFuture(future, timeout, unit, scheduler));
     }
 
     /**
@@ -763,7 +763,7 @@ public static  Single error(@NonNull Throwable exception) {
     @SchedulerSupport(SchedulerSupport.CUSTOM)
     @NonNull
     public static <@NonNull T> Single fromFuture(@NonNull Future future, @NonNull Scheduler scheduler) {
-        return toSingle(Flowable.fromFuture(future, scheduler));
+        return toSingle(Flowable.fromFuture(future, scheduler));
     }
 
     /**
@@ -801,7 +801,7 @@ public static  Single error(@NonNull Throwable exception) {
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Single fromPublisher(@NonNull Publisher publisher) {
         Objects.requireNonNull(publisher, "publisher is null");
-        return RxJavaPlugins.onAssembly(new SingleFromPublisher(publisher));
+        return RxJavaPlugins.onAssembly(new SingleFromPublisher<>(publisher));
     }
 
     /**
@@ -825,11 +825,11 @@ public static  Single fromPublisher(@NonNull Publisher publis
     @SchedulerSupport(SchedulerSupport.NONE)
     public static  Single fromObservable(@NonNull ObservableSource observableSource) {
         Objects.requireNonNull(observableSource, "observableSource is null");
-        return RxJavaPlugins.onAssembly(new ObservableSingleSingle(observableSource, null));
+        return RxJavaPlugins.onAssembly(new ObservableSingleSingle<>(observableSource, null));
     }
 
     /**
-     * Returns a {@link Single} that invokes passed supplierfunction and emits its result
+     * Returns a {@link Single} that invokes passed supplier and emits its result
      * for each new SingleObserver that subscribes.
      * 

* Allows you to defer execution of passed function until SingleObserver subscribes to the {@link Single}. @@ -863,7 +863,7 @@ public static Single fromObservable(@NonNull ObservableSource Single fromSupplier(@NonNull Supplier supplier) { Objects.requireNonNull(supplier, "supplier is null"); - return RxJavaPlugins.onAssembly(new SingleFromSupplier(supplier)); + return RxJavaPlugins.onAssembly(new SingleFromSupplier<>(supplier)); } /** @@ -996,10 +996,9 @@ public static Flowable merge(@NonNull Publisher Single merge(@NonNull SingleSource> source) { Objects.requireNonNull(source, "source is null"); - return RxJavaPlugins.onAssembly(new SingleFlatMap, T>(source, (Function)Functions.identity())); + return RxJavaPlugins.onAssembly(new SingleFlatMap, T>(source, Functions.identity())); } /** @@ -1580,7 +1579,7 @@ public static Single wrap(@NonNull SingleSource source) { public static Single zip(@NonNull Iterable> sources, @NonNull Function zipper) { Objects.requireNonNull(zipper, "zipper is null"); Objects.requireNonNull(sources, "sources is null"); - return RxJavaPlugins.onAssembly(new SingleZipIterable(sources, zipper)); + return RxJavaPlugins.onAssembly(new SingleZipIterable<>(sources, zipper)); } /** @@ -2037,7 +2036,7 @@ public static Single zipArray(@NonNull Function(sources, zipper)); + return RxJavaPlugins.onAssembly(new SingleZipArray<>(sources, zipper)); } /** @@ -3057,7 +3056,7 @@ public final T blockingGet() { @SchedulerSupport(SchedulerSupport.NONE) public final Single lift(@NonNull SingleOperator lift) { Objects.requireNonNull(lift, "lift is null"); - return RxJavaPlugins.onAssembly(new SingleLift(this, lift)); + return RxJavaPlugins.onAssembly(new SingleLift<>(this, lift)); } /** @@ -3081,7 +3080,7 @@ public final Single lift(@NonNull SingleOperator @SchedulerSupport(SchedulerSupport.NONE) public final <@NonNull R> Single map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper is null"); - return RxJavaPlugins.onAssembly(new SingleMap(this, mapper)); + return RxJavaPlugins.onAssembly(new SingleMap<>(this, mapper)); } /** @@ -4068,7 +4067,7 @@ public final Flowable toFlowable() { @SchedulerSupport(SchedulerSupport.NONE) @NonNull public final Future toFuture() { - return subscribeWith(new FutureSingleObserver()); + return subscribeWith(new FutureSingleObserver<>()); } /** @@ -4271,7 +4270,7 @@ private static Single toSingle(@NonNull Flowable source) { *

{@code mapOptional} does not operate by default on a particular {@link Scheduler}.
*
* @param the non-null output type - * @param mapper the function that receives the upstream success iteem and should return a non-empty {@code Optional} + * @param mapper the function that receives the upstream success item and should return a non-empty {@code Optional} * to emit as the success output or an empty {@code Optional} to complete the {@code Maybe} * @return the new Maybe instance * @since 3.0.0 diff --git a/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java b/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java index 85b06bd313..d7a3457fe4 100644 --- a/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java +++ b/src/main/java/io/reactivex/rxjava3/exceptions/CompositeException.java @@ -63,7 +63,6 @@ public CompositeException(@NonNull Throwable... exceptions) { */ public CompositeException(@NonNull Iterable errors) { Set deDupedExceptions = new LinkedHashSet<>(); - List localExceptions = new ArrayList<>(); if (errors != null) { for (Throwable ex : errors) { if (ex instanceof CompositeException) { @@ -81,7 +80,7 @@ public CompositeException(@NonNull Iterable errors) { if (deDupedExceptions.isEmpty()) { throw new IllegalArgumentException("errors is empty"); } - localExceptions.addAll(deDupedExceptions); + List localExceptions = new ArrayList<>(deDupedExceptions); this.exceptions = Collections.unmodifiableList(localExceptions); this.message = exceptions.size() + " exceptions occurred. "; } diff --git a/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java b/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java index 34557ce302..05ebb5bfb3 100644 --- a/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java +++ b/src/main/java/io/reactivex/rxjava3/exceptions/package-info.java @@ -17,7 +17,7 @@ /** * Exception handling utilities ({@link io.reactivex.rxjava3.exceptions.Exceptions Exceptions}), * composite exception container ({@link io.reactivex.rxjava3.exceptions.CompositeException CompositeException}) and - * various lifecycle-reladed ({@link io.reactivex.rxjava3.exceptions.MissingBackpressureException UndeliverableException}) + * various lifecycle-related ({@link io.reactivex.rxjava3.exceptions.MissingBackpressureException UndeliverableException}) * and behavior-violation exception types ({@link io.reactivex.rxjava3.exceptions.OnErrorNotImplementedException OnErrorNotImplementedException}, * {@link io.reactivex.rxjava3.exceptions.MissingBackpressureException MissingBackpressureException}). */ diff --git a/src/main/java/io/reactivex/rxjava3/internal/disposables/EmptyDisposable.java b/src/main/java/io/reactivex/rxjava3/internal/disposables/EmptyDisposable.java index d08ce8d14a..51acc86c75 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/disposables/EmptyDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/disposables/EmptyDisposable.java @@ -95,7 +95,7 @@ public boolean offer(Object v1, Object v2) { @Nullable @Override - public Object poll() throws Exception { + public Object poll() { return null; // always empty } diff --git a/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java b/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java index 90928b64c6..8a6d4dd992 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/disposables/ListCompositeDisposable.java @@ -32,7 +32,7 @@ public ListCompositeDisposable() { public ListCompositeDisposable(Disposable... resources) { Objects.requireNonNull(resources, "resources is null"); - this.resources = new LinkedList(); + this.resources = new LinkedList<>(); for (Disposable d : resources) { Objects.requireNonNull(d, "Disposable item is null"); this.resources.add(d); @@ -41,7 +41,7 @@ public ListCompositeDisposable(Disposable... resources) { public ListCompositeDisposable(Iterable resources) { Objects.requireNonNull(resources, "resources is null"); - this.resources = new LinkedList(); + this.resources = new LinkedList<>(); for (Disposable d : resources) { Objects.requireNonNull(d, "Disposable item is null"); this.resources.add(d); @@ -79,7 +79,7 @@ public boolean add(Disposable d) { if (!disposed) { List set = resources; if (set == null) { - set = new LinkedList(); + set = new LinkedList<>(); resources = set; } set.add(d); @@ -98,7 +98,7 @@ public boolean addAll(Disposable... ds) { if (!disposed) { List set = resources; if (set == null) { - set = new LinkedList(); + set = new LinkedList<>(); resources = set; } for (Disposable d : ds) { @@ -171,7 +171,7 @@ void dispose(List set) { } catch (Throwable ex) { Exceptions.throwIfFatal(ex); if (errors == null) { - errors = new ArrayList(); + errors = new ArrayList<>(); } errors.add(ex); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java index 05d1997893..1fee43d07e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java +++ b/src/main/java/io/reactivex/rxjava3/internal/functions/Functions.java @@ -36,46 +36,46 @@ private Functions() { @NonNull public static Function toFunction(@NonNull BiFunction f) { - return new Array2Func(f); + return new Array2Func<>(f); } @NonNull public static Function toFunction(@NonNull Function3 f) { - return new Array3Func(f); + return new Array3Func<>(f); } @NonNull public static Function toFunction(@NonNull Function4 f) { - return new Array4Func(f); + return new Array4Func<>(f); } @NonNull public static Function toFunction(@NonNull Function5 f) { - return new Array5Func(f); + return new Array5Func<>(f); } @NonNull public static Function toFunction( @NonNull Function6 f) { - return new Array6Func(f); + return new Array6Func<>(f); } @NonNull public static Function toFunction( @NonNull Function7 f) { - return new Array7Func(f); + return new Array7Func<>(f); } @NonNull public static Function toFunction( @NonNull Function8 f) { - return new Array8Func(f); + return new Array8Func<>(f); } @NonNull public static Function toFunction( @NonNull Function9 f) { - return new Array9Func(f); + return new Array9Func<>(f); } /** A singleton identity function. */ @@ -186,17 +186,17 @@ static final class JustValue implements Callable, Supplier, Function } @Override - public U call() throws Exception { + public U call() { return value; } @Override - public U apply(T t) throws Exception { + public U apply(T t) { return value; } @Override - public U get() throws Throwable { + public U get() { return value; } } @@ -209,7 +209,7 @@ public U get() throws Throwable { */ @NonNull public static Callable justCallable(@NonNull T value) { - return new JustValue(value); + return new JustValue<>(value); } /** @@ -220,7 +220,7 @@ public static Callable justCallable(@NonNull T value) { */ @NonNull public static Supplier justSupplier(@NonNull T value) { - return new JustValue(value); + return new JustValue<>(value); } /** @@ -232,7 +232,7 @@ public static Supplier justSupplier(@NonNull T value) { */ @NonNull public static Function justFunction(@NonNull U value) { - return new JustValue(value); + return new JustValue<>(value); } static final class CastToClass implements Function { @@ -243,7 +243,7 @@ static final class CastToClass implements Function { } @Override - public U apply(T t) throws Exception { + public U apply(T t) { return clazz.cast(t); } } @@ -257,7 +257,7 @@ public U apply(T t) throws Exception { */ @NonNull public static Function castFunction(@NonNull Class target) { - return new CastToClass(target); + return new CastToClass<>(target); } static final class ArrayListCapacityCallable implements Supplier> { @@ -268,13 +268,13 @@ static final class ArrayListCapacityCallable implements Supplier> { } @Override - public List get() throws Exception { - return new ArrayList(capacity); + public List get() { + return new ArrayList<>(capacity); } } public static Supplier> createArrayList(int capacity) { - return new ArrayListCapacityCallable(capacity); + return new ArrayListCapacityCallable<>(capacity); } static final class EqualsPredicate implements Predicate { @@ -285,25 +285,25 @@ static final class EqualsPredicate implements Predicate { } @Override - public boolean test(T t) throws Exception { + public boolean test(T t) { return Objects.equals(t, value); } } public static Predicate equalsWith(T value) { - return new EqualsPredicate(value); + return new EqualsPredicate<>(value); } enum HashSetCallable implements Supplier>, Callable> { INSTANCE; @Override - public Set call() throws Exception { - return new HashSet(); + public Set call() { + return new HashSet<>(); } @Override - public Set get() throws Throwable { - return new HashSet(); + public Set get() { + return new HashSet<>(); } } @@ -334,7 +334,7 @@ static final class NotificationOnError implements Consumer { @Override public void accept(Throwable v) throws Throwable { - onNotification.accept(Notification.createOnError(v)); + onNotification.accept(Notification.createOnError(v)); } } @@ -347,20 +347,20 @@ static final class NotificationOnComplete implements Action { @Override public void run() throws Throwable { - onNotification.accept(Notification.createOnComplete()); + onNotification.accept(Notification.createOnComplete()); } } public static Consumer notificationOnNext(Consumer> onNotification) { - return new NotificationOnNext(onNotification); + return new NotificationOnNext<>(onNotification); } public static Consumer notificationOnError(Consumer> onNotification) { - return new NotificationOnError(onNotification); + return new NotificationOnError<>(onNotification); } public static Action notificationOnComplete(Consumer> onNotification) { - return new NotificationOnComplete(onNotification); + return new NotificationOnComplete<>(onNotification); } static final class ActionConsumer implements Consumer { @@ -377,7 +377,7 @@ public void accept(T t) throws Throwable { } public static Consumer actionConsumer(Action action) { - return new ActionConsumer(action); + return new ActionConsumer<>(action); } static final class ClassFilter implements Predicate { @@ -388,13 +388,13 @@ static final class ClassFilter implements Predicate { } @Override - public boolean test(T t) throws Exception { + public boolean test(T t) { return clazz.isInstance(t); } } public static Predicate isInstanceOf(Class clazz) { - return new ClassFilter(clazz); + return new ClassFilter<>(clazz); } static final class BooleanSupplierPredicateReverse implements Predicate { @@ -411,7 +411,7 @@ public boolean test(T t) throws Throwable { } public static Predicate predicateReverseFor(BooleanSupplier supplier) { - return new BooleanSupplierPredicateReverse(supplier); + return new BooleanSupplierPredicateReverse<>(supplier); } static final class TimestampFunction implements Function> { @@ -425,13 +425,13 @@ static final class TimestampFunction implements Function> { } @Override - public Timed apply(T t) throws Exception { - return new Timed(t, scheduler.now(unit), unit); + public Timed apply(T t) { + return new Timed<>(t, scheduler.now(unit), unit); } } public static Function> timestampWith(TimeUnit unit, Scheduler scheduler) { - return new TimestampFunction(unit, scheduler); + return new TimestampFunction<>(unit, scheduler); } static final class ToMapKeySelector implements BiConsumer, T> { @@ -449,7 +449,7 @@ public void accept(Map m, T t) throws Throwable { } public static BiConsumer, T> toMapKeySelector(final Function keySelector) { - return new ToMapKeySelector(keySelector); + return new ToMapKeySelector<>(keySelector); } static final class ToMapKeyValueSelector implements BiConsumer, T> { @@ -471,7 +471,7 @@ public void accept(Map m, T t) throws Throwable { } public static BiConsumer, T> toMapKeyValueSelector(final Function keySelector, final Function valueSelector) { - return new ToMapKeyValueSelector(valueSelector, keySelector); + return new ToMapKeyValueSelector<>(valueSelector, keySelector); } static final class ToMultimapKeyValueSelector implements BiConsumer>, T> { @@ -506,7 +506,7 @@ public void accept(Map> m, T t) throws Throwable { public static BiConsumer>, T> toMultimapKeyValueSelector( final Function keySelector, final Function valueSelector, final Function> collectionFactory) { - return new ToMultimapKeyValueSelector(collectionFactory, valueSelector, keySelector); + return new ToMultimapKeyValueSelector<>(collectionFactory, valueSelector, keySelector); } enum NaturalComparator implements Comparator { @@ -539,7 +539,7 @@ public List apply(List v) { } public static Function, List> listSorter(final Comparator comparator) { - return new ListSorter(comparator); + return new ListSorter<>(comparator); } public static final Consumer REQUEST_MAX = new MaxRequestSubscription(); @@ -762,7 +762,7 @@ public Object call() { } @Override - public Object get() throws Throwable { + public Object get() { return null; } } @@ -777,7 +777,7 @@ public int compare(Object a, Object b) { static final class MaxRequestSubscription implements Consumer { @Override - public void accept(Subscription t) throws Exception { + public void accept(Subscription t) { t.request(Long.MAX_VALUE); } } @@ -796,7 +796,7 @@ public static class BoundedConsumer implements Consumer { } @Override - public void accept(Subscription s) throws Exception { + public void accept(Subscription s) { s.request(bufferSize); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java index 968e581625..210d920aad 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ObservableFlatMapStream.java @@ -112,7 +112,7 @@ public void onNext(@NonNull T t) { done = true; break; } - R value = Objects.requireNonNull(it.next(), "The Stream's Iterator.next retuned a null value"); + R value = Objects.requireNonNull(it.next(), "The Stream's Iterator.next returned a null value"); if (disposed) { done = true; break; diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java index e94f50cb67..c8802de3f6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/ParallelCollector.java @@ -209,7 +209,6 @@ public void onNext(T t) { Exceptions.throwIfFatal(ex); get().cancel(); onError(ex); - return; } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/observers/DeferredScalarDisposable.java b/src/main/java/io/reactivex/rxjava3/internal/observers/DeferredScalarDisposable.java index 8db129efb6..57d5f8c5c7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/observers/DeferredScalarDisposable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/observers/DeferredScalarDisposable.java @@ -114,7 +114,7 @@ public final void complete() { @Nullable @Override - public final T poll() throws Exception { + public final T poll() { if (get() == FUSED_READY) { T v = value; value = null; diff --git a/src/main/java/io/reactivex/rxjava3/internal/observers/FutureObserver.java b/src/main/java/io/reactivex/rxjava3/internal/observers/FutureObserver.java index f3d067f7ca..515e23e6bf 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/observers/FutureObserver.java +++ b/src/main/java/io/reactivex/rxjava3/internal/observers/FutureObserver.java @@ -19,6 +19,7 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.Observer; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; @@ -41,7 +42,7 @@ public final class FutureObserver extends CountDownLatch public FutureObserver() { super(1); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override @@ -90,7 +91,7 @@ public T get() throws InterruptedException, ExecutionException { } @Override - public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + public T get(long timeout, @NonNull TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { if (getCount() != 0) { BlockingHelper.verifyNonBlocking(); if (!await(timeout, unit)) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/observers/FutureSingleObserver.java b/src/main/java/io/reactivex/rxjava3/internal/observers/FutureSingleObserver.java index 80abb70419..3e55963bea 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/observers/FutureSingleObserver.java +++ b/src/main/java/io/reactivex/rxjava3/internal/observers/FutureSingleObserver.java @@ -18,6 +18,7 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.core.SingleObserver; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.internal.disposables.DisposableHelper; @@ -40,7 +41,7 @@ public final class FutureSingleObserver extends CountDownLatch public FutureSingleObserver() { super(1); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override @@ -89,7 +90,7 @@ public T get() throws InterruptedException, ExecutionException { } @Override - public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + public T get(long timeout, @NonNull TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { if (getCount() != 0) { BlockingHelper.verifyNonBlocking(); if (!await(timeout, unit)) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCache.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCache.java index 253e518dad..da002c0dd6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCache.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableCache.java @@ -39,7 +39,7 @@ public final class CompletableCache extends Completable implements CompletableOb public CompletableCache(CompletableSource source) { this.source = source; - this.observers = new AtomicReference(EMPTY); + this.observers = new AtomicReference<>(EMPTY); this.once = new AtomicBoolean(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcat.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcat.java index 9d55b626ea..c16b847c58 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcat.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableConcat.java @@ -106,9 +106,9 @@ public void onSubscribe(Subscription s) { } if (prefetch == Integer.MAX_VALUE) { - queue = new SpscLinkedArrayQueue(Flowable.bufferSize()); + queue = new SpscLinkedArrayQueue<>(Flowable.bufferSize()); } else { - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); } downstream.onSubscribe(this); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromObservable.java index d368a04244..d91133e502 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromObservable.java @@ -26,7 +26,7 @@ public CompletableFromObservable(ObservableSource observable) { @Override protected void subscribeActual(final CompletableObserver observer) { - observable.subscribe(new CompletableFromObservableObserver(observer)); + observable.subscribe(new CompletableFromObservableObserver<>(observer)); } static final class CompletableFromObservableObserver implements Observer { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromPublisher.java index 0e65777e3f..6baeb7a87e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromPublisher.java @@ -29,7 +29,7 @@ public CompletableFromPublisher(Publisher flowable) { @Override protected void subscribeActual(final CompletableObserver downstream) { - flowable.subscribe(new FromPublisherSubscriber(downstream)); + flowable.subscribe(new FromPublisherSubscriber<>(downstream)); } static final class FromPublisherSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSingle.java index adee6c86f0..b035f142d9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableFromSingle.java @@ -26,7 +26,7 @@ public CompletableFromSingle(SingleSource single) { @Override protected void subscribeActual(final CompletableObserver observer) { - single.subscribe(new CompletableFromSingleObserver(observer)); + single.subscribe(new CompletableFromSingleObserver<>(observer)); } static final class CompletableFromSingleObserver implements SingleObserver { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterialize.java index 129084898a..15dd320cad 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableMaterialize.java @@ -34,6 +34,6 @@ public CompletableMaterialize(Completable source) { @Override protected void subscribeActual(SingleObserver> observer) { - source.subscribe(new MaterializeSingleObserver(observer)); + source.subscribe(new MaterializeSingleObserver<>(observer)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToFlowable.java index ccbafd06b5..9ad0d0a8d8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToFlowable.java @@ -28,7 +28,7 @@ public CompletableToFlowable(CompletableSource source) { @Override protected void subscribeActual(Subscriber s) { - SubscriberCompletableObserver os = new SubscriberCompletableObserver(s); + SubscriberCompletableObserver os = new SubscriberCompletableObserver<>(s); source.subscribe(os); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservable.java index 91ce597c3b..095a975dbf 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableToObservable.java @@ -71,7 +71,7 @@ public int requestFusion(int mode) { } @Override - public Void poll() throws Exception { + public Void poll() { return null; // always empty } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java index 7a4e99b332..3476a65b25 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/completable/CompletableUsing.java @@ -80,7 +80,7 @@ protected void subscribeActual(CompletableObserver observer) { return; } - source.subscribe(new UsingObserver(observer, resource, disposer, eager)); + source.subscribe(new UsingObserver<>(observer, resource, disposer, eager)); } static final class UsingObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableIterable.java index 151541f63f..51015ce918 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableIterable.java @@ -38,7 +38,7 @@ public BlockingFlowableIterable(Flowable source, int bufferSize) { @Override public Iterator iterator() { - BlockingFlowableIterator it = new BlockingFlowableIterator(bufferSize); + BlockingFlowableIterator it = new BlockingFlowableIterator<>(bufferSize); source.subscribe(it); return it; } @@ -65,7 +65,7 @@ static final class BlockingFlowableIterator volatile Throwable error; BlockingFlowableIterator(int batchSize) { - this.queue = new SpscArrayQueue(batchSize); + this.queue = new SpscArrayQueue<>(batchSize); this.batchSize = batchSize; this.limit = batchSize - (batchSize >> 2); this.lock = new ReentrantLock(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableLatest.java index 91fce3e427..1cc469a1b9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableLatest.java @@ -39,7 +39,7 @@ public BlockingFlowableLatest(Publisher source) { @Override public Iterator iterator() { - LatestSubscriberIterator lio = new LatestSubscriberIterator(); + LatestSubscriberIterator lio = new LatestSubscriberIterator<>(); Flowable.fromPublisher(source).materialize().subscribe(lio); return lio; } @@ -48,7 +48,7 @@ public Iterator iterator() { static final class LatestSubscriberIterator extends DisposableSubscriber> implements Iterator { final Semaphore notify = new Semaphore(0); // observer's notification - final AtomicReference> value = new AtomicReference>(); + final AtomicReference> value = new AtomicReference<>(); // iterator's notification Notification iteratorNotification; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableMostRecent.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableMostRecent.java index f8fae1d413..bfd206eea9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableMostRecent.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableMostRecent.java @@ -40,7 +40,7 @@ public BlockingFlowableMostRecent(Flowable source, T initialValue) { @Override public Iterator iterator() { - MostRecentSubscriber mostRecentSubscriber = new MostRecentSubscriber(initialValue); + MostRecentSubscriber mostRecentSubscriber = new MostRecentSubscriber<>(initialValue); source.subscribe(mostRecentSubscriber); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNext.java index 4db920bb1f..57ca264f0c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/BlockingFlowableNext.java @@ -41,8 +41,8 @@ public BlockingFlowableNext(Publisher source) { @Override public Iterator iterator() { - NextSubscriber nextSubscriber = new NextSubscriber(); - return new NextIterator(source, nextSubscriber); + NextSubscriber nextSubscriber = new NextSubscriber<>(); + return new NextIterator<>(source, nextSubscriber); } // test needs to access the observer.waiting flag @@ -133,7 +133,7 @@ public void remove() { } static final class NextSubscriber extends DisposableSubscriber> { - private final BlockingQueue> buf = new ArrayBlockingQueue>(1); + private final BlockingQueue> buf = new ArrayBlockingQueue<>(1); final AtomicInteger waiting = new AtomicInteger(); @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAll.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAll.java index 0933703803..84ea50cbf7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAll.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAll.java @@ -31,7 +31,7 @@ public FlowableAll(Flowable source, Predicate predicate) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new AllSubscriber(s, predicate)); + source.subscribe(new AllSubscriber<>(s, predicate)); } static final class AllSubscriber extends DeferredScalarSubscription implements FlowableSubscriber { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllSingle.java index 516d37bf7c..8d1bf998ef 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAllSingle.java @@ -35,12 +35,12 @@ public FlowableAllSingle(Flowable source, Predicate predicate) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new AllSubscriber(observer, predicate)); + source.subscribe(new AllSubscriber<>(observer, predicate)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableAll(source, predicate)); + return RxJavaPlugins.onAssembly(new FlowableAll<>(source, predicate)); } static final class AllSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmb.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmb.java index 546beb0fae..ff650b5719 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmb.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAmb.java @@ -69,7 +69,7 @@ public void subscribeActual(Subscriber s) { return; } - AmbCoordinator ac = new AmbCoordinator(s, count); + AmbCoordinator ac = new AmbCoordinator<>(s, count); ac.subscribe(sources); } @@ -89,7 +89,7 @@ public void subscribe(Publisher[] sources) { AmbInnerSubscriber[] as = subscribers; int len = as.length; for (int i = 0; i < len; i++) { - as[i] = new AmbInnerSubscriber(this, i + 1, downstream); + as[i] = new AmbInnerSubscriber<>(this, i + 1, downstream); } winner.lazySet(0); // release the contents of 'as' downstream.onSubscribe(this); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAny.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAny.java index 401d444cca..d0adcb6dbe 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAny.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAny.java @@ -29,7 +29,7 @@ public FlowableAny(Flowable source, Predicate predicate) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new AnySubscriber(s, predicate)); + source.subscribe(new AnySubscriber<>(s, predicate)); } static final class AnySubscriber extends DeferredScalarSubscription implements FlowableSubscriber { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnySingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnySingle.java index 6a9e4aacbb..7ca8a6b428 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnySingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableAnySingle.java @@ -34,12 +34,12 @@ public FlowableAnySingle(Flowable source, Predicate predicate) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new AnySubscriber(observer, predicate)); + source.subscribe(new AnySubscriber<>(observer, predicate)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableAny(source, predicate)); + return RxJavaPlugins.onAssembly(new FlowableAny<>(source, predicate)); } static final class AnySubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java index 14654d35ee..9caafbc8e5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBlockingSubscribe.java @@ -42,9 +42,9 @@ private FlowableBlockingSubscribe() { * @param the value type */ public static void subscribe(Publisher o, Subscriber subscriber) { - final BlockingQueue queue = new LinkedBlockingQueue(); + final BlockingQueue queue = new LinkedBlockingQueue<>(); - BlockingSubscriber bs = new BlockingSubscriber(queue); + BlockingSubscriber bs = new BlockingSubscriber<>(queue); o.subscribe(bs); @@ -82,7 +82,7 @@ public static void subscribe(Publisher o, Subscriber */ public static void subscribe(Publisher o) { BlockingIgnoringReceiver callback = new BlockingIgnoringReceiver(); - LambdaSubscriber ls = new LambdaSubscriber(Functions.emptyConsumer(), + LambdaSubscriber ls = new LambdaSubscriber<>(Functions.emptyConsumer(), callback, callback, Functions.REQUEST_MAX); o.subscribe(ls); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java index 08f89e99e2..04f5ae182e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBuffer.java @@ -42,11 +42,11 @@ public FlowableBuffer(Flowable source, int size, int skip, Supplier buffer @Override public void subscribeActual(Subscriber s) { if (size == skip) { - source.subscribe(new PublisherBufferExactSubscriber(s, size, bufferSupplier)); + source.subscribe(new PublisherBufferExactSubscriber<>(s, size, bufferSupplier)); } else if (skip > size) { - source.subscribe(new PublisherBufferSkipSubscriber(s, size, skip, bufferSupplier)); + source.subscribe(new PublisherBufferSkipSubscriber<>(s, size, skip, bufferSupplier)); } else { - source.subscribe(new PublisherBufferOverlappingSubscriber(s, size, skip, bufferSupplier)); + source.subscribe(new PublisherBufferOverlappingSubscriber<>(s, size, skip, bufferSupplier)); } } @@ -318,7 +318,7 @@ static final class PublisherBufferOverlappingSubscriber(); + this.buffers = new ArrayDeque<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java index 47f6e6bf06..251144a1d2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferBoundary.java @@ -44,7 +44,7 @@ public FlowableBufferBoundary(Flowable source, Publisher buff @Override protected void subscribeActual(Subscriber s) { BufferBoundarySubscriber parent = - new BufferBoundarySubscriber( + new BufferBoundarySubscriber<>( s, bufferOpen, bufferClose, bufferSupplier ); s.onSubscribe(parent); @@ -93,11 +93,11 @@ static final class BufferBoundarySubscriber, this.bufferSupplier = bufferSupplier; this.bufferOpen = bufferOpen; this.bufferClose = bufferClose; - this.queue = new SpscLinkedArrayQueue(bufferSize()); + this.queue = new SpscLinkedArrayQueue<>(bufferSize()); this.subscribers = new CompositeDisposable(); this.requested = new AtomicLong(); - this.upstream = new AtomicReference(); - this.buffers = new LinkedHashMap(); + this.upstream = new AtomicReference<>(); + this.buffers = new LinkedHashMap<>(); this.errors = new AtomicThrowable(); } @@ -105,7 +105,7 @@ static final class BufferBoundarySubscriber, public void onSubscribe(Subscription s) { if (SubscriptionHelper.setOnce(this.upstream, s)) { - BufferOpenSubscriber open = new BufferOpenSubscriber(this); + BufferOpenSubscriber open = new BufferOpenSubscriber<>(this); subscribers.add(open); bufferOpen.subscribe(open); @@ -199,7 +199,7 @@ void open(Open token) { bufs.put(idx, buf); } - BufferCloseSubscriber bc = new BufferCloseSubscriber(this, idx); + BufferCloseSubscriber bc = new BufferCloseSubscriber<>(this, idx); subscribers.add(bc); p.subscribe(bc); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java index a1ca188b13..8d23e3c7bc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferExactBoundary.java @@ -41,7 +41,7 @@ public FlowableBufferExactBoundary(Flowable source, Publisher boundary, Su @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new BufferExactBoundarySubscriber(new SerializedSubscriber(s), bufferSupplier, boundary)); + source.subscribe(new BufferExactBoundarySubscriber<>(new SerializedSubscriber<>(s), bufferSupplier, boundary)); } static final class BufferExactBoundarySubscriber, B> @@ -58,7 +58,7 @@ static final class BufferExactBoundarySubscriber actual, Supplier bufferSupplier, Publisher boundary) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.boundary = boundary; } @@ -84,7 +84,7 @@ public void onSubscribe(Subscription s) { buffer = b; - BufferBoundarySubscriber bs = new BufferBoundarySubscriber(this); + BufferBoundarySubscriber bs = new BufferBoundarySubscriber<>(this); other = bs; downstream.onSubscribe(this); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java index 0b3faed1c5..854e0460ed 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableBufferTimed.java @@ -56,16 +56,16 @@ public FlowableBufferTimed(Flowable source, long timespan, long timeskip, Tim @Override protected void subscribeActual(Subscriber s) { if (timespan == timeskip && maxSize == Integer.MAX_VALUE) { - source.subscribe(new BufferExactUnboundedSubscriber( - new SerializedSubscriber(s), + source.subscribe(new BufferExactUnboundedSubscriber<>( + new SerializedSubscriber<>(s), bufferSupplier, timespan, unit, scheduler)); return; } Scheduler.Worker w = scheduler.createWorker(); if (timespan == timeskip) { - source.subscribe(new BufferExactBoundedSubscriber( - new SerializedSubscriber(s), + source.subscribe(new BufferExactBoundedSubscriber<>( + new SerializedSubscriber<>(s), bufferSupplier, timespan, unit, maxSize, restartTimerOnMaxSize, w )); @@ -73,8 +73,8 @@ protected void subscribeActual(Subscriber s) { } // Can't use maxSize because what to do if a buffer is full but its // timespan hasn't been elapsed? - source.subscribe(new BufferSkipBoundedSubscriber( - new SerializedSubscriber(s), + source.subscribe(new BufferSkipBoundedSubscriber<>( + new SerializedSubscriber<>(s), bufferSupplier, timespan, timeskip, unit, w)); } @@ -89,12 +89,12 @@ static final class BufferExactUnboundedSubscriber timer = new AtomicReference(); + final AtomicReference timer = new AtomicReference<>(); BufferExactUnboundedSubscriber( Subscriber actual, Supplier bufferSupplier, long timespan, TimeUnit unit, Scheduler scheduler) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.timespan = timespan; this.unit = unit; @@ -238,13 +238,13 @@ static final class BufferSkipBoundedSubscriber actual, Supplier bufferSupplier, long timespan, long timeskip, TimeUnit unit, Worker w) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.timespan = timespan; this.timeskip = timeskip; this.unit = unit; this.w = w; - this.buffers = new LinkedList(); + this.buffers = new LinkedList<>(); } @Override @@ -298,7 +298,7 @@ public void onError(Throwable t) { public void onComplete() { List bs; synchronized (this) { - bs = new ArrayList(buffers); + bs = new ArrayList<>(buffers); buffers.clear(); } @@ -404,7 +404,7 @@ static final class BufferExactBoundedSubscriber bufferSupplier, long timespan, TimeUnit unit, int maxSize, boolean restartOnMaxSize, Worker w) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.timespan = timespan; this.unit = unit; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCache.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCache.java index aea0b3150d..25d5fb71a3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCache.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCache.java @@ -99,15 +99,15 @@ public FlowableCache(Flowable source, int capacityHint) { super(source); this.capacityHint = capacityHint; this.once = new AtomicBoolean(); - Node n = new Node(capacityHint); + Node n = new Node<>(capacityHint); this.head = n; this.tail = n; - this.subscribers = new AtomicReference[]>(EMPTY); + this.subscribers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(Subscriber t) { - CacheSubscription consumer = new CacheSubscription(t, this); + CacheSubscription consumer = new CacheSubscription<>(t, this); t.onSubscribe(consumer); add(consumer); @@ -303,7 +303,7 @@ public void onNext(T t) { int tailOffset = this.tailOffset; // if the current tail node is full, create a fresh node if (tailOffset == capacityHint) { - Node n = new Node(tailOffset); + Node n = new Node<>(tailOffset); n.values[0] = t; this.tailOffset = 1; tail.next = n; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java index c2560cf6e3..b8751a23f9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollect.java @@ -44,7 +44,7 @@ protected void subscribeActual(Subscriber s) { return; } - source.subscribe(new CollectSubscriber(s, u, collector)); + source.subscribe(new CollectSubscriber<>(s, u, collector)); } static final class CollectSubscriber extends DeferredScalarSubscription implements FlowableSubscriber { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java index edf17d95ef..f159911fba 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCollectSingle.java @@ -49,12 +49,12 @@ protected void subscribeActual(SingleObserver observer) { return; } - source.subscribe(new CollectSubscriber(observer, u, collector)); + source.subscribe(new CollectSubscriber<>(observer, u, collector)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableCollect(source, initialSupplier, collector)); + return RxJavaPlugins.onAssembly(new FlowableCollect<>(source, initialSupplier, collector)); } static final class CollectSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java index 24306072bc..392de62d22 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCombineLatest.java @@ -132,12 +132,12 @@ public void subscribeActual(Subscriber s) { return; } if (n == 1) { - ((Publisher)a[0]).subscribe(new MapSubscriber(s, new SingletonArrayFunc())); + a[0].subscribe(new MapSubscriber<>(s, new SingletonArrayFunc())); return; } CombineLatestCoordinator coordinator = - new CombineLatestCoordinator(s, combiner, n, bufferSize, delayErrors); + new CombineLatestCoordinator<>(s, combiner, n, bufferSize, delayErrors); s.onSubscribe(coordinator); @@ -183,13 +183,13 @@ static final class CombineLatestCoordinator @SuppressWarnings("unchecked") CombineLatestInnerSubscriber[] a = new CombineLatestInnerSubscriber[n]; for (int i = 0; i < n; i++) { - a[i] = new CombineLatestInnerSubscriber(this, i, bufferSize); + a[i] = new CombineLatestInnerSubscriber<>(this, i, bufferSize); } this.subscribers = a; this.latest = new Object[n]; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); this.requested = new AtomicLong(); - this.error = new AtomicReference(); + this.error = new AtomicReference<>(); this.delayErrors = delayErrors; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatArray.java index ad1651b261..81834bba61 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatArray.java @@ -34,7 +34,7 @@ public FlowableConcatArray(Publisher[] sources, boolean delayError) @Override protected void subscribeActual(Subscriber s) { - ConcatArraySubscriber parent = new ConcatArraySubscriber(sources, delayError, s); + ConcatArraySubscriber parent = new ConcatArraySubscriber<>(sources, delayError, s); s.onSubscribe(parent); parent.onComplete(); @@ -82,7 +82,7 @@ public void onError(Throwable t) { if (delayError) { List list = errors; if (list == null) { - list = new ArrayList(sources.length - index + 1); + list = new ArrayList<>(sources.length - index + 1); errors = list; } list.add(t); @@ -121,7 +121,7 @@ public void onComplete() { if (delayError) { List list = errors; if (list == null) { - list = new ArrayList(n - i + 1); + list = new ArrayList<>(n - i + 1); errors = list; } list.add(ex); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java index f757fb5f3b..98d8ea4410 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMap.java @@ -46,11 +46,11 @@ public static Subscriber subscribe(Subscriber s, Function(s, mapper, prefetch, false); + return new ConcatMapDelayed<>(s, mapper, prefetch, false); case END: - return new ConcatMapDelayed(s, mapper, prefetch, true); + return new ConcatMapDelayed<>(s, mapper, prefetch, true); default: - return new ConcatMapImmediate(s, mapper, prefetch); + return new ConcatMapImmediate<>(s, mapper, prefetch); } } @@ -100,7 +100,7 @@ abstract static class BaseConcatMapSubscriber this.mapper = mapper; this.prefetch = prefetch; this.limit = prefetch - (prefetch >> 2); - this.inner = new ConcatMapInner(this); + this.inner = new ConcatMapInner<>(this); this.errors = new AtomicThrowable(); } @@ -133,7 +133,7 @@ public final void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); subscribeActual(); @@ -328,7 +328,7 @@ void drain() { continue; } else { active = true; - inner.setSubscription(new WeakScalarSubscription(vr, inner)); + inner.setSubscription(new WeakScalarSubscription<>(vr, inner)); } } else { @@ -527,7 +527,7 @@ void drain() { continue; } else { active = true; - inner.setSubscription(new WeakScalarSubscription(vr, inner)); + inner.setSubscription(new WeakScalarSubscription<>(vr, inner)); } } else { active = true; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java index 9aee3eaebb..ea39af45a2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEager.java @@ -51,7 +51,7 @@ public FlowableConcatMapEager(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ConcatMapEagerDelayErrorSubscriber( + source.subscribe(new ConcatMapEagerDelayErrorSubscriber<>( s, mapper, maxConcurrency, prefetch, errorMode)); } @@ -93,7 +93,7 @@ static final class ConcatMapEagerDelayErrorSubscriber this.maxConcurrency = maxConcurrency; this.prefetch = prefetch; this.errorMode = errorMode; - this.subscribers = new SpscLinkedArrayQueue>(Math.min(prefetch, maxConcurrency)); + this.subscribers = new SpscLinkedArrayQueue<>(Math.min(prefetch, maxConcurrency)); this.errors = new AtomicThrowable(); this.requested = new AtomicLong(); } @@ -123,7 +123,7 @@ public void onNext(T t) { return; } - InnerQueuedSubscriber inner = new InnerQueuedSubscriber(this, prefetch); + InnerQueuedSubscriber inner = new InnerQueuedSubscriber<>(this, prefetch); if (cancelled) { return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerPublisher.java index 8b1e7ec68b..36d465d190 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapEagerPublisher.java @@ -53,7 +53,7 @@ public FlowableConcatMapEagerPublisher(Publisher source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ConcatMapEagerDelayErrorSubscriber( + source.subscribe(new ConcatMapEagerDelayErrorSubscriber<>( s, mapper, maxConcurrency, prefetch, errorMode)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java index 05c984b533..d59677efa1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatMapScheduler.java @@ -50,13 +50,13 @@ public FlowableConcatMapScheduler(Flowable source, protected void subscribeActual(Subscriber s) { switch (errorMode) { case BOUNDARY: - source.subscribe(new ConcatMapDelayed(s, mapper, prefetch, false, scheduler.createWorker())); + source.subscribe(new ConcatMapDelayed<>(s, mapper, prefetch, false, scheduler.createWorker())); break; case END: - source.subscribe(new ConcatMapDelayed(s, mapper, prefetch, true, scheduler.createWorker())); + source.subscribe(new ConcatMapDelayed<>(s, mapper, prefetch, true, scheduler.createWorker())); break; default: - source.subscribe(new ConcatMapImmediate(s, mapper, prefetch, scheduler.createWorker())); + source.subscribe(new ConcatMapImmediate<>(s, mapper, prefetch, scheduler.createWorker())); } } @@ -98,7 +98,7 @@ abstract static class BaseConcatMapSubscriber this.mapper = mapper; this.prefetch = prefetch; this.limit = prefetch - (prefetch >> 2); - this.inner = new ConcatMapInner(this); + this.inner = new ConcatMapInner<>(this); this.errors = new AtomicThrowable(); this.worker = worker; } @@ -132,7 +132,7 @@ public final void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); subscribeActual(); @@ -341,7 +341,7 @@ public void run() { continue; } else { active = true; - inner.setSubscription(new WeakScalarSubscription(vr, inner)); + inner.setSubscription(new WeakScalarSubscription<>(vr, inner)); } } else { @@ -524,7 +524,7 @@ public void run() { continue; } else { active = true; - inner.setSubscription(new WeakScalarSubscription(vr, inner)); + inner.setSubscription(new WeakScalarSubscription<>(vr, inner)); } } else { active = true; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletable.java index 16556a5892..31729e450f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithCompletable.java @@ -40,7 +40,7 @@ public FlowableConcatWithCompletable(Flowable source, CompletableSource other @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ConcatWithSubscriber(s, other)); + source.subscribe(new ConcatWithSubscriber<>(s, other)); } static final class ConcatWithSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybe.java index 92c65809c7..b7db7fd093 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithMaybe.java @@ -41,7 +41,7 @@ public FlowableConcatWithMaybe(Flowable source, MaybeSource othe @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ConcatWithSubscriber(s, other)); + source.subscribe(new ConcatWithSubscriber<>(s, other)); } static final class ConcatWithSubscriber @@ -59,7 +59,7 @@ static final class ConcatWithSubscriber ConcatWithSubscriber(Subscriber actual, MaybeSource other) { super(actual); this.other = other; - this.otherDisposable = new AtomicReference(); + this.otherDisposable = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingle.java index b1b65d9dbf..c93ee51ba8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableConcatWithSingle.java @@ -41,7 +41,7 @@ public FlowableConcatWithSingle(Flowable source, SingleSource ot @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ConcatWithSubscriber(s, other)); + source.subscribe(new ConcatWithSubscriber<>(s, other)); } static final class ConcatWithSubscriber @@ -57,7 +57,7 @@ static final class ConcatWithSubscriber ConcatWithSubscriber(Subscriber actual, SingleSource other) { super(actual); this.other = other; - this.otherDisposable = new AtomicReference(); + this.otherDisposable = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCountSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCountSingle.java index 517d026e3f..928f36209c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCountSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCountSingle.java @@ -36,7 +36,7 @@ protected void subscribeActual(SingleObserver observer) { @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableCount(source)); + return RxJavaPlugins.onAssembly(new FlowableCount<>(source)); } static final class CountSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreate.java index 66cbd7409b..b7091c31f9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableCreate.java @@ -45,23 +45,23 @@ public void subscribeActual(Subscriber t) { switch (backpressure) { case MISSING: { - emitter = new MissingEmitter(t); + emitter = new MissingEmitter<>(t); break; } case ERROR: { - emitter = new ErrorAsyncEmitter(t); + emitter = new ErrorAsyncEmitter<>(t); break; } case DROP: { - emitter = new DropAsyncEmitter(t); + emitter = new DropAsyncEmitter<>(t); break; } case LATEST: { - emitter = new LatestAsyncEmitter(t); + emitter = new LatestAsyncEmitter<>(t); break; } default: { - emitter = new BufferAsyncEmitter(t, bufferSize()); + emitter = new BufferAsyncEmitter<>(t, bufferSize()); break; } } @@ -97,7 +97,7 @@ static final class SerializedEmitter SerializedEmitter(BaseEmitter emitter) { this.emitter = emitter; this.errors = new AtomicThrowable(); - this.queue = new SpscLinkedArrayQueue(16); + this.queue = new SpscLinkedArrayQueue<>(16); } @Override @@ -347,7 +347,7 @@ public final long requested() { @Override public final FlowableEmitter serialize() { - return new SerializedEmitter(this); + return new SerializedEmitter<>(this); } @Override @@ -460,7 +460,7 @@ static final class BufferAsyncEmitter extends BaseEmitter { BufferAsyncEmitter(Subscriber actual, int capacityHint) { super(actual); - this.queue = new SpscLinkedArrayQueue(capacityHint); + this.queue = new SpscLinkedArrayQueue<>(capacityHint); this.wip = new AtomicInteger(); } @@ -598,7 +598,7 @@ static final class LatestAsyncEmitter extends BaseEmitter { LatestAsyncEmitter(Subscriber downstream) { super(downstream); - this.queue = new AtomicReference(); + this.queue = new AtomicReference<>(); this.wip = new AtomicInteger(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java index ab86164622..16c666b283 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounce.java @@ -38,7 +38,7 @@ public FlowableDebounce(Flowable source, Function s) { - source.subscribe(new DebounceSubscriber(new SerializedSubscriber(s), debounceSelector)); + source.subscribe(new DebounceSubscriber<>(new SerializedSubscriber<>(s), debounceSelector)); } static final class DebounceSubscriber extends AtomicLong @@ -50,7 +50,7 @@ static final class DebounceSubscriber extends AtomicLong Subscription upstream; - final AtomicReference debouncer = new AtomicReference(); + final AtomicReference debouncer = new AtomicReference<>(); volatile long index; @@ -96,7 +96,7 @@ public void onNext(T t) { return; } - DebounceInnerSubscriber dis = new DebounceInnerSubscriber(this, idx, t); + DebounceInnerSubscriber dis = new DebounceInnerSubscriber<>(this, idx, t); if (debouncer.compareAndSet(d, dis)) { p.subscribe(dis); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTimed.java index 582c04c6e5..82ae29d497 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDebounceTimed.java @@ -42,8 +42,8 @@ public FlowableDebounceTimed(Flowable source, long timeout, TimeUnit unit, Sc @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new DebounceTimedSubscriber( - new SerializedSubscriber(s), + source.subscribe(new DebounceTimedSubscriber<>( + new SerializedSubscriber<>(s), timeout, unit, scheduler.createWorker())); } @@ -93,7 +93,7 @@ public void onNext(T t) { d.dispose(); } - DebounceEmitter de = new DebounceEmitter(t, idx, this); + DebounceEmitter de = new DebounceEmitter<>(t, idx, this); timer = de; d = worker.schedule(de, timeout, unit); de.setResource(d); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelay.java index 18950363e2..e66acd39e9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelay.java @@ -42,12 +42,12 @@ protected void subscribeActual(Subscriber t) { if (delayError) { downstream = t; } else { - downstream = new SerializedSubscriber(t); + downstream = new SerializedSubscriber<>(t); } Scheduler.Worker w = scheduler.createWorker(); - source.subscribe(new DelaySubscriber(downstream, delay, unit, w, delayError)); + source.subscribe(new DelaySubscriber<>(downstream, delay, unit, w, delayError)); } static final class DelaySubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOther.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOther.java index 1ae7f8210e..c4e199c7e2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOther.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDelaySubscriptionOther.java @@ -37,7 +37,7 @@ public FlowableDelaySubscriptionOther(Publisher main, Publisher @Override public void subscribeActual(final Subscriber child) { - MainSubscriber parent = new MainSubscriber(child, main); + MainSubscriber parent = new MainSubscriber<>(child, main); child.onSubscribe(parent); other.subscribe(parent.other); } @@ -58,7 +58,7 @@ static final class MainSubscriber extends AtomicLong implements FlowableSubsc this.downstream = downstream; this.main = main; this.other = new OtherSubscriber(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } void next() { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java index d5e46e6d23..dc2de8e389 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDematerialize.java @@ -34,7 +34,7 @@ public FlowableDematerialize(Flowable source, Function subscriber) { - source.subscribe(new DematerializeSubscriber(subscriber, selector)); + source.subscribe(new DematerializeSubscriber<>(subscriber, selector)); } static final class DematerializeSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetach.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetach.java index e3f266094e..e03ee6094b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetach.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDetach.java @@ -27,7 +27,7 @@ public FlowableDetach(Flowable source) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new DetachSubscriber(s)); + source.subscribe(new DetachSubscriber<>(s)); } static final class DetachSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java index 1cd4be2458..bc62429611 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinct.java @@ -52,7 +52,7 @@ protected void subscribeActual(Subscriber subscriber) { return; } - source.subscribe(new DistinctSubscriber(subscriber, keySelector, collection)); + source.subscribe(new DistinctSubscriber<>(subscriber, keySelector, collection)); } static final class DistinctSubscriber extends BasicFuseableSubscriber { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctUntilChanged.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctUntilChanged.java index f3ee070429..6eb323625d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctUntilChanged.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDistinctUntilChanged.java @@ -37,9 +37,9 @@ public FlowableDistinctUntilChanged(Flowable source, Function k protected void subscribeActual(Subscriber s) { if (s instanceof ConditionalSubscriber) { ConditionalSubscriber cs = (ConditionalSubscriber) s; - source.subscribe(new DistinctUntilChangedConditionalSubscriber(cs, keySelector, comparer)); + source.subscribe(new DistinctUntilChangedConditionalSubscriber<>(cs, keySelector, comparer)); } else { - source.subscribe(new DistinctUntilChangedSubscriber(s, keySelector, comparer)); + source.subscribe(new DistinctUntilChangedSubscriber<>(s, keySelector, comparer)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNext.java index ed5d969328..7aa451e1f6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoAfterNext.java @@ -39,9 +39,9 @@ public FlowableDoAfterNext(Flowable source, Consumer onAfterNext) @Override protected void subscribeActual(Subscriber s) { if (s instanceof ConditionalSubscriber) { - source.subscribe(new DoAfterConditionalSubscriber((ConditionalSubscriber)s, onAfterNext)); + source.subscribe(new DoAfterConditionalSubscriber<>((ConditionalSubscriber) s, onAfterNext)); } else { - source.subscribe(new DoAfterSubscriber(s, onAfterNext)); + source.subscribe(new DoAfterSubscriber<>(s, onAfterNext)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinally.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinally.java index 8cc70f171c..9373819291 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinally.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoFinally.java @@ -41,9 +41,9 @@ public FlowableDoFinally(Flowable source, Action onFinally) { @Override protected void subscribeActual(Subscriber s) { if (s instanceof ConditionalSubscriber) { - source.subscribe(new DoFinallyConditionalSubscriber((ConditionalSubscriber)s, onFinally)); + source.subscribe(new DoFinallyConditionalSubscriber<>((ConditionalSubscriber) s, onFinally)); } else { - source.subscribe(new DoFinallySubscriber(s, onFinally)); + source.subscribe(new DoFinallySubscriber<>(s, onFinally)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java index 74d74f41cf..2e6d45d5e7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnEach.java @@ -44,10 +44,10 @@ public FlowableDoOnEach(Flowable source, Consumer onNext, @Override protected void subscribeActual(Subscriber s) { if (s instanceof ConditionalSubscriber) { - source.subscribe(new DoOnEachConditionalSubscriber( - (ConditionalSubscriber)s, onNext, onError, onComplete, onAfterTerminate)); + source.subscribe(new DoOnEachConditionalSubscriber<>( + (ConditionalSubscriber) s, onNext, onError, onComplete, onAfterTerminate)); } else { - source.subscribe(new DoOnEachSubscriber( + source.subscribe(new DoOnEachSubscriber<>( s, onNext, onError, onComplete, onAfterTerminate)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnLifecycle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnLifecycle.java index 687474a403..a75881d2ba 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnLifecycle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableDoOnLifecycle.java @@ -35,7 +35,7 @@ public FlowableDoOnLifecycle(Flowable source, Consumer @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SubscriptionLambdaSubscriber(s, onSubscribe, onRequest, onCancel)); + source.subscribe(new SubscriptionLambdaSubscriber<>(s, onSubscribe, onRequest, onCancel)); } static final class SubscriptionLambdaSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAt.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAt.java index 31c21e2552..91c9c0bee7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAt.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAt.java @@ -35,7 +35,7 @@ public FlowableElementAt(Flowable source, long index, T defaultValue, boolean @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ElementAtSubscriber(s, index, defaultValue, errorOnFewer)); + source.subscribe(new ElementAtSubscriber<>(s, index, defaultValue, errorOnFewer)); } static final class ElementAtSubscriber extends DeferredScalarSubscription implements FlowableSubscriber { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtMaybe.java index 67f4744b76..fc30483fd2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtMaybe.java @@ -33,12 +33,12 @@ public FlowableElementAtMaybe(Flowable source, long index) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new ElementAtSubscriber(observer, index)); + source.subscribe(new ElementAtSubscriber<>(observer, index)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableElementAt(source, index, null, false)); + return RxJavaPlugins.onAssembly(new FlowableElementAt<>(source, index, null, false)); } static final class ElementAtSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtSingle.java index 3260307745..b743b6a3b7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableElementAtSingle.java @@ -38,12 +38,12 @@ public FlowableElementAtSingle(Flowable source, long index, T defaultValue) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new ElementAtSubscriber(observer, index, defaultValue)); + source.subscribe(new ElementAtSubscriber<>(observer, index, defaultValue)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableElementAt(source, index, defaultValue, true)); + return RxJavaPlugins.onAssembly(new FlowableElementAt<>(source, index, defaultValue, true)); } static final class ElementAtSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilter.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilter.java index 52aac8e1f4..790881a160 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilter.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFilter.java @@ -31,10 +31,10 @@ public FlowableFilter(Flowable source, Predicate predicate) { @Override protected void subscribeActual(Subscriber s) { if (s instanceof ConditionalSubscriber) { - source.subscribe(new FilterConditionalSubscriber( - (ConditionalSubscriber)s, predicate)); + source.subscribe(new FilterConditionalSubscriber<>( + (ConditionalSubscriber) s, predicate)); } else { - source.subscribe(new FilterSubscriber(s, predicate)); + source.subscribe(new FilterSubscriber<>(s, predicate)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java index 1c14ea54af..ea6a4f76ed 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMap.java @@ -55,7 +55,7 @@ protected void subscribeActual(Subscriber s) { public static FlowableSubscriber subscribe(Subscriber s, Function> mapper, boolean delayErrors, int maxConcurrency, int bufferSize) { - return new MergeSubscriber(s, mapper, delayErrors, maxConcurrency, bufferSize); + return new MergeSubscriber<>(s, mapper, delayErrors, maxConcurrency, bufferSize); } static final class MergeSubscriber extends AtomicInteger implements FlowableSubscriber, Subscription { @@ -76,7 +76,7 @@ static final class MergeSubscriber extends AtomicInteger implements Flowab volatile boolean cancelled; - final AtomicReference[]> subscribers = new AtomicReference[]>(); + final AtomicReference[]> subscribers = new AtomicReference<>(); static final InnerSubscriber[] EMPTY = new InnerSubscriber[0]; @@ -157,7 +157,7 @@ public void onNext(T t) { } } } else { - InnerSubscriber inner = new InnerSubscriber(this, uniqueId++); + InnerSubscriber inner = new InnerSubscriber<>(this, uniqueId++); if (addInner(inner)) { p.subscribe(inner); } @@ -216,9 +216,9 @@ SimpleQueue getMainQueue() { SimplePlainQueue q = queue; if (q == null) { if (maxConcurrency == Integer.MAX_VALUE) { - q = new SpscLinkedArrayQueue(bufferSize); + q = new SpscLinkedArrayQueue<>(bufferSize); } else { - q = new SpscArrayQueue(maxConcurrency); + q = new SpscArrayQueue<>(maxConcurrency); } queue = q; } @@ -267,7 +267,7 @@ void tryEmitScalar(U value) { SimpleQueue getInnerQueue(InnerSubscriber inner) { SimpleQueue q = inner.queue; if (q == null) { - q = new SpscArrayQueue(bufferSize); + q = new SpscArrayQueue<>(bufferSize); inner.queue = q; } return q; @@ -298,7 +298,7 @@ void tryEmit(U value, InnerSubscriber inner) { } else { SimpleQueue q = inner.queue; if (q == null) { - q = new SpscArrayQueue(bufferSize); + q = new SpscArrayQueue<>(bufferSize); inner.queue = q; } if (!q.offer(value)) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java index 5c544add23..945e6a877e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletable.java @@ -50,7 +50,7 @@ public FlowableFlatMapCompletable(Flowable source, @Override protected void subscribeActual(Subscriber subscriber) { - source.subscribe(new FlatMapCompletableMainSubscriber(subscriber, mapper, delayErrors, maxConcurrency)); + source.subscribe(new FlatMapCompletableMainSubscriber<>(subscriber, mapper, delayErrors, maxConcurrency)); } static final class FlatMapCompletableMainSubscriber extends BasicIntQueueSubscription @@ -171,7 +171,7 @@ public void request(long n) { @Nullable @Override - public T poll() throws Exception { + public T poll() { return null; // always empty } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java index 94f0fa8653..00445c8018 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapCompletableCompletable.java @@ -53,12 +53,12 @@ public FlowableFlatMapCompletableCompletable(Flowable source, @Override protected void subscribeActual(CompletableObserver observer) { - source.subscribe(new FlatMapCompletableMainSubscriber(observer, mapper, delayErrors, maxConcurrency)); + source.subscribe(new FlatMapCompletableMainSubscriber<>(observer, mapper, delayErrors, maxConcurrency)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableFlatMapCompletable(source, mapper, delayErrors, maxConcurrency)); + return RxJavaPlugins.onAssembly(new FlowableFlatMapCompletable<>(source, mapper, delayErrors, maxConcurrency)); } static final class FlatMapCompletableMainSubscriber extends AtomicInteger diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java index 18c2e21f63..1036e73973 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapMaybe.java @@ -50,7 +50,7 @@ public FlowableFlatMapMaybe(Flowable source, Function s) { - source.subscribe(new FlatMapMaybeSubscriber(s, mapper, delayErrors, maxConcurrency)); + source.subscribe(new FlatMapMaybeSubscriber<>(s, mapper, delayErrors, maxConcurrency)); } static final class FlatMapMaybeSubscriber @@ -91,7 +91,7 @@ static final class FlatMapMaybeSubscriber this.set = new CompositeDisposable(); this.errors = new AtomicThrowable(); this.active = new AtomicInteger(1); - this.queue = new AtomicReference>(); + this.queue = new AtomicReference<>(); } @Override @@ -210,7 +210,7 @@ SpscLinkedArrayQueue getOrCreateQueue() { if (current != null) { return current; } - current = new SpscLinkedArrayQueue(Flowable.bufferSize()); + current = new SpscLinkedArrayQueue<>(Flowable.bufferSize()); if (queue.compareAndSet(null, current)) { return current; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java index 366b48be09..d0c9002669 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlatMapSingle.java @@ -50,7 +50,7 @@ public FlowableFlatMapSingle(Flowable source, Function s) { - source.subscribe(new FlatMapSingleSubscriber(s, mapper, delayErrors, maxConcurrency)); + source.subscribe(new FlatMapSingleSubscriber<>(s, mapper, delayErrors, maxConcurrency)); } static final class FlatMapSingleSubscriber @@ -91,7 +91,7 @@ static final class FlatMapSingleSubscriber this.set = new CompositeDisposable(); this.errors = new AtomicThrowable(); this.active = new AtomicInteger(1); - this.queue = new AtomicReference>(); + this.queue = new AtomicReference<>(); } @Override @@ -210,7 +210,7 @@ SpscLinkedArrayQueue getOrCreateQueue() { if (current != null) { return current; } - current = new SpscLinkedArrayQueue(Flowable.bufferSize()); + current = new SpscLinkedArrayQueue<>(Flowable.bufferSize()); if (queue.compareAndSet(null, current)) { return current; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java index b275337841..06f0b650e6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFlattenIterable.java @@ -76,7 +76,7 @@ public void subscribeActual(Subscriber s) { return; } - source.subscribe(new FlattenIterableSubscriber(s, mapper, prefetch)); + source.subscribe(new FlattenIterableSubscriber<>(s, mapper, prefetch)); } /** diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java index 83371ab447..9b9a4c66f4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromArray.java @@ -33,10 +33,10 @@ public FlowableFromArray(T[] array) { @Override public void subscribeActual(Subscriber s) { if (s instanceof ConditionalSubscriber) { - s.onSubscribe(new ArrayConditionalSubscription( + s.onSubscribe(new ArrayConditionalSubscription<>( (ConditionalSubscriber)s, array)); } else { - s.onSubscribe(new ArraySubscription(s, array)); + s.onSubscribe(new ArraySubscription<>(s, array)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java index 80afc65910..603f09abae 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromCallable.java @@ -32,7 +32,7 @@ public FlowableFromCallable(Callable callable) { @Override public void subscribeActual(Subscriber s) { - DeferredScalarSubscription deferred = new DeferredScalarSubscription(s); + DeferredScalarSubscription deferred = new DeferredScalarSubscription<>(s); s.onSubscribe(deferred); T t; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromFuture.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromFuture.java index 8356c92915..57dc02774f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromFuture.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromFuture.java @@ -35,7 +35,7 @@ public FlowableFromFuture(Future future, long timeout, TimeUnit uni @Override public void subscribeActual(Subscriber s) { - DeferredScalarSubscription deferred = new DeferredScalarSubscription(s); + DeferredScalarSubscription deferred = new DeferredScalarSubscription<>(s); s.onSubscribe(deferred); T v; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromObservable.java index e88fd35c73..105c6b6ee2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromObservable.java @@ -26,7 +26,7 @@ public FlowableFromObservable(Observable upstream) { @Override protected void subscribeActual(Subscriber s) { - upstream.subscribe(new SubscriberObserver(s)); + upstream.subscribe(new SubscriberObserver<>(s)); } static final class SubscriberObserver implements Observer, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java index 7be86d618b..84fbae7eec 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableFromSupplier.java @@ -38,7 +38,7 @@ public FlowableFromSupplier(Supplier supplier) { @Override public void subscribeActual(Subscriber s) { - DeferredScalarSubscription deferred = new DeferredScalarSubscription(s); + DeferredScalarSubscription deferred = new DeferredScalarSubscription<>(s); s.onSubscribe(deferred); T t; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGenerate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGenerate.java index 893ec19ff9..2feda7ad97 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGenerate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGenerate.java @@ -48,7 +48,7 @@ public void subscribeActual(Subscriber s) { return; } - s.onSubscribe(new GeneratorSubscription(s, generator, disposeState, state)); + s.onSubscribe(new GeneratorSubscription<>(s, generator, disposeState, state)); } static final class GeneratorSubscription diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupBy.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupBy.java index 85c7d08289..3cb84d4d6b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupBy.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupBy.java @@ -56,10 +56,10 @@ protected void subscribeActual(Subscriber> s) { try { if (mapFactory == null) { evictedGroups = null; - groups = new ConcurrentHashMap>(); + groups = new ConcurrentHashMap<>(); } else { - evictedGroups = new ConcurrentLinkedQueue>(); - Consumer evictionAction = (Consumer) new EvictionAction(evictedGroups); + evictedGroups = new ConcurrentLinkedQueue<>(); + Consumer evictionAction = (Consumer) new EvictionAction<>(evictedGroups); groups = (Map) mapFactory.apply(evictionAction); } } catch (Throwable e) { @@ -69,7 +69,7 @@ protected void subscribeActual(Subscriber> s) { return; } GroupBySubscriber subscriber = - new GroupBySubscriber(s, keySelector, valueSelector, bufferSize, delayError, groups, evictedGroups); + new GroupBySubscriber<>(s, keySelector, valueSelector, bufferSize, delayError, groups, evictedGroups); source.subscribe(subscriber); } @@ -327,8 +327,8 @@ static final class GroupedUnicast extends GroupedFlowable { final State state; public static GroupedUnicast createWith(K key, int bufferSize, GroupBySubscriber parent, boolean delayError) { - State state = new State(bufferSize, parent, key, delayError); - return new GroupedUnicast(key, state); + State state = new State<>(bufferSize, parent, key, delayError); + return new GroupedUnicast<>(key, state); } protected GroupedUnicast(K key, State state) { @@ -370,7 +370,7 @@ static final class State extends BasicIntQueueSubscription implements P final AtomicBoolean cancelled = new AtomicBoolean(); - final AtomicReference> actual = new AtomicReference>(); + final AtomicReference> actual = new AtomicReference<>(); boolean outputFused; int produced; @@ -383,7 +383,7 @@ static final class State extends BasicIntQueueSubscription implements P static final int ABANDONED_HAS_SUBSCRIBER = ABANDONED | HAS_SUBSCRIBER; State(int bufferSize, GroupBySubscriber parent, K key, boolean delayError) { - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); this.parent = parent; this.key = key; this.delayError = delayError; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java index ccd80c53d9..a950de7a96 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableGroupJoin.java @@ -59,7 +59,7 @@ public FlowableGroupJoin( protected void subscribeActual(Subscriber s) { GroupJoinSubscription parent = - new GroupJoinSubscription(s, leftEnd, rightEnd, resultSelector); + new GroupJoinSubscription<>(s, leftEnd, rightEnd, resultSelector); s.onSubscribe(parent); @@ -132,10 +132,10 @@ static final class GroupJoinSubscription this.downstream = actual; this.requested = new AtomicLong(); this.disposables = new CompositeDisposable(); - this.queue = new SpscLinkedArrayQueue(bufferSize()); - this.lefts = new LinkedHashMap>(); - this.rights = new LinkedHashMap(); - this.error = new AtomicReference(); + this.queue = new SpscLinkedArrayQueue<>(bufferSize()); + this.lefts = new LinkedHashMap<>(); + this.rights = new LinkedHashMap<>(); + this.error = new AtomicReference<>(); this.leftEnd = leftEnd; this.rightEnd = rightEnd; this.resultSelector = resultSelector; @@ -239,7 +239,7 @@ void drain() { @SuppressWarnings("unchecked") TLeft left = (TLeft)val; - UnicastProcessor up = UnicastProcessor.create(); + UnicastProcessor up = UnicastProcessor.create(); int idx = leftIndex++; lefts.put(idx, up); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableHide.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableHide.java index 512269f58f..17d84f178a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableHide.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableHide.java @@ -32,7 +32,7 @@ public FlowableHide(Flowable source) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new HideSubscriber(s)); + source.subscribe(new HideSubscriber<>(s)); } static final class HideSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElements.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElements.java index c30935c569..9afee2e50d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElements.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElements.java @@ -28,7 +28,7 @@ public FlowableIgnoreElements(Flowable source) { @Override protected void subscribeActual(final Subscriber t) { - source.subscribe(new IgnoreElementsSubscriber(t)); + source.subscribe(new IgnoreElementsSubscriber<>(t)); } static final class IgnoreElementsSubscriber implements FlowableSubscriber, QueueSubscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsCompletable.java index e68e5dd784..4eaced342e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIgnoreElementsCompletable.java @@ -31,12 +31,12 @@ public FlowableIgnoreElementsCompletable(Flowable source) { @Override protected void subscribeActual(final CompletableObserver t) { - source.subscribe(new IgnoreElementsSubscriber(t)); + source.subscribe(new IgnoreElementsSubscriber<>(t)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableIgnoreElements(source)); + return RxJavaPlugins.onAssembly(new FlowableIgnoreElements<>(source)); } static final class IgnoreElementsSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java index f8f97191fd..41dc08d920 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInternalHelper.java @@ -47,7 +47,7 @@ public S apply(S t1, Emitter t2) throws Throwable { } public static BiFunction, S> simpleGenerator(Consumer> consumer) { - return new SimpleGenerator(consumer); + return new SimpleGenerator<>(consumer); } static final class SimpleBiGenerator implements BiFunction, S> { @@ -65,7 +65,7 @@ public S apply(S t1, Emitter t2) throws Throwable { } public static BiFunction, S> simpleBiGenerator(BiConsumer> consumer) { - return new SimpleBiGenerator(consumer); + return new SimpleBiGenerator<>(consumer); } static final class ItemDelayFunction implements Function> { @@ -78,12 +78,12 @@ static final class ItemDelayFunction implements Function> @Override public Publisher apply(final T v) throws Throwable { Publisher p = Objects.requireNonNull(itemDelay.apply(v), "The itemDelay returned a null Publisher"); - return new FlowableTakePublisher(p, 1).map(Functions.justFunction(v)).defaultIfEmpty(v); + return new FlowableTakePublisher<>(p, 1).map(Functions.justFunction(v)).defaultIfEmpty(v); } } public static Function> itemDelay(final Function> itemDelay) { - return new ItemDelayFunction(itemDelay); + return new ItemDelayFunction<>(itemDelay); } static final class SubscriberOnNext implements Consumer { @@ -94,7 +94,7 @@ static final class SubscriberOnNext implements Consumer { } @Override - public void accept(T v) throws Exception { + public void accept(T v) { subscriber.onNext(v); } } @@ -107,7 +107,7 @@ static final class SubscriberOnError implements Consumer { } @Override - public void accept(Throwable v) throws Exception { + public void accept(Throwable v) { subscriber.onError(v); } } @@ -120,21 +120,21 @@ static final class SubscriberOnComplete implements Action { } @Override - public void run() throws Exception { + public void run() { subscriber.onComplete(); } } public static Consumer subscriberOnNext(Subscriber subscriber) { - return new SubscriberOnNext(subscriber); + return new SubscriberOnNext<>(subscriber); } public static Consumer subscriberOnError(Subscriber subscriber) { - return new SubscriberOnError(subscriber); + return new SubscriberOnError<>(subscriber); } public static Action subscriberOnComplete(Subscriber subscriber) { - return new SubscriberOnComplete(subscriber); + return new SubscriberOnComplete<>(subscriber); } static final class FlatMapWithCombinerInner implements Function { @@ -166,14 +166,14 @@ static final class FlatMapWithCombinerOuter implements Function apply(final T t) throws Throwable { @SuppressWarnings("unchecked") Publisher u = (Publisher)Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Publisher"); - return new FlowableMapPublisher(u, new FlatMapWithCombinerInner(combiner, t)); + return new FlowableMapPublisher<>(u, new FlatMapWithCombinerInner(combiner, t)); } } public static Function> flatMapWithCombiner( final Function> mapper, final BiFunction combiner) { - return new FlatMapWithCombinerOuter(combiner, mapper); + return new FlatMapWithCombinerOuter<>(combiner, mapper); } static final class FlatMapIntoIterable implements Function> { @@ -185,34 +185,34 @@ static final class FlatMapIntoIterable implements Function @Override public Publisher apply(T t) throws Throwable { - return new FlowableFromIterable(Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); + return new FlowableFromIterable<>(Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); } } public static Function> flatMapIntoIterable(final Function> mapper) { - return new FlatMapIntoIterable(mapper); + return new FlatMapIntoIterable<>(mapper); } public static Supplier> replaySupplier(final Flowable parent) { - return new ReplaySupplier(parent); + return new ReplaySupplier<>(parent); } public static Supplier> replaySupplier(final Flowable parent, final int bufferSize, boolean eagerTruncate) { - return new BufferedReplaySupplier(parent, bufferSize, eagerTruncate); + return new BufferedReplaySupplier<>(parent, bufferSize, eagerTruncate); } public static Supplier> replaySupplier(final Flowable parent, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - return new BufferedTimedReplay(parent, bufferSize, time, unit, scheduler, eagerTruncate); + return new BufferedTimedReplay<>(parent, bufferSize, time, unit, scheduler, eagerTruncate); } public static Supplier> replaySupplier(final Flowable parent, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - return new TimedReplay(parent, time, unit, scheduler, eagerTruncate); + return new TimedReplay<>(parent, time, unit, scheduler, eagerTruncate); } public enum RequestMax implements Consumer { INSTANCE; @Override - public void accept(Subscription t) throws Exception { + public void accept(Subscription t) { t.request(Long.MAX_VALUE); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInterval.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInterval.java index 44e735be37..9ef04ea11c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInterval.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableInterval.java @@ -66,7 +66,7 @@ static final class IntervalSubscriber extends AtomicLong long count; - final AtomicReference resource = new AtomicReference(); + final AtomicReference resource = new AtomicReference<>(); IntervalSubscriber(Subscriber downstream) { this.downstream = downstream; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalRange.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalRange.java index 83ff94ecaa..e9580f40c6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalRange.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableIntervalRange.java @@ -71,7 +71,7 @@ static final class IntervalRangeSubscriber extends AtomicLong long count; - final AtomicReference resource = new AtomicReference(); + final AtomicReference resource = new AtomicReference<>(); IntervalRangeSubscriber(Subscriber actual, long start, long end) { this.downstream = actual; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java index df8eff7049..01cf71023f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJoin.java @@ -56,7 +56,7 @@ public FlowableJoin( protected void subscribeActual(Subscriber s) { JoinSubscription parent = - new JoinSubscription(s, leftEnd, rightEnd, resultSelector); + new JoinSubscription<>(s, leftEnd, rightEnd, resultSelector); s.onSubscribe(parent); @@ -116,10 +116,10 @@ static final class JoinSubscription this.downstream = actual; this.requested = new AtomicLong(); this.disposables = new CompositeDisposable(); - this.queue = new SpscLinkedArrayQueue(bufferSize()); - this.lefts = new LinkedHashMap(); - this.rights = new LinkedHashMap(); - this.error = new AtomicReference(); + this.queue = new SpscLinkedArrayQueue<>(bufferSize()); + this.lefts = new LinkedHashMap<>(); + this.rights = new LinkedHashMap<>(); + this.error = new AtomicReference<>(); this.leftEnd = leftEnd; this.rightEnd = rightEnd; this.resultSelector = resultSelector; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJust.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJust.java index d498befefc..d17c9644fa 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJust.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableJust.java @@ -31,7 +31,7 @@ public FlowableJust(final T value) { @Override protected void subscribeActual(Subscriber s) { - s.onSubscribe(new ScalarSubscription(s, value)); + s.onSubscribe(new ScalarSubscription<>(s, value)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastMaybe.java index 1c6a46f71c..936e68c211 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastMaybe.java @@ -36,7 +36,7 @@ public FlowableLastMaybe(Publisher source) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new LastSubscriber(observer)); + source.subscribe(new LastSubscriber<>(observer)); } static final class LastSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastSingle.java index 7694067609..70346f3555 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableLastSingle.java @@ -42,7 +42,7 @@ public FlowableLastSingle(Publisher source, T defaultItem) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new LastSubscriber(observer, defaultItem)); + source.subscribe(new LastSubscriber<>(observer, defaultItem)); } static final class LastSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java index cdfb0b0bb5..b01d6dc5a0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMapNotification.java @@ -41,7 +41,7 @@ public FlowableMapNotification( @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new MapNotificationSubscriber(s, onNextMapper, onErrorMapper, onCompleteSupplier)); + source.subscribe(new MapNotificationSubscriber<>(s, onNextMapper, onErrorMapper, onCompleteSupplier)); } static final class MapNotificationSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterialize.java index e3e23c18e5..2b361427b1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMaterialize.java @@ -27,7 +27,7 @@ public FlowableMaterialize(Flowable source) { @Override protected void subscribeActual(Subscriber> s) { - source.subscribe(new MaterializeSubscriber(s)); + source.subscribe(new MaterializeSubscriber<>(s)); } static final class MaterializeSubscriber extends SinglePostCompleteSubscriber> { @@ -46,12 +46,12 @@ public void onNext(T t) { @Override public void onError(Throwable t) { - complete(Notification.createOnError(t)); + complete(Notification.createOnError(t)); } @Override public void onComplete() { - complete(Notification.createOnComplete()); + complete(Notification.createOnComplete()); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletable.java index efb978083c..a0725a1d64 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithCompletable.java @@ -41,7 +41,7 @@ public FlowableMergeWithCompletable(Flowable source, CompletableSource other) @Override protected void subscribeActual(Subscriber subscriber) { - MergeWithSubscriber parent = new MergeWithSubscriber(subscriber); + MergeWithSubscriber parent = new MergeWithSubscriber<>(subscriber); subscriber.onSubscribe(parent); source.subscribe(parent); other.subscribe(parent.otherObserver); @@ -68,7 +68,7 @@ static final class MergeWithSubscriber extends AtomicInteger MergeWithSubscriber(Subscriber downstream) { this.downstream = downstream; - this.mainSubscription = new AtomicReference(); + this.mainSubscription = new AtomicReference<>(); this.otherObserver = new OtherObserver(this); this.errors = new AtomicThrowable(); this.requested = new AtomicLong(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybe.java index 4ad7f92d07..7d3be91bea 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithMaybe.java @@ -43,7 +43,7 @@ public FlowableMergeWithMaybe(Flowable source, MaybeSource other @Override protected void subscribeActual(Subscriber subscriber) { - MergeWithObserver parent = new MergeWithObserver(subscriber); + MergeWithObserver parent = new MergeWithObserver<>(subscriber); subscriber.onSubscribe(parent); source.subscribe(parent); other.subscribe(parent.otherObserver); @@ -88,8 +88,8 @@ static final class MergeWithObserver extends AtomicInteger MergeWithObserver(Subscriber downstream) { this.downstream = downstream; - this.mainSubscription = new AtomicReference(); - this.otherObserver = new OtherObserver(this); + this.mainSubscription = new AtomicReference<>(); + this.otherObserver = new OtherObserver<>(this); this.errors = new AtomicThrowable(); this.requested = new AtomicLong(); this.prefetch = bufferSize(); @@ -211,7 +211,7 @@ void otherComplete() { SimplePlainQueue getOrCreateQueue() { SimplePlainQueue q = queue; if (q == null) { - q = new SpscArrayQueue(bufferSize()); + q = new SpscArrayQueue<>(bufferSize()); queue = q; } return q; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingle.java index 33fe6b8454..997ad3fa2e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableMergeWithSingle.java @@ -43,7 +43,7 @@ public FlowableMergeWithSingle(Flowable source, SingleSource oth @Override protected void subscribeActual(Subscriber subscriber) { - MergeWithObserver parent = new MergeWithObserver(subscriber); + MergeWithObserver parent = new MergeWithObserver<>(subscriber); subscriber.onSubscribe(parent); source.subscribe(parent); other.subscribe(parent.otherObserver); @@ -88,8 +88,8 @@ static final class MergeWithObserver extends AtomicInteger MergeWithObserver(Subscriber downstream) { this.downstream = downstream; - this.mainSubscription = new AtomicReference(); - this.otherObserver = new OtherObserver(this); + this.mainSubscription = new AtomicReference<>(); + this.otherObserver = new OtherObserver<>(this); this.errors = new AtomicThrowable(); this.requested = new AtomicLong(); this.prefetch = bufferSize(); @@ -206,7 +206,7 @@ void otherError(Throwable ex) { SimplePlainQueue getOrCreateQueue() { SimplePlainQueue q = queue; if (q == null) { - q = new SpscArrayQueue(bufferSize()); + q = new SpscArrayQueue<>(bufferSize()); queue = q; } return q; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOn.java index 66605c9520..3a4f237e5b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableObserveOn.java @@ -50,10 +50,10 @@ public void subscribeActual(Subscriber s) { Worker worker = scheduler.createWorker(); if (s instanceof ConditionalSubscriber) { - source.subscribe(new ObserveOnConditionalSubscriber( + source.subscribe(new ObserveOnConditionalSubscriber<>( (ConditionalSubscriber) s, worker, delayError, prefetch)); } else { - source.subscribe(new ObserveOnSubscriber(s, worker, delayError, prefetch)); + source.subscribe(new ObserveOnSubscriber<>(s, worker, delayError, prefetch)); } } @@ -289,7 +289,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); downstream.onSubscribe(this); @@ -533,7 +533,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); downstream.onSubscribe(this); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBuffer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBuffer.java index df04ac54db..19d28291ba 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBuffer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBuffer.java @@ -43,7 +43,7 @@ public FlowableOnBackpressureBuffer(Flowable source, int bufferSize, boolean @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new BackpressureBufferSubscriber(s, bufferSize, unbounded, delayError, onOverflow)); + source.subscribe(new BackpressureBufferSubscriber<>(s, bufferSize, unbounded, delayError, onOverflow)); } static final class BackpressureBufferSubscriber extends BasicIntQueueSubscription implements FlowableSubscriber { @@ -75,9 +75,9 @@ static final class BackpressureBufferSubscriber extends BasicIntQueueSubscrip SimplePlainQueue q; if (unbounded) { - q = new SpscLinkedArrayQueue(bufferSize); + q = new SpscLinkedArrayQueue<>(bufferSize); } else { - q = new SpscArrayQueue(bufferSize); + q = new SpscArrayQueue<>(bufferSize); } this.queue = q; @@ -255,7 +255,7 @@ public int requestFusion(int mode) { @Nullable @Override - public T poll() throws Exception { + public T poll() { return queue.poll(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategy.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategy.java index 70b65ab39a..6e458356b6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategy.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureBufferStrategy.java @@ -48,7 +48,7 @@ public FlowableOnBackpressureBufferStrategy(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new OnBackpressureBufferStrategySubscriber(s, onOverflow, strategy, bufferSize)); + source.subscribe(new OnBackpressureBufferStrategySubscriber<>(s, onOverflow, strategy, bufferSize)); } static final class OnBackpressureBufferStrategySubscriber @@ -83,7 +83,7 @@ static final class OnBackpressureBufferStrategySubscriber this.strategy = strategy; this.bufferSize = bufferSize; this.requested = new AtomicLong(); - this.deque = new ArrayDeque(); + this.deque = new ArrayDeque<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDrop.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDrop.java index 8f3f2d4946..0db3cd3cc2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDrop.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureDrop.java @@ -45,7 +45,7 @@ public void accept(T t) { @Override protected void subscribeActual(Subscriber s) { - this.source.subscribe(new BackpressureDropSubscriber(s, onDrop)); + this.source.subscribe(new BackpressureDropSubscriber<>(s, onDrop)); } static final class BackpressureDropSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureError.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureError.java index 10304a4b84..b85e1281fb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureError.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureError.java @@ -31,7 +31,7 @@ public FlowableOnBackpressureError(Flowable source) { @Override protected void subscribeActual(Subscriber s) { - this.source.subscribe(new BackpressureErrorSubscriber(s)); + this.source.subscribe(new BackpressureErrorSubscriber<>(s)); } static final class BackpressureErrorSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatest.java index b7ba9116a2..6123475ef0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnBackpressureLatest.java @@ -29,7 +29,7 @@ public FlowableOnBackpressureLatest(Flowable source) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new BackpressureLatestSubscriber(s)); + source.subscribe(new BackpressureLatestSubscriber<>(s)); } static final class BackpressureLatestSubscriber extends AtomicInteger implements FlowableSubscriber, Subscription { @@ -47,7 +47,7 @@ static final class BackpressureLatestSubscriber extends AtomicInteger impleme final AtomicLong requested = new AtomicLong(); - final AtomicReference current = new AtomicReference(); + final AtomicReference current = new AtomicReference<>(); BackpressureLatestSubscriber(Subscriber downstream) { this.downstream = downstream; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java index 1d939022b4..7c9de4a748 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorNext.java @@ -34,7 +34,7 @@ public FlowableOnErrorNext(Flowable source, @Override protected void subscribeActual(Subscriber s) { - OnErrorNextSubscriber parent = new OnErrorNextSubscriber(s, nextSupplier); + OnErrorNextSubscriber parent = new OnErrorNextSubscriber<>(s, nextSupplier); s.onSubscribe(parent); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java index 4a908cc113..4fe88985a4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableOnErrorReturn.java @@ -31,7 +31,7 @@ public FlowableOnErrorReturn(Flowable source, Function s) { - source.subscribe(new OnErrorReturnSubscriber(s, valueSupplier)); + source.subscribe(new OnErrorReturnSubscriber<>(s, valueSupplier)); } static final class OnErrorReturnSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublish.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublish.java index 452dc83cce..49e75fc1fd 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublish.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublish.java @@ -34,7 +34,7 @@ * completes or the connection is disposed. *

* The difference to FlowablePublish is that when the upstream terminates, - * late subscriberss will receive that terminal event until the connection is + * late subscribers will receive that terminal event until the connection is * disposed and the ConnectableFlowable is reset to its fresh state. * * @param the element type @@ -52,7 +52,7 @@ public final class FlowablePublish extends ConnectableFlowable public FlowablePublish(Publisher source, int bufferSize) { this.source = source; this.bufferSize = bufferSize; - this.current = new AtomicReference>(); + this.current = new AtomicReference<>(); } @Override @@ -61,8 +61,8 @@ public Publisher source() { } /** - * The internal buffer size of this FloawblePublishAlt operator. - * @return The internal buffer size of this FloawblePublishAlt operator. + * The internal buffer size of this FlowablePublishAlt operator. + * @return The internal buffer size of this FlowablePublishAlt operator. */ public int publishBufferSize() { return bufferSize; @@ -77,7 +77,7 @@ public void connect(Consumer connection) { conn = current.get(); if (conn == null || conn.isDisposed()) { - PublishConnection fresh = new PublishConnection(current, bufferSize); + PublishConnection fresh = new PublishConnection<>(current, bufferSize); if (!current.compareAndSet(conn, fresh)) { continue; } @@ -109,7 +109,7 @@ protected void subscribeActual(Subscriber s) { // don't create a fresh connection if the current is disposed if (conn == null) { - PublishConnection fresh = new PublishConnection(current, bufferSize); + PublishConnection fresh = new PublishConnection<>(current, bufferSize); if (!current.compareAndSet(conn, fresh)) { continue; } @@ -119,7 +119,7 @@ protected void subscribeActual(Subscriber s) { break; } - InnerSubscription inner = new InnerSubscription(s, conn); + InnerSubscription inner = new InnerSubscription<>(s, conn); s.onSubscribe(inner); if (conn.add(inner)) { @@ -180,10 +180,10 @@ static final class PublishConnection @SuppressWarnings("unchecked") PublishConnection(AtomicReference> current, int bufferSize) { this.current = current; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.connect = new AtomicBoolean(); this.bufferSize = bufferSize; - this.subscribers = new AtomicReference[]>(EMPTY); + this.subscribers = new AtomicReference<>(EMPTY); } @SuppressWarnings("unchecked") @@ -222,7 +222,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(bufferSize); + queue = new SpscArrayQueue<>(bufferSize); s.request(bufferSize); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java index 702955bf02..05e0e7a907 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowablePublishMulticast.java @@ -55,7 +55,7 @@ public FlowablePublishMulticast(Flowable source, @Override protected void subscribeActual(Subscriber s) { - MulticastProcessor mp = new MulticastProcessor(prefetch, delayError); + MulticastProcessor mp = new MulticastProcessor<>(prefetch, delayError); Publisher other; @@ -67,7 +67,7 @@ protected void subscribeActual(Subscriber s) { return; } - OutputCanceller out = new OutputCanceller(s, mp); + OutputCanceller out = new OutputCanceller<>(s, mp); other.subscribe(out); @@ -159,8 +159,8 @@ static final class MulticastProcessor extends Flowable implements Flowable this.limit = prefetch - (prefetch >> 2); // request after 75% consumption this.delayError = delayError; this.wip = new AtomicInteger(); - this.upstream = new AtomicReference(); - this.subscribers = new AtomicReference[]>(EMPTY); + this.upstream = new AtomicReference<>(); + this.subscribers = new AtomicReference<>(EMPTY); } @Override @@ -293,7 +293,7 @@ void remove(MulticastSubscription s) { @Override protected void subscribeActual(Subscriber s) { - MulticastSubscription ms = new MulticastSubscription(s, this); + MulticastSubscription ms = new MulticastSubscription<>(s, this); s.onSubscribe(ms); if (add(ms)) { if (ms.isCancelled()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java index 9d3d3f4794..2aa1d71db0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduce.java @@ -40,7 +40,7 @@ public FlowableReduce(Flowable source, BiFunction reducer) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ReduceSubscriber(s, reducer)); + source.subscribe(new ReduceSubscriber<>(s, reducer)); } static final class ReduceSubscriber extends DeferredScalarSubscription implements FlowableSubscriber { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java index 2de3b633fe..aded7967b8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceMaybe.java @@ -50,12 +50,12 @@ public Publisher source() { @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableReduce(source, reducer)); + return RxJavaPlugins.onAssembly(new FlowableReduce<>(source, reducer)); } @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new ReduceSubscriber(observer, reducer)); + source.subscribe(new ReduceSubscriber<>(observer, reducer)); } static final class ReduceSubscriber implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java index ec570639c6..715cb49c0a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceSeedSingle.java @@ -47,7 +47,7 @@ public FlowableReduceSeedSingle(Publisher source, R seed, BiFunction observer) { - source.subscribe(new ReduceSeedObserver(observer, reducer, seed)); + source.subscribe(new ReduceSeedObserver<>(observer, reducer, seed)); } static final class ReduceSeedObserver implements FlowableSubscriber, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java index 343d72c4e3..b2aaf836d7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReduceWithSingle.java @@ -55,6 +55,6 @@ protected void subscribeActual(SingleObserver observer) { EmptyDisposable.error(ex, observer); return; } - source.subscribe(new ReduceSeedObserver(observer, reducer, seed)); + source.subscribe(new ReduceSeedObserver<>(observer, reducer, seed)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCount.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCount.java index e8f7f895ba..1bce5a4dd7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCount.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRefCount.java @@ -84,7 +84,7 @@ protected void subscribeActual(Subscriber s) { } } - source.subscribe(new RefCountSubscriber(s, this, conn)); + source.subscribe(new RefCountSubscriber<>(s, this, conn)); if (connect) { source.connect(conn); @@ -168,7 +168,7 @@ public void run() { } @Override - public void accept(Disposable t) throws Exception { + public void accept(Disposable t) { DisposableHelper.replace(this, t); synchronized (parent) { if (disconnectedEarly) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeat.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeat.java index 092703b343..3acb03fb76 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeat.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeat.java @@ -32,7 +32,7 @@ public void subscribeActual(Subscriber s) { SubscriptionArbiter sa = new SubscriptionArbiter(false); s.onSubscribe(sa); - RepeatSubscriber rs = new RepeatSubscriber(s, count != Long.MAX_VALUE ? count - 1 : Long.MAX_VALUE, sa, source); + RepeatSubscriber rs = new RepeatSubscriber<>(s, count != Long.MAX_VALUE ? count - 1 : Long.MAX_VALUE, sa, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatUntil.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatUntil.java index 9e24575c58..70df0e949d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatUntil.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatUntil.java @@ -34,7 +34,7 @@ public void subscribeActual(Subscriber s) { SubscriptionArbiter sa = new SubscriptionArbiter(false); s.onSubscribe(sa); - RepeatSubscriber rs = new RepeatSubscriber(s, until, sa, source); + RepeatSubscriber rs = new RepeatSubscriber<>(s, until, sa, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java index dcd807aa2a..5371930b5f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRepeatWhen.java @@ -37,9 +37,9 @@ public FlowableRepeatWhen(Flowable source, @Override public void subscribeActual(Subscriber s) { - SerializedSubscriber z = new SerializedSubscriber(s); + SerializedSubscriber z = new SerializedSubscriber<>(s); - FlowableProcessor processor = UnicastProcessor.create(8).toSerialized(); + FlowableProcessor processor = UnicastProcessor.create(8).toSerialized(); Publisher when; @@ -51,9 +51,9 @@ public void subscribeActual(Subscriber s) { return; } - WhenReceiver receiver = new WhenReceiver(source); + WhenReceiver receiver = new WhenReceiver<>(source); - RepeatWhenSubscriber subscriber = new RepeatWhenSubscriber(z, processor, receiver); + RepeatWhenSubscriber subscriber = new RepeatWhenSubscriber<>(z, processor, receiver); receiver.subscriber = subscriber; @@ -80,7 +80,7 @@ static final class WhenReceiver WhenReceiver(Publisher source) { this.source = source; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.requested = new AtomicLong(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java index 6a352e2fc4..db8718a433 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableReplay.java @@ -56,7 +56,7 @@ public final class FlowableReplay extends ConnectableFlowable implements H public static Flowable multicastSelector( final Supplier> connectableFactory, final Function, ? extends Publisher> selector) { - return new MulticastFlowable(connectableFactory, selector); + return new MulticastFlowable<>(connectableFactory, selector); } /** @@ -83,7 +83,7 @@ public static ConnectableFlowable create(Flowable source, if (bufferSize == Integer.MAX_VALUE) { return createFrom(source); } - return create(source, new ReplayBufferSupplier(bufferSize, eagerTruncate)); + return create(source, new ReplayBufferSupplier<>(bufferSize, eagerTruncate)); } /** @@ -114,7 +114,7 @@ public static ConnectableFlowable create(Flowable source, */ public static ConnectableFlowable create(Flowable source, final long maxAge, final TimeUnit unit, final Scheduler scheduler, final int bufferSize, boolean eagerTruncate) { - return create(source, new ScheduledReplayBufferSupplier(bufferSize, maxAge, unit, scheduler, eagerTruncate)); + return create(source, new ScheduledReplayBufferSupplier<>(bufferSize, maxAge, unit, scheduler, eagerTruncate)); } /** @@ -126,9 +126,9 @@ public static ConnectableFlowable create(Flowable source, static ConnectableFlowable create(Flowable source, final Supplier> bufferFactory) { // the current connection to source needs to be shared between the operator and its onSubscribe call - final AtomicReference> curr = new AtomicReference>(); - Publisher onSubscribe = new ReplayPublisher(curr, bufferFactory); - return RxJavaPlugins.onAssembly(new FlowableReplay(onSubscribe, source, curr, bufferFactory)); + final AtomicReference> curr = new AtomicReference<>(); + Publisher onSubscribe = new ReplayPublisher<>(curr, bufferFactory); + return RxJavaPlugins.onAssembly(new FlowableReplay<>(onSubscribe, source, curr, bufferFactory)); } private FlowableReplay(Publisher onSubscribe, Flowable source, @@ -179,7 +179,7 @@ public void connect(Consumer connection) { } // create a new subscriber-to-source - ReplaySubscriber u = new ReplaySubscriber(buf); + ReplaySubscriber u = new ReplaySubscriber<>(buf); // try setting it as the current subscriber-to-source if (!current.compareAndSet(ps, u)) { // did not work, perhaps a new subscriber arrived @@ -255,7 +255,7 @@ static final class ReplaySubscriber ReplaySubscriber(ReplayBuffer buffer) { this.buffer = buffer; this.management = new AtomicInteger(); - this.subscribers = new AtomicReference[]>(EMPTY); + this.subscribers = new AtomicReference<>(EMPTY); this.shouldConnect = new AtomicBoolean(); } @@ -1004,7 +1004,7 @@ static final class SizeAndTimeBoundReplayBuffer extends BoundedReplayBuffer(value, terminal ? Long.MAX_VALUE : scheduler.now(unit), unit); + return new Timed<>(value, terminal ? Long.MAX_VALUE : scheduler.now(unit), unit); } @Override @@ -1128,7 +1128,7 @@ protected void subscribeActual(Subscriber child) { return; } - final SubscriberResourceWrapper srw = new SubscriberResourceWrapper(child); + final SubscriberResourceWrapper srw = new SubscriberResourceWrapper<>(child); observable.subscribe(srw); @@ -1187,7 +1187,7 @@ static final class ReplayBufferSupplier implements Supplier> @Override public ReplayBuffer get() { - return new SizeBoundReplayBuffer(bufferSize, eagerTruncate); + return new SizeBoundReplayBuffer<>(bufferSize, eagerTruncate); } } @@ -1209,7 +1209,7 @@ static final class ScheduledReplayBufferSupplier implements Supplier get() { - return new SizeAndTimeBoundReplayBuffer(bufferSize, maxAge, unit, scheduler, eagerTruncate); + return new SizeAndTimeBoundReplayBuffer<>(bufferSize, maxAge, unit, scheduler, eagerTruncate); } } @@ -1241,7 +1241,7 @@ public void subscribe(Subscriber child) { return; } // create a new subscriber to source - ReplaySubscriber u = new ReplaySubscriber(buf); + ReplaySubscriber u = new ReplaySubscriber<>(buf); // let's try setting it as the current subscriber-to-source if (!curr.compareAndSet(null, u)) { // didn't work, maybe someone else did it or the current subscriber @@ -1253,7 +1253,7 @@ public void subscribe(Subscriber child) { } // create the backpressure-managing producer for this child - InnerSubscription inner = new InnerSubscription(r, child); + InnerSubscription inner = new InnerSubscription<>(r, child); // the producer has been registered with the current subscriber-to-source so // at least it will receive the next terminal event // setting the producer will trigger the first request to be considered by @@ -1282,7 +1282,7 @@ public void subscribe(Subscriber child) { static final class DefaultUnboundedFactory implements Supplier { @Override public Object get() { - return new UnboundedReplayBuffer(16); + return new UnboundedReplayBuffer<>(16); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryBiPredicate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryBiPredicate.java index 1a06e02bb6..62b4852849 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryBiPredicate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryBiPredicate.java @@ -36,7 +36,7 @@ public void subscribeActual(Subscriber s) { SubscriptionArbiter sa = new SubscriptionArbiter(false); s.onSubscribe(sa); - RetryBiSubscriber rs = new RetryBiSubscriber(s, predicate, sa, source); + RetryBiSubscriber rs = new RetryBiSubscriber<>(s, predicate, sa, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryPredicate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryPredicate.java index 07d7207d74..75350871c4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryPredicate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryPredicate.java @@ -38,7 +38,7 @@ public void subscribeActual(Subscriber s) { SubscriptionArbiter sa = new SubscriptionArbiter(false); s.onSubscribe(sa); - RetrySubscriber rs = new RetrySubscriber(s, count, predicate, sa, source); + RetrySubscriber rs = new RetrySubscriber<>(s, count, predicate, sa, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java index 7307664353..f4129d7be8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableRetryWhen.java @@ -36,7 +36,7 @@ public FlowableRetryWhen(Flowable source, @Override public void subscribeActual(Subscriber s) { - SerializedSubscriber z = new SerializedSubscriber(s); + SerializedSubscriber z = new SerializedSubscriber<>(s); FlowableProcessor processor = UnicastProcessor.create(8).toSerialized(); @@ -50,9 +50,9 @@ public void subscribeActual(Subscriber s) { return; } - WhenReceiver receiver = new WhenReceiver(source); + WhenReceiver receiver = new WhenReceiver<>(source); - RetryWhenSubscriber subscriber = new RetryWhenSubscriber(z, processor, receiver); + RetryWhenSubscriber subscriber = new RetryWhenSubscriber<>(z, processor, receiver); receiver.subscriber = subscriber; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSamplePublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSamplePublisher.java index 59a6ec3a14..4a20a2b95f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSamplePublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSamplePublisher.java @@ -37,11 +37,11 @@ public FlowableSamplePublisher(Publisher source, Publisher other, boolean @Override protected void subscribeActual(Subscriber s) { - SerializedSubscriber serial = new SerializedSubscriber(s); + SerializedSubscriber serial = new SerializedSubscriber<>(s); if (emitLast) { - source.subscribe(new SampleMainEmitLast(serial, other)); + source.subscribe(new SampleMainEmitLast<>(serial, other)); } else { - source.subscribe(new SampleMainNoLast(serial, other)); + source.subscribe(new SampleMainNoLast<>(serial, other)); } } @@ -54,7 +54,7 @@ abstract static class SamplePublisherSubscriber extends AtomicReference im final AtomicLong requested = new AtomicLong(); - final AtomicReference other = new AtomicReference(); + final AtomicReference other = new AtomicReference<>(); Subscription upstream; @@ -69,7 +69,7 @@ public void onSubscribe(Subscription s) { this.upstream = s; downstream.onSubscribe(this); if (other.get() == null) { - sampler.subscribe(new SamplerSubscriber(this)); + sampler.subscribe(new SamplerSubscriber<>(this)); s.request(Long.MAX_VALUE); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSampleTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSampleTimed.java index ea9a4d5ae2..17e1644749 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSampleTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSampleTimed.java @@ -42,11 +42,11 @@ public FlowableSampleTimed(Flowable source, long period, TimeUnit unit, Sched @Override protected void subscribeActual(Subscriber s) { - SerializedSubscriber serial = new SerializedSubscriber(s); + SerializedSubscriber serial = new SerializedSubscriber<>(s); if (emitLast) { - source.subscribe(new SampleTimedEmitLast(serial, period, unit, scheduler)); + source.subscribe(new SampleTimedEmitLast<>(serial, period, unit, scheduler)); } else { - source.subscribe(new SampleTimedNoLast(serial, period, unit, scheduler)); + source.subscribe(new SampleTimedNoLast<>(serial, period, unit, scheduler)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java index d78e83a3d2..4abae183b4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScalarXMap.java @@ -87,7 +87,7 @@ public static boolean tryScalarXMapSubscribe(Publisher source, EmptySubscription.complete(subscriber); return true; } - subscriber.onSubscribe(new ScalarSubscription(subscriber, u)); + subscriber.onSubscribe(new ScalarSubscription<>(subscriber, u)); } else { r.subscribe(subscriber); } @@ -108,7 +108,7 @@ public static boolean tryScalarXMapSubscribe(Publisher source, * @return the new Flowable instance */ public static Flowable scalarXMap(final T value, final Function> mapper) { - return RxJavaPlugins.onAssembly(new ScalarXMapFlowable(value, mapper)); + return RxJavaPlugins.onAssembly(new ScalarXMapFlowable<>(value, mapper)); } /** @@ -155,7 +155,7 @@ public void subscribeActual(Subscriber s) { EmptySubscription.complete(s); return; } - s.onSubscribe(new ScalarSubscription(s, u)); + s.onSubscribe(new ScalarSubscription<>(s, u)); } else { other.subscribe(s); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java index 2fab24a484..0199189204 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScan.java @@ -32,7 +32,7 @@ public FlowableScan(Flowable source, BiFunction accumulator) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ScanSubscriber(s, accumulator)); + source.subscribe(new ScanSubscriber<>(s, accumulator)); } static final class ScanSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java index da0b1870da..21c88ab7bf 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableScanSeed.java @@ -48,7 +48,7 @@ protected void subscribeActual(Subscriber s) { return; } - source.subscribe(new ScanSeedSubscriber(s, accumulator, r, bufferSize())); + source.subscribe(new ScanSeedSubscriber<>(s, accumulator, r, bufferSize())); } static final class ScanSeedSubscriber @@ -85,7 +85,7 @@ static final class ScanSeedSubscriber this.value = value; this.prefetch = prefetch; this.limit = prefetch - (prefetch >> 2); - this.queue = new SpscArrayQueue(prefetch); + this.queue = new SpscArrayQueue<>(prefetch); this.queue.offer(value); this.requested = new AtomicLong(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqual.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqual.java index 05be9542be..cc65e1d37e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqual.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqual.java @@ -41,7 +41,7 @@ public FlowableSequenceEqual(Publisher first, Publisher s) { - EqualCoordinator parent = new EqualCoordinator(s, prefetch, comparer); + EqualCoordinator parent = new EqualCoordinator<>(s, prefetch, comparer); s.onSubscribe(parent); parent.subscribe(first, second); } @@ -79,8 +79,8 @@ static final class EqualCoordinator extends DeferredScalarSubscription(this, prefetch); - this.second = new EqualSubscriber(this, prefetch); + this.first = new EqualSubscriber<>(this, prefetch); + this.second = new EqualSubscriber<>(this, prefetch); this.errors = new AtomicThrowable(); } @@ -289,7 +289,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); s.request(prefetch); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualSingle.java index 7761c93041..05d99e4e49 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSequenceEqualSingle.java @@ -43,14 +43,14 @@ public FlowableSequenceEqualSingle(Publisher first, Publisher observer) { - EqualCoordinator parent = new EqualCoordinator(observer, prefetch, comparer); + EqualCoordinator parent = new EqualCoordinator<>(observer, prefetch, comparer); observer.onSubscribe(parent); parent.subscribe(first, second); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableSequenceEqual(first, second, comparer, prefetch)); + return RxJavaPlugins.onAssembly(new FlowableSequenceEqual<>(first, second, comparer, prefetch)); } static final class EqualCoordinator @@ -76,8 +76,8 @@ static final class EqualCoordinator EqualCoordinator(SingleObserver actual, int prefetch, BiPredicate comparer) { this.downstream = actual; this.comparer = comparer; - this.first = new EqualSubscriber(this, prefetch); - this.second = new EqualSubscriber(this, prefetch); + this.first = new EqualSubscriber<>(this, prefetch); + this.second = new EqualSubscriber<>(this, prefetch); this.errors = new AtomicThrowable(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSerialized.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSerialized.java index ec924bb0d1..2cd7df3369 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSerialized.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSerialized.java @@ -24,6 +24,6 @@ public FlowableSerialized(Flowable source) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SerializedSubscriber(s)); + source.subscribe(new SerializedSubscriber<>(s)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingle.java index 11a27d372b..068385289a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingle.java @@ -35,7 +35,7 @@ public FlowableSingle(Flowable source, T defaultValue, boolean failOnEmpty) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SingleElementSubscriber(s, defaultValue, failOnEmpty)); + source.subscribe(new SingleElementSubscriber<>(s, defaultValue, failOnEmpty)); } static final class SingleElementSubscriber extends DeferredScalarSubscription diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleMaybe.java index 1d701014cb..fac74c9cd8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleMaybe.java @@ -31,12 +31,12 @@ public FlowableSingleMaybe(Flowable source) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new SingleElementSubscriber(observer)); + source.subscribe(new SingleElementSubscriber<>(observer)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableSingle(source, null, false)); + return RxJavaPlugins.onAssembly(new FlowableSingle<>(source, null, false)); } static final class SingleElementSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleSingle.java index 50c4d06ccc..c24b45a760 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSingleSingle.java @@ -36,12 +36,12 @@ public FlowableSingleSingle(Flowable source, T defaultValue) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new SingleElementSubscriber(observer, defaultValue)); + source.subscribe(new SingleElementSubscriber<>(observer, defaultValue)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableSingle(source, defaultValue, true)); + return RxJavaPlugins.onAssembly(new FlowableSingle<>(source, defaultValue, true)); } static final class SingleElementSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkip.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkip.java index f60d36e8f1..2aecef3e10 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkip.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkip.java @@ -27,7 +27,7 @@ public FlowableSkip(Flowable source, long n) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SkipSubscriber(s, n)); + source.subscribe(new SkipSubscriber<>(s, n)); } static final class SkipSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLast.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLast.java index 8b3a6a2487..5908d6d0b0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLast.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLast.java @@ -30,7 +30,7 @@ public FlowableSkipLast(Flowable source, int skip) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SkipLastSubscriber(s, skip)); + source.subscribe(new SkipLastSubscriber<>(s, skip)); } static final class SkipLastSubscriber extends ArrayDeque implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTimed.java index 099c5ec59c..b56224cbdf 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipLastTimed.java @@ -41,7 +41,7 @@ public FlowableSkipLastTimed(Flowable source, long time, TimeUnit unit, Sched @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SkipLastTimedSubscriber(s, time, unit, scheduler, bufferSize, delayError)); + source.subscribe(new SkipLastTimedSubscriber<>(s, time, unit, scheduler, bufferSize, delayError)); } static final class SkipLastTimedSubscriber extends AtomicInteger implements FlowableSubscriber, Subscription { @@ -68,7 +68,7 @@ static final class SkipLastTimedSubscriber extends AtomicInteger implements F this.time = time; this.unit = unit; this.scheduler = scheduler; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); this.delayError = delayError; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipUntil.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipUntil.java index 8052bc7fbf..8de843142c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipUntil.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipUntil.java @@ -31,7 +31,7 @@ public FlowableSkipUntil(Flowable source, Publisher other) { @Override protected void subscribeActual(Subscriber child) { - SkipUntilMainSubscriber parent = new SkipUntilMainSubscriber(child); + SkipUntilMainSubscriber parent = new SkipUntilMainSubscriber<>(child); child.onSubscribe(parent); other.subscribe(parent.other); @@ -57,7 +57,7 @@ static final class SkipUntilMainSubscriber extends AtomicInteger SkipUntilMainSubscriber(Subscriber downstream) { this.downstream = downstream; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.requested = new AtomicLong(); this.other = new OtherSubscriber(); this.error = new AtomicThrowable(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipWhile.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipWhile.java index 7ed91df231..62ee5ebdbe 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipWhile.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSkipWhile.java @@ -29,7 +29,7 @@ public FlowableSkipWhile(Flowable source, Predicate predicate) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SkipWhileSubscriber(s, predicate)); + source.subscribe(new SkipWhileSubscriber<>(s, predicate)); } static final class SkipWhileSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOn.java index 63394a09ab..b7fdd91e9f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSubscribeOn.java @@ -42,7 +42,7 @@ public FlowableSubscribeOn(Flowable source, Scheduler scheduler, boolean nonS @Override public void subscribeActual(final Subscriber s) { Scheduler.Worker w = scheduler.createWorker(); - final SubscribeOnSubscriber sos = new SubscribeOnSubscriber(s, w, source, nonScheduledRequests); + final SubscribeOnSubscriber sos = new SubscribeOnSubscriber<>(s, w, source, nonScheduledRequests); s.onSubscribe(sos); w.schedule(sos); @@ -69,7 +69,7 @@ static final class SubscribeOnSubscriber extends AtomicReference this.downstream = actual; this.worker = worker; this.source = source; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.requested = new AtomicLong(); this.nonScheduledRequests = !requestOn; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmpty.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmpty.java index a2e81eac58..b2ad9db0d0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmpty.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchIfEmpty.java @@ -27,7 +27,7 @@ public FlowableSwitchIfEmpty(Flowable source, Publisher other) { @Override protected void subscribeActual(Subscriber s) { - SwitchIfEmptySubscriber parent = new SwitchIfEmptySubscriber(s, other); + SwitchIfEmptySubscriber parent = new SwitchIfEmptySubscriber<>(s, other); s.onSubscribe(parent.arbiter); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java index c618a0252b..7c6c1006ca 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableSwitchMap.java @@ -46,7 +46,7 @@ protected void subscribeActual(Subscriber s) { if (FlowableScalarXMap.tryScalarXMapSubscribe(source, s, mapper)) { return; } - source.subscribe(new SwitchMapSubscriber(s, mapper, bufferSize, delayErrors)); + source.subscribe(new SwitchMapSubscriber<>(s, mapper, bufferSize, delayErrors)); } static final class SwitchMapSubscriber extends AtomicInteger implements FlowableSubscriber, Subscription { @@ -64,13 +64,13 @@ static final class SwitchMapSubscriber extends AtomicInteger implements Fl Subscription upstream; - final AtomicReference> active = new AtomicReference>(); + final AtomicReference> active = new AtomicReference<>(); final AtomicLong requested = new AtomicLong(); static final SwitchMapInnerSubscriber CANCELLED; static { - CANCELLED = new SwitchMapInnerSubscriber(null, -1L, 1); + CANCELLED = new SwitchMapInnerSubscriber<>(null, -1L, 1); CANCELLED.cancel(); } @@ -118,7 +118,7 @@ public void onNext(T t) { return; } - SwitchMapInnerSubscriber nextInner = new SwitchMapInnerSubscriber(this, c, bufferSize); + SwitchMapInnerSubscriber nextInner = new SwitchMapInnerSubscriber<>(this, c, bufferSize); for (;;) { inner = active.get(); @@ -371,7 +371,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(bufferSize); + queue = new SpscArrayQueue<>(bufferSize); s.request(bufferSize); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTake.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTake.java index 9b8e995322..15d06f087f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTake.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTake.java @@ -32,7 +32,7 @@ public FlowableTake(Flowable source, long n) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new TakeSubscriber(s, n)); + source.subscribe(new TakeSubscriber<>(s, n)); } static final class TakeSubscriber @@ -105,12 +105,7 @@ public void request(long n) { if (r == 0L) { break; } - long toRequest; - if (r <= n) { - toRequest = r; - } else { - toRequest = n; - } + long toRequest = Math.min(r, n); long u = r - toRequest; if (compareAndSet(r, u)) { upstream.request(toRequest); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLast.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLast.java index 5252bd808d..062135b47c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLast.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLast.java @@ -32,7 +32,7 @@ public FlowableTakeLast(Flowable source, int count) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new TakeLastSubscriber(s, count)); + source.subscribe(new TakeLastSubscriber<>(s, count)); } static final class TakeLastSubscriber extends ArrayDeque implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOne.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOne.java index fb11b74c0c..e4b674704d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOne.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastOne.java @@ -25,7 +25,7 @@ public FlowableTakeLastOne(Flowable source) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new TakeLastOneSubscriber(s)); + source.subscribe(new TakeLastOneSubscriber<>(s)); } static final class TakeLastOneSubscriber extends DeferredScalarSubscription diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimed.java index 2538fc178e..a7c2ab3343 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeLastTimed.java @@ -45,7 +45,7 @@ public FlowableTakeLastTimed(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new TakeLastTimedSubscriber(s, count, time, unit, scheduler, bufferSize, delayError)); + source.subscribe(new TakeLastTimedSubscriber<>(s, count, time, unit, scheduler, bufferSize, delayError)); } static final class TakeLastTimedSubscriber extends AtomicInteger implements FlowableSubscriber, Subscription { @@ -74,7 +74,7 @@ static final class TakeLastTimedSubscriber extends AtomicInteger implements F this.time = time; this.unit = unit; this.scheduler = scheduler; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); this.delayError = delayError; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakePublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakePublisher.java index bda003e7da..9a30bda905 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakePublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakePublisher.java @@ -35,6 +35,6 @@ public FlowableTakePublisher(Publisher source, long limit) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new TakeSubscriber(s, limit)); + source.subscribe(new TakeSubscriber<>(s, limit)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntil.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntil.java index 1bbe14cd57..9c927cca5d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntil.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntil.java @@ -30,7 +30,7 @@ public FlowableTakeUntil(Flowable source, Publisher other) { @Override protected void subscribeActual(Subscriber child) { - TakeUntilMainSubscriber parent = new TakeUntilMainSubscriber(child); + TakeUntilMainSubscriber parent = new TakeUntilMainSubscriber<>(child); child.onSubscribe(parent); other.subscribe(parent.other); @@ -55,7 +55,7 @@ static final class TakeUntilMainSubscriber extends AtomicInteger implements F TakeUntilMainSubscriber(Subscriber downstream) { this.downstream = downstream; this.requested = new AtomicLong(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.other = new OtherSubscriber(); this.error = new AtomicThrowable(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicate.java index 8bb8f27531..a8ec19c22a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeUntilPredicate.java @@ -30,7 +30,7 @@ public FlowableTakeUntilPredicate(Flowable source, Predicate predi @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new InnerSubscriber(s, predicate)); + source.subscribe(new InnerSubscriber<>(s, predicate)); } static final class InnerSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhile.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhile.java index ba9a115da8..5bcd8157d3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhile.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTakeWhile.java @@ -30,7 +30,7 @@ public FlowableTakeWhile(Flowable source, Predicate predicate) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new TakeWhileSubscriber(s, predicate)); + source.subscribe(new TakeWhileSubscriber<>(s, predicate)); } static final class TakeWhileSubscriber implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleFirstTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleFirstTimed.java index 4b1441b731..918e2ea6d7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleFirstTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleFirstTimed.java @@ -42,8 +42,8 @@ public FlowableThrottleFirstTimed(Flowable source, long timeout, TimeUnit uni @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new DebounceTimedSubscriber( - new SerializedSubscriber(s), + source.subscribe(new DebounceTimedSubscriber<>( + new SerializedSubscriber<>(s), timeout, unit, scheduler.createWorker())); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleLatest.java index fe32addcc6..989eaaf4b2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableThrottleLatest.java @@ -56,7 +56,7 @@ public FlowableThrottleLatest(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ThrottleLatestSubscriber(s, timeout, unit, scheduler.createWorker(), emitLast)); + source.subscribe(new ThrottleLatestSubscriber<>(s, timeout, unit, scheduler.createWorker(), emitLast)); } static final class ThrottleLatestSubscriber @@ -100,7 +100,7 @@ static final class ThrottleLatestSubscriber this.unit = unit; this.worker = worker; this.emitLast = emitLast; - this.latest = new AtomicReference(); + this.latest = new AtomicReference<>(); this.requested = new AtomicLong(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeInterval.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeInterval.java index 4f7c1e899b..eb9ec1fbd9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeInterval.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeInterval.java @@ -33,7 +33,7 @@ public FlowableTimeInterval(Flowable source, TimeUnit unit, Scheduler schedul @Override protected void subscribeActual(Subscriber> s) { - source.subscribe(new TimeIntervalSubscriber(s, unit, scheduler)); + source.subscribe(new TimeIntervalSubscriber<>(s, unit, scheduler)); } static final class TimeIntervalSubscriber implements FlowableSubscriber, Subscription { @@ -66,7 +66,7 @@ public void onNext(T t) { long last = lastTime; lastTime = now; long delta = now - last; - downstream.onNext(new Timed(t, delta, unit)); + downstream.onNext(new Timed<>(t, delta, unit)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java index c5879c2e74..976ab10a05 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeout.java @@ -47,12 +47,12 @@ public FlowableTimeout( @Override protected void subscribeActual(Subscriber s) { if (other == null) { - TimeoutSubscriber parent = new TimeoutSubscriber(s, itemTimeoutIndicator); + TimeoutSubscriber parent = new TimeoutSubscriber<>(s, itemTimeoutIndicator); s.onSubscribe(parent); parent.startFirstTimeout(firstTimeoutIndicator); source.subscribe(parent); } else { - TimeoutFallbackSubscriber parent = new TimeoutFallbackSubscriber(s, itemTimeoutIndicator, other); + TimeoutFallbackSubscriber parent = new TimeoutFallbackSubscriber<>(s, itemTimeoutIndicator, other); s.onSubscribe(parent); parent.startFirstTimeout(firstTimeoutIndicator); source.subscribe(parent); @@ -82,7 +82,7 @@ static final class TimeoutSubscriber extends AtomicLong this.downstream = actual; this.itemTimeoutIndicator = itemTimeoutIndicator; this.task = new SequentialDisposable(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.requested = new AtomicLong(); } @@ -212,7 +212,7 @@ static final class TimeoutFallbackSubscriber extends SubscriptionArbiter this.downstream = actual; this.itemTimeoutIndicator = itemTimeoutIndicator; this.task = new SequentialDisposable(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.fallback = fallback; this.index = new AtomicLong(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTimed.java index 8963788992..351099500e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableTimeoutTimed.java @@ -43,12 +43,12 @@ public FlowableTimeoutTimed(Flowable source, @Override protected void subscribeActual(Subscriber s) { if (other == null) { - TimeoutSubscriber parent = new TimeoutSubscriber(s, timeout, unit, scheduler.createWorker()); + TimeoutSubscriber parent = new TimeoutSubscriber<>(s, timeout, unit, scheduler.createWorker()); s.onSubscribe(parent); parent.startTimeout(0L); source.subscribe(parent); } else { - TimeoutFallbackSubscriber parent = new TimeoutFallbackSubscriber(s, timeout, unit, scheduler.createWorker(), other); + TimeoutFallbackSubscriber parent = new TimeoutFallbackSubscriber<>(s, timeout, unit, scheduler.createWorker(), other); s.onSubscribe(parent); parent.startTimeout(0L); source.subscribe(parent); @@ -80,7 +80,7 @@ static final class TimeoutSubscriber extends AtomicLong this.unit = unit; this.worker = worker; this.task = new SequentialDisposable(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.requested = new AtomicLong(); } @@ -203,7 +203,7 @@ static final class TimeoutFallbackSubscriber extends SubscriptionArbiter this.worker = worker; this.fallback = fallback; this.task = new SequentialDisposable(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.index = new AtomicLong(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToList.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToList.java index 439fba1074..18d8a86de4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToList.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToList.java @@ -41,7 +41,7 @@ protected void subscribeActual(Subscriber s) { EmptySubscription.error(e, s); return; } - source.subscribe(new ToListSubscriber(s, coll)); + source.subscribe(new ToListSubscriber<>(s, coll)); } static final class ToListSubscriber> diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListSingle.java index b90869ad62..aea4eb3270 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableToListSingle.java @@ -53,12 +53,12 @@ protected void subscribeActual(SingleObserver observer) { EmptyDisposable.error(e, observer); return; } - source.subscribe(new ToListSubscriber(observer, coll)); + source.subscribe(new ToListSubscriber<>(observer, coll)); } @Override public Flowable fuseToFlowable() { - return RxJavaPlugins.onAssembly(new FlowableToList(source, collectionSupplier)); + return RxJavaPlugins.onAssembly(new FlowableToList<>(source, collectionSupplier)); } static final class ToListSubscriber> diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOn.java index 3a7d29a47b..ee9d8f3cc7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUnsubscribeOn.java @@ -30,7 +30,7 @@ public FlowableUnsubscribeOn(Flowable source, Scheduler scheduler) { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new UnsubscribeSubscriber(s, scheduler)); + source.subscribe(new UnsubscribeSubscriber<>(s, scheduler)); } static final class UnsubscribeSubscriber extends AtomicBoolean implements FlowableSubscriber, Subscription { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java index 7e2e91c672..5a5b177d35 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableUsing.java @@ -68,7 +68,7 @@ public void subscribeActual(Subscriber s) { return; } - UsingSubscriber us = new UsingSubscriber(s, resource, disposer, eager); + UsingSubscriber us = new UsingSubscriber<>(s, resource, disposer, eager); source.subscribe(us); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java index ae027e2d04..e00ed99edf 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindow.java @@ -42,12 +42,12 @@ public FlowableWindow(Flowable source, long size, long skip, int bufferSize) @Override public void subscribeActual(Subscriber> s) { if (skip == size) { - source.subscribe(new WindowExactSubscriber(s, size, bufferSize)); + source.subscribe(new WindowExactSubscriber<>(s, size, bufferSize)); } else if (skip > size) { - source.subscribe(new WindowSkipSubscriber(s, size, skip, bufferSize)); + source.subscribe(new WindowSkipSubscriber<>(s, size, skip, bufferSize)); } else { - source.subscribe(new WindowOverlapSubscriber(s, size, skip, bufferSize)); + source.subscribe(new WindowOverlapSubscriber<>(s, size, skip, bufferSize)); } } @@ -96,10 +96,10 @@ public void onNext(T t) { if (i == 0) { getAndIncrement(); - w = UnicastProcessor.create(bufferSize, this); + w = UnicastProcessor.create(bufferSize, this); window = w; - intercept = new FlowableWindowSubscribeIntercept(w); + intercept = new FlowableWindowSubscribeIntercept<>(w); downstream.onNext(intercept); } @@ -216,10 +216,10 @@ public void onNext(T t) { if (i == 0) { getAndIncrement(); - w = UnicastProcessor.create(bufferSize, this); + w = UnicastProcessor.create(bufferSize, this); window = w; - intercept = new FlowableWindowSubscribeIntercept(w); + intercept = new FlowableWindowSubscribeIntercept<>(w); downstream.onNext(intercept); } @@ -339,8 +339,8 @@ static final class WindowOverlapSubscriber this.downstream = actual; this.size = size; this.skip = skip; - this.queue = new SpscLinkedArrayQueue>(bufferSize); - this.windows = new ArrayDeque>(); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); + this.windows = new ArrayDeque<>(); this.once = new AtomicBoolean(); this.firstRequest = new AtomicBoolean(); this.requested = new AtomicLong(); @@ -369,7 +369,7 @@ public void onNext(T t) { if (!cancelled) { getAndIncrement(); - newWindow = UnicastProcessor.create(bufferSize, this); + newWindow = UnicastProcessor.create(bufferSize, this); windows.offer(newWindow); } @@ -477,7 +477,7 @@ void drain() { break; } - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(t); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(t); a.onNext(intercept); if (intercept.tryAbandon()) { @@ -488,7 +488,7 @@ void drain() { if (e == r) { if (cancelled) { - continue outer; + continue; } if (checkTerminated(done, q.isEmpty(), a, q)) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java index 2e07d3eec8..c86d3faab1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundary.java @@ -38,7 +38,7 @@ public FlowableWindowBoundary(Flowable source, Publisher other, int capaci @Override protected void subscribeActual(Subscriber> subscriber) { - WindowBoundaryMainSubscriber parent = new WindowBoundaryMainSubscriber(subscriber, capacityHint); + WindowBoundaryMainSubscriber parent = new WindowBoundaryMainSubscriber<>(subscriber, capacityHint); subscriber.onSubscribe(parent); @@ -84,10 +84,10 @@ static final class WindowBoundaryMainSubscriber WindowBoundaryMainSubscriber(Subscriber> downstream, int capacityHint) { this.downstream = downstream; this.capacityHint = capacityHint; - this.boundarySubscriber = new WindowBoundaryInnerSubscriber(this); - this.upstream = new AtomicReference(); + this.boundarySubscriber = new WindowBoundaryInnerSubscriber<>(this); + this.upstream = new AtomicReference<>(); this.windows = new AtomicInteger(1); - this.queue = new MpscLinkedQueue(); + this.queue = new MpscLinkedQueue<>(); this.errors = new AtomicThrowable(); this.stopWindows = new AtomicBoolean(); this.requested = new AtomicLong(); @@ -240,7 +240,7 @@ void drain() { if (emitted != requested.get()) { emitted++; - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(w); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(w); downstream.onNext(intercept); if (intercept.tryAbandon()) { w.onComplete(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java index d55198010a..f6248b3355 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowBoundarySelector.java @@ -46,7 +46,7 @@ public FlowableWindowBoundarySelector( @Override protected void subscribeActual(Subscriber> s) { - source.subscribe(new WindowBoundaryMainSubscriber( + source.subscribe(new WindowBoundaryMainSubscriber<>( s, open, closingIndicator, bufferSize)); } @@ -85,16 +85,16 @@ static final class WindowBoundaryMainSubscriber WindowBoundaryMainSubscriber(Subscriber> actual, Publisher open, Function> closingIndicator, int bufferSize) { this.downstream = actual; - this.queue = new MpscLinkedQueue(); + this.queue = new MpscLinkedQueue<>(); this.open = open; this.closingIndicator = closingIndicator; this.bufferSize = bufferSize; this.resources = new CompositeDisposable(); - this.windows = new ArrayList>(); + this.windows = new ArrayList<>(); this.windowCount = new AtomicLong(1L); this.downstreamCancelled = new AtomicBoolean(); this.error = new AtomicThrowable(); - this.startSubscriber = new WindowStartSubscriber(this); + this.startSubscriber = new WindowStartSubscriber<>(this); this.requested = new AtomicLong(); } @@ -171,7 +171,7 @@ public void run() { } void open(B startValue) { - queue.offer(new WindowStartItem(startValue)); + queue.offer(new WindowStartItem<>(startValue)); drain(); } @@ -257,7 +257,7 @@ void drain() { windowCount.getAndIncrement(); UnicastProcessor newWindow = UnicastProcessor.create(bufferSize, this); - WindowEndSubscriberIntercept endSubscriber = new WindowEndSubscriberIntercept(this, newWindow); + WindowEndSubscriberIntercept endSubscriber = new WindowEndSubscriberIntercept<>(this, newWindow); downstream.onNext(endSubscriber); @@ -388,7 +388,7 @@ static final class WindowEndSubscriberIntercept extends Flowable WindowEndSubscriberIntercept(WindowBoundaryMainSubscriber parent, UnicastProcessor window) { this.parent = parent; this.window = window; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.once = new AtomicBoolean(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java index 0dfec4e4af..89b49e857a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWindowTimed.java @@ -55,18 +55,18 @@ public FlowableWindowTimed(Flowable source, protected void subscribeActual(Subscriber> downstream) { if (timespan == timeskip) { if (maxSize == Long.MAX_VALUE) { - source.subscribe(new WindowExactUnboundedSubscriber( + source.subscribe(new WindowExactUnboundedSubscriber<>( downstream, timespan, unit, scheduler, bufferSize)); return; } - source.subscribe(new WindowExactBoundedSubscriber( + source.subscribe(new WindowExactBoundedSubscriber<>( downstream, timespan, unit, scheduler, bufferSize, maxSize, restartTimerOnMaxSize)); return; } - source.subscribe(new WindowSkipSubscriber(downstream, + source.subscribe(new WindowSkipSubscriber<>(downstream, timespan, timeskip, unit, scheduler.createWorker(), bufferSize)); } @@ -99,7 +99,7 @@ abstract static class AbstractWindowSubscriber AbstractWindowSubscriber(Subscriber> downstream, long timespan, TimeUnit unit, int bufferSize) { this.downstream = downstream; - this.queue = new MpscLinkedQueue(); + this.queue = new MpscLinkedQueue<>(); this.timespan = timespan; this.unit = unit; this.bufferSize = bufferSize; @@ -201,7 +201,7 @@ void createFirstWindow() { emitted = 1; - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); timer.replace(scheduler.schedulePeriodicallyDirect(this, timespan, timespan, unit)); @@ -290,7 +290,7 @@ else if (!isEmpty) { window = UnicastProcessor.create(bufferSize, windowRunnable); this.window = window; - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); if (intercept.tryAbandon()) { @@ -369,7 +369,7 @@ void createFirstWindow() { windowCount.getAndIncrement(); window = UnicastProcessor.create(bufferSize, this); - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); Runnable boundaryTask = new WindowBoundaryRunnable(this, 1L); @@ -507,7 +507,7 @@ UnicastProcessor createNewWindow(UnicastProcessor window) { window = UnicastProcessor.create(bufferSize, this); this.window = window; - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); if (restartTimerOnMaxSize) { @@ -557,7 +557,7 @@ static final class WindowSkipSubscriber super(actual, timespan, unit, bufferSize); this.timeskip = timeskip; this.worker = worker; - this.windows = new LinkedList>(); + this.windows = new LinkedList<>(); } @Override @@ -570,7 +570,7 @@ void createFirstWindow() { UnicastProcessor window = UnicastProcessor.create(bufferSize, this); windows.add(window); - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); @@ -644,7 +644,7 @@ void drain() { UnicastProcessor window = UnicastProcessor.create(bufferSize, this); windows.add(window); - FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept(window); + FlowableWindowSubscribeIntercept intercept = new FlowableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java index 049befd1f4..81d3544dd0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFrom.java @@ -36,8 +36,8 @@ public FlowableWithLatestFrom(Flowable source, BiFunction s) { - final SerializedSubscriber serial = new SerializedSubscriber(s); - final WithLatestFromSubscriber wlf = new WithLatestFromSubscriber(serial, combiner); + final SerializedSubscriber serial = new SerializedSubscriber<>(s); + final WithLatestFromSubscriber wlf = new WithLatestFromSubscriber<>(serial, combiner); serial.onSubscribe(wlf); @@ -55,11 +55,11 @@ static final class WithLatestFromSubscriber extends AtomicReference final BiFunction combiner; - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); final AtomicLong requested = new AtomicLong(); - final AtomicReference other = new AtomicReference(); + final AtomicReference other = new AtomicReference<>(); WithLatestFromSubscriber(Subscriber actual, BiFunction combiner) { this.downstream = actual; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java index 1d1be81e05..be9166a16d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableWithLatestFromMany.java @@ -82,11 +82,11 @@ protected void subscribeActual(Subscriber s) { } if (n == 0) { - new FlowableMap(source, new SingletonArrayFunc()).subscribeActual(s); + new FlowableMap<>(source, new SingletonArrayFunc()).subscribeActual(s); return; } - WithLatestFromSubscriber parent = new WithLatestFromSubscriber(s, combiner, n); + WithLatestFromSubscriber parent = new WithLatestFromSubscriber<>(s, combiner, n); s.onSubscribe(parent); parent.subscribe(others, n); @@ -123,8 +123,8 @@ static final class WithLatestFromSubscriber s[i] = new WithLatestInnerSubscriber(this, i); } this.subscribers = s; - this.values = new AtomicReferenceArray(n); - this.upstream = new AtomicReference(); + this.values = new AtomicReferenceArray<>(n); + this.upstream = new AtomicReference<>(); this.requested = new AtomicLong(); this.error = new AtomicThrowable(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java index 27f1f8e2dc..77011a5a2d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/flowable/FlowableZip.java @@ -71,7 +71,7 @@ public void subscribeActual(Subscriber s) { return; } - ZipCoordinator coordinator = new ZipCoordinator(s, zipper, count, bufferSize, delayError); + ZipCoordinator coordinator = new ZipCoordinator<>(s, zipper, count, bufferSize, delayError); s.onSubscribe(coordinator); @@ -108,7 +108,7 @@ static final class ZipCoordinator @SuppressWarnings("unchecked") ZipSubscriber[] a = new ZipSubscriber[n]; for (int i = 0; i < n; i++) { - a[i] = new ZipSubscriber(this, prefetch); + a[i] = new ZipSubscriber<>(this, prefetch); } this.current = new Object[n]; this.subscribers = a; @@ -354,7 +354,7 @@ public void onSubscribe(Subscription s) { } } - queue = new SpscArrayQueue(prefetch); + queue = new SpscArrayQueue<>(prefetch); s.request(prefetch); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCache.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCache.java index 64d57a0605..05c1593707 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCache.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCache.java @@ -41,13 +41,13 @@ public final class MaybeCache extends Maybe implements MaybeObserver { @SuppressWarnings("unchecked") public MaybeCache(MaybeSource source) { - this.source = new AtomicReference>(source); - this.observers = new AtomicReference[]>(EMPTY); + this.source = new AtomicReference<>(source); + this.observers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(MaybeObserver observer) { - CacheDisposable parent = new CacheDisposable(observer, this); + CacheDisposable parent = new CacheDisposable<>(observer, this); observer.onSubscribe(parent); if (add(parent)) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArray.java index bab81330b2..f77a59c364 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArray.java @@ -38,7 +38,7 @@ public MaybeConcatArray(MaybeSource[] sources) { @Override protected void subscribeActual(Subscriber s) { - ConcatMaybeObserver parent = new ConcatMaybeObserver(s, sources); + ConcatMaybeObserver parent = new ConcatMaybeObserver<>(s, sources); s.onSubscribe(parent); parent.drain(); } @@ -68,7 +68,7 @@ static final class ConcatMaybeObserver this.sources = sources; this.requested = new AtomicLong(); this.disposables = new SequentialDisposable(); - this.current = new AtomicReference(NotificationLite.COMPLETE); // as if a previous completed + this.current = new AtomicReference<>(NotificationLite.COMPLETE); // as if a previous completed } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayDelayError.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayDelayError.java index a928db62a6..e889adfe1a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayDelayError.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatArrayDelayError.java @@ -39,7 +39,7 @@ public MaybeConcatArrayDelayError(MaybeSource[] sources) { @Override protected void subscribeActual(Subscriber s) { - ConcatMaybeObserver parent = new ConcatMaybeObserver(s, sources); + ConcatMaybeObserver parent = new ConcatMaybeObserver<>(s, sources); s.onSubscribe(parent); parent.drain(); } @@ -71,7 +71,7 @@ static final class ConcatMaybeObserver this.sources = sources; this.requested = new AtomicLong(); this.disposables = new SequentialDisposable(); - this.current = new AtomicReference(NotificationLite.COMPLETE); // as if a previous completed + this.current = new AtomicReference<>(NotificationLite.COMPLETE); // as if a previous completed this.errors = new AtomicThrowable(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java index f3abd7ddb8..ee6e87cbe0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeConcatIterable.java @@ -52,7 +52,7 @@ protected void subscribeActual(Subscriber s) { return; } - ConcatMaybeObserver parent = new ConcatMaybeObserver(s, it); + ConcatMaybeObserver parent = new ConcatMaybeObserver<>(s, it); s.onSubscribe(parent); parent.drain(); } @@ -80,7 +80,7 @@ static final class ConcatMaybeObserver this.sources = sources; this.requested = new AtomicLong(); this.disposables = new SequentialDisposable(); - this.current = new AtomicReference(NotificationLite.COMPLETE); // as if a previous completed + this.current = new AtomicReference<>(NotificationLite.COMPLETE); // as if a previous completed } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreate.java index 2fc0f48414..24293b895b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeCreate.java @@ -39,7 +39,7 @@ public MaybeCreate(MaybeOnSubscribe source) { @Override protected void subscribeActual(MaybeObserver observer) { - Emitter parent = new Emitter(observer); + Emitter parent = new Emitter<>(observer); observer.onSubscribe(parent); try { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelay.java index 8509d90ab4..6885763cbd 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelay.java @@ -42,7 +42,7 @@ public MaybeDelay(MaybeSource source, long delay, TimeUnit unit, Scheduler sc @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new DelayMaybeObserver(observer, delay, unit, scheduler)); + source.subscribe(new DelayMaybeObserver<>(observer, delay, unit, scheduler)); } static final class DelayMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayOtherPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayOtherPublisher.java index 0e3e8758fe..7797a968a5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayOtherPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayOtherPublisher.java @@ -40,7 +40,7 @@ public MaybeDelayOtherPublisher(MaybeSource source, Publisher other) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new DelayMaybeObserver(observer, other)); + source.subscribe(new DelayMaybeObserver<>(observer, other)); } static final class DelayMaybeObserver @@ -52,7 +52,7 @@ static final class DelayMaybeObserver Disposable upstream; DelayMaybeObserver(MaybeObserver actual, Publisher otherSource) { - this.other = new OtherSubscriber(actual); + this.other = new OtherSubscriber<>(actual); this.otherSource = otherSource; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelaySubscriptionOtherPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelaySubscriptionOtherPublisher.java index 09851e4af8..7de18c70a5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelaySubscriptionOtherPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelaySubscriptionOtherPublisher.java @@ -40,7 +40,7 @@ public MaybeDelaySubscriptionOtherPublisher(MaybeSource source, Publisher @Override protected void subscribeActual(MaybeObserver observer) { - other.subscribe(new OtherSubscriber(observer, source)); + other.subscribe(new OtherSubscriber<>(observer, source)); } static final class OtherSubscriber implements FlowableSubscriber, Disposable { @@ -51,7 +51,7 @@ static final class OtherSubscriber implements FlowableSubscriber, Dis Subscription upstream; OtherSubscriber(MaybeObserver actual, MaybeSource source) { - this.main = new DelayMaybeObserver(actual); + this.main = new DelayMaybeObserver<>(actual); this.source = source; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayWithCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayWithCompletable.java index 65a9460ec0..6bdc5d1a98 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayWithCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDelayWithCompletable.java @@ -32,7 +32,7 @@ public MaybeDelayWithCompletable(MaybeSource source, CompletableSource other) @Override protected void subscribeActual(MaybeObserver observer) { - other.subscribe(new OtherObserver(observer, source)); + other.subscribe(new OtherObserver<>(observer, source)); } static final class OtherObserver @@ -64,7 +64,7 @@ public void onError(Throwable e) { @Override public void onComplete() { - source.subscribe(new DelayWithMainObserver(this, downstream)); + source.subscribe(new DelayWithMainObserver<>(this, downstream)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetach.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetach.java index 9c4e4b9070..595c362917 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetach.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDetach.java @@ -30,7 +30,7 @@ public MaybeDetach(MaybeSource source) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new DetachMaybeObserver(observer)); + source.subscribe(new DetachMaybeObserver<>(observer)); } static final class DetachMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccess.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccess.java index 939f817ac9..28943223f0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccess.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoAfterSuccess.java @@ -37,7 +37,7 @@ public MaybeDoAfterSuccess(MaybeSource source, Consumer onAfterSuc @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new DoAfterObserver(observer, onAfterSuccess)); + source.subscribe(new DoAfterObserver<>(observer, onAfterSuccess)); } static final class DoAfterObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoFinally.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoFinally.java index eaeae0234f..0078cf88b1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoFinally.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoFinally.java @@ -39,7 +39,7 @@ public MaybeDoFinally(MaybeSource source, Action onFinally) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new DoFinallyObserver(observer, onFinally)); + source.subscribe(new DoFinallyObserver<>(observer, onFinally)); } static final class DoFinallyObserver extends AtomicInteger implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEvent.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEvent.java index c70397473b..eccc36c606 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEvent.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeDoOnEvent.java @@ -36,7 +36,7 @@ public MaybeDoOnEvent(MaybeSource source, BiConsumer observer) { - source.subscribe(new DoOnEventMaybeObserver(observer, onEvent)); + source.subscribe(new DoOnEventMaybeObserver<>(observer, onEvent)); } static final class DoOnEventMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeEqualSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeEqualSingle.java index e1e7e9fcbf..ddcfc7b3c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeEqualSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeEqualSingle.java @@ -44,7 +44,7 @@ public MaybeEqualSingle(MaybeSource source1, MaybeSource observer) { - EqualCoordinator parent = new EqualCoordinator(observer, isEqual); + EqualCoordinator parent = new EqualCoordinator<>(observer, isEqual); observer.onSubscribe(parent); parent.subscribe(source1, source2); } @@ -65,8 +65,8 @@ static final class EqualCoordinator super(2); this.downstream = actual; this.isEqual = isEqual; - this.observer1 = new EqualObserver(this); - this.observer2 = new EqualObserver(this); + this.observer1 = new EqualObserver<>(this); + this.observer2 = new EqualObserver<>(this); } void subscribe(MaybeSource source1, MaybeSource source2) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilter.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilter.java index a8c4f12176..2ddf5fff27 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilter.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilter.java @@ -36,7 +36,7 @@ public MaybeFilter(MaybeSource source, Predicate predicate) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new FilterMaybeObserver(observer, predicate)); + source.subscribe(new FilterMaybeObserver<>(observer, predicate)); } static final class FilterMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilterSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilterSingle.java index ff8e71b745..0535447881 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilterSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFilterSingle.java @@ -37,7 +37,7 @@ public MaybeFilterSingle(SingleSource source, Predicate predicate) @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new FilterMaybeObserver(observer, predicate)); + source.subscribe(new FilterMaybeObserver<>(observer, predicate)); } static final class FilterMaybeObserver implements SingleObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java index afb27373c5..da5cfe56b8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapBiSelector.java @@ -59,7 +59,7 @@ static final class FlatMapBiMainObserver FlatMapBiMainObserver(MaybeObserver actual, Function> mapper, BiFunction resultSelector) { - this.inner = new InnerObserver(actual, resultSelector); + this.inner = new InnerObserver<>(actual, resultSelector); this.mapper = mapper; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java index fb8bb9d9f1..10fc003bf8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapCompletable.java @@ -39,7 +39,7 @@ public MaybeFlatMapCompletable(MaybeSource source, Function parent = new FlatMapCompletableObserver(observer, mapper); + FlatMapCompletableObserver parent = new FlatMapCompletableObserver<>(observer, mapper); observer.onSubscribe(parent); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java index 8460405dc9..40ca98c351 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableFlowable.java @@ -48,7 +48,7 @@ public MaybeFlatMapIterableFlowable(MaybeSource source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new FlatMapIterableObserver(s, mapper)); + source.subscribe(new FlatMapIterableObserver<>(s, mapper)); } static final class FlatMapIterableObserver @@ -279,7 +279,7 @@ public boolean isEmpty() { @Nullable @Override - public R poll() throws Exception { + public R poll() { Iterator iterator = it; if (iterator != null) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java index afbb56f8e9..c5c24995b1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapIterableObservable.java @@ -44,7 +44,7 @@ public MaybeFlatMapIterableObservable(MaybeSource source, @Override protected void subscribeActual(Observer observer) { - source.subscribe(new FlatMapIterableObserver(observer, mapper)); + source.subscribe(new FlatMapIterableObserver<>(observer, mapper)); } static final class FlatMapIterableObserver @@ -189,7 +189,7 @@ public boolean isEmpty() { @Nullable @Override - public R poll() throws Exception { + public R poll() { Iterator iterator = it; if (iterator != null) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java index 2f50f8cd7d..e357938db2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapNotification.java @@ -48,7 +48,7 @@ public MaybeFlatMapNotification(MaybeSource source, @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new FlatMapMaybeObserver(observer, onSuccessMapper, onErrorMapper, onCompleteSupplier)); + source.subscribe(new FlatMapMaybeObserver<>(observer, onSuccessMapper, onErrorMapper, onCompleteSupplier)); } static final class FlatMapMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java index 38ec9417d9..2199ea470b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingle.java @@ -41,7 +41,7 @@ public MaybeFlatMapSingle(MaybeSource source, Function downstream) { - source.subscribe(new FlatMapMaybeObserver(downstream, mapper)); + source.subscribe(new FlatMapMaybeObserver<>(downstream, mapper)); } static final class FlatMapMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java index b917a5885e..49b7100575 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatMapSingleElement.java @@ -42,7 +42,7 @@ public MaybeFlatMapSingleElement(MaybeSource source, Function downstream) { - source.subscribe(new FlatMapMaybeObserver(downstream, mapper)); + source.subscribe(new FlatMapMaybeObserver<>(downstream, mapper)); } static final class FlatMapMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java index 325c4db2ea..21d306095c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFlatten.java @@ -39,7 +39,7 @@ public MaybeFlatten(MaybeSource source, Function observer) { - source.subscribe(new FlatMapMaybeObserver(observer, mapper)); + source.subscribe(new FlatMapMaybeObserver<>(observer, mapper)); } static final class FlatMapMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java index 2b7b658390..4c83bbeee7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromRunnable.java @@ -58,7 +58,7 @@ protected void subscribeActual(MaybeObserver observer) { } @Override - public T get() throws Throwable { + public T get() { runnable.run(); return null; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSingle.java index 7ea38ea986..c74a73ee25 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeFromSingle.java @@ -38,7 +38,7 @@ public SingleSource source() { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new FromSingleObserver(observer)); + source.subscribe(new FromSingleObserver<>(observer)); } static final class FromSingleObserver implements SingleObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeHide.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeHide.java index 2950734c37..4dcddd0c27 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeHide.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeHide.java @@ -30,7 +30,7 @@ public MaybeHide(MaybeSource source) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new HideMaybeObserver(observer)); + source.subscribe(new HideMaybeObserver<>(observer)); } static final class HideMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElement.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElement.java index 8750bc4f7e..1c9242548d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElement.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElement.java @@ -30,7 +30,7 @@ public MaybeIgnoreElement(MaybeSource source) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new IgnoreMaybeObserver(observer)); + source.subscribe(new IgnoreMaybeObserver<>(observer)); } static final class IgnoreMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElementCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElementCompletable.java index 4b2ea8d462..6e9c1e9d67 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElementCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIgnoreElementCompletable.java @@ -34,12 +34,12 @@ public MaybeIgnoreElementCompletable(MaybeSource source) { @Override protected void subscribeActual(CompletableObserver observer) { - source.subscribe(new IgnoreMaybeObserver(observer)); + source.subscribe(new IgnoreMaybeObserver<>(observer)); } @Override public Maybe fuseToMaybe() { - return RxJavaPlugins.onAssembly(new MaybeIgnoreElement(source)); + return RxJavaPlugins.onAssembly(new MaybeIgnoreElement<>(source)); } static final class IgnoreMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmpty.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmpty.java index 83ec1d278a..64fb3046ba 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmpty.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmpty.java @@ -31,7 +31,7 @@ public MaybeIsEmpty(MaybeSource source) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new IsEmptyMaybeObserver(observer)); + source.subscribe(new IsEmptyMaybeObserver<>(observer)); } static final class IsEmptyMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmptySingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmptySingle.java index c2f3bf4428..a7fa1882b8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmptySingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeIsEmptySingle.java @@ -41,12 +41,12 @@ public MaybeSource source() { @Override public Maybe fuseToMaybe() { - return RxJavaPlugins.onAssembly(new MaybeIsEmpty(source)); + return RxJavaPlugins.onAssembly(new MaybeIsEmpty<>(source)); } @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new IsEmptyMaybeObserver(observer)); + source.subscribe(new IsEmptyMaybeObserver<>(observer)); } static final class IsEmptyMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterialize.java index 02e6691578..2690bc4797 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMaterialize.java @@ -34,6 +34,6 @@ public MaybeMaterialize(Maybe source) { @Override protected void subscribeActual(SingleObserver> observer) { - source.subscribe(new MaterializeSingleObserver(observer)); + source.subscribe(new MaterializeSingleObserver<>(observer)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java index 2910a2e803..e550b16fba 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeMergeArray.java @@ -47,11 +47,11 @@ protected void subscribeActual(Subscriber s) { SimpleQueueWithConsumerIndex queue; if (n <= bufferSize()) { - queue = new MpscFillOnceSimpleQueue(n); + queue = new MpscFillOnceSimpleQueue<>(n); } else { - queue = new ClqSimpleQueue(); + queue = new ClqSimpleQueue<>(); } - MergeMaybeObserver parent = new MergeMaybeObserver(s, n, queue); + MergeMaybeObserver parent = new MergeMaybeObserver<>(s, n, queue); s.onSubscribe(parent); @@ -110,7 +110,7 @@ public int requestFusion(int mode) { @Nullable @SuppressWarnings("unchecked") @Override - public T poll() throws Exception { + public T poll() { for (;;) { Object o = queue.poll(); if (o != NotificationLite.COMPLETE) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeObserveOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeObserveOn.java index 0bf89494e9..7bf03b080e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeObserveOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeObserveOn.java @@ -35,7 +35,7 @@ public MaybeObserveOn(MaybeSource source, Scheduler scheduler) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new ObserveOnMaybeObserver(observer, scheduler)); + source.subscribe(new ObserveOnMaybeObserver<>(observer, scheduler)); } static final class ObserveOnMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorComplete.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorComplete.java index cd82570f58..8792a93cc5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorComplete.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorComplete.java @@ -37,7 +37,7 @@ public MaybeOnErrorComplete(MaybeSource source, @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new OnErrorCompleteMaybeObserver(observer, predicate)); + source.subscribe(new OnErrorCompleteMaybeObserver<>(observer, predicate)); } static final class OnErrorCompleteMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java index 901016f39a..434685c074 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorNext.java @@ -43,7 +43,7 @@ public MaybeOnErrorNext(MaybeSource source, @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new OnErrorNextMaybeObserver(observer, resumeFunction, allowFatal)); + source.subscribe(new OnErrorNextMaybeObserver<>(observer, resumeFunction, allowFatal)); } static final class OnErrorNextMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java index c7ab7b8524..1a75120e84 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeOnErrorReturn.java @@ -37,7 +37,7 @@ public MaybeOnErrorReturn(MaybeSource source, @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new OnErrorReturnMaybeObserver(observer, valueSupplier)); + source.subscribe(new OnErrorReturnMaybeObserver<>(observer, valueSupplier)); } static final class OnErrorReturnMaybeObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeek.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeek.java index 1b4b717367..3b5427ba0f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeek.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybePeek.java @@ -53,7 +53,7 @@ public MaybePeek(MaybeSource source, Consumer onSubscribe @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new MaybePeekObserver(observer, this)); + source.subscribe(new MaybePeekObserver<>(observer, this)); } static final class MaybePeekObserver implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSubscribeOn.java index 8a934d5216..543b7c2b6a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSubscribeOn.java @@ -34,10 +34,10 @@ public MaybeSubscribeOn(MaybeSource source, Scheduler scheduler) { @Override protected void subscribeActual(MaybeObserver observer) { - SubscribeOnMaybeObserver parent = new SubscribeOnMaybeObserver(observer); + SubscribeOnMaybeObserver parent = new SubscribeOnMaybeObserver<>(observer); observer.onSubscribe(parent); - parent.task.replace(scheduler.scheduleDirect(new SubscribeTask(parent, source))); + parent.task.replace(scheduler.scheduleDirect(new SubscribeTask<>(parent, source))); } static final class SubscribeTask implements Runnable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmpty.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmpty.java index e1b218078e..3be260441c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmpty.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmpty.java @@ -35,7 +35,7 @@ public MaybeSwitchIfEmpty(MaybeSource source, MaybeSource other) @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new SwitchIfEmptyMaybeObserver(observer, other)); + source.subscribe(new SwitchIfEmptyMaybeObserver<>(observer, other)); } static final class SwitchIfEmptyMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmptySingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmptySingle.java index 3533092a14..142e39f1f6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmptySingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeSwitchIfEmptySingle.java @@ -42,7 +42,7 @@ public MaybeSource source() { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new SwitchIfEmptyMaybeObserver(observer, other)); + source.subscribe(new SwitchIfEmptyMaybeObserver<>(observer, other)); } static final class SwitchIfEmptyMaybeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilMaybe.java index 9b53acf077..6c9498c065 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilMaybe.java @@ -38,7 +38,7 @@ public MaybeTakeUntilMaybe(MaybeSource source, MaybeSource other) { @Override protected void subscribeActual(MaybeObserver observer) { - TakeUntilMainMaybeObserver parent = new TakeUntilMainMaybeObserver(observer); + TakeUntilMainMaybeObserver parent = new TakeUntilMainMaybeObserver<>(observer); observer.onSubscribe(parent); other.subscribe(parent.other); @@ -57,7 +57,7 @@ static final class TakeUntilMainMaybeObserver TakeUntilMainMaybeObserver(MaybeObserver downstream) { this.downstream = downstream; - this.other = new TakeUntilOtherMaybeObserver(this); + this.other = new TakeUntilOtherMaybeObserver<>(this); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilPublisher.java index 0ef77d40e2..e86cc7c324 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTakeUntilPublisher.java @@ -41,7 +41,7 @@ public MaybeTakeUntilPublisher(MaybeSource source, Publisher other) { @Override protected void subscribeActual(MaybeObserver observer) { - TakeUntilMainMaybeObserver parent = new TakeUntilMainMaybeObserver(observer); + TakeUntilMainMaybeObserver parent = new TakeUntilMainMaybeObserver<>(observer); observer.onSubscribe(parent); other.subscribe(parent.other); @@ -60,7 +60,7 @@ static final class TakeUntilMainMaybeObserver TakeUntilMainMaybeObserver(MaybeObserver downstream) { this.downstream = downstream; - this.other = new TakeUntilOtherMaybeObserver(this); + this.other = new TakeUntilOtherMaybeObserver<>(this); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutMaybe.java index aee0b524c2..03783489a3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutMaybe.java @@ -42,7 +42,7 @@ public MaybeTimeoutMaybe(MaybeSource source, MaybeSource other, MaybeSourc @Override protected void subscribeActual(MaybeObserver observer) { - TimeoutMainMaybeObserver parent = new TimeoutMainMaybeObserver(observer, fallback); + TimeoutMainMaybeObserver parent = new TimeoutMainMaybeObserver<>(observer, fallback); observer.onSubscribe(parent); other.subscribe(parent.other); @@ -66,9 +66,9 @@ static final class TimeoutMainMaybeObserver TimeoutMainMaybeObserver(MaybeObserver actual, MaybeSource fallback) { this.downstream = actual; - this.other = new TimeoutOtherMaybeObserver(this); + this.other = new TimeoutOtherMaybeObserver<>(this); this.fallback = fallback; - this.otherObserver = fallback != null ? new TimeoutFallbackMaybeObserver(actual) : null; + this.otherObserver = fallback != null ? new TimeoutFallbackMaybeObserver<>(actual) : null; } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutPublisher.java index ae1a2efd18..76545730f3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeTimeoutPublisher.java @@ -45,7 +45,7 @@ public MaybeTimeoutPublisher(MaybeSource source, Publisher other, MaybeSou @Override protected void subscribeActual(MaybeObserver observer) { - TimeoutMainMaybeObserver parent = new TimeoutMainMaybeObserver(observer, fallback); + TimeoutMainMaybeObserver parent = new TimeoutMainMaybeObserver<>(observer, fallback); observer.onSubscribe(parent); other.subscribe(parent.other); @@ -69,9 +69,9 @@ static final class TimeoutMainMaybeObserver TimeoutMainMaybeObserver(MaybeObserver actual, MaybeSource fallback) { this.downstream = actual; - this.other = new TimeoutOtherMaybeObserver(this); + this.other = new TimeoutOtherMaybeObserver<>(this); this.fallback = fallback; - this.otherObserver = fallback != null ? new TimeoutFallbackMaybeObserver(actual) : null; + this.otherObserver = fallback != null ? new TimeoutFallbackMaybeObserver<>(actual) : null; } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToFlowable.java index 11b713265d..c31ad262ab 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToFlowable.java @@ -42,7 +42,7 @@ public MaybeSource source() { @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new MaybeToFlowableSubscriber(s)); + source.subscribe(new MaybeToFlowableSubscriber<>(s)); } static final class MaybeToFlowableSubscriber extends DeferredScalarSubscription diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToObservable.java index 75129b3514..4b27918621 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToObservable.java @@ -52,7 +52,7 @@ protected void subscribeActual(Observer observer) { * @since 2.2 */ public static MaybeObserver create(Observer downstream) { - return new MaybeToObservableObserver(downstream); + return new MaybeToObservableObserver<>(downstream); } static final class MaybeToObservableObserver extends DeferredScalarDisposable diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToPublisher.java index b6552e9726..914021c914 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToPublisher.java @@ -30,7 +30,7 @@ public static Function, Publisher> instance() { } @Override - public Publisher apply(MaybeSource t) throws Exception { - return new MaybeToFlowable(t); + public Publisher apply(MaybeSource t) { + return new MaybeToFlowable<>(t); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToSingle.java index c620cef68e..3199521639 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeToSingle.java @@ -43,7 +43,7 @@ public MaybeSource source() { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new ToSingleMaybeSubscriber(observer, defaultValue)); + source.subscribe(new ToSingleMaybeSubscriber<>(observer, defaultValue)); } static final class ToSingleMaybeSubscriber implements MaybeObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUnsubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUnsubscribeOn.java index 6e29ffa610..2f07161dc9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUnsubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeUnsubscribeOn.java @@ -35,7 +35,7 @@ public MaybeUnsubscribeOn(MaybeSource source, Scheduler scheduler) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new UnsubscribeOnMaybeObserver(observer, scheduler)); + source.subscribe(new UnsubscribeOnMaybeObserver<>(observer, scheduler)); } static final class UnsubscribeOnMaybeObserver extends AtomicReference diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java index d4bf3b0a77..b155e019fc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipArray.java @@ -40,11 +40,11 @@ protected void subscribeActual(MaybeObserver observer) { int n = sources.length; if (n == 1) { - sources[0].subscribe(new MaybeMap.MapMaybeObserver(observer, new SingletonArrayFunc())); + sources[0].subscribe(new MaybeMap.MapMaybeObserver<>(observer, new SingletonArrayFunc())); return; } - ZipCoordinator parent = new ZipCoordinator(observer, n, zipper); + ZipCoordinator parent = new ZipCoordinator<>(observer, n, zipper); observer.onSubscribe(parent); @@ -82,7 +82,7 @@ static final class ZipCoordinator extends AtomicInteger implements Disposa this.zipper = zipper; ZipMaybeObserver[] o = new ZipMaybeObserver[n]; for (int i = 0; i < n; i++) { - o[i] = new ZipMaybeObserver(this, i); + o[i] = new ZipMaybeObserver<>(this, i); } this.observers = o; this.values = new Object[n]; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java index 7c2d85c97b..839b373920 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/maybe/MaybeZipIterable.java @@ -62,11 +62,11 @@ protected void subscribeActual(MaybeObserver observer) { } if (n == 1) { - a[0].subscribe(new MaybeMap.MapMaybeObserver(observer, new SingletonArrayFunc())); + a[0].subscribe(new MaybeMap.MapMaybeObserver<>(observer, new SingletonArrayFunc())); return; } - ZipCoordinator parent = new ZipCoordinator(observer, n, zipper); + ZipCoordinator parent = new ZipCoordinator<>(observer, n, zipper); observer.onSubscribe(parent); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/CompletableAndThenObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/CompletableAndThenObservable.java index 28bdae282a..2a3263a50a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/CompletableAndThenObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/CompletableAndThenObservable.java @@ -40,7 +40,7 @@ public CompletableAndThenObservable(CompletableSource source, @Override protected void subscribeActual(Observer observer) { - AndThenObservableObserver parent = new AndThenObservableObserver(observer, other); + AndThenObservableObserver parent = new AndThenObservableObserver<>(observer, other); observer.onSubscribe(parent); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java index 986af99049..e4a3f36c05 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapCompletable.java @@ -57,7 +57,7 @@ public FlowableConcatMapCompletable(Flowable source, @Override protected void subscribeActual(CompletableObserver observer) { - source.subscribe(new ConcatMapCompletableObserver(observer, mapper, errorMode, prefetch)); + source.subscribe(new ConcatMapCompletableObserver<>(observer, mapper, errorMode, prefetch)); } static final class ConcatMapCompletableObserver @@ -99,7 +99,7 @@ static final class ConcatMapCompletableObserver this.prefetch = prefetch; this.errors = new AtomicThrowable(); this.inner = new ConcatMapInnerObserver(this); - this.queue = new SpscArrayQueue(prefetch); + this.queue = new SpscArrayQueue<>(prefetch); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java index cd8b9fa77f..5f71148e0d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapMaybe.java @@ -58,7 +58,7 @@ public FlowableConcatMapMaybe(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ConcatMapMaybeSubscriber(s, mapper, prefetch, errorMode)); + source.subscribe(new ConcatMapMaybeSubscriber<>(s, mapper, prefetch, errorMode)); } static final class ConcatMapMaybeSubscriber @@ -113,8 +113,8 @@ static final class ConcatMapMaybeSubscriber this.errorMode = errorMode; this.requested = new AtomicLong(); this.errors = new AtomicThrowable(); - this.inner = new ConcatMapMaybeObserver(this); - this.queue = new SpscArrayQueue(prefetch); + this.inner = new ConcatMapMaybeObserver<>(this); + this.queue = new SpscArrayQueue<>(prefetch); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java index 78334d4a1f..f6cd939546 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableConcatMapSingle.java @@ -58,7 +58,7 @@ public FlowableConcatMapSingle(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new ConcatMapSingleSubscriber(s, mapper, prefetch, errorMode)); + source.subscribe(new ConcatMapSingleSubscriber<>(s, mapper, prefetch, errorMode)); } static final class ConcatMapSingleSubscriber @@ -113,8 +113,8 @@ static final class ConcatMapSingleSubscriber this.errorMode = errorMode; this.requested = new AtomicLong(); this.errors = new AtomicThrowable(); - this.inner = new ConcatMapSingleObserver(this); - this.queue = new SpscArrayQueue(prefetch); + this.inner = new ConcatMapSingleObserver<>(this); + this.queue = new SpscArrayQueue<>(prefetch); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java index a2d66067ab..1bc0fb709e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapCompletable.java @@ -52,7 +52,7 @@ public FlowableSwitchMapCompletable(Flowable source, @Override protected void subscribeActual(CompletableObserver observer) { - source.subscribe(new SwitchMapCompletableObserver(observer, mapper, delayErrors)); + source.subscribe(new SwitchMapCompletableObserver<>(observer, mapper, delayErrors)); } static final class SwitchMapCompletableObserver implements FlowableSubscriber, Disposable { @@ -79,7 +79,7 @@ static final class SwitchMapCompletableObserver implements FlowableSubscriber this.mapper = mapper; this.delayErrors = delayErrors; this.errors = new AtomicThrowable(); - this.inner = new AtomicReference(); + this.inner = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java index 11ea0a4c77..7933b4ec13 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapMaybe.java @@ -54,7 +54,7 @@ public FlowableSwitchMapMaybe(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SwitchMapMaybeSubscriber(s, mapper, delayErrors)); + source.subscribe(new SwitchMapMaybeSubscriber<>(s, mapper, delayErrors)); } static final class SwitchMapMaybeSubscriber extends AtomicInteger @@ -75,7 +75,7 @@ static final class SwitchMapMaybeSubscriber extends AtomicInteger final AtomicReference> inner; static final SwitchMapMaybeObserver INNER_DISPOSED = - new SwitchMapMaybeObserver(null); + new SwitchMapMaybeObserver<>(null); Subscription upstream; @@ -93,7 +93,7 @@ static final class SwitchMapMaybeSubscriber extends AtomicInteger this.delayErrors = delayErrors; this.errors = new AtomicThrowable(); this.requested = new AtomicLong(); - this.inner = new AtomicReference>(); + this.inner = new AtomicReference<>(); } @Override @@ -125,7 +125,7 @@ public void onNext(T t) { return; } - SwitchMapMaybeObserver observer = new SwitchMapMaybeObserver(this); + SwitchMapMaybeObserver observer = new SwitchMapMaybeObserver<>(this); for (;;) { current = inner.get(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java index c41b52f6e5..4fe83126e5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/FlowableSwitchMapSingle.java @@ -54,7 +54,7 @@ public FlowableSwitchMapSingle(Flowable source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new SwitchMapSingleSubscriber(s, mapper, delayErrors)); + source.subscribe(new SwitchMapSingleSubscriber<>(s, mapper, delayErrors)); } static final class SwitchMapSingleSubscriber extends AtomicInteger @@ -75,7 +75,7 @@ static final class SwitchMapSingleSubscriber extends AtomicInteger final AtomicReference> inner; static final SwitchMapSingleObserver INNER_DISPOSED = - new SwitchMapSingleObserver(null); + new SwitchMapSingleObserver<>(null); Subscription upstream; @@ -93,7 +93,7 @@ static final class SwitchMapSingleSubscriber extends AtomicInteger this.delayErrors = delayErrors; this.errors = new AtomicThrowable(); this.requested = new AtomicLong(); - this.inner = new AtomicReference>(); + this.inner = new AtomicReference<>(); } @Override @@ -125,7 +125,7 @@ public void onNext(T t) { return; } - SwitchMapSingleObserver observer = new SwitchMapSingleObserver(this); + SwitchMapSingleObserver observer = new SwitchMapSingleObserver<>(this); for (;;) { current = inner.get(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaterializeSingleObserver.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaterializeSingleObserver.java index 551b32d4b1..d9cb5592ac 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaterializeSingleObserver.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaterializeSingleObserver.java @@ -45,17 +45,17 @@ public void onSubscribe(Disposable d) { @Override public void onComplete() { - downstream.onSuccess(Notification.createOnComplete()); + downstream.onSuccess(Notification.createOnComplete()); } @Override public void onSuccess(T t) { - downstream.onSuccess(Notification.createOnNext(t)); + downstream.onSuccess(Notification.createOnNext(t)); } @Override public void onError(Throwable e) { - downstream.onSuccess(Notification.createOnError(e)); + downstream.onSuccess(Notification.createOnError(e)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java index 891c2a0f33..ae42405f74 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapObservable.java @@ -44,7 +44,7 @@ public MaybeFlatMapObservable(MaybeSource source, @Override protected void subscribeActual(Observer observer) { - FlatMapObserver parent = new FlatMapObserver(observer, mapper); + FlatMapObserver parent = new FlatMapObserver<>(observer, mapper); observer.onSubscribe(parent); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java index 1ac179bcca..c590755704 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/MaybeFlatMapPublisher.java @@ -47,7 +47,7 @@ public MaybeFlatMapPublisher(MaybeSource source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new FlatMapPublisherSubscriber(s, mapper)); + source.subscribe(new FlatMapPublisherSubscriber<>(s, mapper)); } static final class FlatMapPublisherSubscriber diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java index 44f322e39d..2295f4397e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapCompletable.java @@ -55,7 +55,7 @@ public ObservableConcatMapCompletable(Observable source, @Override protected void subscribeActual(CompletableObserver observer) { if (!ScalarXMapZHelper.tryAsCompletable(source, mapper, observer)) { - source.subscribe(new ConcatMapCompletableObserver(observer, mapper, errorMode, prefetch)); + source.subscribe(new ConcatMapCompletableObserver<>(observer, mapper, errorMode, prefetch)); } } @@ -120,7 +120,7 @@ public void onSubscribe(Disposable d) { return; } } - queue = new SpscLinkedArrayQueue(prefetch); + queue = new SpscLinkedArrayQueue<>(prefetch); downstream.onSubscribe(this); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java index c76b0115cb..1177d5a06e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapMaybe.java @@ -56,7 +56,7 @@ public ObservableConcatMapMaybe(Observable source, @Override protected void subscribeActual(Observer observer) { if (!ScalarXMapZHelper.tryAsMaybe(source, mapper, observer)) { - source.subscribe(new ConcatMapMaybeMainObserver(observer, mapper, prefetch, errorMode)); + source.subscribe(new ConcatMapMaybeMainObserver<>(observer, mapper, prefetch, errorMode)); } } @@ -102,8 +102,8 @@ static final class ConcatMapMaybeMainObserver this.mapper = mapper; this.errorMode = errorMode; this.errors = new AtomicThrowable(); - this.inner = new ConcatMapMaybeObserver(this); - this.queue = new SpscLinkedArrayQueue(prefetch); + this.inner = new ConcatMapMaybeObserver<>(this); + this.queue = new SpscLinkedArrayQueue<>(prefetch); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java index 34d7d04734..5fc6b8a520 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableConcatMapSingle.java @@ -56,7 +56,7 @@ public ObservableConcatMapSingle(Observable source, @Override protected void subscribeActual(Observer observer) { if (!ScalarXMapZHelper.tryAsSingle(source, mapper, observer)) { - source.subscribe(new ConcatMapSingleMainObserver(observer, mapper, prefetch, errorMode)); + source.subscribe(new ConcatMapSingleMainObserver<>(observer, mapper, prefetch, errorMode)); } } @@ -102,8 +102,8 @@ static final class ConcatMapSingleMainObserver this.mapper = mapper; this.errorMode = errorMode; this.errors = new AtomicThrowable(); - this.inner = new ConcatMapSingleObserver(this); - this.queue = new SpscLinkedArrayQueue(prefetch); + this.inner = new ConcatMapSingleObserver<>(this); + this.queue = new SpscLinkedArrayQueue<>(prefetch); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java index a7a6a7fd99..bb6ec70f85 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapCompletable.java @@ -50,7 +50,7 @@ public ObservableSwitchMapCompletable(Observable source, @Override protected void subscribeActual(CompletableObserver observer) { if (!ScalarXMapZHelper.tryAsCompletable(source, mapper, observer)) { - source.subscribe(new SwitchMapCompletableObserver(observer, mapper, delayErrors)); + source.subscribe(new SwitchMapCompletableObserver<>(observer, mapper, delayErrors)); } } @@ -78,7 +78,7 @@ static final class SwitchMapCompletableObserver implements Observer, Dispo this.mapper = mapper; this.delayErrors = delayErrors; this.errors = new AtomicThrowable(); - this.inner = new AtomicReference(); + this.inner = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java index bf3c9f43c2..f4c9bf6c0c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapMaybe.java @@ -52,7 +52,7 @@ public ObservableSwitchMapMaybe(Observable source, @Override protected void subscribeActual(Observer observer) { if (!ScalarXMapZHelper.tryAsMaybe(source, mapper, observer)) { - source.subscribe(new SwitchMapMaybeMainObserver(observer, mapper, delayErrors)); + source.subscribe(new SwitchMapMaybeMainObserver<>(observer, mapper, delayErrors)); } } @@ -72,7 +72,7 @@ static final class SwitchMapMaybeMainObserver extends AtomicInteger final AtomicReference> inner; static final SwitchMapMaybeObserver INNER_DISPOSED = - new SwitchMapMaybeObserver(null); + new SwitchMapMaybeObserver<>(null); Disposable upstream; @@ -87,7 +87,7 @@ static final class SwitchMapMaybeMainObserver extends AtomicInteger this.mapper = mapper; this.delayErrors = delayErrors; this.errors = new AtomicThrowable(); - this.inner = new AtomicReference>(); + this.inner = new AtomicReference<>(); } @Override @@ -118,7 +118,7 @@ public void onNext(T t) { return; } - SwitchMapMaybeObserver observer = new SwitchMapMaybeObserver(this); + SwitchMapMaybeObserver observer = new SwitchMapMaybeObserver<>(this); for (;;) { current = inner.get(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java index bc19c88b55..f5db39d5d5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/ObservableSwitchMapSingle.java @@ -52,7 +52,7 @@ public ObservableSwitchMapSingle(Observable source, @Override protected void subscribeActual(Observer observer) { if (!ScalarXMapZHelper.tryAsSingle(source, mapper, observer)) { - source.subscribe(new SwitchMapSingleMainObserver(observer, mapper, delayErrors)); + source.subscribe(new SwitchMapSingleMainObserver<>(observer, mapper, delayErrors)); } } @@ -72,7 +72,7 @@ static final class SwitchMapSingleMainObserver extends AtomicInteger final AtomicReference> inner; static final SwitchMapSingleObserver INNER_DISPOSED = - new SwitchMapSingleObserver(null); + new SwitchMapSingleObserver<>(null); Disposable upstream; @@ -87,7 +87,7 @@ static final class SwitchMapSingleMainObserver extends AtomicInteger this.mapper = mapper; this.delayErrors = delayErrors; this.errors = new AtomicThrowable(); - this.inner = new AtomicReference>(); + this.inner = new AtomicReference<>(); } @Override @@ -118,7 +118,7 @@ public void onNext(T t) { return; } - SwitchMapSingleObserver observer = new SwitchMapSingleObserver(this); + SwitchMapSingleObserver observer = new SwitchMapSingleObserver<>(this); for (;;) { current = inner.get(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java index bfb466bac1..c3dd9b59c1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/mixed/SingleFlatMapObservable.java @@ -44,7 +44,7 @@ public SingleFlatMapObservable(SingleSource source, @Override protected void subscribeActual(Observer observer) { - FlatMapObserver parent = new FlatMapObserver(observer, mapper); + FlatMapObserver parent = new FlatMapObserver<>(observer, mapper); observer.onSubscribe(parent); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableIterable.java index 0ba2da1720..dce3e2bc21 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableIterable.java @@ -35,7 +35,7 @@ public BlockingObservableIterable(ObservableSource source, int buff @Override public Iterator iterator() { - BlockingObservableIterator it = new BlockingObservableIterator(bufferSize); + BlockingObservableIterator it = new BlockingObservableIterator<>(bufferSize); source.subscribe(it); return it; } @@ -56,7 +56,7 @@ static final class BlockingObservableIterator volatile Throwable error; BlockingObservableIterator(int batchSize) { - this.queue = new SpscLinkedArrayQueue(batchSize); + this.queue = new SpscLinkedArrayQueue<>(batchSize); this.lock = new ReentrantLock(); this.condition = lock.newCondition(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableLatest.java index aaa975dbf7..e431db42a7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableLatest.java @@ -38,7 +38,7 @@ public BlockingObservableLatest(ObservableSource source) { @Override public Iterator iterator() { - BlockingObservableLatestIterator lio = new BlockingObservableLatestIterator(); + BlockingObservableLatestIterator lio = new BlockingObservableLatestIterator<>(); Observable> materialized = Observable.wrap(source).materialize(); @@ -52,7 +52,7 @@ static final class BlockingObservableLatestIterator extends DisposableObserve final Semaphore notify = new Semaphore(0); // observer's notification - final AtomicReference> value = new AtomicReference>(); + final AtomicReference> value = new AtomicReference<>(); @Override public void onNext(Notification args) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableMostRecent.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableMostRecent.java index 20408d2607..1332351cac 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableMostRecent.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableMostRecent.java @@ -40,7 +40,7 @@ public BlockingObservableMostRecent(ObservableSource source, T initialValue) @Override public Iterator iterator() { - MostRecentObserver mostRecentObserver = new MostRecentObserver(initialValue); + MostRecentObserver mostRecentObserver = new MostRecentObserver<>(initialValue); source.subscribe(mostRecentObserver); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNext.java index 1d5cca354b..53aed40999 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/BlockingObservableNext.java @@ -39,8 +39,8 @@ public BlockingObservableNext(ObservableSource source) { @Override public Iterator iterator() { - NextObserver nextObserver = new NextObserver(); - return new NextIterator(source, nextObserver); + NextObserver nextObserver = new NextObserver<>(); + return new NextIterator<>(source, nextObserver); } // test needs to access the observer.waiting flag @@ -80,7 +80,7 @@ private boolean moveToNext() { started = true; // if not started, start now observer.setWaiting(); - new ObservableMaterialize(items).subscribe(observer); + new ObservableMaterialize<>(items).subscribe(observer); } Notification nextNotification; @@ -130,7 +130,7 @@ public void remove() { } static final class NextObserver extends DisposableObserver> { - private final BlockingQueue> buf = new ArrayBlockingQueue>(1); + private final BlockingQueue> buf = new ArrayBlockingQueue<>(1); final AtomicInteger waiting = new AtomicInteger(); @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAll.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAll.java index a28da3650d..3f62b4732c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAll.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAll.java @@ -28,7 +28,7 @@ public ObservableAll(ObservableSource source, Predicate predicate) @Override protected void subscribeActual(Observer t) { - source.subscribe(new AllObserver(t, predicate)); + source.subscribe(new AllObserver<>(t, predicate)); } static final class AllObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllSingle.java index db004ef966..8c167d4c47 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAllSingle.java @@ -31,12 +31,12 @@ public ObservableAllSingle(ObservableSource source, Predicate pred @Override protected void subscribeActual(SingleObserver t) { - source.subscribe(new AllObserver(t, predicate)); + source.subscribe(new AllObserver<>(t, predicate)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableAll(source, predicate)); + return RxJavaPlugins.onAssembly(new ObservableAll<>(source, predicate)); } static final class AllObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmb.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmb.java index aa8fae849d..8bc43aa41c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmb.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAmb.java @@ -68,7 +68,7 @@ public void subscribeActual(Observer observer) { return; } - AmbCoordinator ac = new AmbCoordinator(observer, count); + AmbCoordinator ac = new AmbCoordinator<>(observer, count); ac.subscribe(sources); } @@ -88,7 +88,7 @@ public void subscribe(ObservableSource[] sources) { AmbInnerObserver[] as = observers; int len = as.length; for (int i = 0; i < len; i++) { - as[i] = new AmbInnerObserver(this, i + 1, downstream); + as[i] = new AmbInnerObserver<>(this, i + 1, downstream); } winner.lazySet(0); // release the contents of 'as' downstream.onSubscribe(this); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAny.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAny.java index 69d3298d40..bda497f7ed 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAny.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAny.java @@ -28,7 +28,7 @@ public ObservableAny(ObservableSource source, Predicate predicate) @Override protected void subscribeActual(Observer t) { - source.subscribe(new AnyObserver(t, predicate)); + source.subscribe(new AnyObserver<>(t, predicate)); } static final class AnyObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnySingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnySingle.java index d3c3e495a5..6e263a0d48 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnySingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableAnySingle.java @@ -32,12 +32,12 @@ public ObservableAnySingle(ObservableSource source, Predicate pred @Override protected void subscribeActual(SingleObserver t) { - source.subscribe(new AnyObserver(t, predicate)); + source.subscribe(new AnyObserver<>(t, predicate)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableAny(source, predicate)); + return RxJavaPlugins.onAssembly(new ObservableAny<>(source, predicate)); } static final class AnyObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java index c02bcfca88..2c88f007e9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBlockingSubscribe.java @@ -41,9 +41,9 @@ private ObservableBlockingSubscribe() { * @param the value type */ public static void subscribe(ObservableSource o, Observer observer) { - final BlockingQueue queue = new LinkedBlockingQueue(); + final BlockingQueue queue = new LinkedBlockingQueue<>(); - BlockingObserver bs = new BlockingObserver(queue); + BlockingObserver bs = new BlockingObserver<>(queue); observer.onSubscribe(bs); o.subscribe(bs); @@ -76,7 +76,7 @@ public static void subscribe(ObservableSource o, Observer void subscribe(ObservableSource o) { BlockingIgnoringReceiver callback = new BlockingIgnoringReceiver(); - LambdaObserver ls = new LambdaObserver(Functions.emptyConsumer(), + LambdaObserver ls = new LambdaObserver<>(Functions.emptyConsumer(), callback, callback, Functions.emptyConsumer()); o.subscribe(ls); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java index bd7f4b1baa..2688212575 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBuffer.java @@ -39,12 +39,12 @@ public ObservableBuffer(ObservableSource source, int count, int skip, Supplie @Override protected void subscribeActual(Observer t) { if (skip == count) { - BufferExactObserver bes = new BufferExactObserver(t, count, bufferSupplier); + BufferExactObserver bes = new BufferExactObserver<>(t, count, bufferSupplier); if (bes.createBuffer()) { source.subscribe(bes); } } else { - source.subscribe(new BufferSkipObserver(t, count, skip, bufferSupplier)); + source.subscribe(new BufferSkipObserver<>(t, count, skip, bufferSupplier)); } } @@ -157,7 +157,7 @@ static final class BufferSkipObserver> this.count = count; this.skip = skip; this.bufferSupplier = bufferSupplier; - this.buffers = new ArrayDeque(); + this.buffers = new ArrayDeque<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java index cef9ef41d8..4b0027f4fe 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferBoundary.java @@ -43,7 +43,7 @@ public ObservableBufferBoundary(ObservableSource source, ObservableSource t) { BufferBoundaryObserver parent = - new BufferBoundaryObserver( + new BufferBoundaryObserver<>( t, bufferOpen, bufferClose, bufferSupplier ); t.onSubscribe(parent); @@ -88,10 +88,10 @@ static final class BufferBoundaryObserver, Op this.bufferSupplier = bufferSupplier; this.bufferOpen = bufferOpen; this.bufferClose = bufferClose; - this.queue = new SpscLinkedArrayQueue(bufferSize()); + this.queue = new SpscLinkedArrayQueue<>(bufferSize()); this.observers = new CompositeDisposable(); - this.upstream = new AtomicReference(); - this.buffers = new LinkedHashMap(); + this.upstream = new AtomicReference<>(); + this.buffers = new LinkedHashMap<>(); this.errors = new AtomicThrowable(); } @@ -99,7 +99,7 @@ static final class BufferBoundaryObserver, Op public void onSubscribe(Disposable d) { if (DisposableHelper.setOnce(this.upstream, d)) { - BufferOpenObserver open = new BufferOpenObserver(this); + BufferOpenObserver open = new BufferOpenObserver<>(this); observers.add(open); bufferOpen.subscribe(open); @@ -190,7 +190,7 @@ void open(Open token) { bufs.put(idx, buf); } - BufferCloseObserver bc = new BufferCloseObserver(this, idx); + BufferCloseObserver bc = new BufferCloseObserver<>(this, idx); observers.add(bc); p.subscribe(bc); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java index 7d76354cf9..849222505e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferExactBoundary.java @@ -39,7 +39,7 @@ public ObservableBufferExactBoundary(ObservableSource source, ObservableSourc @Override protected void subscribeActual(Observer t) { - source.subscribe(new BufferExactBoundaryObserver(new SerializedObserver(t), bufferSupplier, boundary)); + source.subscribe(new BufferExactBoundaryObserver<>(new SerializedObserver<>(t), bufferSupplier, boundary)); } static final class BufferExactBoundaryObserver, B> @@ -56,7 +56,7 @@ static final class BufferExactBoundaryObserver actual, Supplier bufferSupplier, ObservableSource boundary) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.boundary = boundary; } @@ -80,7 +80,7 @@ public void onSubscribe(Disposable d) { buffer = b; - BufferBoundaryObserver bs = new BufferBoundaryObserver(this); + BufferBoundaryObserver bs = new BufferBoundaryObserver<>(this); other = bs; downstream.onSubscribe(this); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java index 3b12c3d0f5..8a7becc241 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableBufferTimed.java @@ -55,16 +55,16 @@ public ObservableBufferTimed(ObservableSource source, long timespan, long tim @Override protected void subscribeActual(Observer t) { if (timespan == timeskip && maxSize == Integer.MAX_VALUE) { - source.subscribe(new BufferExactUnboundedObserver( - new SerializedObserver(t), + source.subscribe(new BufferExactUnboundedObserver<>( + new SerializedObserver<>(t), bufferSupplier, timespan, unit, scheduler)); return; } Scheduler.Worker w = scheduler.createWorker(); if (timespan == timeskip) { - source.subscribe(new BufferExactBoundedObserver( - new SerializedObserver(t), + source.subscribe(new BufferExactBoundedObserver<>( + new SerializedObserver<>(t), bufferSupplier, timespan, unit, maxSize, restartTimerOnMaxSize, w )); @@ -72,8 +72,8 @@ protected void subscribeActual(Observer t) { } // Can't use maxSize because what to do if a buffer is full but its // timespan hasn't been elapsed? - source.subscribe(new BufferSkipBoundedObserver( - new SerializedObserver(t), + source.subscribe(new BufferSkipBoundedObserver<>( + new SerializedObserver<>(t), bufferSupplier, timespan, timeskip, unit, w)); } @@ -89,12 +89,12 @@ static final class BufferExactUnboundedObserver timer = new AtomicReference(); + final AtomicReference timer = new AtomicReference<>(); BufferExactUnboundedObserver( Observer actual, Supplier bufferSupplier, long timespan, TimeUnit unit, Scheduler scheduler) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.timespan = timespan; this.unit = unit; @@ -228,13 +228,13 @@ static final class BufferSkipBoundedObserver> BufferSkipBoundedObserver(Observer actual, Supplier bufferSupplier, long timespan, long timeskip, TimeUnit unit, Worker w) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.timespan = timespan; this.timeskip = timeskip; this.unit = unit; this.w = w; - this.buffers = new LinkedList(); + this.buffers = new LinkedList<>(); } @Override @@ -285,7 +285,7 @@ public void onError(Throwable t) { public void onComplete() { List bs; synchronized (this) { - bs = new ArrayList(buffers); + bs = new ArrayList<>(buffers); buffers.clear(); } @@ -409,7 +409,7 @@ static final class BufferExactBoundedObserver Supplier bufferSupplier, long timespan, TimeUnit unit, int maxSize, boolean restartOnMaxSize, Worker w) { - super(actual, new MpscLinkedQueue()); + super(actual, new MpscLinkedQueue<>()); this.bufferSupplier = bufferSupplier; this.timespan = timespan; this.unit = unit; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCache.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCache.java index d03ad2008e..c3e7447e6e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCache.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCache.java @@ -95,15 +95,15 @@ public ObservableCache(Observable source, int capacityHint) { super(source); this.capacityHint = capacityHint; this.once = new AtomicBoolean(); - Node n = new Node(capacityHint); + Node n = new Node<>(capacityHint); this.head = n; this.tail = n; - this.observers = new AtomicReference[]>(EMPTY); + this.observers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(Observer t) { - CacheDisposable consumer = new CacheDisposable(t, this); + CacheDisposable consumer = new CacheDisposable<>(t, this); t.onSubscribe(consumer); add(consumer); @@ -292,7 +292,7 @@ public void onNext(T t) { int tailOffset = this.tailOffset; // if the current tail node is full, create a fresh node if (tailOffset == capacityHint) { - Node n = new Node(tailOffset); + Node n = new Node<>(tailOffset); n.values[0] = t; this.tailOffset = 1; tail.next = n; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java index 76b03eddb6..6bddb34aac 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollect.java @@ -43,7 +43,7 @@ protected void subscribeActual(Observer t) { return; } - source.subscribe(new CollectObserver(t, u, collector)); + source.subscribe(new CollectObserver<>(t, u, collector)); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java index 07db8d5bf3..82f8923de3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCollectSingle.java @@ -47,12 +47,12 @@ protected void subscribeActual(SingleObserver t) { return; } - source.subscribe(new CollectObserver(t, u, collector)); + source.subscribe(new CollectObserver<>(t, u, collector)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableCollect(source, initialSupplier, collector)); + return RxJavaPlugins.onAssembly(new ObservableCollect<>(source, initialSupplier, collector)); } static final class CollectObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java index 0cc8e3e45e..34dce80431 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCombineLatest.java @@ -66,7 +66,7 @@ public void subscribeActual(Observer observer) { return; } - LatestCoordinator lc = new LatestCoordinator(observer, combiner, count, bufferSize, delayError); + LatestCoordinator lc = new LatestCoordinator<>(observer, combiner, count, bufferSize, delayError); lc.subscribe(sources); } @@ -99,10 +99,10 @@ static final class LatestCoordinator extends AtomicInteger implements Disp this.latest = new Object[count]; CombinerObserver[] as = new CombinerObserver[count]; for (int i = 0; i < count; i++) { - as[i] = new CombinerObserver(this, i); + as[i] = new CombinerObserver<>(this, i); } this.observers = as; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); } public void subscribe(ObservableSource[] sources) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java index b96b1ec8a0..9b03b9fc35 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMap.java @@ -48,10 +48,10 @@ public void subscribeActual(Observer observer) { } if (delayErrors == ErrorMode.IMMEDIATE) { - SerializedObserver serial = new SerializedObserver(observer); - source.subscribe(new SourceObserver(serial, mapper, bufferSize)); + SerializedObserver serial = new SerializedObserver<>(observer); + source.subscribe(new SourceObserver<>(serial, mapper, bufferSize)); } else { - source.subscribe(new ConcatMapDelayErrorObserver(observer, mapper, bufferSize, delayErrors == ErrorMode.END)); + source.subscribe(new ConcatMapDelayErrorObserver<>(observer, mapper, bufferSize, delayErrors == ErrorMode.END)); } } @@ -80,7 +80,7 @@ static final class SourceObserver extends AtomicInteger implements Observe this.downstream = actual; this.mapper = mapper; this.bufferSize = bufferSize; - this.inner = new InnerObserver(actual, this); + this.inner = new InnerObserver<>(actual, this); } @Override @@ -113,7 +113,7 @@ public void onSubscribe(Disposable d) { } } - queue = new SpscLinkedArrayQueue(bufferSize); + queue = new SpscLinkedArrayQueue<>(bufferSize); downstream.onSubscribe(this); } @@ -305,7 +305,7 @@ static final class ConcatMapDelayErrorObserver this.bufferSize = bufferSize; this.tillTheEnd = tillTheEnd; this.errors = new AtomicThrowable(); - this.observer = new DelayErrorInnerObserver(actual, this); + this.observer = new DelayErrorInnerObserver<>(actual, this); } @Override @@ -338,7 +338,7 @@ public void onSubscribe(Disposable d) { } } - queue = new SpscLinkedArrayQueue(bufferSize); + queue = new SpscLinkedArrayQueue<>(bufferSize); downstream.onSubscribe(this); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java index fe2bed8c8c..792940b5e0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapEager.java @@ -50,7 +50,7 @@ public ObservableConcatMapEager(ObservableSource source, @Override protected void subscribeActual(Observer observer) { - source.subscribe(new ConcatMapEagerMainObserver(observer, mapper, maxConcurrency, prefetch, errorMode)); + source.subscribe(new ConcatMapEagerMainObserver<>(observer, mapper, maxConcurrency, prefetch, errorMode)); } static final class ConcatMapEagerMainObserver @@ -96,7 +96,7 @@ static final class ConcatMapEagerMainObserver this.prefetch = prefetch; this.errorMode = errorMode; this.errors = new AtomicThrowable(); - this.observers = new ArrayDeque>(); + this.observers = new ArrayDeque<>(); } @SuppressWarnings("unchecked") @@ -129,7 +129,7 @@ public void onSubscribe(Disposable d) { } } - queue = new SpscLinkedArrayQueue(prefetch); + queue = new SpscLinkedArrayQueue<>(prefetch); downstream.onSubscribe(this); } @@ -282,7 +282,7 @@ public void drain() { return; } - InnerQueuedObserver inner = new InnerQueuedObserver(this, prefetch); + InnerQueuedObserver inner = new InnerQueuedObserver<>(this, prefetch); observers.offer(inner); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java index f45dd92f1f..b4cd8352b5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatMapScheduler.java @@ -48,10 +48,10 @@ public ObservableConcatMapScheduler(ObservableSource source, Function observer) { if (delayErrors == ErrorMode.IMMEDIATE) { - SerializedObserver serial = new SerializedObserver(observer); - source.subscribe(new ConcatMapObserver(serial, mapper, bufferSize, scheduler.createWorker())); + SerializedObserver serial = new SerializedObserver<>(observer); + source.subscribe(new ConcatMapObserver<>(serial, mapper, bufferSize, scheduler.createWorker())); } else { - source.subscribe(new ConcatMapDelayErrorObserver(observer, mapper, bufferSize, delayErrors == ErrorMode.END, scheduler.createWorker())); + source.subscribe(new ConcatMapDelayErrorObserver<>(observer, mapper, bufferSize, delayErrors == ErrorMode.END, scheduler.createWorker())); } } @@ -81,7 +81,7 @@ static final class ConcatMapObserver extends AtomicInteger implements Obse this.downstream = actual; this.mapper = mapper; this.bufferSize = bufferSize; - this.inner = new InnerObserver(actual, this); + this.inner = new InnerObserver<>(actual, this); this.worker = worker; } @@ -115,7 +115,7 @@ public void onSubscribe(Disposable d) { } } - queue = new SpscLinkedArrayQueue(bufferSize); + queue = new SpscLinkedArrayQueue<>(bufferSize); downstream.onSubscribe(this); } @@ -318,7 +318,7 @@ static final class ConcatMapDelayErrorObserver this.bufferSize = bufferSize; this.tillTheEnd = tillTheEnd; this.errors = new AtomicThrowable(); - this.observer = new DelayErrorInnerObserver(actual, this); + this.observer = new DelayErrorInnerObserver<>(actual, this); this.worker = worker; } @@ -352,7 +352,7 @@ public void onSubscribe(Disposable d) { } } - queue = new SpscLinkedArrayQueue(bufferSize); + queue = new SpscLinkedArrayQueue<>(bufferSize); downstream.onSubscribe(this); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletable.java index 333428a80b..4ffc8f5490 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithCompletable.java @@ -37,7 +37,7 @@ public ObservableConcatWithCompletable(Observable source, CompletableSource o @Override protected void subscribeActual(Observer observer) { - source.subscribe(new ConcatWithObserver(observer, other)); + source.subscribe(new ConcatWithObserver<>(observer, other)); } static final class ConcatWithObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybe.java index 7f3d8d8716..19af4e5641 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithMaybe.java @@ -37,7 +37,7 @@ public ObservableConcatWithMaybe(Observable source, MaybeSource @Override protected void subscribeActual(Observer observer) { - source.subscribe(new ConcatWithObserver(observer, other)); + source.subscribe(new ConcatWithObserver<>(observer, other)); } static final class ConcatWithObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingle.java index 3c91beb30f..b2e1860110 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableConcatWithSingle.java @@ -37,7 +37,7 @@ public ObservableConcatWithSingle(Observable source, SingleSource observer) { - source.subscribe(new ConcatWithObserver(observer, other)); + source.subscribe(new ConcatWithObserver<>(observer, other)); } static final class ConcatWithObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCountSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCountSingle.java index 84042bfbea..5796519f74 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCountSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCountSingle.java @@ -32,7 +32,7 @@ public void subscribeActual(SingleObserver t) { @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableCount(source)); + return RxJavaPlugins.onAssembly(new ObservableCount<>(source)); } static final class CountObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreate.java index fe1f394ff3..66445bc8d1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableCreate.java @@ -33,7 +33,7 @@ public ObservableCreate(ObservableOnSubscribe source) { @Override protected void subscribeActual(Observer observer) { - CreateEmitter parent = new CreateEmitter(observer); + CreateEmitter parent = new CreateEmitter<>(observer); observer.onSubscribe(parent); try { @@ -113,7 +113,7 @@ public void setCancellable(Cancellable c) { @Override public ObservableEmitter serialize() { - return new SerializedEmitter(this); + return new SerializedEmitter<>(this); } @Override @@ -154,7 +154,7 @@ static final class SerializedEmitter SerializedEmitter(ObservableEmitter emitter) { this.emitter = emitter; this.errors = new AtomicThrowable(); - this.queue = new SpscLinkedArrayQueue(16); + this.queue = new SpscLinkedArrayQueue<>(16); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java index 20f98731c1..16f882d6c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounce.java @@ -34,7 +34,7 @@ public ObservableDebounce(ObservableSource source, Function t) { - source.subscribe(new DebounceObserver(new SerializedObserver(t), debounceSelector)); + source.subscribe(new DebounceObserver<>(new SerializedObserver<>(t), debounceSelector)); } static final class DebounceObserver @@ -44,7 +44,7 @@ static final class DebounceObserver Disposable upstream; - final AtomicReference debouncer = new AtomicReference(); + final AtomicReference debouncer = new AtomicReference<>(); volatile long index; @@ -89,7 +89,7 @@ public void onNext(T t) { return; } - DebounceInnerObserver dis = new DebounceInnerObserver(this, idx, t); + DebounceInnerObserver dis = new DebounceInnerObserver<>(this, idx, t); if (debouncer.compareAndSet(d, dis)) { p.subscribe(dis); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTimed.java index 626fe77928..b0b19ef0dc 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDebounceTimed.java @@ -37,8 +37,8 @@ public ObservableDebounceTimed(ObservableSource source, long timeout, TimeUni @Override public void subscribeActual(Observer t) { - source.subscribe(new DebounceTimedObserver( - new SerializedObserver(t), + source.subscribe(new DebounceTimedObserver<>( + new SerializedObserver<>(t), timeout, unit, scheduler.createWorker())); } @@ -85,7 +85,7 @@ public void onNext(T t) { d.dispose(); } - DebounceEmitter de = new DebounceEmitter(t, idx, this); + DebounceEmitter de = new DebounceEmitter<>(t, idx, this); timer = de; d = worker.schedule(de, timeout, unit); de.setResource(d); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelay.java index 61e424a3eb..93cbb00389 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDelay.java @@ -42,12 +42,12 @@ public void subscribeActual(Observer t) { if (delayError) { observer = (Observer)t; } else { - observer = new SerializedObserver(t); + observer = new SerializedObserver<>(t); } Scheduler.Worker w = scheduler.createWorker(); - source.subscribe(new DelayObserver(observer, delay, unit, w, delayError)); + source.subscribe(new DelayObserver<>(observer, delay, unit, w, delayError)); } static final class DelayObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java index 45b42e1ecc..37f5fdadd8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDematerialize.java @@ -33,7 +33,7 @@ public ObservableDematerialize(ObservableSource source, Function observer) { - source.subscribe(new DematerializeObserver(observer, selector)); + source.subscribe(new DematerializeObserver<>(observer, selector)); } static final class DematerializeObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetach.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetach.java index 7665a6b4f8..763fcdde97 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetach.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDetach.java @@ -32,7 +32,7 @@ public ObservableDetach(ObservableSource source) { @Override protected void subscribeActual(Observer observer) { - source.subscribe(new DetachObserver(observer)); + source.subscribe(new DetachObserver<>(observer)); } static final class DetachObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java index aadeafb2b1..cd766ef559 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinct.java @@ -49,7 +49,7 @@ protected void subscribeActual(Observer observer) { return; } - source.subscribe(new DistinctObserver(observer, keySelector, collection)); + source.subscribe(new DistinctObserver<>(observer, keySelector, collection)); } static final class DistinctObserver extends BasicFuseableObserver { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChanged.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChanged.java index a36e92a3c4..f18e47030d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChanged.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDistinctUntilChanged.java @@ -32,7 +32,7 @@ public ObservableDistinctUntilChanged(ObservableSource source, Function observer) { - source.subscribe(new DistinctUntilChangedObserver(observer, keySelector, comparer)); + source.subscribe(new DistinctUntilChangedObserver<>(observer, keySelector, comparer)); } static final class DistinctUntilChangedObserver extends BasicFuseableObserver { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNext.java index 212b84fccc..91051e7562 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoAfterNext.java @@ -35,7 +35,7 @@ public ObservableDoAfterNext(ObservableSource source, Consumer onA @Override protected void subscribeActual(Observer observer) { - source.subscribe(new DoAfterObserver(observer, onAfterNext)); + source.subscribe(new DoAfterObserver<>(observer, onAfterNext)); } static final class DoAfterObserver extends BasicFuseableObserver { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinally.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinally.java index e5087d0ee4..8c7ee32d09 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinally.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoFinally.java @@ -40,7 +40,7 @@ public ObservableDoFinally(ObservableSource source, Action onFinally) { @Override protected void subscribeActual(Observer observer) { - source.subscribe(new DoFinallyObserver(observer, onFinally)); + source.subscribe(new DoFinallyObserver<>(observer, onFinally)); } static final class DoFinallyObserver extends BasicIntQueueDisposable implements Observer { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEach.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEach.java index bbc34fed83..ff884192fa 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEach.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnEach.java @@ -39,7 +39,7 @@ public ObservableDoOnEach(ObservableSource source, Consumer onNext @Override public void subscribeActual(Observer t) { - source.subscribe(new DoOnEachObserver(t, onNext, onError, onComplete, onAfterTerminate)); + source.subscribe(new DoOnEachObserver<>(t, onNext, onError, onComplete, onAfterTerminate)); } static final class DoOnEachObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnLifecycle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnLifecycle.java index b59cfdca64..835cd886b2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnLifecycle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableDoOnLifecycle.java @@ -30,6 +30,6 @@ public ObservableDoOnLifecycle(Observable upstream, Consumer observer) { - source.subscribe(new DisposableLambdaObserver(observer, onSubscribe, onDispose)); + source.subscribe(new DisposableLambdaObserver<>(observer, onSubscribe, onDispose)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAt.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAt.java index 28ff883d03..b8d042a532 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAt.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAt.java @@ -34,7 +34,7 @@ public ObservableElementAt(ObservableSource source, long index, T defaultValu @Override public void subscribeActual(Observer t) { - source.subscribe(new ElementAtObserver(t, index, defaultValue, errorOnFewer)); + source.subscribe(new ElementAtObserver<>(t, index, defaultValue, errorOnFewer)); } static final class ElementAtObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtMaybe.java index ad987cccd2..53b1312092 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtMaybe.java @@ -29,12 +29,12 @@ public ObservableElementAtMaybe(ObservableSource source, long index) { @Override public void subscribeActual(MaybeObserver t) { - source.subscribe(new ElementAtObserver(t, index)); + source.subscribe(new ElementAtObserver<>(t, index)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableElementAt(source, index, null, false)); + return RxJavaPlugins.onAssembly(new ObservableElementAt<>(source, index, null, false)); } static final class ElementAtObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtSingle.java index 397e678265..bb006ca2dd 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableElementAtSingle.java @@ -34,12 +34,12 @@ public ObservableElementAtSingle(ObservableSource source, long index, T defau @Override public void subscribeActual(SingleObserver t) { - source.subscribe(new ElementAtObserver(t, index, defaultValue)); + source.subscribe(new ElementAtObserver<>(t, index, defaultValue)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableElementAt(source, index, defaultValue, true)); + return RxJavaPlugins.onAssembly(new ObservableElementAt<>(source, index, defaultValue, true)); } static final class ElementAtObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilter.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilter.java index 3bbacc6250..51fec2cc68 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilter.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFilter.java @@ -27,7 +27,7 @@ public ObservableFilter(ObservableSource source, Predicate predica @Override public void subscribeActual(Observer observer) { - source.subscribe(new FilterObserver(observer, predicate)); + source.subscribe(new FilterObserver<>(observer, predicate)); } static final class FilterObserver extends BasicFuseableObserver { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java index 7e01d7cbc1..a169627ed3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMap.java @@ -50,7 +50,7 @@ public void subscribeActual(Observer t) { return; } - source.subscribe(new MergeObserver(t, mapper, delayErrors, maxConcurrency, bufferSize)); + source.subscribe(new MergeObserver<>(t, mapper, delayErrors, maxConcurrency, bufferSize)); } static final class MergeObserver extends AtomicInteger implements Disposable, Observer { @@ -95,9 +95,9 @@ static final class MergeObserver extends AtomicInteger implements Disposab this.maxConcurrency = maxConcurrency; this.bufferSize = bufferSize; if (maxConcurrency != Integer.MAX_VALUE) { - sources = new ArrayDeque>(maxConcurrency); + sources = new ArrayDeque<>(maxConcurrency); } - this.observers = new AtomicReference[]>(EMPTY); + this.observers = new AtomicReference<>(EMPTY); } @Override @@ -158,7 +158,7 @@ void subscribeInner(ObservableSource p) { break; } } else { - InnerObserver inner = new InnerObserver(this, uniqueId++); + InnerObserver inner = new InnerObserver<>(this, uniqueId++); if (addInner(inner)) { p.subscribe(inner); } @@ -239,9 +239,9 @@ boolean tryEmitScalar(Supplier value) { SimplePlainQueue q = queue; if (q == null) { if (maxConcurrency == Integer.MAX_VALUE) { - q = new SpscLinkedArrayQueue(bufferSize); + q = new SpscLinkedArrayQueue<>(bufferSize); } else { - q = new SpscArrayQueue(maxConcurrency); + q = new SpscArrayQueue<>(maxConcurrency); } queue = q; } @@ -267,7 +267,7 @@ void tryEmit(U value, InnerObserver inner) { } else { SimpleQueue q = inner.queue; if (q == null) { - q = new SpscLinkedArrayQueue(bufferSize); + q = new SpscLinkedArrayQueue<>(bufferSize); inner.queue = q; } q.offer(value); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java index 75fc80ac61..729e8cf6c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletable.java @@ -44,7 +44,7 @@ public ObservableFlatMapCompletable(ObservableSource source, @Override protected void subscribeActual(Observer observer) { - source.subscribe(new FlatMapCompletableMainObserver(observer, mapper, delayErrors)); + source.subscribe(new FlatMapCompletableMainObserver<>(observer, mapper, delayErrors)); } static final class FlatMapCompletableMainObserver extends BasicIntQueueDisposable @@ -145,7 +145,7 @@ public boolean isDisposed() { @Nullable @Override - public T poll() throws Exception { + public T poll() { return null; // always empty } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java index ddf9602a05..2d488ac2c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapCompletableCompletable.java @@ -46,12 +46,12 @@ public ObservableFlatMapCompletableCompletable(ObservableSource source, @Override protected void subscribeActual(CompletableObserver observer) { - source.subscribe(new FlatMapCompletableMainObserver(observer, mapper, delayErrors)); + source.subscribe(new FlatMapCompletableMainObserver<>(observer, mapper, delayErrors)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableFlatMapCompletable(source, mapper, delayErrors)); + return RxJavaPlugins.onAssembly(new ObservableFlatMapCompletable<>(source, mapper, delayErrors)); } static final class FlatMapCompletableMainObserver extends AtomicInteger implements Disposable, Observer { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java index ac951a571c..7ac5ca6f3c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapMaybe.java @@ -44,7 +44,7 @@ public ObservableFlatMapMaybe(ObservableSource source, Function observer) { - source.subscribe(new FlatMapMaybeObserver(observer, mapper, delayErrors)); + source.subscribe(new FlatMapMaybeObserver<>(observer, mapper, delayErrors)); } static final class FlatMapMaybeObserver @@ -79,7 +79,7 @@ static final class FlatMapMaybeObserver this.set = new CompositeDisposable(); this.errors = new AtomicThrowable(); this.active = new AtomicInteger(1); - this.queue = new AtomicReference>(); + this.queue = new AtomicReference<>(); } @Override @@ -177,7 +177,7 @@ SpscLinkedArrayQueue getOrCreateQueue() { if (current != null) { return current; } - current = new SpscLinkedArrayQueue(Observable.bufferSize()); + current = new SpscLinkedArrayQueue<>(Observable.bufferSize()); if (queue.compareAndSet(null, current)) { return current; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java index 6d5dd3da63..8ed45d5f39 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlatMapSingle.java @@ -44,7 +44,7 @@ public ObservableFlatMapSingle(ObservableSource source, Function observer) { - source.subscribe(new FlatMapSingleObserver(observer, mapper, delayErrors)); + source.subscribe(new FlatMapSingleObserver<>(observer, mapper, delayErrors)); } static final class FlatMapSingleObserver @@ -79,7 +79,7 @@ static final class FlatMapSingleObserver this.set = new CompositeDisposable(); this.errors = new AtomicThrowable(); this.active = new AtomicInteger(1); - this.queue = new AtomicReference>(); + this.queue = new AtomicReference<>(); } @Override @@ -177,7 +177,7 @@ SpscLinkedArrayQueue getOrCreateQueue() { if (current != null) { return current; } - current = new SpscLinkedArrayQueue(Observable.bufferSize()); + current = new SpscLinkedArrayQueue<>(Observable.bufferSize()); if (queue.compareAndSet(null, current)) { return current; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java index 3f90f31acd..732eb50dd3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFlattenIterable.java @@ -41,7 +41,7 @@ public ObservableFlattenIterable(ObservableSource source, @Override protected void subscribeActual(Observer observer) { - source.subscribe(new FlattenIterableObserver(observer, mapper)); + source.subscribe(new FlattenIterableObserver<>(observer, mapper)); } static final class FlattenIterableObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java index fb4beaaf21..97ebfe0d4f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromArray.java @@ -27,7 +27,7 @@ public ObservableFromArray(T[] array) { @Override public void subscribeActual(Observer observer) { - FromArrayDisposable d = new FromArrayDisposable(observer, array); + FromArrayDisposable d = new FromArrayDisposable<>(observer, array); observer.onSubscribe(d); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallable.java index db08487f81..0285b83ff2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromCallable.java @@ -36,7 +36,7 @@ public ObservableFromCallable(Callable callable) { @Override public void subscribeActual(Observer observer) { - DeferredScalarDisposable d = new DeferredScalarDisposable(observer); + DeferredScalarDisposable d = new DeferredScalarDisposable<>(observer); observer.onSubscribe(d); if (d.isDisposed()) { return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromFuture.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromFuture.java index aa9e7da0bb..5459429f4d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromFuture.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromFuture.java @@ -33,7 +33,7 @@ public ObservableFromFuture(Future future, long timeout, TimeUnit u @Override public void subscribeActual(Observer observer) { - DeferredScalarDisposable d = new DeferredScalarDisposable(observer); + DeferredScalarDisposable d = new DeferredScalarDisposable<>(observer); observer.onSubscribe(d); if (!d.isDisposed()) { T v; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java index 9edf260931..187d0cecc3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromIterable.java @@ -51,7 +51,7 @@ public void subscribeActual(Observer observer) { return; } - FromIterableDisposable d = new FromIterableDisposable(observer, it); + FromIterableDisposable d = new FromIterableDisposable<>(observer, it); observer.onSubscribe(d); if (!d.fusionMode) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplier.java index 44b6dbdd00..e46cf8fe8f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableFromSupplier.java @@ -35,7 +35,7 @@ public ObservableFromSupplier(Supplier supplier) { @Override public void subscribeActual(Observer observer) { - DeferredScalarDisposable d = new DeferredScalarDisposable(observer); + DeferredScalarDisposable d = new DeferredScalarDisposable<>(observer); observer.onSubscribe(d); if (d.isDisposed()) { return; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGenerate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGenerate.java index df94a37dbd..2bd9a866a1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGenerate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGenerate.java @@ -45,7 +45,7 @@ public void subscribeActual(Observer observer) { return; } - GeneratorDisposable gd = new GeneratorDisposable(observer, generator, disposeState, state); + GeneratorDisposable gd = new GeneratorDisposable<>(observer, generator, disposeState, state); observer.onSubscribe(gd); gd.run(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java index 78edf33cea..8e259db47c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupBy.java @@ -44,7 +44,7 @@ public ObservableGroupBy(ObservableSource source, @Override public void subscribeActual(Observer> t) { - source.subscribe(new GroupByObserver(t, keySelector, valueSelector, bufferSize, delayError)); + source.subscribe(new GroupByObserver<>(t, keySelector, valueSelector, bufferSize, delayError)); } public static final class GroupByObserver extends AtomicInteger implements Observer, Disposable { @@ -70,7 +70,7 @@ public GroupByObserver(Observer> actual, Functio this.valueSelector = valueSelector; this.bufferSize = bufferSize; this.delayError = delayError; - this.groups = new ConcurrentHashMap>(); + this.groups = new ConcurrentHashMap<>(); this.lazySet(1); } @@ -139,7 +139,7 @@ public void onNext(T t) { @Override public void onError(Throwable t) { - List> list = new ArrayList>(groups.values()); + List> list = new ArrayList<>(groups.values()); groups.clear(); for (GroupedUnicast e : list) { @@ -151,7 +151,7 @@ public void onError(Throwable t) { @Override public void onComplete() { - List> list = new ArrayList>(groups.values()); + List> list = new ArrayList<>(groups.values()); groups.clear(); for (GroupedUnicast e : list) { @@ -191,8 +191,8 @@ static final class GroupedUnicast extends GroupedObservable { final State state; public static GroupedUnicast createWith(K key, int bufferSize, GroupByObserver parent, boolean delayError) { - State state = new State(bufferSize, parent, key, delayError); - return new GroupedUnicast(key, state); + State state = new State<>(bufferSize, parent, key, delayError); + return new GroupedUnicast<>(key, state); } protected GroupedUnicast(K key, State state) { @@ -232,7 +232,7 @@ static final class State extends AtomicInteger implements Disposable, Obse final AtomicBoolean cancelled = new AtomicBoolean(); - final AtomicReference> actual = new AtomicReference>(); + final AtomicReference> actual = new AtomicReference<>(); final AtomicInteger once = new AtomicInteger(); @@ -242,7 +242,7 @@ static final class State extends AtomicInteger implements Disposable, Obse static final int ABANDONED_HAS_SUBSCRIBER = ABANDONED | HAS_SUBSCRIBER; State(int bufferSize, GroupByObserver parent, K key, boolean delayError) { - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); this.parent = parent; this.key = key; this.delayError = delayError; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java index 7b12579966..3605ee5203 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableGroupJoin.java @@ -58,7 +58,7 @@ public ObservableGroupJoin( protected void subscribeActual(Observer observer) { GroupJoinDisposable parent = - new GroupJoinDisposable(observer, leftEnd, rightEnd, resultSelector); + new GroupJoinDisposable<>(observer, leftEnd, rightEnd, resultSelector); observer.onSubscribe(parent); @@ -130,10 +130,10 @@ static final class GroupJoinDisposable BiFunction, ? extends R> resultSelector) { this.downstream = actual; this.disposables = new CompositeDisposable(); - this.queue = new SpscLinkedArrayQueue(bufferSize()); - this.lefts = new LinkedHashMap>(); - this.rights = new LinkedHashMap(); - this.error = new AtomicReference(); + this.queue = new SpscLinkedArrayQueue<>(bufferSize()); + this.lefts = new LinkedHashMap<>(); + this.rights = new LinkedHashMap<>(); + this.error = new AtomicReference<>(); this.leftEnd = leftEnd; this.rightEnd = rightEnd; this.resultSelector = resultSelector; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableHide.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableHide.java index b929da4f78..5b216cd44c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableHide.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableHide.java @@ -31,7 +31,7 @@ public ObservableHide(ObservableSource source) { @Override protected void subscribeActual(Observer o) { - source.subscribe(new HideDisposable(o)); + source.subscribe(new HideDisposable<>(o)); } static final class HideDisposable implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElements.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElements.java index a633c80dbb..ec4a07e92a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElements.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElements.java @@ -24,7 +24,7 @@ public ObservableIgnoreElements(ObservableSource source) { @Override public void subscribeActual(final Observer t) { - source.subscribe(new IgnoreObservable(t)); + source.subscribe(new IgnoreObservable<>(t)); } static final class IgnoreObservable implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsCompletable.java index f19ea85008..14b3a77684 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableIgnoreElementsCompletable.java @@ -28,12 +28,12 @@ public ObservableIgnoreElementsCompletable(ObservableSource source) { @Override public void subscribeActual(final CompletableObserver t) { - source.subscribe(new IgnoreObservable(t)); + source.subscribe(new IgnoreObservable<>(t)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableIgnoreElements(source)); + return RxJavaPlugins.onAssembly(new ObservableIgnoreElements<>(source)); } static final class IgnoreObservable implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java index fcd5f57fa5..5f9dce646d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableInternalHelper.java @@ -44,7 +44,7 @@ public S apply(S t1, Emitter t2) throws Throwable { } public static BiFunction, S> simpleGenerator(Consumer> consumer) { - return new SimpleGenerator(consumer); + return new SimpleGenerator<>(consumer); } static final class SimpleBiGenerator implements BiFunction, S> { @@ -62,7 +62,7 @@ public S apply(S t1, Emitter t2) throws Throwable { } public static BiFunction, S> simpleBiGenerator(BiConsumer> consumer) { - return new SimpleBiGenerator(consumer); + return new SimpleBiGenerator<>(consumer); } static final class ItemDelayFunction implements Function> { @@ -75,12 +75,12 @@ static final class ItemDelayFunction implements Function apply(final T v) throws Throwable { ObservableSource o = Objects.requireNonNull(itemDelay.apply(v), "The itemDelay returned a null ObservableSource"); - return new ObservableTake(o, 1).map(Functions.justFunction(v)).defaultIfEmpty(v); + return new ObservableTake<>(o, 1).map(Functions.justFunction(v)).defaultIfEmpty(v); } } public static Function> itemDelay(final Function> itemDelay) { - return new ItemDelayFunction(itemDelay); + return new ItemDelayFunction<>(itemDelay); } static final class ObserverOnNext implements Consumer { @@ -91,7 +91,7 @@ static final class ObserverOnNext implements Consumer { } @Override - public void accept(T v) throws Exception { + public void accept(T v) { observer.onNext(v); } } @@ -104,7 +104,7 @@ static final class ObserverOnError implements Consumer { } @Override - public void accept(Throwable v) throws Exception { + public void accept(Throwable v) { observer.onError(v); } } @@ -117,21 +117,21 @@ static final class ObserverOnComplete implements Action { } @Override - public void run() throws Exception { + public void run() { observer.onComplete(); } } public static Consumer observerOnNext(Observer observer) { - return new ObserverOnNext(observer); + return new ObserverOnNext<>(observer); } public static Consumer observerOnError(Observer observer) { - return new ObserverOnError(observer); + return new ObserverOnError<>(observer); } public static Action observerOnComplete(Observer observer) { - return new ObserverOnComplete(observer); + return new ObserverOnComplete<>(observer); } static final class FlatMapWithCombinerInner implements Function { @@ -163,14 +163,14 @@ static final class FlatMapWithCombinerOuter implements Function apply(final T t) throws Throwable { @SuppressWarnings("unchecked") ObservableSource u = (ObservableSource)Objects.requireNonNull(mapper.apply(t), "The mapper returned a null ObservableSource"); - return new ObservableMap(u, new FlatMapWithCombinerInner(combiner, t)); + return new ObservableMap<>(u, new FlatMapWithCombinerInner(combiner, t)); } } public static Function> flatMapWithCombiner( final Function> mapper, final BiFunction combiner) { - return new FlatMapWithCombinerOuter(combiner, mapper); + return new FlatMapWithCombinerOuter<>(combiner, mapper); } static final class FlatMapIntoIterable implements Function> { @@ -182,36 +182,36 @@ static final class FlatMapIntoIterable implements Function apply(T t) throws Throwable { - return new ObservableFromIterable(Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); + return new ObservableFromIterable<>(Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Iterable")); } } public static Function> flatMapIntoIterable(final Function> mapper) { - return new FlatMapIntoIterable(mapper); + return new FlatMapIntoIterable<>(mapper); } enum MapToInt implements Function { INSTANCE; @Override - public Object apply(Object t) throws Exception { + public Object apply(Object t) { return 0; } } public static Supplier> replaySupplier(final Observable parent) { - return new ReplaySupplier(parent); + return new ReplaySupplier<>(parent); } public static Supplier> replaySupplier(final Observable parent, final int bufferSize, boolean eagerTruncate) { - return new BufferedReplaySupplier(parent, bufferSize, eagerTruncate); + return new BufferedReplaySupplier<>(parent, bufferSize, eagerTruncate); } public static Supplier> replaySupplier(final Observable parent, final int bufferSize, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - return new BufferedTimedReplaySupplier(parent, bufferSize, time, unit, scheduler, eagerTruncate); + return new BufferedTimedReplaySupplier<>(parent, bufferSize, time, unit, scheduler, eagerTruncate); } public static Supplier> replaySupplier(final Observable parent, final long time, final TimeUnit unit, final Scheduler scheduler, boolean eagerTruncate) { - return new TimedReplayCallable(parent, time, unit, scheduler, eagerTruncate); + return new TimedReplayCallable<>(parent, time, unit, scheduler, eagerTruncate); } static final class ReplaySupplier implements Supplier> { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java index 6660d342f3..70f62aec03 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJoin.java @@ -56,7 +56,7 @@ public ObservableJoin( protected void subscribeActual(Observer observer) { JoinDisposable parent = - new JoinDisposable( + new JoinDisposable<>( observer, leftEnd, rightEnd, resultSelector); observer.onSubscribe(parent); @@ -115,10 +115,10 @@ static final class JoinDisposable BiFunction resultSelector) { this.downstream = actual; this.disposables = new CompositeDisposable(); - this.queue = new SpscLinkedArrayQueue(bufferSize()); - this.lefts = new LinkedHashMap(); - this.rights = new LinkedHashMap(); - this.error = new AtomicReference(); + this.queue = new SpscLinkedArrayQueue<>(bufferSize()); + this.lefts = new LinkedHashMap<>(); + this.rights = new LinkedHashMap<>(); + this.error = new AtomicReference<>(); this.leftEnd = leftEnd; this.rightEnd = rightEnd; this.resultSelector = resultSelector; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJust.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJust.java index 0fe64a5291..d3448d0b41 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJust.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableJust.java @@ -30,7 +30,7 @@ public ObservableJust(final T value) { @Override protected void subscribeActual(Observer observer) { - ScalarDisposable sd = new ScalarDisposable(observer, value); + ScalarDisposable sd = new ScalarDisposable<>(observer, value); observer.onSubscribe(sd); sd.run(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastMaybe.java index 79457e0e80..73e26a3972 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastMaybe.java @@ -35,7 +35,7 @@ public ObservableLastMaybe(ObservableSource source) { @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new LastObserver(observer)); + source.subscribe(new LastObserver<>(observer)); } static final class LastObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastSingle.java index 2ba8f28856..7c5433e105 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableLastSingle.java @@ -40,7 +40,7 @@ public ObservableLastSingle(ObservableSource source, T defaultItem) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new LastObserver(observer, defaultItem)); + source.subscribe(new LastObserver<>(observer, defaultItem)); } static final class LastObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java index 147c21ae37..1622b504d4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMapNotification.java @@ -40,7 +40,7 @@ public ObservableMapNotification( @Override public void subscribeActual(Observer> t) { - source.subscribe(new MapNotificationObserver(t, onNextMapper, onErrorMapper, onCompleteSupplier)); + source.subscribe(new MapNotificationObserver<>(t, onNextMapper, onErrorMapper, onCompleteSupplier)); } static final class MapNotificationObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterialize.java index 6041961bcb..e55430c378 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMaterialize.java @@ -25,7 +25,7 @@ public ObservableMaterialize(ObservableSource source) { @Override public void subscribeActual(Observer> t) { - source.subscribe(new MaterializeObserver(t)); + source.subscribe(new MaterializeObserver<>(t)); } static final class MaterializeObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletable.java index 6d022ebf62..1f086f7175 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithCompletable.java @@ -38,7 +38,7 @@ public ObservableMergeWithCompletable(Observable source, CompletableSource ot @Override protected void subscribeActual(Observer observer) { - MergeWithObserver parent = new MergeWithObserver(observer); + MergeWithObserver parent = new MergeWithObserver<>(observer); observer.onSubscribe(parent); source.subscribe(parent); other.subscribe(parent.otherObserver); @@ -63,7 +63,7 @@ static final class MergeWithObserver extends AtomicInteger MergeWithObserver(Observer downstream) { this.downstream = downstream; - this.mainDisposable = new AtomicReference(); + this.mainDisposable = new AtomicReference<>(); this.otherObserver = new OtherObserver(this); this.errors = new AtomicThrowable(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybe.java index d852bff7dd..f462075a50 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithMaybe.java @@ -40,7 +40,7 @@ public ObservableMergeWithMaybe(Observable source, MaybeSource o @Override protected void subscribeActual(Observer observer) { - MergeWithObserver parent = new MergeWithObserver(observer); + MergeWithObserver parent = new MergeWithObserver<>(observer); observer.onSubscribe(parent); source.subscribe(parent); other.subscribe(parent.otherObserver); @@ -75,8 +75,8 @@ static final class MergeWithObserver extends AtomicInteger MergeWithObserver(Observer downstream) { this.downstream = downstream; - this.mainDisposable = new AtomicReference(); - this.otherObserver = new OtherObserver(this); + this.mainDisposable = new AtomicReference<>(); + this.otherObserver = new OtherObserver<>(this); this.errors = new AtomicThrowable(); } @@ -162,7 +162,7 @@ void otherComplete() { SimplePlainQueue getOrCreateQueue() { SimplePlainQueue q = queue; if (q == null) { - q = new SpscLinkedArrayQueue(bufferSize()); + q = new SpscLinkedArrayQueue<>(bufferSize()); queue = q; } return q; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingle.java index 30e80cf4de..08940cf645 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableMergeWithSingle.java @@ -40,7 +40,7 @@ public ObservableMergeWithSingle(Observable source, SingleSource @Override protected void subscribeActual(Observer observer) { - MergeWithObserver parent = new MergeWithObserver(observer); + MergeWithObserver parent = new MergeWithObserver<>(observer); observer.onSubscribe(parent); source.subscribe(parent); other.subscribe(parent.otherObserver); @@ -75,8 +75,8 @@ static final class MergeWithObserver extends AtomicInteger MergeWithObserver(Observer downstream) { this.downstream = downstream; - this.mainDisposable = new AtomicReference(); - this.otherObserver = new OtherObserver(this); + this.mainDisposable = new AtomicReference<>(); + this.otherObserver = new OtherObserver<>(this); this.errors = new AtomicThrowable(); } @@ -157,7 +157,7 @@ void otherError(Throwable ex) { SimplePlainQueue getOrCreateQueue() { SimplePlainQueue q = queue; if (q == null) { - q = new SpscLinkedArrayQueue(bufferSize()); + q = new SpscLinkedArrayQueue<>(bufferSize()); queue = q; } return q; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOn.java index 9bc2b39117..af5e78df3f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableObserveOn.java @@ -42,7 +42,7 @@ protected void subscribeActual(Observer observer) { } else { Scheduler.Worker w = scheduler.createWorker(); - source.subscribe(new ObserveOnObserver(observer, w, delayError, bufferSize)); + source.subscribe(new ObserveOnObserver<>(observer, w, delayError, bufferSize)); } } @@ -101,7 +101,7 @@ public void onSubscribe(Disposable d) { } } - queue = new SpscLinkedArrayQueue(bufferSize); + queue = new SpscLinkedArrayQueue<>(bufferSize); downstream.onSubscribe(this); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorNext.java index 093e8d1299..3cf1849011 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorNext.java @@ -31,7 +31,7 @@ public ObservableOnErrorNext(ObservableSource source, @Override public void subscribeActual(Observer t) { - OnErrorNextObserver parent = new OnErrorNextObserver(t, nextSupplier); + OnErrorNextObserver parent = new OnErrorNextObserver<>(t, nextSupplier); t.onSubscribe(parent.arbiter); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturn.java index df868ab410..54efdc62e5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableOnErrorReturn.java @@ -28,7 +28,7 @@ public ObservableOnErrorReturn(ObservableSource source, Function t) { - source.subscribe(new OnErrorReturnObserver(t, valueSupplier)); + source.subscribe(new OnErrorReturnObserver<>(t, valueSupplier)); } static final class OnErrorReturnObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublish.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublish.java index 629945369e..8589ee3c8a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublish.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublish.java @@ -46,7 +46,7 @@ public final class ObservablePublish extends ConnectableObservable public ObservablePublish(ObservableSource source) { this.source = source; - this.current = new AtomicReference>(); + this.current = new AtomicReference<>(); } @Override @@ -58,7 +58,7 @@ public void connect(Consumer connection) { conn = current.get(); if (conn == null || conn.isDisposed()) { - PublishConnection fresh = new PublishConnection(current); + PublishConnection fresh = new PublishConnection<>(current); if (!current.compareAndSet(conn, fresh)) { continue; } @@ -89,7 +89,7 @@ protected void subscribeActual(Observer observer) { conn = current.get(); // we don't create a fresh connection if the current is terminated if (conn == null) { - PublishConnection fresh = new PublishConnection(current); + PublishConnection fresh = new PublishConnection<>(current); if (!current.compareAndSet(conn, fresh)) { continue; } @@ -98,7 +98,7 @@ protected void subscribeActual(Observer observer) { break; } - InnerDisposable inner = new InnerDisposable(observer, conn); + InnerDisposable inner = new InnerDisposable<>(observer, conn); observer.onSubscribe(inner); if (conn.add(inner)) { if (inner.isDisposed()) { @@ -152,7 +152,7 @@ static final class PublishConnection PublishConnection(AtomicReference> current) { this.connect = new AtomicBoolean(); this.current = current; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); lazySet(EMPTY); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java index 8247a6abfc..687bee9548 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservablePublishSelector.java @@ -52,14 +52,14 @@ protected void subscribeActual(Observer observer) { return; } - TargetObserver o = new TargetObserver(observer); + TargetObserver o = new TargetObserver<>(observer); target.subscribe(o); - source.subscribe(new SourceObserver(subject, o)); + source.subscribe(new SourceObserver<>(subject, o)); } - static final class SourceObserver implements Observer { + static final class SourceObserver implements Observer { final PublishSubject subject; @@ -91,7 +91,7 @@ public void onComplete() { } } - static final class TargetObserver + static final class TargetObserver extends AtomicReference implements Observer, Disposable { private static final long serialVersionUID = 854110278590336484L; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRange.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRange.java index 2b845fec51..56e277dc32 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRange.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRange.java @@ -71,7 +71,7 @@ void run() { @Nullable @Override - public Integer poll() throws Exception { + public Integer poll() { long i = index; if (i != end) { index = i + 1; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLong.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLong.java index 2a4122ccad..f92decb11e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLong.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRangeLong.java @@ -68,7 +68,7 @@ void run() { @Nullable @Override - public Long poll() throws Exception { + public Long poll() { long i = index; if (i != end) { index = i + 1; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java index bab7bbb065..f5421a666e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceMaybe.java @@ -41,7 +41,7 @@ public ObservableReduceMaybe(ObservableSource source, BiFunction red @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new ReduceObserver(observer, reducer)); + source.subscribe(new ReduceObserver<>(observer, reducer)); } static final class ReduceObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java index e6b0d1d0ef..67347debb8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceSeedSingle.java @@ -45,7 +45,7 @@ public ObservableReduceSeedSingle(ObservableSource source, R seed, BiFunction @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new ReduceSeedObserver(observer, reducer, seed)); + source.subscribe(new ReduceSeedObserver<>(observer, reducer, seed)); } static final class ReduceSeedObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java index 956d7bf7c9..1de6cb1e6c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReduceWithSingle.java @@ -53,6 +53,6 @@ protected void subscribeActual(SingleObserver observer) { EmptyDisposable.error(ex, observer); return; } - source.subscribe(new ReduceSeedObserver(observer, reducer, seed)); + source.subscribe(new ReduceSeedObserver<>(observer, reducer, seed)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCount.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCount.java index f0b9157204..a806a537c6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCount.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRefCount.java @@ -81,7 +81,7 @@ protected void subscribeActual(Observer observer) { } } - source.subscribe(new RefCountObserver(observer, this, conn)); + source.subscribe(new RefCountObserver<>(observer, this, conn)); if (connect) { source.connect(conn); @@ -166,7 +166,7 @@ public void run() { } @Override - public void accept(Disposable t) throws Exception { + public void accept(Disposable t) { DisposableHelper.replace(this, t); synchronized (parent) { if (disconnectedEarly) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeat.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeat.java index ad9159137c..2816dac708 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeat.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeat.java @@ -31,7 +31,7 @@ public void subscribeActual(Observer observer) { SequentialDisposable sd = new SequentialDisposable(); observer.onSubscribe(sd); - RepeatObserver rs = new RepeatObserver(observer, count != Long.MAX_VALUE ? count - 1 : Long.MAX_VALUE, sd, source); + RepeatObserver rs = new RepeatObserver<>(observer, count != Long.MAX_VALUE ? count - 1 : Long.MAX_VALUE, sd, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatUntil.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatUntil.java index 60637fe802..51c8b9dd65 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatUntil.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatUntil.java @@ -33,7 +33,7 @@ public void subscribeActual(Observer observer) { SequentialDisposable sd = new SequentialDisposable(); observer.onSubscribe(sd); - RepeatUntilObserver rs = new RepeatUntilObserver(observer, until, sd, source); + RepeatUntilObserver rs = new RepeatUntilObserver<>(observer, until, sd, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java index eff1e248ac..d3fcdf5971 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRepeatWhen.java @@ -52,7 +52,7 @@ protected void subscribeActual(Observer observer) { return; } - RepeatWhenObserver parent = new RepeatWhenObserver(observer, signaller, source); + RepeatWhenObserver parent = new RepeatWhenObserver<>(observer, signaller, source); observer.onSubscribe(parent); other.subscribe(parent.inner); @@ -87,7 +87,7 @@ static final class RepeatWhenObserver extends AtomicInteger implements Observ this.wip = new AtomicInteger(); this.error = new AtomicThrowable(); this.inner = new InnerRepeatObserver(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java index b6a945e8b7..7dabfcc72d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableReplay.java @@ -59,7 +59,7 @@ interface BufferSupplier { public static Observable multicastSelector( final Supplier> connectableFactory, final Function, ? extends ObservableSource> selector) { - return RxJavaPlugins.onAssembly(new MulticastReplay(connectableFactory, selector)); + return RxJavaPlugins.onAssembly(new MulticastReplay<>(connectableFactory, selector)); } /** @@ -86,7 +86,7 @@ public static ConnectableObservable create(ObservableSource source, if (bufferSize == Integer.MAX_VALUE) { return createFrom(source); } - return create(source, new ReplayBufferSupplier(bufferSize, eagerTruncate)); + return create(source, new ReplayBufferSupplier<>(bufferSize, eagerTruncate)); } /** @@ -117,7 +117,7 @@ public static ConnectableObservable create(ObservableSource source, */ public static ConnectableObservable create(ObservableSource source, final long maxAge, final TimeUnit unit, final Scheduler scheduler, final int bufferSize, boolean eagerTruncate) { - return create(source, new ScheduledReplaySupplier(bufferSize, maxAge, unit, scheduler, eagerTruncate)); + return create(source, new ScheduledReplaySupplier<>(bufferSize, maxAge, unit, scheduler, eagerTruncate)); } /** @@ -129,9 +129,9 @@ public static ConnectableObservable create(ObservableSource source, static ConnectableObservable create(ObservableSource source, final BufferSupplier bufferFactory) { // the current connection to source needs to be shared between the operator and its onSubscribe call - final AtomicReference> curr = new AtomicReference>(); - ObservableSource onSubscribe = new ReplaySource(curr, bufferFactory); - return RxJavaPlugins.onAssembly(new ObservableReplay(onSubscribe, source, curr, bufferFactory)); + final AtomicReference> curr = new AtomicReference<>(); + ObservableSource onSubscribe = new ReplaySource<>(curr, bufferFactory); + return RxJavaPlugins.onAssembly(new ObservableReplay<>(onSubscribe, source, curr, bufferFactory)); } private ObservableReplay(ObservableSource onSubscribe, ObservableSource source, @@ -174,7 +174,7 @@ public void connect(Consumer connection) { // create a new subscriber-to-source ReplayBuffer buf = bufferFactory.call(); - ReplayObserver u = new ReplayObserver(buf); + ReplayObserver u = new ReplayObserver<>(buf); // try setting it as the current subscriber-to-source if (!current.compareAndSet(ps, u)) { // did not work, perhaps a new subscriber arrived @@ -243,7 +243,7 @@ static final class ReplayObserver ReplayObserver(ReplayBuffer buffer) { this.buffer = buffer; - this.observers = new AtomicReference(EMPTY); + this.observers = new AtomicReference<>(EMPTY); this.shouldConnect = new AtomicBoolean(); } @@ -829,7 +829,7 @@ static final class SizeAndTimeBoundReplayBuffer extends BoundedReplayBuffer(value, scheduler.now(unit), unit); + return new Timed<>(value, scheduler.now(unit), unit); } @Override @@ -927,7 +927,7 @@ Node getHead() { static final class UnBoundedFactory implements BufferSupplier { @Override public ReplayBuffer call() { - return new UnboundedReplayBuffer(16); + return new UnboundedReplayBuffer<>(16); } } @@ -957,7 +957,7 @@ static final class ReplayBufferSupplier implements BufferSupplier { @Override public ReplayBuffer call() { - return new SizeBoundReplayBuffer(bufferSize, eagerTruncate); + return new SizeBoundReplayBuffer<>(bufferSize, eagerTruncate); } } @@ -979,7 +979,7 @@ static final class ScheduledReplaySupplier implements BufferSupplier { @Override public ReplayBuffer call() { - return new SizeAndTimeBoundReplayBuffer(bufferSize, maxAge, unit, scheduler, eagerTruncate); + return new SizeAndTimeBoundReplayBuffer<>(bufferSize, maxAge, unit, scheduler, eagerTruncate); } } @@ -1004,7 +1004,7 @@ public void subscribe(Observer child) { // create a new subscriber to source ReplayBuffer buf = bufferFactory.call(); - ReplayObserver u = new ReplayObserver(buf); + ReplayObserver u = new ReplayObserver<>(buf); // let's try setting it as the current subscriber-to-source if (!curr.compareAndSet(null, u)) { // didn't work, maybe someone else did it or the current subscriber @@ -1016,7 +1016,7 @@ public void subscribe(Observer child) { } // create the backpressure-managing producer for this child - InnerDisposable inner = new InnerDisposable(r, child); + InnerDisposable inner = new InnerDisposable<>(r, child); // the producer has been registered with the current subscriber-to-source so // at least it will receive the next terminal event // setting the producer will trigger the first request to be considered by @@ -1062,11 +1062,11 @@ protected void subscribeActual(Observer child) { return; } - final ObserverResourceWrapper srw = new ObserverResourceWrapper(child); + final ObserverResourceWrapper srw = new ObserverResourceWrapper<>(child); observable.subscribe(srw); - co.connect(new DisposeConsumer(srw)); + co.connect(new DisposeConsumer<>(srw)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryBiPredicate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryBiPredicate.java index 48edbdaa26..56de8606be 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryBiPredicate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryBiPredicate.java @@ -35,7 +35,7 @@ public void subscribeActual(Observer observer) { SequentialDisposable sa = new SequentialDisposable(); observer.onSubscribe(sa); - RetryBiObserver rs = new RetryBiObserver(observer, predicate, sa, source); + RetryBiObserver rs = new RetryBiObserver<>(observer, predicate, sa, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryPredicate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryPredicate.java index 963e5c8890..9c4d5c748a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryPredicate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryPredicate.java @@ -37,7 +37,7 @@ public void subscribeActual(Observer observer) { SequentialDisposable sa = new SequentialDisposable(); observer.onSubscribe(sa); - RepeatObserver rs = new RepeatObserver(observer, count, predicate, sa, source); + RepeatObserver rs = new RepeatObserver<>(observer, count, predicate, sa, source); rs.subscribeNext(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java index 0eb5fbb5b7..2a3f0b6b07 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableRetryWhen.java @@ -52,7 +52,7 @@ protected void subscribeActual(Observer observer) { return; } - RepeatWhenObserver parent = new RepeatWhenObserver(observer, signaller, source); + RepeatWhenObserver parent = new RepeatWhenObserver<>(observer, signaller, source); observer.onSubscribe(parent); other.subscribe(parent.inner); @@ -87,7 +87,7 @@ static final class RepeatWhenObserver extends AtomicInteger implements Observ this.wip = new AtomicInteger(); this.error = new AtomicThrowable(); this.inner = new InnerRepeatObserver(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTimed.java index 4a91ff30b8..3f516e8cca 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleTimed.java @@ -38,11 +38,11 @@ public ObservableSampleTimed(ObservableSource source, long period, TimeUnit u @Override public void subscribeActual(Observer t) { - SerializedObserver serial = new SerializedObserver(t); + SerializedObserver serial = new SerializedObserver<>(t); if (emitLast) { - source.subscribe(new SampleTimedEmitLast(serial, period, unit, scheduler)); + source.subscribe(new SampleTimedEmitLast<>(serial, period, unit, scheduler)); } else { - source.subscribe(new SampleTimedNoLast(serial, period, unit, scheduler)); + source.subscribe(new SampleTimedNoLast<>(serial, period, unit, scheduler)); } } @@ -55,7 +55,7 @@ abstract static class SampleTimedObserver extends AtomicReference implemen final TimeUnit unit; final Scheduler scheduler; - final AtomicReference timer = new AtomicReference(); + final AtomicReference timer = new AtomicReference<>(); Disposable upstream; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleWithObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleWithObservable.java index 4c0fccc654..cae3259d12 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleWithObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSampleWithObservable.java @@ -34,11 +34,11 @@ public ObservableSampleWithObservable(ObservableSource source, ObservableSour @Override public void subscribeActual(Observer t) { - SerializedObserver serial = new SerializedObserver(t); + SerializedObserver serial = new SerializedObserver<>(t); if (emitLast) { - source.subscribe(new SampleMainEmitLast(serial, other)); + source.subscribe(new SampleMainEmitLast<>(serial, other)); } else { - source.subscribe(new SampleMainNoLast(serial, other)); + source.subscribe(new SampleMainNoLast<>(serial, other)); } } @@ -50,7 +50,7 @@ abstract static class SampleMainObserver extends AtomicReference final Observer downstream; final ObservableSource sampler; - final AtomicReference other = new AtomicReference(); + final AtomicReference other = new AtomicReference<>(); Disposable upstream; @@ -65,7 +65,7 @@ public void onSubscribe(Disposable d) { this.upstream = d; downstream.onSubscribe(this); if (other.get() == null) { - sampler.subscribe(new SamplerObserver(this)); + sampler.subscribe(new SamplerObserver<>(this)); } } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java index f4a1bce25b..c62c23c0c5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScalarXMap.java @@ -88,7 +88,7 @@ public static boolean tryScalarXMapSubscribe(ObservableSource source, EmptyDisposable.complete(observer); return true; } - ScalarDisposable sd = new ScalarDisposable(observer, u); + ScalarDisposable sd = new ScalarDisposable<>(observer, u); observer.onSubscribe(sd); sd.run(); } else { @@ -112,7 +112,7 @@ public static boolean tryScalarXMapSubscribe(ObservableSource source, */ public static Observable scalarXMap(T value, Function> mapper) { - return RxJavaPlugins.onAssembly(new ScalarXMapObservable(value, mapper)); + return RxJavaPlugins.onAssembly(new ScalarXMapObservable<>(value, mapper)); } /** @@ -159,7 +159,7 @@ public void subscribeActual(Observer observer) { EmptyDisposable.complete(observer); return; } - ScalarDisposable sd = new ScalarDisposable(observer, u); + ScalarDisposable sd = new ScalarDisposable<>(observer, u); observer.onSubscribe(sd); sd.run(); } else { @@ -205,7 +205,7 @@ public boolean offer(T v1, T v2) { @Nullable @Override - public T poll() throws Exception { + public T poll() { if (get() == FUSED) { lazySet(ON_COMPLETE); return value; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java index 94918b6c39..bd0ba2b12a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScan.java @@ -31,7 +31,7 @@ public ObservableScan(ObservableSource source, BiFunction accumulato @Override public void subscribeActual(Observer t) { - source.subscribe(new ScanObserver(t, accumulator)); + source.subscribe(new ScanObserver<>(t, accumulator)); } static final class ScanObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java index 24e4b580ad..89eb4046f0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableScanSeed.java @@ -43,7 +43,7 @@ public void subscribeActual(Observer t) { return; } - source.subscribe(new ScanSeedObserver(t, accumulator, r)); + source.subscribe(new ScanSeedObserver<>(t, accumulator, r)); } static final class ScanSeedObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqual.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqual.java index 1e779db18e..11eb694696 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqual.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqual.java @@ -38,7 +38,7 @@ public ObservableSequenceEqual(ObservableSource first, ObservableSo @Override public void subscribeActual(Observer observer) { - EqualCoordinator ec = new EqualCoordinator(observer, bufferSize, first, second, comparer); + EqualCoordinator ec = new EqualCoordinator<>(observer, bufferSize, first, second, comparer); observer.onSubscribe(ec); ec.subscribe(); } @@ -69,8 +69,8 @@ static final class EqualCoordinator extends AtomicInteger implements Disposab @SuppressWarnings("unchecked") EqualObserver[] as = new EqualObserver[2]; this.observers = as; - as[0] = new EqualObserver(this, 0, bufferSize); - as[1] = new EqualObserver(this, 1, bufferSize); + as[0] = new EqualObserver<>(this, 0, bufferSize); + as[1] = new EqualObserver<>(this, 1, bufferSize); this.resources = new ArrayCompositeDisposable(2); } @@ -226,7 +226,7 @@ static final class EqualObserver implements Observer { EqualObserver(EqualCoordinator parent, int index, int bufferSize) { this.parent = parent; this.index = index; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqualSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqualSingle.java index 165a8ab64a..9169cb6465 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqualSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSequenceEqualSingle.java @@ -40,14 +40,14 @@ public ObservableSequenceEqualSingle(ObservableSource first, Observ @Override public void subscribeActual(SingleObserver observer) { - EqualCoordinator ec = new EqualCoordinator(observer, bufferSize, first, second, comparer); + EqualCoordinator ec = new EqualCoordinator<>(observer, bufferSize, first, second, comparer); observer.onSubscribe(ec); ec.subscribe(); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableSequenceEqual(first, second, comparer, bufferSize)); + return RxJavaPlugins.onAssembly(new ObservableSequenceEqual<>(first, second, comparer, bufferSize)); } static final class EqualCoordinator extends AtomicInteger implements Disposable { @@ -76,8 +76,8 @@ static final class EqualCoordinator extends AtomicInteger implements Disposab @SuppressWarnings("unchecked") EqualObserver[] as = new EqualObserver[2]; this.observers = as; - as[0] = new EqualObserver(this, 0, bufferSize); - as[1] = new EqualObserver(this, 1, bufferSize); + as[0] = new EqualObserver<>(this, 0, bufferSize); + as[1] = new EqualObserver<>(this, 1, bufferSize); this.resources = new ArrayCompositeDisposable(2); } @@ -230,7 +230,7 @@ static final class EqualObserver implements Observer { EqualObserver(EqualCoordinator parent, int index, int bufferSize) { this.parent = parent; this.index = index; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerialized.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerialized.java index 35ce558744..98f757e6f7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerialized.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSerialized.java @@ -22,6 +22,6 @@ public ObservableSerialized(Observable upstream) { @Override protected void subscribeActual(Observer observer) { - source.subscribe(new SerializedObserver(observer)); + source.subscribe(new SerializedObserver<>(observer)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleMaybe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleMaybe.java index b0bba864b9..83a8569f05 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleMaybe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleMaybe.java @@ -28,7 +28,7 @@ public ObservableSingleMaybe(ObservableSource source) { @Override public void subscribeActual(MaybeObserver t) { - source.subscribe(new SingleElementObserver(t)); + source.subscribe(new SingleElementObserver<>(t)); } static final class SingleElementObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleSingle.java index f7c8f597e1..de4729f5e5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSingleSingle.java @@ -33,7 +33,7 @@ public ObservableSingleSingle(ObservableSource source, T defaultVal @Override public void subscribeActual(SingleObserver t) { - source.subscribe(new SingleElementObserver(t, defaultValue)); + source.subscribe(new SingleElementObserver<>(t, defaultValue)); } static final class SingleElementObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkip.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkip.java index 9812b33663..e0000dc7a5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkip.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkip.java @@ -26,7 +26,7 @@ public ObservableSkip(ObservableSource source, long n) { @Override public void subscribeActual(Observer observer) { - source.subscribe(new SkipObserver(observer, n)); + source.subscribe(new SkipObserver<>(observer, n)); } static final class SkipObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLast.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLast.java index 7d7ea2383e..3e45e38a79 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLast.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLast.java @@ -29,7 +29,7 @@ public ObservableSkipLast(ObservableSource source, int skip) { @Override public void subscribeActual(Observer observer) { - source.subscribe(new SkipLastObserver(observer, skip)); + source.subscribe(new SkipLastObserver<>(observer, skip)); } static final class SkipLastObserver extends ArrayDeque implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTimed.java index 41b34b6186..f70620b5c0 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipLastTimed.java @@ -40,7 +40,7 @@ public ObservableSkipLastTimed(ObservableSource source, @Override public void subscribeActual(Observer t) { - source.subscribe(new SkipLastTimedObserver(t, time, unit, scheduler, bufferSize, delayError)); + source.subscribe(new SkipLastTimedObserver<>(t, time, unit, scheduler, bufferSize, delayError)); } static final class SkipLastTimedObserver extends AtomicInteger implements Observer, Disposable { @@ -65,7 +65,7 @@ static final class SkipLastTimedObserver extends AtomicInteger implements Obs this.time = time; this.unit = unit; this.scheduler = scheduler; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); this.delayError = delayError; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipUntil.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipUntil.java index 9f0377b784..457158200d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipUntil.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipUntil.java @@ -28,13 +28,13 @@ public ObservableSkipUntil(ObservableSource source, ObservableSource other @Override public void subscribeActual(Observer child) { - final SerializedObserver serial = new SerializedObserver(child); + final SerializedObserver serial = new SerializedObserver<>(child); final ArrayCompositeDisposable frc = new ArrayCompositeDisposable(2); serial.onSubscribe(frc); - final SkipUntilObserver sus = new SkipUntilObserver(serial, frc); + final SkipUntilObserver sus = new SkipUntilObserver<>(serial, frc); other.subscribe(new SkipUntil(frc, sus, serial)); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipWhile.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipWhile.java index 04326b2647..d6dad232b5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipWhile.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSkipWhile.java @@ -28,7 +28,7 @@ public ObservableSkipWhile(ObservableSource source, Predicate pred @Override public void subscribeActual(Observer observer) { - source.subscribe(new SkipWhileObserver(observer, predicate)); + source.subscribe(new SkipWhileObserver<>(observer, predicate)); } static final class SkipWhileObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOn.java index f1fce38571..8eac3ae04e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSubscribeOn.java @@ -29,7 +29,7 @@ public ObservableSubscribeOn(ObservableSource source, Scheduler scheduler) { @Override public void subscribeActual(final Observer observer) { - final SubscribeOnObserver parent = new SubscribeOnObserver(observer); + final SubscribeOnObserver parent = new SubscribeOnObserver<>(observer); observer.onSubscribe(parent); @@ -45,7 +45,7 @@ static final class SubscribeOnObserver extends AtomicReference im SubscribeOnObserver(Observer downstream) { this.downstream = downstream; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmpty.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmpty.java index 4b41661f1d..b697afc1c9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmpty.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchIfEmpty.java @@ -26,7 +26,7 @@ public ObservableSwitchIfEmpty(ObservableSource source, ObservableSource t) { - SwitchIfEmptyObserver parent = new SwitchIfEmptyObserver(t, other); + SwitchIfEmptyObserver parent = new SwitchIfEmptyObserver<>(t, other); t.onSubscribe(parent.arbiter); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java index bca0f1beeb..e533834541 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableSwitchMap.java @@ -48,7 +48,7 @@ public void subscribeActual(Observer t) { return; } - source.subscribe(new SwitchMapObserver(t, mapper, bufferSize, delayErrors)); + source.subscribe(new SwitchMapObserver<>(t, mapper, bufferSize, delayErrors)); } static final class SwitchMapObserver extends AtomicInteger implements Observer, Disposable { @@ -68,11 +68,11 @@ static final class SwitchMapObserver extends AtomicInteger implements Obse Disposable upstream; - final AtomicReference> active = new AtomicReference>(); + final AtomicReference> active = new AtomicReference<>(); static final SwitchMapInnerObserver CANCELLED; static { - CANCELLED = new SwitchMapInnerObserver(null, -1L, 1); + CANCELLED = new SwitchMapInnerObserver<>(null, -1L, 1); CANCELLED.cancel(); } @@ -116,7 +116,7 @@ public void onNext(T t) { return; } - SwitchMapInnerObserver nextInner = new SwitchMapInnerObserver(this, c, bufferSize); + SwitchMapInnerObserver nextInner = new SwitchMapInnerObserver<>(this, c, bufferSize); for (;;) { inner = active.get(); @@ -364,7 +364,7 @@ public void onSubscribe(Disposable d) { } } - queue = new SpscLinkedArrayQueue(bufferSize); + queue = new SpscLinkedArrayQueue<>(bufferSize); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTake.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTake.java index 7e675f061c..63f37db46e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTake.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTake.java @@ -27,7 +27,7 @@ public ObservableTake(ObservableSource source, long limit) { @Override protected void subscribeActual(Observer observer) { - source.subscribe(new TakeObserver(observer, limit)); + source.subscribe(new TakeObserver<>(observer, limit)); } static final class TakeObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLast.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLast.java index 45566a64e7..bfa2ee0bdd 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLast.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLast.java @@ -29,7 +29,7 @@ public ObservableTakeLast(ObservableSource source, int count) { @Override public void subscribeActual(Observer t) { - source.subscribe(new TakeLastObserver(t, count)); + source.subscribe(new TakeLastObserver<>(t, count)); } static final class TakeLastObserver extends ArrayDeque implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOne.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOne.java index 8f176bfebd..f2766db765 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOne.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastOne.java @@ -24,7 +24,7 @@ public ObservableTakeLastOne(ObservableSource source) { @Override public void subscribeActual(Observer observer) { - source.subscribe(new TakeLastOneObserver(observer)); + source.subscribe(new TakeLastOneObserver<>(observer)); } static final class TakeLastOneObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTimed.java index 0b9e16e26f..be6f83db79 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeLastTimed.java @@ -42,7 +42,7 @@ public ObservableTakeLastTimed(ObservableSource source, @Override public void subscribeActual(Observer t) { - source.subscribe(new TakeLastTimedObserver(t, count, time, unit, scheduler, bufferSize, delayError)); + source.subscribe(new TakeLastTimedObserver<>(t, count, time, unit, scheduler, bufferSize, delayError)); } static final class TakeLastTimedObserver @@ -69,7 +69,7 @@ static final class TakeLastTimedObserver this.time = time; this.unit = unit; this.scheduler = scheduler; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); this.delayError = delayError; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntil.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntil.java index fdb376c57e..f1d5d00a0d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntil.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntil.java @@ -31,7 +31,7 @@ public ObservableTakeUntil(ObservableSource source, ObservableSource child) { - TakeUntilMainObserver parent = new TakeUntilMainObserver(child); + TakeUntilMainObserver parent = new TakeUntilMainObserver<>(child); child.onSubscribe(parent); other.subscribe(parent.otherObserver); @@ -53,7 +53,7 @@ static final class TakeUntilMainObserver extends AtomicInteger TakeUntilMainObserver(Observer downstream) { this.downstream = downstream; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.otherObserver = new OtherObserver(); this.error = new AtomicThrowable(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicate.java index d38aabd839..1727a24fb4 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeUntilPredicate.java @@ -29,7 +29,7 @@ public ObservableTakeUntilPredicate(ObservableSource source, Predicate observer) { - source.subscribe(new TakeUntilPredicateObserver(observer, predicate)); + source.subscribe(new TakeUntilPredicateObserver<>(observer, predicate)); } static final class TakeUntilPredicateObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhile.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhile.java index 62cb16d791..5c251c969e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhile.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTakeWhile.java @@ -29,7 +29,7 @@ public ObservableTakeWhile(ObservableSource source, Predicate pred @Override public void subscribeActual(Observer t) { - source.subscribe(new TakeWhileObserver(t, predicate)); + source.subscribe(new TakeWhileObserver<>(t, predicate)); } static final class TakeWhileObserver implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTimed.java index 5708f04816..041ce80840 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleFirstTimed.java @@ -38,8 +38,8 @@ public ObservableThrottleFirstTimed(ObservableSource source, @Override public void subscribeActual(Observer t) { - source.subscribe(new DebounceTimedObserver( - new SerializedObserver(t), + source.subscribe(new DebounceTimedObserver<>( + new SerializedObserver<>(t), timeout, unit, scheduler.createWorker())); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleLatest.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleLatest.java index 9d23714549..253f5d3501 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleLatest.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableThrottleLatest.java @@ -53,7 +53,7 @@ public ObservableThrottleLatest(Observable source, @Override protected void subscribeActual(Observer observer) { - source.subscribe(new ThrottleLatestObserver(observer, timeout, unit, scheduler.createWorker(), emitLast)); + source.subscribe(new ThrottleLatestObserver<>(observer, timeout, unit, scheduler.createWorker(), emitLast)); } static final class ThrottleLatestObserver @@ -93,7 +93,7 @@ static final class ThrottleLatestObserver this.unit = unit; this.worker = worker; this.emitLast = emitLast; - this.latest = new AtomicReference(); + this.latest = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeInterval.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeInterval.java index f5ad5a52f5..7dd47ec03b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeInterval.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeInterval.java @@ -32,7 +32,7 @@ public ObservableTimeInterval(ObservableSource source, TimeUnit unit, Schedul @Override public void subscribeActual(Observer> t) { - source.subscribe(new TimeIntervalObserver(t, unit, scheduler)); + source.subscribe(new TimeIntervalObserver<>(t, unit, scheduler)); } static final class TimeIntervalObserver implements Observer, Disposable { @@ -75,7 +75,7 @@ public void onNext(T t) { long last = lastTime; lastTime = now; long delta = now - last; - downstream.onNext(new Timed(t, delta, unit)); + downstream.onNext(new Timed<>(t, delta, unit)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java index a9ada29730..dfef0e74e6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeout.java @@ -44,12 +44,12 @@ public ObservableTimeout( @Override protected void subscribeActual(Observer observer) { if (other == null) { - TimeoutObserver parent = new TimeoutObserver(observer, itemTimeoutIndicator); + TimeoutObserver parent = new TimeoutObserver<>(observer, itemTimeoutIndicator); observer.onSubscribe(parent); parent.startFirstTimeout(firstTimeoutIndicator); source.subscribe(parent); } else { - TimeoutFallbackObserver parent = new TimeoutFallbackObserver(observer, itemTimeoutIndicator, other); + TimeoutFallbackObserver parent = new TimeoutFallbackObserver<>(observer, itemTimeoutIndicator, other); observer.onSubscribe(parent); parent.startFirstTimeout(firstTimeoutIndicator); source.subscribe(parent); @@ -77,7 +77,7 @@ static final class TimeoutObserver extends AtomicLong this.downstream = actual; this.itemTimeoutIndicator = itemTimeoutIndicator; this.task = new SequentialDisposable(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override @@ -206,7 +206,7 @@ static final class TimeoutFallbackObserver this.task = new SequentialDisposable(); this.fallback = fallback; this.index = new AtomicLong(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTimed.java index 0553c9474f..29b1772930 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableTimeoutTimed.java @@ -41,12 +41,12 @@ public ObservableTimeoutTimed(Observable source, @Override protected void subscribeActual(Observer observer) { if (other == null) { - TimeoutObserver parent = new TimeoutObserver(observer, timeout, unit, scheduler.createWorker()); + TimeoutObserver parent = new TimeoutObserver<>(observer, timeout, unit, scheduler.createWorker()); observer.onSubscribe(parent); parent.startTimeout(0L); source.subscribe(parent); } else { - TimeoutFallbackObserver parent = new TimeoutFallbackObserver(observer, timeout, unit, scheduler.createWorker(), other); + TimeoutFallbackObserver parent = new TimeoutFallbackObserver<>(observer, timeout, unit, scheduler.createWorker(), other); observer.onSubscribe(parent); parent.startTimeout(0L); source.subscribe(parent); @@ -76,7 +76,7 @@ static final class TimeoutObserver extends AtomicLong this.unit = unit; this.worker = worker; this.task = new SequentialDisposable(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override @@ -196,7 +196,7 @@ static final class TimeoutFallbackObserver extends AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToList.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToList.java index 9cde3b4033..6a3c5322f2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToList.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToList.java @@ -49,7 +49,7 @@ public void subscribeActual(Observer t) { EmptyDisposable.error(e, t); return; } - source.subscribe(new ToListObserver(t, coll)); + source.subscribe(new ToListObserver<>(t, coll)); } static final class ToListObserver> implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListSingle.java index 6258313b61..0ac41b12d6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableToListSingle.java @@ -53,12 +53,12 @@ public void subscribeActual(SingleObserver t) { EmptyDisposable.error(e, t); return; } - source.subscribe(new ToListObserver(t, coll)); + source.subscribe(new ToListObserver<>(t, coll)); } @Override public Observable fuseToObservable() { - return RxJavaPlugins.onAssembly(new ObservableToList(source, collectionSupplier)); + return RxJavaPlugins.onAssembly(new ObservableToList<>(source, collectionSupplier)); } static final class ToListObserver> implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOn.java index d012b7cc4d..2c5f748477 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUnsubscribeOn.java @@ -29,7 +29,7 @@ public ObservableUnsubscribeOn(ObservableSource source, Scheduler scheduler) @Override public void subscribeActual(Observer t) { - source.subscribe(new UnsubscribeObserver(t, scheduler)); + source.subscribe(new UnsubscribeObserver<>(t, scheduler)); } static final class UnsubscribeObserver extends AtomicBoolean implements Observer, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java index 60fad9e740..dec3da5ebe 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableUsing.java @@ -67,7 +67,7 @@ public void subscribeActual(Observer observer) { return; } - UsingObserver us = new UsingObserver(observer, resource, disposer, eager); + UsingObserver us = new UsingObserver<>(observer, resource, disposer, eager); source.subscribe(us); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java index 3f11f3952a..3ada9cd93c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindow.java @@ -36,9 +36,9 @@ public ObservableWindow(ObservableSource source, long count, long skip, int c @Override public void subscribeActual(Observer> t) { if (count == skip) { - source.subscribe(new WindowExactObserver(t, count, capacityHint)); + source.subscribe(new WindowExactObserver<>(t, count, capacityHint)); } else { - source.subscribe(new WindowSkipObserver(t, count, skip, capacityHint)); + source.subscribe(new WindowSkipObserver<>(t, count, skip, capacityHint)); } } @@ -81,7 +81,7 @@ public void onNext(T t) { if (w == null && !cancelled) { w = UnicastSubject.create(capacityHint, this); window = w; - intercept = new ObservableWindowSubscribeIntercept(w); + intercept = new ObservableWindowSubscribeIntercept<>(w); downstream.onNext(intercept); } @@ -169,7 +169,7 @@ static final class WindowSkipObserver extends AtomicBoolean this.count = count; this.skip = skip; this.capacityHint = capacityHint; - this.windows = new ArrayDeque>(); + this.windows = new ArrayDeque<>(); } @Override @@ -194,7 +194,7 @@ public void onNext(T t) { if (i % s == 0 && !cancelled) { wip.getAndIncrement(); UnicastSubject w = UnicastSubject.create(capacityHint, this); - intercept = new ObservableWindowSubscribeIntercept(w); + intercept = new ObservableWindowSubscribeIntercept<>(w); ws.offer(w); downstream.onNext(intercept); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java index 48de711a04..503a5cbd29 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundary.java @@ -36,7 +36,7 @@ public ObservableWindowBoundary(ObservableSource source, ObservableSource @Override public void subscribeActual(Observer> observer) { - WindowBoundaryMainObserver parent = new WindowBoundaryMainObserver(observer, capacityHint); + WindowBoundaryMainObserver parent = new WindowBoundaryMainObserver<>(observer, capacityHint); observer.onSubscribe(parent); other.subscribe(parent.boundaryObserver); @@ -75,10 +75,10 @@ static final class WindowBoundaryMainObserver WindowBoundaryMainObserver(Observer> downstream, int capacityHint) { this.downstream = downstream; this.capacityHint = capacityHint; - this.boundaryObserver = new WindowBoundaryInnerObserver(this); - this.upstream = new AtomicReference(); + this.boundaryObserver = new WindowBoundaryInnerObserver<>(this); + this.upstream = new AtomicReference<>(); this.windows = new AtomicInteger(1); - this.queue = new MpscLinkedQueue(); + this.queue = new MpscLinkedQueue<>(); this.errors = new AtomicThrowable(); this.stopWindows = new AtomicBoolean(); } @@ -230,7 +230,7 @@ void drain() { window = w; windows.getAndIncrement(); - ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(w); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept<>(w); downstream.onNext(intercept); if (intercept.tryAbandon()) { w.onComplete(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java index 8722d6df0c..367ddf2d02 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowBoundarySelector.java @@ -46,7 +46,7 @@ public ObservableWindowBoundarySelector( @Override public void subscribeActual(Observer> t) { - source.subscribe(new WindowBoundaryMainObserver( + source.subscribe(new WindowBoundaryMainObserver<>( t, open, closingIndicator, bufferSize)); } @@ -85,16 +85,16 @@ static final class WindowBoundaryMainObserver WindowBoundaryMainObserver(Observer> downstream, ObservableSource open, Function> closingIndicator, int bufferSize) { this.downstream = downstream; - this.queue = new MpscLinkedQueue(); + this.queue = new MpscLinkedQueue<>(); this.open = open; this.closingIndicator = closingIndicator; this.bufferSize = bufferSize; this.resources = new CompositeDisposable(); - this.windows = new ArrayList>(); + this.windows = new ArrayList<>(); this.windowCount = new AtomicLong(1L); this.downstreamDisposed = new AtomicBoolean(); this.error = new AtomicThrowable(); - this.startObserver = new WindowStartObserver(this); + this.startObserver = new WindowStartObserver<>(this); this.requested = new AtomicLong(); } @@ -167,7 +167,7 @@ public void run() { } void open(B startValue) { - queue.offer(new WindowStartItem(startValue)); + queue.offer(new WindowStartItem<>(startValue)); drain(); } @@ -249,7 +249,7 @@ void drain() { windowCount.getAndIncrement(); UnicastSubject newWindow = UnicastSubject.create(bufferSize, this); - WindowEndObserverIntercept endObserver = new WindowEndObserverIntercept(this, newWindow); + WindowEndObserverIntercept endObserver = new WindowEndObserverIntercept<>(this, newWindow); downstream.onNext(endObserver); @@ -371,7 +371,7 @@ static final class WindowEndObserverIntercept extends Observable WindowEndObserverIntercept(WindowBoundaryMainObserver parent, UnicastSubject window) { this.parent = parent; this.window = window; - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.once = new AtomicBoolean(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java index 3913bf2911..5c295f4608 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWindowTimed.java @@ -53,18 +53,18 @@ public ObservableWindowTimed(Observable source, protected void subscribeActual(Observer> downstream) { if (timespan == timeskip) { if (maxSize == Long.MAX_VALUE) { - source.subscribe(new WindowExactUnboundedObserver( + source.subscribe(new WindowExactUnboundedObserver<>( downstream, timespan, unit, scheduler, bufferSize)); return; } - source.subscribe(new WindowExactBoundedObserver( + source.subscribe(new WindowExactBoundedObserver<>( downstream, timespan, unit, scheduler, bufferSize, maxSize, restartTimerOnMaxSize)); return; } - source.subscribe(new WindowSkipObserver(downstream, + source.subscribe(new WindowSkipObserver<>(downstream, timespan, timeskip, unit, scheduler.createWorker(), bufferSize)); } @@ -96,7 +96,7 @@ abstract static class AbstractWindowObserver AbstractWindowObserver(Observer> downstream, long timespan, TimeUnit unit, int bufferSize) { this.downstream = downstream; - this.queue = new MpscLinkedQueue(); + this.queue = new MpscLinkedQueue<>(); this.timespan = timespan; this.unit = unit; this.bufferSize = bufferSize; @@ -194,7 +194,7 @@ void createFirstWindow() { emitted = 1; - ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); timer.replace(scheduler.schedulePeriodicallyDirect(this, timespan, timespan, unit)); @@ -267,7 +267,7 @@ else if (!isEmpty) { window = UnicastSubject.create(bufferSize, windowRunnable); this.window = window; - ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); if (intercept.tryAbandon()) { @@ -344,7 +344,7 @@ void createFirstWindow() { windowCount.getAndIncrement(); window = UnicastSubject.create(bufferSize, this); - ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); Runnable boundaryTask = new WindowBoundaryRunnable(this, 1L); @@ -466,7 +466,7 @@ UnicastSubject createNewWindow(UnicastSubject window) { window = UnicastSubject.create(bufferSize, this); this.window = window; - ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); if (restartTimerOnMaxSize) { @@ -515,7 +515,7 @@ static final class WindowSkipObserver super(actual, timespan, unit, bufferSize); this.timeskip = timeskip; this.worker = worker; - this.windows = new LinkedList>(); + this.windows = new LinkedList<>(); } @Override @@ -527,7 +527,7 @@ void createFirstWindow() { UnicastSubject window = UnicastSubject.create(bufferSize, this); windows.add(window); - ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); @@ -591,7 +591,7 @@ void drain() { UnicastSubject window = UnicastSubject.create(bufferSize, this); windows.add(window); - ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept(window); + ObservableWindowSubscribeIntercept intercept = new ObservableWindowSubscribeIntercept<>(window); downstream.onNext(intercept); worker.schedule(new WindowBoundaryRunnable(this, false), timespan, unit); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java index 6ac680db30..b7e283c515 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFrom.java @@ -35,8 +35,8 @@ public ObservableWithLatestFrom(ObservableSource source, @Override public void subscribeActual(Observer t) { - final SerializedObserver serial = new SerializedObserver(t); - final WithLatestFromObserver wlf = new WithLatestFromObserver(serial, combiner); + final SerializedObserver serial = new SerializedObserver<>(t); + final WithLatestFromObserver wlf = new WithLatestFromObserver<>(serial, combiner); serial.onSubscribe(wlf); @@ -53,9 +53,9 @@ static final class WithLatestFromObserver extends AtomicReference im final BiFunction combiner; - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); - final AtomicReference other = new AtomicReference(); + final AtomicReference other = new AtomicReference<>(); WithLatestFromObserver(Observer actual, BiFunction combiner) { this.downstream = actual; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java index 73d28032fc..c9f2a6b57a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableWithLatestFromMany.java @@ -82,11 +82,11 @@ protected void subscribeActual(Observer observer) { } if (n == 0) { - new ObservableMap(source, new SingletonArrayFunc()).subscribeActual(observer); + new ObservableMap<>(source, new SingletonArrayFunc()).subscribeActual(observer); return; } - WithLatestFromObserver parent = new WithLatestFromObserver(observer, combiner, n); + WithLatestFromObserver parent = new WithLatestFromObserver<>(observer, combiner, n); observer.onSubscribe(parent); parent.subscribe(others, n); @@ -121,8 +121,8 @@ static final class WithLatestFromObserver s[i] = new WithLatestInnerObserver(this, i); } this.observers = s; - this.values = new AtomicReferenceArray(n); - this.upstream = new AtomicReference(); + this.values = new AtomicReferenceArray<>(n); + this.upstream = new AtomicReference<>(); this.error = new AtomicThrowable(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java index 80ae9b174b..ace2d748e5 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObservableZip.java @@ -68,7 +68,7 @@ public void subscribeActual(Observer observer) { return; } - ZipCoordinator zc = new ZipCoordinator(observer, zipper, count, delayError); + ZipCoordinator zc = new ZipCoordinator<>(observer, zipper, count, delayError); zc.subscribe(sources, bufferSize); } @@ -98,7 +98,7 @@ public void subscribe(ObservableSource[] sources, int bufferSize) { ZipObserver[] s = observers; int len = s.length; for (int i = 0; i < len; i++) { - s[i] = new ZipObserver(this, bufferSize); + s[i] = new ZipObserver<>(this, bufferSize); } // this makes sure the contents of the observers array is visible this.lazySet(0); @@ -263,11 +263,11 @@ static final class ZipObserver implements Observer { volatile boolean done; Throwable error; - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); ZipObserver(ZipCoordinator parent, int bufferSize) { this.parent = parent; - this.queue = new SpscLinkedArrayQueue(bufferSize); + this.queue = new SpscLinkedArrayQueue<>(bufferSize); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObserverResourceWrapper.java b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObserverResourceWrapper.java index 4111a36af4..4245417ce7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObserverResourceWrapper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/observable/ObserverResourceWrapper.java @@ -25,7 +25,7 @@ public final class ObserverResourceWrapper extends AtomicReference downstream; - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); public ObserverResourceWrapper(Observer downstream) { this.downstream = downstream; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCache.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCache.java index 48220d16b1..a8cb63f0ed 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCache.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCache.java @@ -39,12 +39,12 @@ public final class SingleCache extends Single implements SingleObserver public SingleCache(SingleSource source) { this.source = source; this.wip = new AtomicInteger(); - this.observers = new AtomicReference[]>(EMPTY); + this.observers = new AtomicReference<>(EMPTY); } @Override protected void subscribeActual(final SingleObserver observer) { - CacheDisposable d = new CacheDisposable(observer, this); + CacheDisposable d = new CacheDisposable<>(observer, this); observer.onSubscribe(d); if (add(d)) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCreate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCreate.java index a2164d2ec5..8c5331c87b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCreate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleCreate.java @@ -33,7 +33,7 @@ public SingleCreate(SingleOnSubscribe source) { @Override protected void subscribeActual(SingleObserver observer) { - Emitter parent = new Emitter(observer); + Emitter parent = new Emitter<>(observer); observer.onSubscribe(parent); try { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithCompletable.java index 9915ff1c66..c3f7458459 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithCompletable.java @@ -33,7 +33,7 @@ public SingleDelayWithCompletable(SingleSource source, CompletableSource othe @Override protected void subscribeActual(SingleObserver observer) { - other.subscribe(new OtherObserver(observer, source)); + other.subscribe(new OtherObserver<>(observer, source)); } static final class OtherObserver @@ -66,7 +66,7 @@ public void onError(Throwable e) { @Override public void onComplete() { - source.subscribe(new ResumeSingleObserver(this, downstream)); + source.subscribe(new ResumeSingleObserver<>(this, downstream)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithObservable.java index 78c843db4b..a0eb8ab860 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithObservable.java @@ -34,7 +34,7 @@ public SingleDelayWithObservable(SingleSource source, ObservableSource oth @Override protected void subscribeActual(SingleObserver observer) { - other.subscribe(new OtherSubscriber(observer, source)); + other.subscribe(new OtherSubscriber<>(observer, source)); } static final class OtherSubscriber @@ -84,7 +84,7 @@ public void onComplete() { return; } done = true; - source.subscribe(new ResumeSingleObserver(this, downstream)); + source.subscribe(new ResumeSingleObserver<>(this, downstream)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithPublisher.java index 647f55916b..1d8a601a02 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithPublisher.java @@ -37,7 +37,7 @@ public SingleDelayWithPublisher(SingleSource source, Publisher other) { @Override protected void subscribeActual(SingleObserver observer) { - other.subscribe(new OtherSubscriber(observer, source)); + other.subscribe(new OtherSubscriber<>(observer, source)); } static final class OtherSubscriber @@ -92,7 +92,7 @@ public void onComplete() { return; } done = true; - source.subscribe(new ResumeSingleObserver(this, downstream)); + source.subscribe(new ResumeSingleObserver<>(this, downstream)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithSingle.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithSingle.java index 721d4dc2a5..d8e11665c8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithSingle.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDelayWithSingle.java @@ -33,7 +33,7 @@ public SingleDelayWithSingle(SingleSource source, SingleSource other) { @Override protected void subscribeActual(SingleObserver observer) { - other.subscribe(new OtherObserver(observer, source)); + other.subscribe(new OtherObserver<>(observer, source)); } static final class OtherObserver @@ -61,7 +61,7 @@ public void onSubscribe(Disposable d) { @Override public void onSuccess(U value) { - source.subscribe(new ResumeSingleObserver(this, downstream)); + source.subscribe(new ResumeSingleObserver<>(this, downstream)); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java index 4582e4d4da..6d072f4cea 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDematerialize.java @@ -42,7 +42,7 @@ public SingleDematerialize(Single source, Function @Override protected void subscribeActual(MaybeObserver observer) { - source.subscribe(new DematerializeObserver(observer, selector)); + source.subscribe(new DematerializeObserver<>(observer, selector)); } static final class DematerializeObserver implements SingleObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDetach.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDetach.java index 70e5ae029c..72e1f3def3 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDetach.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDetach.java @@ -33,7 +33,7 @@ public SingleDetach(SingleSource source) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new DetachSingleObserver(observer)); + source.subscribe(new DetachSingleObserver<>(observer)); } static final class DetachSingleObserver implements SingleObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccess.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccess.java index 01fa4ce590..d7ab5e4c79 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccess.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterSuccess.java @@ -39,7 +39,7 @@ public SingleDoAfterSuccess(SingleSource source, Consumer onAfterS @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new DoAfterObserver(observer, onAfterSuccess)); + source.subscribe(new DoAfterObserver<>(observer, onAfterSuccess)); } static final class DoAfterObserver implements SingleObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminate.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminate.java index bcb604e2ca..eca7ebff15 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminate.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoAfterTerminate.java @@ -39,7 +39,7 @@ public SingleDoAfterTerminate(SingleSource source, Action onAfterTerminate) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new DoAfterTerminateObserver(observer, onAfterTerminate)); + source.subscribe(new DoAfterTerminateObserver<>(observer, onAfterTerminate)); } static final class DoAfterTerminateObserver implements SingleObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoFinally.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoFinally.java index bfa309be03..0ba60a6493 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoFinally.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoFinally.java @@ -41,7 +41,7 @@ public SingleDoFinally(SingleSource source, Action onFinally) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new DoFinallyObserver(observer, onFinally)); + source.subscribe(new DoFinallyObserver<>(observer, onFinally)); } static final class DoFinallyObserver extends AtomicInteger implements SingleObserver, Disposable { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnDispose.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnDispose.java index 31dd0183f4..262f4f7d43 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnDispose.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnDispose.java @@ -35,7 +35,7 @@ public SingleDoOnDispose(SingleSource source, Action onDispose) { @Override protected void subscribeActual(final SingleObserver observer) { - source.subscribe(new DoOnDisposeObserver(observer, onDispose)); + source.subscribe(new DoOnDisposeObserver<>(observer, onDispose)); } static final class DoOnDisposeObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnSubscribe.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnSubscribe.java index d7c9230fb7..24a24caac9 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnSubscribe.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleDoOnSubscribe.java @@ -38,7 +38,7 @@ public SingleDoOnSubscribe(SingleSource source, Consumer @Override protected void subscribeActual(final SingleObserver observer) { - source.subscribe(new DoOnSubscribeSingleObserver(observer, onSubscribe)); + source.subscribe(new DoOnSubscribeSingleObserver<>(observer, onSubscribe)); } static final class DoOnSubscribeSingleObserver implements SingleObserver { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java index d4a9f54737..d914799640 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapCompletable.java @@ -39,7 +39,7 @@ public SingleFlatMapCompletable(SingleSource source, Function parent = new FlatMapCompletableObserver(observer, mapper); + FlatMapCompletableObserver parent = new FlatMapCompletableObserver<>(observer, mapper); observer.onSubscribe(parent); source.subscribe(parent); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java index 0d9e4aee0f..9e05a25ef8 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableFlowable.java @@ -48,7 +48,7 @@ public SingleFlatMapIterableFlowable(SingleSource source, @Override protected void subscribeActual(Subscriber s) { - source.subscribe(new FlatMapIterableObserver(s, mapper)); + source.subscribe(new FlatMapIterableObserver<>(s, mapper)); } static final class FlatMapIterableObserver @@ -273,7 +273,7 @@ public boolean isEmpty() { @Nullable @Override - public R poll() throws Exception { + public R poll() { Iterator iterator = it; if (iterator != null) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java index 795f60a3ca..add137d7cb 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapIterableObservable.java @@ -44,7 +44,7 @@ public SingleFlatMapIterableObservable(SingleSource source, @Override protected void subscribeActual(Observer observer) { - source.subscribe(new FlatMapIterableObserver(observer, mapper)); + source.subscribe(new FlatMapIterableObserver<>(observer, mapper)); } static final class FlatMapIterableObserver @@ -183,7 +183,7 @@ public boolean isEmpty() { @Nullable @Override - public R poll() throws Exception { + public R poll() { Iterator iterator = it; if (iterator != null) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java index 4a636ba337..f474830e40 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleFlatMapPublisher.java @@ -56,7 +56,7 @@ public SingleFlatMapPublisher(SingleSource source, @Override protected void subscribeActual(Subscriber downstream) { - source.subscribe(new SingleFlatMapPublisherObserver(downstream, mapper)); + source.subscribe(new SingleFlatMapPublisherObserver<>(downstream, mapper)); } static final class SingleFlatMapPublisherObserver extends AtomicLong @@ -73,7 +73,7 @@ static final class SingleFlatMapPublisherObserver extends AtomicLong Function> mapper) { this.downstream = actual; this.mapper = mapper; - this.parent = new AtomicReference(); + this.parent = new AtomicReference<>(); } @Override diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleInternalHelper.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleInternalHelper.java index 8c322ada64..95ed46144c 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleInternalHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleInternalHelper.java @@ -36,17 +36,17 @@ enum NoSuchElementCallable implements Supplier, Callable INSTANCE; @Override - public NoSuchElementException call() throws Exception { + public NoSuchElementException call() { return new NoSuchElementException(); } @Override - public NoSuchElementException get() throws Throwable { + public NoSuchElementException get() { return new NoSuchElementException(); } } - public static Supplier emptyThrower() { + public static Supplier emptyThrower() { return NoSuchElementCallable.INSTANCE; } @@ -79,7 +79,7 @@ public boolean hasNext() { @Override public Flowable next() { - return new SingleToFlowable(sit.next()); + return new SingleToFlowable<>(sit.next()); } @Override @@ -98,12 +98,12 @@ static final class ToFlowableIterable implements Iterable> { @Override public Iterator> iterator() { - return new ToFlowableIterator(sources.iterator()); + return new ToFlowableIterator<>(sources.iterator()); } } public static Iterable> iterableToFlowable(final Iterable> sources) { - return new ToFlowableIterable(sources); + return new ToFlowableIterable<>(sources); } @SuppressWarnings("rawtypes") diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterialize.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterialize.java index fd0afc2f98..1ffbfe073a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterialize.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleMaterialize.java @@ -34,6 +34,6 @@ public SingleMaterialize(Single source) { @Override protected void subscribeActual(SingleObserver> observer) { - source.subscribe(new MaterializeSingleObserver(observer)); + source.subscribe(new MaterializeSingleObserver<>(observer)); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleObserveOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleObserveOn.java index 035f458e11..168a88ee8e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleObserveOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleObserveOn.java @@ -32,7 +32,7 @@ public SingleObserveOn(SingleSource source, Scheduler scheduler) { @Override protected void subscribeActual(final SingleObserver observer) { - source.subscribe(new ObserveOnSingleObserver(observer, scheduler)); + source.subscribe(new ObserveOnSingleObserver<>(observer, scheduler)); } static final class ObserveOnSingleObserver extends AtomicReference diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java index 6f768591e8..070b244747 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleResumeNext.java @@ -36,7 +36,7 @@ public SingleResumeNext(SingleSource source, @Override protected void subscribeActual(final SingleObserver observer) { - source.subscribe(new ResumeMainSingleObserver(observer, nextFunction)); + source.subscribe(new ResumeMainSingleObserver<>(observer, nextFunction)); } static final class ResumeMainSingleObserver extends AtomicReference diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleSubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleSubscribeOn.java index 79187148cd..3347fc1989 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleSubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleSubscribeOn.java @@ -31,7 +31,7 @@ public SingleSubscribeOn(SingleSource source, Scheduler scheduler) @Override protected void subscribeActual(final SingleObserver observer) { - final SubscribeOnObserver parent = new SubscribeOnObserver(observer, source); + final SubscribeOnObserver parent = new SubscribeOnObserver<>(observer, source); observer.onSubscribe(parent); Disposable f = scheduler.scheduleDirect(parent); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTakeUntil.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTakeUntil.java index e69ac1b118..50e8b89763 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTakeUntil.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTakeUntil.java @@ -43,7 +43,7 @@ public SingleTakeUntil(SingleSource source, Publisher other) { @Override protected void subscribeActual(SingleObserver observer) { - TakeUntilMainObserver parent = new TakeUntilMainObserver(observer); + TakeUntilMainObserver parent = new TakeUntilMainObserver<>(observer); observer.onSubscribe(parent); other.subscribe(parent.other); diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTimeout.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTimeout.java index 1cb417c8db..f529d02446 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTimeout.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleTimeout.java @@ -47,7 +47,7 @@ public SingleTimeout(SingleSource source, long timeout, TimeUnit unit, Schedu @Override protected void subscribeActual(final SingleObserver observer) { - TimeoutMainObserver parent = new TimeoutMainObserver(observer, other, timeout, unit); + TimeoutMainObserver parent = new TimeoutMainObserver<>(observer, other, timeout, unit); observer.onSubscribe(parent); DisposableHelper.replace(parent.task, scheduler.scheduleDirect(parent, timeout, unit)); @@ -103,9 +103,9 @@ public void onError(Throwable e) { this.other = other; this.timeout = timeout; this.unit = unit; - this.task = new AtomicReference(); + this.task = new AtomicReference<>(); if (other != null) { - this.fallback = new TimeoutFallbackObserver(actual); + this.fallback = new TimeoutFallbackObserver<>(actual); } else { this.fallback = null; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleToObservable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleToObservable.java index 18ecef340f..810adaea3f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleToObservable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleToObservable.java @@ -44,7 +44,7 @@ public void subscribeActual(final Observer observer) { * @since 2.2 */ public static SingleObserver create(Observer downstream) { - return new SingleToObservableObserver(downstream); + return new SingleToObservableObserver<>(downstream); } static final class SingleToObservableObserver diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUnsubscribeOn.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUnsubscribeOn.java index 3f5b20b20c..9a78d61bb7 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUnsubscribeOn.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleUnsubscribeOn.java @@ -37,7 +37,7 @@ public SingleUnsubscribeOn(SingleSource source, Scheduler scheduler) { @Override protected void subscribeActual(SingleObserver observer) { - source.subscribe(new UnsubscribeOnSingleObserver(observer, scheduler)); + source.subscribe(new UnsubscribeOnSingleObserver<>(observer, scheduler)); } static final class UnsubscribeOnSingleObserver extends AtomicReference diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java index b207476c95..9130a2bf35 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipArray.java @@ -40,11 +40,11 @@ protected void subscribeActual(SingleObserver observer) { int n = sources.length; if (n == 1) { - sources[0].subscribe(new SingleMap.MapSingleObserver(observer, new SingletonArrayFunc())); + sources[0].subscribe(new SingleMap.MapSingleObserver<>(observer, new SingletonArrayFunc())); return; } - ZipCoordinator parent = new ZipCoordinator(observer, n, zipper); + ZipCoordinator parent = new ZipCoordinator<>(observer, n, zipper); observer.onSubscribe(parent); @@ -83,7 +83,7 @@ static final class ZipCoordinator extends AtomicInteger implements Disposa this.zipper = zipper; ZipSingleObserver[] o = new ZipSingleObserver[n]; for (int i = 0; i < n; i++) { - o[i] = new ZipSingleObserver(this, i); + o[i] = new ZipSingleObserver<>(this, i); } this.observers = o; this.values = new Object[n]; diff --git a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java index b41538aeb4..ad393987be 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java +++ b/src/main/java/io/reactivex/rxjava3/internal/operators/single/SingleZipIterable.java @@ -61,11 +61,11 @@ protected void subscribeActual(SingleObserver observer) { } if (n == 1) { - a[0].subscribe(new SingleMap.MapSingleObserver(observer, new SingletonArrayFunc())); + a[0].subscribe(new SingleMap.MapSingleObserver<>(observer, new SingletonArrayFunc())); return; } - ZipCoordinator parent = new ZipCoordinator(observer, n, zipper); + ZipCoordinator parent = new ZipCoordinator<>(observer, n, zipper); observer.onSubscribe(parent); diff --git a/src/main/java/io/reactivex/rxjava3/internal/queue/MpscLinkedQueue.java b/src/main/java/io/reactivex/rxjava3/internal/queue/MpscLinkedQueue.java index 8458e8d876..31fe6eca9b 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/queue/MpscLinkedQueue.java +++ b/src/main/java/io/reactivex/rxjava3/internal/queue/MpscLinkedQueue.java @@ -32,9 +32,9 @@ public final class MpscLinkedQueue implements SimplePlainQueue { private final AtomicReference> consumerNode; public MpscLinkedQueue() { - producerNode = new AtomicReference>(); - consumerNode = new AtomicReference>(); - LinkedQueueNode node = new LinkedQueueNode(); + producerNode = new AtomicReference<>(); + consumerNode = new AtomicReference<>(); + LinkedQueueNode node = new LinkedQueueNode<>(); spConsumerNode(node); xchgProducerNode(node); // this ensures correct construction: StoreLoad } @@ -59,7 +59,7 @@ public boolean offer(final T e) { if (null == e) { throw new NullPointerException("Null is not a valid element"); } - final LinkedQueueNode nextNode = new LinkedQueueNode(e); + final LinkedQueueNode nextNode = new LinkedQueueNode<>(e); final LinkedQueueNode prevProducerNode = xchgProducerNode(nextNode); // Should a producer thread get interrupted here the chain WILL be broken until that thread is resumed // and completes the store in prev.next. diff --git a/src/main/java/io/reactivex/rxjava3/internal/queue/SpscLinkedArrayQueue.java b/src/main/java/io/reactivex/rxjava3/internal/queue/SpscLinkedArrayQueue.java index 4a2a182dfc..2bd61dcc15 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/queue/SpscLinkedArrayQueue.java +++ b/src/main/java/io/reactivex/rxjava3/internal/queue/SpscLinkedArrayQueue.java @@ -48,7 +48,7 @@ public final class SpscLinkedArrayQueue implements SimplePlainQueue { public SpscLinkedArrayQueue(final int bufferSize) { int p2capacity = Pow2.roundToPowerOfTwo(Math.max(8, bufferSize)); int mask = p2capacity - 1; - AtomicReferenceArray buffer = new AtomicReferenceArray(p2capacity + 1); + AtomicReferenceArray buffer = new AtomicReferenceArray<>(p2capacity + 1); producerBuffer = buffer; producerMask = mask; adjustLookAheadStep(p2capacity); @@ -100,7 +100,7 @@ private boolean writeToQueue(final AtomicReferenceArray buffer, final T 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); + final AtomicReferenceArray newBuffer = new AtomicReferenceArray<>(capacity); producerBuffer = newBuffer; producerLookAhead = currIndex + mask - 1; soElement(newBuffer, offset, e); // StoreStore @@ -247,7 +247,7 @@ private static void soElement(AtomicReferenceArray buffer, int offset, O buffer.lazySet(offset, e); } - private static Object lvElement(AtomicReferenceArray buffer, int offset) { + private static Object lvElement(AtomicReferenceArray buffer, int offset) { return buffer.get(offset); } @@ -273,7 +273,7 @@ public boolean offer(T first, T second) { soProducerIndex(p + 2); } else { final int capacity = buffer.length(); - final AtomicReferenceArray newBuffer = new AtomicReferenceArray(capacity); + final AtomicReferenceArray newBuffer = new AtomicReferenceArray<>(capacity); producerBuffer = newBuffer; pi = calcWrappedOffset(p, m); diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/AbstractDirectTask.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/AbstractDirectTask.java index 2788bdfec6..a1c4bc2ce1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/AbstractDirectTask.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/AbstractDirectTask.java @@ -37,9 +37,9 @@ abstract class AbstractDirectTask protected Thread runner; - protected static final FutureTask FINISHED = new FutureTask(Functions.EMPTY_RUNNABLE, null); + protected static final FutureTask FINISHED = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); - protected static final FutureTask DISPOSED = new FutureTask(Functions.EMPTY_RUNNABLE, null); + protected static final FutureTask DISPOSED = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); AbstractDirectTask(Runnable runnable) { this.runnable = runnable; diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ComputationScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ComputationScheduler.java index eb686b17e0..e0bf388772 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ComputationScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ComputationScheduler.java @@ -135,7 +135,7 @@ public ComputationScheduler() { */ public ComputationScheduler(ThreadFactory threadFactory) { this.threadFactory = threadFactory; - this.pool = new AtomicReference(NONE); + this.pool = new AtomicReference<>(NONE); start(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancel.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancel.java index 231cb5b3f2..370fb5d8a2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancel.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/DisposeOnCancel.java @@ -15,6 +15,7 @@ import java.util.concurrent.*; +import io.reactivex.rxjava3.annotations.NonNull; import io.reactivex.rxjava3.disposables.Disposable; /** @@ -46,13 +47,12 @@ public boolean isDone() { } @Override - public Object get() throws InterruptedException, ExecutionException { + public Object get() { return null; } @Override - public Object get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { + public Object get(long timeout, @NonNull TimeUnit unit) { return null; } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ExecutorScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ExecutorScheduler.java index 8fd7b33720..a9547e64e1 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ExecutorScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ExecutorScheduler.java @@ -140,7 +140,7 @@ public static final class ExecutorWorker extends Scheduler.Worker implements Run public ExecutorWorker(Executor executor, boolean interruptibleWorker, boolean fair) { this.executor = executor; - this.queue = new MpscLinkedQueue(); + this.queue = new MpscLinkedQueue<>(); this.interruptibleWorker = interruptibleWorker; this.fair = fair; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java index 5df8793ebb..a855b44683 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/InstantPeriodicTask.java @@ -39,18 +39,18 @@ final class InstantPeriodicTask implements Callable, Disposable { Thread runner; - static final FutureTask CANCELLED = new FutureTask(Functions.EMPTY_RUNNABLE, null); + static final FutureTask CANCELLED = new FutureTask<>(Functions.EMPTY_RUNNABLE, null); InstantPeriodicTask(Runnable task, ExecutorService executor) { super(); this.task = task; - this.first = new AtomicReference>(); - this.rest = new AtomicReference>(); + this.first = new AtomicReference<>(); + this.rest = new AtomicReference<>(); this.executor = executor; } @Override - public Void call() throws Exception { + public Void call() { runner = Thread.currentThread(); try { task.run(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/IoScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/IoScheduler.java index f4edcf0208..3a4bac58b2 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/IoScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/IoScheduler.java @@ -77,7 +77,7 @@ static final class CachedWorkerPool implements Runnable { CachedWorkerPool(long keepAliveTime, TimeUnit unit, ThreadFactory threadFactory) { this.keepAliveTime = unit != null ? unit.toNanos(keepAliveTime) : 0L; - this.expiringWorkerQueue = new ConcurrentLinkedQueue(); + this.expiringWorkerQueue = new ConcurrentLinkedQueue<>(); this.allWorkers = new CompositeDisposable(); this.threadFactory = threadFactory; @@ -164,7 +164,7 @@ public IoScheduler() { */ public IoScheduler(ThreadFactory threadFactory) { this.threadFactory = threadFactory; - this.pool = new AtomicReference(NONE); + this.pool = new AtomicReference<>(NONE); start(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/RxThreadFactory.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/RxThreadFactory.java index 7d0646056d..5bd0614b89 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/RxThreadFactory.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/RxThreadFactory.java @@ -13,6 +13,8 @@ package io.reactivex.rxjava3.internal.schedulers; +import io.reactivex.rxjava3.annotations.NonNull; + import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicLong; @@ -47,7 +49,7 @@ public RxThreadFactory(String prefix, int priority, boolean nonBlocking) { } @Override - public Thread newThread(Runnable r) { + public Thread newThread(@NonNull Runnable r) { StringBuilder nameBuilder = new StringBuilder(prefix).append('-').append(incrementAndGet()); // if (CREATE_TRACE) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectTask.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectTask.java index 89bd9d3d01..d48ccf8ab6 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectTask.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/ScheduledDirectTask.java @@ -32,7 +32,7 @@ public ScheduledDirectTask(Runnable runnable) { } @Override - public Void call() throws Exception { + public Void call() { runner = Thread.currentThread(); try { runnable.run(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java index 392a3c3051..333d4ded9e 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SchedulerPoolFactory.java @@ -47,12 +47,12 @@ private SchedulerPoolFactory() { public static final int PURGE_PERIOD_SECONDS; static final AtomicReference PURGE_THREAD = - new AtomicReference(); + new AtomicReference<>(); // Upcast to the Map interface here to avoid 8.x compatibility issues. // See http://stackoverflow.com/a/32955708/61158 static final Map POOLS = - new ConcurrentHashMap(); + new ConcurrentHashMap<>(); /** * Starts the purge thread if not already started. @@ -134,7 +134,7 @@ static boolean getBooleanProperty(boolean enabled, String key, boolean defaultNo static final class SystemPropertyAccessor implements Function { @Override - public String apply(String t) throws Throwable { + public String apply(String t) { return System.getProperty(t); } } @@ -160,7 +160,7 @@ static void tryPutIntoPool(boolean purgeEnabled, ScheduledExecutorService exec) static final class ScheduledTask implements Runnable { @Override public void run() { - for (ScheduledThreadPoolExecutor e : new ArrayList(POOLS.keySet())) { + for (ScheduledThreadPoolExecutor e : new ArrayList<>(POOLS.keySet())) { if (e.isShutdown()) { POOLS.remove(e); } else { diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SingleScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SingleScheduler.java index a1dba3036b..13c7430f94 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/SingleScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/SingleScheduler.java @@ -28,7 +28,7 @@ public final class SingleScheduler extends Scheduler { final ThreadFactory threadFactory; - final AtomicReference executor = new AtomicReference(); + final AtomicReference executor = new AtomicReference<>(); /** The name of the system property for setting the thread priority for this Scheduler. */ private static final String KEY_SINGLE_PRIORITY = "rx3.single-priority"; diff --git a/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java b/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java index b8a164f9e1..5ddd867783 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java +++ b/src/main/java/io/reactivex/rxjava3/internal/schedulers/TrampolineScheduler.java @@ -66,7 +66,7 @@ public Disposable scheduleDirect(@NonNull Runnable run, long delay, TimeUnit uni } static final class TrampolineWorker extends Scheduler.Worker implements Disposable { - final PriorityBlockingQueue queue = new PriorityBlockingQueue(); + final PriorityBlockingQueue queue = new PriorityBlockingQueue<>(); private final AtomicInteger wip = new AtomicInteger(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriber.java b/src/main/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriber.java index 1e09f5b414..59edd4f80d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/internal/subscribers/FutureSubscriber.java @@ -19,6 +19,7 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; +import io.reactivex.rxjava3.annotations.NonNull; import org.reactivestreams.Subscription; import io.reactivex.rxjava3.core.FlowableSubscriber; @@ -42,7 +43,7 @@ public final class FutureSubscriber extends CountDownLatch public FutureSubscriber() { super(1); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); } @Override @@ -91,7 +92,7 @@ public T get() throws InterruptedException, ExecutionException { } @Override - public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + public T get(long timeout, @NonNull TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { if (getCount() != 0) { BlockingHelper.verifyNonBlocking(); if (!await(timeout, unit)) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriber.java b/src/main/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriber.java index 32668ed4d9..f351126044 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriber.java +++ b/src/main/java/io/reactivex/rxjava3/internal/subscribers/StrictSubscriber.java @@ -57,7 +57,7 @@ public StrictSubscriber(Subscriber downstream) { this.downstream = downstream; this.error = new AtomicThrowable(); this.requested = new AtomicLong(); - this.upstream = new AtomicReference(); + this.upstream = new AtomicReference<>(); this.once = new AtomicBoolean(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapper.java b/src/main/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapper.java index 55d052c705..4c9b6b4930 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/subscribers/SubscriberResourceWrapper.java @@ -28,7 +28,7 @@ public final class SubscriberResourceWrapper extends AtomicReference downstream; - final AtomicReference upstream = new AtomicReference(); + final AtomicReference upstream = new AtomicReference<>(); public SubscriberResourceWrapper(Subscriber downstream) { this.downstream = downstream; diff --git a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/AsyncSubscription.java b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/AsyncSubscription.java index 370c45ffbe..62973e7440 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/AsyncSubscription.java +++ b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/AsyncSubscription.java @@ -35,8 +35,8 @@ public final class AsyncSubscription extends AtomicLong implements Subscription, final AtomicReference resource; public AsyncSubscription() { - resource = new AtomicReference(); - actual = new AtomicReference(); + resource = new AtomicReference<>(); + actual = new AtomicReference<>(); } public AsyncSubscription(Disposable resource) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java index 6a50ab07dc..44c5fe5845 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java +++ b/src/main/java/io/reactivex/rxjava3/internal/subscriptions/SubscriptionArbiter.java @@ -63,7 +63,7 @@ public class SubscriptionArbiter extends AtomicInteger implements Subscription { public SubscriptionArbiter(boolean cancelOnReplace) { this.cancelOnReplace = cancelOnReplace; - missedSubscription = new AtomicReference(); + missedSubscription = new AtomicReference<>(); missedRequested = new AtomicLong(); missedProduced = new AtomicLong(); } diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/ArrayListSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/util/ArrayListSupplier.java index 1f82daf237..7338c99040 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/ArrayListSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/ArrayListSupplier.java @@ -32,10 +32,10 @@ public static Function> asFunction() { @Override public List get() { - return new ArrayList(); + return new ArrayList<>(); } @Override public List apply(Object o) { - return new ArrayList(); + return new ArrayList<>(); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/ConnectConsumer.java b/src/main/java/io/reactivex/rxjava3/internal/util/ConnectConsumer.java index 4d4f160a8a..63fd12dc99 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/ConnectConsumer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/ConnectConsumer.java @@ -23,7 +23,7 @@ public final class ConnectConsumer implements Consumer { public Disposable disposable; @Override - public void accept(Disposable t) throws Exception { + public void accept(Disposable t) { this.disposable = t; } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java b/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java index c4aaadb343..8b9d62317a 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/ExceptionHelper.java @@ -52,7 +52,7 @@ public static RuntimeException wrapOrThrow(Throwable error) { */ public static final Throwable TERMINATED = new Termination(); - public static boolean addThrowable(AtomicReference field, Throwable exception) { + public static boolean addThrowable(AtomicReference field, Throwable exception) { for (;;) { Throwable current = field.get(); @@ -73,7 +73,7 @@ public static boolean addThrowable(AtomicReference field, Throwab } } - public static Throwable terminate(AtomicReference field) { + public static Throwable terminate(AtomicReference field) { Throwable current = field.get(); if (current != TERMINATED) { current = field.getAndSet(TERMINATED); @@ -87,8 +87,8 @@ public static Throwable terminate(AtomicReference field) { * @return the list of Throwables flattened in a depth-first manner */ public static List flatten(Throwable t) { - List list = new ArrayList(); - ArrayDeque deque = new ArrayDeque(); + List list = new ArrayList<>(); + ArrayDeque deque = new ArrayDeque<>(); deque.offer(t); while (!deque.isEmpty()) { diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/HashMapSupplier.java b/src/main/java/io/reactivex/rxjava3/internal/util/HashMapSupplier.java index 22d9900d19..46236ff293 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/HashMapSupplier.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/HashMapSupplier.java @@ -26,6 +26,6 @@ public static Supplier> asSupplier() { } @Override public Map get() { - return new HashMap(); + return new HashMap<>(); } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/LinkedArrayList.java b/src/main/java/io/reactivex/rxjava3/internal/util/LinkedArrayList.java index f4b1d4aae2..08d675a4af 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/LinkedArrayList.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/LinkedArrayList.java @@ -92,7 +92,7 @@ public int size() { public String toString() { final int cap = capacityHint; final int s = size; - final List list = new ArrayList(s + 1); + final List list = new ArrayList<>(s + 1); Object[] h = head(); int j = 0; diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/ListAddBiConsumer.java b/src/main/java/io/reactivex/rxjava3/internal/util/ListAddBiConsumer.java index 9f484f1700..cd6c8e9750 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/ListAddBiConsumer.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/ListAddBiConsumer.java @@ -28,7 +28,7 @@ public static BiFunction, T, List> instance() { @SuppressWarnings("unchecked") @Override - public List apply(List t1, Object t2) throws Exception { + public List apply(List t1, Object t2) { t1.add(t2); return t1; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/MergerBiFunction.java b/src/main/java/io/reactivex/rxjava3/internal/util/MergerBiFunction.java index 9efe097a7c..64d9faff71 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/MergerBiFunction.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/MergerBiFunction.java @@ -30,12 +30,12 @@ public MergerBiFunction(Comparator comparator) { } @Override - public List apply(List a, List b) throws Exception { + public List apply(List a, List b) { int n = a.size() + b.size(); if (n == 0) { - return new ArrayList(); + return new ArrayList<>(); } - List both = new ArrayList(n); + List both = new ArrayList<>(n); Iterator at = a.iterator(); Iterator bt = b.iterator(); diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java b/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java index 9cdfab07ce..9808e2fe0d 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/QueueDrainHelper.java @@ -212,9 +212,9 @@ public static boolean checkTerminated(boolean d, boolean empty, */ public static SimpleQueue createQueue(int capacityHint) { if (capacityHint < 0) { - return new SpscLinkedArrayQueue(-capacityHint); + return new SpscLinkedArrayQueue<>(-capacityHint); } - return new SpscArrayQueue(capacityHint); + return new SpscArrayQueue<>(capacityHint); } /** diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/SorterFunction.java b/src/main/java/io/reactivex/rxjava3/internal/util/SorterFunction.java index 5a9929f604..a62efd154f 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/SorterFunction.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/SorterFunction.java @@ -26,7 +26,7 @@ public SorterFunction(Comparator comparator) { } @Override - public List apply(List t) throws Exception { + public List apply(List t) { Collections.sort(t, comparator); return t; } diff --git a/src/main/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayList.java b/src/main/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayList.java index 301dc634fa..33e1e18a56 100644 --- a/src/main/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayList.java +++ b/src/main/java/io/reactivex/rxjava3/internal/util/VolatileSizeArrayList.java @@ -13,6 +13,8 @@ package io.reactivex.rxjava3.internal.util; +import io.reactivex.rxjava3.annotations.NonNull; + import java.util.*; import java.util.concurrent.atomic.AtomicInteger; @@ -29,11 +31,11 @@ public final class VolatileSizeArrayList extends AtomicInteger implements Lis final ArrayList list; public VolatileSizeArrayList() { - list = new ArrayList(); + list = new ArrayList<>(); } public VolatileSizeArrayList(int initialCapacity) { - list = new ArrayList(initialCapacity); + list = new ArrayList<>(initialCapacity); } @Override @@ -62,7 +64,7 @@ public Object[] toArray() { } @Override - public E[] toArray(E[] a) { + public E[] toArray(@NonNull E[] a) { return list.toArray(a); } @@ -81,33 +83,33 @@ public boolean remove(Object o) { } @Override - public boolean containsAll(Collection c) { + public boolean containsAll(@NonNull Collection c) { return list.containsAll(c); } @Override - public boolean addAll(Collection c) { + public boolean addAll(@NonNull Collection c) { boolean b = list.addAll(c); lazySet(list.size()); return b; } @Override - public boolean addAll(int index, Collection c) { + public boolean addAll(int index, @NonNull Collection c) { boolean b = list.addAll(index, c); lazySet(list.size()); return b; } @Override - public boolean removeAll(Collection c) { + public boolean removeAll(@NonNull Collection c) { boolean b = list.removeAll(c); lazySet(list.size()); return b; } @Override - public boolean retainAll(Collection c) { + public boolean retainAll(@NonNull Collection c) { boolean b = list.retainAll(c); lazySet(list.size()); return b; diff --git a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java index c8fbf203ef..ea09fb53d4 100644 --- a/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java +++ b/src/main/java/io/reactivex/rxjava3/parallel/ParallelFlowable.java @@ -24,7 +24,6 @@ import io.reactivex.rxjava3.internal.functions.*; import io.reactivex.rxjava3.internal.jdk8.*; import io.reactivex.rxjava3.internal.operators.parallel.*; -import io.reactivex.rxjava3.internal.operators.parallel.ParallelReduceFull; import io.reactivex.rxjava3.internal.subscriptions.EmptySubscription; import io.reactivex.rxjava3.internal.util.*; import io.reactivex.rxjava3.plugins.RxJavaPlugins; @@ -161,7 +160,7 @@ public static ParallelFlowable from(@NonNull Publisher sourc ObjectHelper.verifyPositive(parallelism, "parallelism"); ObjectHelper.verifyPositive(prefetch, "prefetch"); - return RxJavaPlugins.onAssembly(new ParallelFromPublisher(source, parallelism, prefetch)); + return RxJavaPlugins.onAssembly(new ParallelFromPublisher<>(source, parallelism, prefetch)); } /** @@ -185,7 +184,7 @@ public static ParallelFlowable from(@NonNull Publisher sourc @BackpressureSupport(BackpressureKind.PASS_THROUGH) public final ParallelFlowable map(@NonNull Function mapper) { Objects.requireNonNull(mapper, "mapper"); - return RxJavaPlugins.onAssembly(new ParallelMap(this, mapper)); + return RxJavaPlugins.onAssembly(new ParallelMap<>(this, mapper)); } /** @@ -215,7 +214,7 @@ public final ParallelFlowable map(@NonNull Function ParallelFlowable map(@NonNull Function mapper, @NonNull ParallelFailureHandling errorHandler) { Objects.requireNonNull(mapper, "mapper"); Objects.requireNonNull(errorHandler, "errorHandler is null"); - return RxJavaPlugins.onAssembly(new ParallelMapTry(this, mapper, errorHandler)); + return RxJavaPlugins.onAssembly(new ParallelMapTry<>(this, mapper, errorHandler)); } /** @@ -246,7 +245,7 @@ public final ParallelFlowable map(@NonNull Function ParallelFlowable map(@NonNull Function mapper, @NonNull BiFunction errorHandler) { Objects.requireNonNull(mapper, "mapper"); Objects.requireNonNull(errorHandler, "errorHandler is null"); - return RxJavaPlugins.onAssembly(new ParallelMapTry(this, mapper, errorHandler)); + return RxJavaPlugins.onAssembly(new ParallelMapTry<>(this, mapper, errorHandler)); } /** @@ -620,8 +619,8 @@ public final Flowable sorted(@NonNull Comparator comparator, int c Objects.requireNonNull(comparator, "comparator is null"); ObjectHelper.verifyPositive(capacityHint, "capacityHint"); int ch = capacityHint / parallelism() + 1; - ParallelFlowable> railReduced = reduce(Functions.createArrayList(ch), ListAddBiConsumer.instance()); - ParallelFlowable> railSorted = railReduced.map(new SorterFunction(comparator)); + ParallelFlowable> railReduced = reduce(Functions.createArrayList(ch), ListAddBiConsumer.instance()); + ParallelFlowable> railSorted = railReduced.map(new SorterFunction<>(comparator)); return RxJavaPlugins.onAssembly(new ParallelSortedJoin<>(railSorted, comparator)); } @@ -673,10 +672,10 @@ public final Flowable> toSortedList(@NonNull Comparator compa ObjectHelper.verifyPositive(capacityHint, "capacityHint"); int ch = capacityHint / parallelism() + 1; - ParallelFlowable> railReduced = reduce(Functions.createArrayList(ch), ListAddBiConsumer.instance()); - ParallelFlowable> railSorted = railReduced.map(new SorterFunction(comparator)); + ParallelFlowable> railReduced = reduce(Functions.createArrayList(ch), ListAddBiConsumer.instance()); + ParallelFlowable> railSorted = railReduced.map(new SorterFunction<>(comparator)); - Flowable> merged = railSorted.reduce(new MergerBiFunction(comparator)); + Flowable> merged = railSorted.reduce(new MergerBiFunction<>(comparator)); return RxJavaPlugins.onAssembly(merged); } @@ -1008,7 +1007,7 @@ public final ParallelFlowable doOnCancel(@NonNull Action onCancel) { public final ParallelFlowable collect(@NonNull Supplier collectionSupplier, @NonNull BiConsumer collector) { Objects.requireNonNull(collectionSupplier, "collectionSupplier is null"); Objects.requireNonNull(collector, "collector is null"); - return RxJavaPlugins.onAssembly(new ParallelCollect(this, collectionSupplier, collector)); + return RxJavaPlugins.onAssembly(new ParallelCollect<>(this, collectionSupplier, collector)); } /** diff --git a/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java index ea300afe34..8a20c9140d 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/MulticastProcessor.java @@ -558,7 +558,7 @@ void drain() { } if (as != bs) { - continue outer; + continue; } if (done && q.isEmpty()) { diff --git a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java index 9b722057b9..b7880d6833 100644 --- a/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java +++ b/src/main/java/io/reactivex/rxjava3/processors/ReplayProcessor.java @@ -30,7 +30,7 @@ /** * Replays events to Subscribers. *

- * The {@code ReplayProcessor} supports the following item retainment strategies: + * The {@code ReplayProcessor} supports the following item retention strategies: *

    *
  • {@link #create()} and {@link #create(int)}: retains and replays all events to current and * future {@code Subscriber}s. @@ -173,7 +173,7 @@ public final class ReplayProcessor extends FlowableProcessor { @CheckReturnValue @NonNull public static ReplayProcessor create() { - return new ReplayProcessor<>(new UnboundedReplayBuffer(16)); + return new ReplayProcessor<>(new UnboundedReplayBuffer<>(16)); } /** @@ -194,7 +194,7 @@ public static ReplayProcessor create() { @CheckReturnValue @NonNull public static ReplayProcessor create(int capacityHint) { - return new ReplayProcessor<>(new UnboundedReplayBuffer(capacityHint)); + return new ReplayProcessor<>(new UnboundedReplayBuffer<>(capacityHint)); } /** @@ -220,7 +220,7 @@ public static ReplayProcessor create(int capacityHint) { @CheckReturnValue @NonNull public static ReplayProcessor createWithSize(int maxSize) { - return new ReplayProcessor<>(new SizeBoundReplayBuffer(maxSize)); + return new ReplayProcessor<>(new SizeBoundReplayBuffer<>(maxSize)); } /** @@ -238,7 +238,7 @@ public static ReplayProcessor createWithSize(int maxSize) { */ @CheckReturnValue /* test */ static ReplayProcessor createUnbounded() { - return new ReplayProcessor<>(new SizeBoundReplayBuffer(Integer.MAX_VALUE)); + return new ReplayProcessor<>(new SizeBoundReplayBuffer<>(Integer.MAX_VALUE)); } /** @@ -276,7 +276,7 @@ public static ReplayProcessor createWithSize(int maxSize) { @CheckReturnValue @NonNull public static ReplayProcessor createWithTime(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { - return new ReplayProcessor<>(new SizeAndTimeBoundReplayBuffer(Integer.MAX_VALUE, maxAge, unit, scheduler)); + return new ReplayProcessor<>(new SizeAndTimeBoundReplayBuffer<>(Integer.MAX_VALUE, maxAge, unit, scheduler)); } /** @@ -316,7 +316,7 @@ public static ReplayProcessor createWithTime(long maxAge, @NonNull TimeUn @CheckReturnValue @NonNull public static ReplayProcessor createWithTimeAndSize(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int maxSize) { - return new ReplayProcessor<>(new SizeAndTimeBoundReplayBuffer(maxSize, maxAge, unit, scheduler)); + return new ReplayProcessor<>(new SizeAndTimeBoundReplayBuffer<>(maxSize, maxAge, unit, scheduler)); } /** diff --git a/src/main/java/io/reactivex/rxjava3/schedulers/Schedulers.java b/src/main/java/io/reactivex/rxjava3/schedulers/Schedulers.java index 254511f5e0..4d4c3517a5 100644 --- a/src/main/java/io/reactivex/rxjava3/schedulers/Schedulers.java +++ b/src/main/java/io/reactivex/rxjava3/schedulers/Schedulers.java @@ -112,8 +112,8 @@ private Schedulers() { * before the {@link Schedulers} class is referenced in your code. *

    Supported system properties ({@code System.getProperty()}): *

      - *
    • {@code rx3.computation-threads} (int): sets the number of threads in the {@link #computation()} Scheduler, default is the number of available CPUs
    • - *
    • {@code rx3.computation-priority} (int): sets the thread priority of the {@link #computation()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • + *
    • {@code rx3.computation-threads} (int): sets the number of threads in the {@code computation()} Scheduler, default is the number of available CPUs
    • + *
    • {@code rx3.computation-priority} (int): sets the thread priority of the {@code computation()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • *
    *

    * The default value of this scheduler can be overridden at initialization time via the @@ -157,8 +157,8 @@ public static Scheduler computation() { * before the {@link Schedulers} class is referenced in your code. *

    Supported system properties ({@code System.getProperty()}): *

      - *
    • {@code rx3.io-keep-alive-time} (long): sets the keep-alive time of the {@link #io()} Scheduler workers, default is {@link IoScheduler#KEEP_ALIVE_TIME_DEFAULT}
    • - *
    • {@code rx3.io-priority} (int): sets the thread priority of the {@link #io()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • + *
    • {@code rx3.io-keep-alive-time} (long): sets the keep-alive time of the {@code io()} Scheduler workers, default is {@link IoScheduler#KEEP_ALIVE_TIME_DEFAULT}
    • + *
    • {@code rx3.io-priority} (int): sets the thread priority of the {@code io()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • *
    *

    * The default value of this scheduler can be overridden at initialization time via the @@ -216,7 +216,7 @@ public static Scheduler trampoline() { * before the {@link Schedulers} class is referenced in your code. *

    Supported system properties ({@code System.getProperty()}): *

      - *
    • {@code rx3.newthread-priority} (int): sets the thread priority of the {@link #newThread()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • + *
    • {@code rx3.newthread-priority} (int): sets the thread priority of the {@code newThread()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • *
    *

    * The default value of this scheduler can be overridden at initialization time via the @@ -265,7 +265,7 @@ public static Scheduler newThread() { * before the {@link Schedulers} class is referenced in your code. *

    Supported system properties ({@code System.getProperty()}): *

      - *
    • {@code rx3.single-priority} (int): sets the thread priority of the {@link #single()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • + *
    • {@code rx3.single-priority} (int): sets the thread priority of the {@code single()} Scheduler, default is {@link Thread#NORM_PRIORITY}
    • *
    *

    * The default value of this scheduler can be overridden at initialization time via the @@ -522,28 +522,28 @@ public static void start() { static final class IOTask implements Supplier { @Override - public Scheduler get() throws Exception { + public Scheduler get() { return IoHolder.DEFAULT; } } static final class NewThreadTask implements Supplier { @Override - public Scheduler get() throws Exception { + public Scheduler get() { return NewThreadHolder.DEFAULT; } } static final class SingleTask implements Supplier { @Override - public Scheduler get() throws Exception { + public Scheduler get() { return SingleHolder.DEFAULT; } } static final class ComputationTask implements Supplier { @Override - public Scheduler get() throws Exception { + public Scheduler get() { return ComputationHolder.DEFAULT; } } diff --git a/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java b/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java index 9bb3d93b99..efad31e669 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/ReplaySubject.java @@ -160,7 +160,7 @@ public final class ReplaySubject extends Subject { @CheckReturnValue @NonNull public static ReplaySubject create() { - return new ReplaySubject<>(new UnboundedReplayBuffer(16)); + return new ReplaySubject<>(new UnboundedReplayBuffer<>(16)); } /** @@ -181,7 +181,7 @@ public static ReplaySubject create() { @CheckReturnValue @NonNull public static ReplaySubject create(int capacityHint) { - return new ReplaySubject<>(new UnboundedReplayBuffer(capacityHint)); + return new ReplaySubject<>(new UnboundedReplayBuffer<>(capacityHint)); } /** @@ -207,7 +207,7 @@ public static ReplaySubject create(int capacityHint) { @CheckReturnValue @NonNull public static ReplaySubject createWithSize(int maxSize) { - return new ReplaySubject<>(new SizeBoundReplayBuffer(maxSize)); + return new ReplaySubject<>(new SizeBoundReplayBuffer<>(maxSize)); } /** @@ -224,7 +224,7 @@ public static ReplaySubject createWithSize(int maxSize) { * @return the created subject */ /* test */ static ReplaySubject createUnbounded() { - return new ReplaySubject<>(new SizeBoundReplayBuffer(Integer.MAX_VALUE)); + return new ReplaySubject<>(new SizeBoundReplayBuffer<>(Integer.MAX_VALUE)); } /** @@ -262,7 +262,7 @@ public static ReplaySubject createWithSize(int maxSize) { @CheckReturnValue @NonNull public static ReplaySubject createWithTime(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler) { - return new ReplaySubject<>(new SizeAndTimeBoundReplayBuffer(Integer.MAX_VALUE, maxAge, unit, scheduler)); + return new ReplaySubject<>(new SizeAndTimeBoundReplayBuffer<>(Integer.MAX_VALUE, maxAge, unit, scheduler)); } /** @@ -302,7 +302,7 @@ public static ReplaySubject createWithTime(long maxAge, @NonNull TimeUnit @CheckReturnValue @NonNull public static ReplaySubject createWithTimeAndSize(long maxAge, @NonNull TimeUnit unit, @NonNull Scheduler scheduler, int maxSize) { - return new ReplaySubject<>(new SizeAndTimeBoundReplayBuffer(maxSize, maxAge, unit, scheduler)); + return new ReplaySubject<>(new SizeAndTimeBoundReplayBuffer<>(maxSize, maxAge, unit, scheduler)); } /** diff --git a/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java b/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java index 8f495a9ca5..5e19604eed 100644 --- a/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java +++ b/src/main/java/io/reactivex/rxjava3/subjects/UnicastSubject.java @@ -30,7 +30,7 @@ /** * A Subject that queues up events until a single {@link Observer} subscribes to it, replays * those events to it until the {@code Observer} catches up and then switches to relaying events live to - * this single {@code Observer} until this {@code UnicastSubject} terminates or the {@code Observer} unsubscribes. + * this single {@code Observer} until this {@code UnicastSubject} terminates or the {@code Observer} disposes. *

    * *

    @@ -48,10 +48,10 @@ * optionally delays an error it receives and replays it after the regular items have been emitted.

  • *
  • {@link #create(int, Runnable)} - creates an empty, unbounded {@code UnicastSubject} * with a hint about how many total items one expects to retain and a callback that will be - * called exactly once when the {@code UnicastSubject} gets terminated or the single {@code Observer} unsubscribes.
  • + * called exactly once when the {@code UnicastSubject} gets terminated or the single {@code Observer} disposes. *
  • {@link #create(int, Runnable, boolean)} - creates an empty, unbounded {@code UnicastSubject} * with a hint about how many total items one expects to retain and a callback that will be - * called exactly once when the {@code UnicastSubject} gets terminated or the single {@code Observer} unsubscribes + * called exactly once when the {@code UnicastSubject} gets terminated or the single {@code Observer} disposes * and optionally delays an error it receives and replays it after the regular items have been emitted.
  • *
*

@@ -510,7 +510,7 @@ public int requestFusion(int mode) { @Nullable @Override - public T poll() throws Exception { + public T poll() { return queue.poll(); } From 34f381ceec3fa0b5fc0a03c76161dc1cde2f09bc Mon Sep 17 00:00:00 2001 From: David Karnok Date: Sat, 28 Dec 2019 18:05:30 +0100 Subject: [PATCH 031/665] 3.x: [Java 8] Add flattenStreamAsX to Maybe/Single (#6805) * 3.x: [Java 8] Add flattenStreamAsX to Maybe/Single * Add RS TCK tests for flattenStreamAsFlowable --- .../java/io/reactivex/rxjava3/core/Maybe.java | 85 ++++ .../io/reactivex/rxjava3/core/Observable.java | 1 + .../io/reactivex/rxjava3/core/Single.java | 87 ++++ .../jdk8/MaybeFlattenStreamAsFlowable.java | 285 +++++++++++ .../jdk8/MaybeFlattenStreamAsObservable.java | 262 ++++++++++ .../jdk8/SingleFlattenStreamAsFlowable.java | 47 ++ .../jdk8/SingleFlattenStreamAsObservable.java | 45 ++ .../MaybeFlattenStreamAsFlowableTckTest.java | 40 ++ .../MaybeFlattenStreamAsFlowableTest.java | 453 ++++++++++++++++++ .../MaybeFlattenStreamAsObservableTest.java | 431 +++++++++++++++++ .../SingleFlattenStreamAsFlowableTckTest.java | 40 ++ .../SingleFlattenStreamAsFlowableTest.java | 440 +++++++++++++++++ .../SingleFlattenStreamAsObservableTest.java | 418 ++++++++++++++++ .../validators/JavadocForAnnotations.java | 6 +- .../rxjava3/validators/JavadocWording.java | 14 +- 15 files changed, 2647 insertions(+), 7 deletions(-) create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowable.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservable.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowable.java create mode 100644 src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservable.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservableTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTckTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTest.java create mode 100644 src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservableTest.java diff --git a/src/main/java/io/reactivex/rxjava3/core/Maybe.java b/src/main/java/io/reactivex/rxjava3/core/Maybe.java index dfe8c4f0b4..84a088d5c7 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Maybe.java +++ b/src/main/java/io/reactivex/rxjava3/core/Maybe.java @@ -15,6 +15,7 @@ import java.util.*; import java.util.concurrent.*; +import java.util.stream.*; import org.reactivestreams.*; @@ -3152,6 +3153,7 @@ public final Maybe flatMap(@NonNull FunctionReactiveX operators documentation: FlatMap + * @see #flattenStreamAsFlowable(Function) */ @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @@ -5004,4 +5006,87 @@ public final CompletionStage toCompletionStage() { public final CompletionStage toCompletionStage(@Nullable T defaultItem) { return subscribeWith(new CompletionStageConsumer<>(true, defaultItem)); } + + /** + * Maps the upstream succecss value into a Java {@link Stream} and emits its + * items to the downstream consumer as a {@link Flowable}. + *

+ * + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flattenAsFlowable(Function)}: + *


+     * source.flattenAsFlowable(item -> createStream(item)::iterator);
+     * 
+ *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flattenStreamAsFlowable(item -> IntStream.rangeClosed(1, 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors backpressure from downstream and iterates the given {@code Stream} + * on demand (i.e., when requested).
+ *
Scheduler:
+ *
{@code flattenStreamAsFlowable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the {@code Stream} and the output {@code Flowable} + * @param mapper the function that receives the upstream success item and should + * return a {@code Stream} of values to emit. + * @return the new Flowable instance + * @since 3.0.0 + * @see #flattenAsFlowable(Function) + * @see #flattenStreamAsObservable(Function) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.FULL) + @NonNull + public final Flowable flattenStreamAsFlowable(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new MaybeFlattenStreamAsFlowable<>(this, mapper)); + } + + /** + * Maps the upstream succecss value into a Java {@link Stream} and emits its + * items to the downstream consumer as an {@link Observable}. + * + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flattenAsObservable(Function)}: + *


+     * source.flattenAsObservable(item -> createStream(item)::iterator);
+     * 
+ *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flattenStreamAsObservable(item -> IntStream.rangeClosed(1, 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Scheduler:
+ *
{@code flattenStreamAsObservable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the {@code Stream} and the output {@code Observable} + * @param mapper the function that receives the upstream success item and should + * return a {@code Stream} of values to emit. + * @return the new Observable instance + * @since 3.0.0 + * @see #flattenAsObservable(Function) + * @see #flattenStreamAsFlowable(Function) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final Observable flattenStreamAsObservable(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new MaybeFlattenStreamAsObservable<>(this, mapper)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/core/Observable.java b/src/main/java/io/reactivex/rxjava3/core/Observable.java index 373a576888..7c2558036e 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Observable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Observable.java @@ -6517,6 +6517,7 @@ public final Observable concatMap(@NonNull FunctionReactiveX operators documentation: FlatMap */ @CheckReturnValue diff --git a/src/main/java/io/reactivex/rxjava3/core/Single.java b/src/main/java/io/reactivex/rxjava3/core/Single.java index f0022aef17..28177cc9e3 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Single.java +++ b/src/main/java/io/reactivex/rxjava3/core/Single.java @@ -15,10 +15,12 @@ import java.util.*; import java.util.concurrent.*; +import java.util.stream.*; import org.reactivestreams.Publisher; import io.reactivex.rxjava3.annotations.*; +import io.reactivex.rxjava3.core.Observable; import io.reactivex.rxjava3.disposables.Disposable; import io.reactivex.rxjava3.exceptions.Exceptions; import io.reactivex.rxjava3.functions.*; @@ -2799,6 +2801,7 @@ public final Flowable flatMapPublisher(@NonNull FunctionReactiveX operators documentation: FlatMap + * @see #flattenStreamAsFlowable(Function) */ @BackpressureSupport(BackpressureKind.FULL) @CheckReturnValue @@ -2826,6 +2829,7 @@ public final Flowable flattenAsFlowable(@NonNull FunctionReactiveX operators documentation: FlatMap + * @see #flattenStreamAsObservable(Function) */ @CheckReturnValue @NonNull @@ -4308,4 +4312,87 @@ private static Single toSingle(@NonNull Flowable source) { public final CompletionStage toCompletionStage() { return subscribeWith(new CompletionStageConsumer<>(false, null)); } + + /** + * Maps the upstream succecss value into a Java {@link Stream} and emits its + * items to the downstream consumer as a {@link Flowable}. + * + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flattenAsFlowable(Function)}: + *


+     * source.flattenAsFlowable(item -> createStream(item)::iterator);
+     * 
+ *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flattenStreamAsFlowable(item -> IntStream.rangeClosed(1, 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Backpressure:
+ *
The operator honors backpressure from downstream and iterates the given {@code Stream} + * on demand (i.e., when requested).
+ *
Scheduler:
+ *
{@code flattenStreamAsFlowable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the {@code Stream} and the output {@code Flowable} + * @param mapper the function that receives the upstream success item and should + * return a {@code Stream} of values to emit. + * @return the new Flowable instance + * @since 3.0.0 + * @see #flattenAsFlowable(Function) + * @see #flattenStreamAsObservable(Function) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @BackpressureSupport(BackpressureKind.FULL) + @NonNull + public final Flowable flattenStreamAsFlowable(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new SingleFlattenStreamAsFlowable<>(this, mapper)); + } + + /** + * Maps the upstream succecss value into a Java {@link Stream} and emits its + * items to the downstream consumer as an {@link Observable}. + *

+ * + *

+ * The operator closes the {@code Stream} upon cancellation and when it terminates. Exceptions raised when + * closing a {@code Stream} are routed to the global error handler ({@link RxJavaPlugins#onError(Throwable)}. + * If a {@code Stream} should not be closed, turn it into an {@link Iterable} and use {@link #flattenAsFlowable(Function)}: + *


+     * source.flattenAsObservable(item -> createStream(item)::iterator);
+     * 
+ *

+ * Primitive streams are not supported and items have to be boxed manually (e.g., via {@link IntStream#boxed()}): + *


+     * source.flattenStreamAsObservable(item -> IntStream.rangeClosed(1, 10).boxed());
+     * 
+ *

+ * {@code Stream} does not support concurrent usage so creating and/or consuming the same instance multiple times + * from multiple threads can lead to undefined behavior. + *

+ *
Scheduler:
+ *
{@code flattenStreamAsObservable} does not operate by default on a particular {@link Scheduler}.
+ *
+ * @param the element type of the {@code Stream} and the output {@code Observable} + * @param mapper the function that receives the upstream success item and should + * return a {@code Stream} of values to emit. + * @return the new Observable instance + * @since 3.0.0 + * @see #flattenAsObservable(Function) + * @see #flattenStreamAsFlowable(Function) + */ + @CheckReturnValue + @SchedulerSupport(SchedulerSupport.NONE) + @NonNull + public final Observable flattenStreamAsObservable(@NonNull Function> mapper) { + Objects.requireNonNull(mapper, "mapper is null"); + return RxJavaPlugins.onAssembly(new SingleFlattenStreamAsObservable<>(this, mapper)); + } } diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowable.java new file mode 100644 index 0000000000..0ac724bdf4 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowable.java @@ -0,0 +1,285 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; + +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.annotations.*; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.disposables.DisposableHelper; +import io.reactivex.rxjava3.internal.subscriptions.*; +import io.reactivex.rxjava3.internal.util.BackpressureHelper; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Map the success value into a Java {@link Stream} and emits its values. + * + * @param the source value type + * @param the output value type + * @since 3.0.0 + */ +public final class MaybeFlattenStreamAsFlowable extends Flowable { + + final Maybe source; + + final Function> mapper; + + public MaybeFlattenStreamAsFlowable(Maybe source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(@NonNull Subscriber s) { + source.subscribe(new FlattenStreamMultiObserver<>(s, mapper)); + } + + static final class FlattenStreamMultiObserver + extends BasicIntQueueSubscription + implements MaybeObserver, SingleObserver { + + private static final long serialVersionUID = 7363336003027148283L; + + final Subscriber downstream; + + final Function> mapper; + + final AtomicLong requested; + + Disposable upstream; + + volatile Iterator iterator; + + AutoCloseable close; + + boolean once; + + volatile boolean cancelled; + + boolean outputFused; + + long emitted; + + FlattenStreamMultiObserver(Subscriber downstream, Function> mapper) { + this.downstream = downstream; + this.mapper = mapper; + this.requested = new AtomicLong(); + } + + @Override + public void onSubscribe(@NonNull Disposable d) { + if (DisposableHelper.validate(this.upstream, d)) { + this.upstream = d; + + downstream.onSubscribe(this); + } + } + + @Override + public void onSuccess(@NonNull T t) { + try { + Stream stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream"); + Iterator iterator = stream.iterator(); + AutoCloseable c = stream; + + if (!iterator.hasNext()) { + downstream.onComplete(); + close(c); + return; + } + this.iterator = iterator; + this.close = stream; + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + drain(); + } + + @Override + public void onError(@NonNull Throwable e) { + downstream.onError(e); + } + + @Override + public void onComplete() { + downstream.onComplete(); + } + + @Override + public void request(long n) { + if (SubscriptionHelper.validate(n)) { + BackpressureHelper.add(requested, n); + drain(); + } + } + + @Override + public void cancel() { + cancelled = true; + upstream.dispose(); + if (!outputFused) { + drain(); + } + } + + @Override + public int requestFusion(int mode) { + if ((mode & ASYNC) != 0) { + outputFused = true; + return ASYNC; + } + return NONE; + } + + @Override + public @Nullable R poll() throws Throwable { + Iterator it = iterator; + if (it != null) { + if (once) { + if (!it.hasNext()) { + clear(); + return null; + } + } else { + once = true; + } + return it.next(); + } + return null; + } + + @Override + public boolean isEmpty() { + Iterator it = iterator; + if (it != null) { + if (!once) { + return false; + } + if (it.hasNext()) { + return false; + } + clear(); + } + return true; + } + + @Override + public void clear() { + iterator = null; + AutoCloseable close = this.close; + this.close = null; + close(close); + } + + void close(AutoCloseable c) { + try { + if (c != null) { + c.close(); + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + RxJavaPlugins.onError(ex); + } + } + + void drain() { + if (getAndIncrement() != 0) { + return; + } + + int missed = 1; + Subscriber downstream = this.downstream; + long emitted = this.emitted; + long requested = this.requested.get(); + Iterator it = iterator; + + for (;;) { + + if (cancelled) { + clear(); + } else { + if (outputFused) { + if (it != null) { + downstream.onNext(null); + downstream.onComplete(); + } + } else { + if (it != null && emitted != requested) { + R item; + try { + item = it.next(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + cancelled = true; + continue; + } + + if (cancelled) { + continue; + } + + downstream.onNext(item); + emitted++; + + if (cancelled) { + continue; + } + + boolean has; + try { + has = it.hasNext(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + cancelled = true; + continue; + } + + if (cancelled) { + continue; + } + + if (!has) { + downstream.onComplete(); + cancelled = true; + } + continue; + } + } + } + + this.emitted = emitted; + missed = addAndGet(-missed); + if (missed == 0) { + return; + } + + requested = this.requested.get(); + if (it == null) { + it = iterator; + } + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservable.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservable.java new file mode 100644 index 0000000000..e7a74f0593 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservable.java @@ -0,0 +1,262 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.*; +import java.util.stream.Stream; + +import io.reactivex.rxjava3.annotations.*; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.core.Observer; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.Exceptions; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.disposables.DisposableHelper; +import io.reactivex.rxjava3.internal.observers.BasicIntQueueDisposable; +import io.reactivex.rxjava3.plugins.RxJavaPlugins; + +/** + * Map the success value into a Java {@link Stream} and emits its values. + * + * @param the source value type + * @param the output value type + * @since 3.0.0 + */ +public final class MaybeFlattenStreamAsObservable extends Observable { + + final Maybe source; + + final Function> mapper; + + public MaybeFlattenStreamAsObservable(Maybe source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(@NonNull Observer s) { + source.subscribe(new FlattenStreamMultiObserver<>(s, mapper)); + } + + static final class FlattenStreamMultiObserver + extends BasicIntQueueDisposable + implements MaybeObserver, SingleObserver { + + private static final long serialVersionUID = 7363336003027148283L; + + final Observer downstream; + + final Function> mapper; + + Disposable upstream; + + volatile Iterator iterator; + + AutoCloseable close; + + boolean once; + + volatile boolean disposed; + + boolean outputFused; + + FlattenStreamMultiObserver(Observer downstream, Function> mapper) { + this.downstream = downstream; + this.mapper = mapper; + } + + @Override + public void onSubscribe(@NonNull Disposable d) { + if (DisposableHelper.validate(this.upstream, d)) { + this.upstream = d; + + downstream.onSubscribe(this); + } + } + + @Override + public void onSuccess(@NonNull T t) { + try { + Stream stream = Objects.requireNonNull(mapper.apply(t), "The mapper returned a null Stream"); + Iterator iterator = stream.iterator(); + AutoCloseable c = stream; + + if (!iterator.hasNext()) { + downstream.onComplete(); + close(c); + return; + } + this.iterator = iterator; + this.close = stream; + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + return; + } + drain(); + } + + @Override + public void onError(@NonNull Throwable e) { + downstream.onError(e); + } + + @Override + public void onComplete() { + downstream.onComplete(); + } + + @Override + public void dispose() { + disposed = true; + upstream.dispose(); + if (!outputFused) { + drain(); + } + } + + @Override + public boolean isDisposed() { + return disposed; + } + + @Override + public int requestFusion(int mode) { + if ((mode & ASYNC) != 0) { + outputFused = true; + return ASYNC; + } + return NONE; + } + + @Override + public @Nullable R poll() throws Throwable { + Iterator it = iterator; + if (it != null) { + if (once) { + if (!it.hasNext()) { + clear(); + return null; + } + } else { + once = true; + } + return it.next(); + } + return null; + } + + @Override + public boolean isEmpty() { + Iterator it = iterator; + if (it != null) { + if (!once) { + return false; + } + if (it.hasNext()) { + return false; + } + clear(); + } + return true; + } + + @Override + public void clear() { + iterator = null; + AutoCloseable close = this.close; + this.close = null; + close(close); + } + + void close(AutoCloseable c) { + try { + if (c != null) { + c.close(); + } + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + RxJavaPlugins.onError(ex); + } + } + + void drain() { + if (getAndIncrement() != 0) { + return; + } + + int missed = 1; + Observer downstream = this.downstream; + Iterator it = iterator; + + for (;;) { + + if (disposed) { + clear(); + } else { + if (outputFused) { + downstream.onNext(null); + downstream.onComplete(); + } else { + R item; + try { + item = it.next(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + disposed = true; + continue; + } + + if (disposed) { + continue; + } + + downstream.onNext(item); + + if (disposed) { + continue; + } + + boolean has; + try { + has = it.hasNext(); + } catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + downstream.onError(ex); + disposed = true; + continue; + } + + if (disposed) { + continue; + } + + if (!has) { + downstream.onComplete(); + disposed = true; + } + continue; + } + } + + missed = addAndGet(-missed); + if (missed == 0) { + return; + } + } + } + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowable.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowable.java new file mode 100644 index 0000000000..d7286d5932 --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowable.java @@ -0,0 +1,47 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.Stream; + +import org.reactivestreams.Subscriber; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.jdk8.MaybeFlattenStreamAsFlowable.FlattenStreamMultiObserver; + +/** + * Map the success value into a Java {@link Stream} and emits its values. + * + * @param the source value type + * @param the output value type + * @since 3.0.0 + */ +public final class SingleFlattenStreamAsFlowable extends Flowable { + + final Single source; + + final Function> mapper; + + public SingleFlattenStreamAsFlowable(Single source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(@NonNull Subscriber s) { + source.subscribe(new FlattenStreamMultiObserver<>(s, mapper)); + } +} diff --git a/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservable.java b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservable.java new file mode 100644 index 0000000000..81b2d5fd7b --- /dev/null +++ b/src/main/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservable.java @@ -0,0 +1,45 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.Stream; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.jdk8.MaybeFlattenStreamAsObservable.FlattenStreamMultiObserver; + +/** + * Map the success value into a Java {@link Stream} and emits its values. + * + * @param the source value type + * @param the output value type + * @since 3.0.0 + */ +public final class SingleFlattenStreamAsObservable extends Observable { + + final Single source; + + final Function> mapper; + + public SingleFlattenStreamAsObservable(Single source, Function> mapper) { + this.source = source; + this.mapper = mapper; + } + + @Override + protected void subscribeActual(@NonNull Observer s) { + source.subscribe(new FlattenStreamMultiObserver<>(s, mapper)); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTckTest.java new file mode 100644 index 0000000000..94906fdd2f --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTckTest.java @@ -0,0 +1,40 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class MaybeFlattenStreamAsFlowableTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Maybe.just(1).flattenStreamAsFlowable(v -> IntStream.range(0, (int)elements).boxed()) + ; + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Maybe.just(1).flattenStreamAsFlowable(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTest.java new file mode 100644 index 0000000000..8e1e75b8e0 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsFlowableTest.java @@ -0,0 +1,453 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.*; + +import org.junit.Test; +import org.reactivestreams.Subscription; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; +import io.reactivex.rxjava3.subjects.MaybeSubject; +import io.reactivex.rxjava3.subscribers.TestSubscriber; +import io.reactivex.rxjava3.testsupport.*; + +public class MaybeFlattenStreamAsFlowableTest extends RxJavaTest { + + @Test + public void successJust() { + Maybe.just(1) + .flattenStreamAsFlowable(Stream::of) + .test() + .assertResult(1); + } + + @Test + public void successEmpty() { + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of()) + .test() + .assertResult(); + } + + @Test + public void successMany() { + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of(2, 3, 4, 5, 6)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void successManyTake() { + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of(2, 3, 4, 5, 6)) + .take(3) + .test() + .assertResult(2, 3, 4); + } + + @Test + public void empty() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Maybe.empty() + .flattenStreamAsFlowable(f) + .test() + .assertResult(); + + verify(f, never()).apply(any()); + } + + @Test + public void error() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Maybe.error(new TestException()) + .flattenStreamAsFlowable(f) + .test() + .assertFailure(TestException.class); + + verify(f, never()).apply(any()); + } + + @Test + public void mapperCrash() { + Maybe.just(1) + .flattenStreamAsFlowable(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(Maybe.never().flattenStreamAsFlowable(Stream::of)); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeMaybeToFlowable(m -> m.flattenStreamAsFlowable(Stream::of)); + } + + @Test + public void badRequest() { + TestHelper.assertBadRequestReported(MaybeSubject.create().flattenStreamAsFlowable(Stream::of)); + } + + @Test + public void fusedEmpty() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of()) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void fusedJust() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1); + } + + @Test + public void fusedMany() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3); + } + + @Test + public void fusedManyRejected() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.SYNC); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3); + } + + @Test + public void manyBackpressured() { + Maybe.just(1) + .flattenStreamAsFlowable(v -> IntStream.rangeClosed(1, 5).boxed()) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(1, 2) + .requestMore(2) + .assertValuesOnly(1, 2, 3, 4) + .requestMore(1) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void manyBackpressured2() { + Maybe.just(1) + .flattenStreamAsFlowable(v -> IntStream.rangeClosed(1, 5).boxed()) + .rebatchRequests(1) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(1, 2) + .requestMore(2) + .assertValuesOnly(1, 2, 3, 4) + .requestMore(1) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void fusedStreamAvailableLater() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + MaybeSubject ms = MaybeSubject.create(); + + ms + .flattenStreamAsFlowable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertEmpty(); + + ms.onSuccess(1); + + ts + .assertResult(1, 2, 3); + } + + @Test + public void fused() throws Throwable { + AtomicReference> qsr = new AtomicReference<>(); + + MaybeSubject ms = MaybeSubject.create(); + + ms + .flattenStreamAsFlowable(Stream::of) + .subscribe(new FlowableSubscriber() { + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + + @Override + @SuppressWarnings("unchecked") + public void onSubscribe(@NonNull Subscription s) { + qsr.set((QueueSubscription)s); + } + }); + + QueueSubscription qs = qsr.get(); + + assertEquals(QueueFuseable.ASYNC, qs.requestFusion(QueueFuseable.ASYNC)); + + assertTrue(qs.isEmpty()); + assertNull(qs.poll()); + + ms.onSuccess(1); + + assertFalse(qs.isEmpty()); + assertEquals(1, qs.poll().intValue()); + + assertTrue(qs.isEmpty()); + assertNull(qs.poll()); + + qs.cancel(); + + assertTrue(qs.isEmpty()); + assertNull(qs.poll()); + } + + @Test + public void requestOneByOne() { + TestSubscriber ts = new TestSubscriber<>(); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of(1, 2, 3, 4, 5)) + .subscribe(new FlowableSubscriber() { + + Subscription upstream; + + @Override + public void onSubscribe(@NonNull Subscription s) { + ts.onSubscribe(new BooleanSubscription()); + upstream = s; + s.request(1); + } + + @Override + public void onNext(Integer t) { + ts.onNext(t); + upstream.request(1); + } + + @Override + public void onError(Throwable t) { + ts.onError(t); + } + + @Override + public void onComplete() { + ts.onComplete(); + } + }); + + ts.assertResult(1, 2, 3, 4, 5); + } + + @Test + public void streamCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Maybe.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v).onClose(() -> { throw new TestException(); })) + .test() + .assertResult(1); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void hasNextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + throw new TestException(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> stream) + .test() + .assertFailure(TestException.class, 1); + } + + @Test + public void nextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + throw new TestException(); + } + }); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> stream) + .test() + .assertFailure(TestException.class); + } + + @Test + public void cancelAfterHasNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestSubscriber ts = new TestSubscriber<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + ts.cancel(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> stream) + .subscribeWith(ts) + .assertValuesOnly(1); + } + + @Test + public void cancelAfterNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestSubscriber ts = new TestSubscriber<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + ts.cancel(); + return 1; + } + }); + + Maybe.just(1) + .flattenStreamAsFlowable(v -> stream) + .subscribeWith(ts) + .assertEmpty(); + } + + @Test + public void requestSuccessRace() { + for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { + MaybeSubject ms = MaybeSubject.create(); + + TestSubscriber ts = new TestSubscriber<>(0L); + + ms.flattenStreamAsFlowable(Stream::of) + .subscribe(ts); + + Runnable r1 = () -> ms.onSuccess(1); + Runnable r2 = () -> ts.request(1); + + TestHelper.race(r1, r2); + + ts.assertResult(1); + } + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservableTest.java new file mode 100644 index 0000000000..dd0fb5c2ed --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/MaybeFlattenStreamAsObservableTest.java @@ -0,0 +1,431 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Stream; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.subjects.MaybeSubject; +import io.reactivex.rxjava3.testsupport.*; + +public class MaybeFlattenStreamAsObservableTest extends RxJavaTest { + + @Test + public void successJust() { + Maybe.just(1) + .flattenStreamAsObservable(Stream::of) + .test() + .assertResult(1); + } + + @Test + public void successEmpty() { + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of()) + .test() + .assertResult(); + } + + @Test + public void successMany() { + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of(2, 3, 4, 5, 6)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void successManyTake() { + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of(2, 3, 4, 5, 6)) + .take(3) + .test() + .assertResult(2, 3, 4); + } + + @Test + public void empty() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Maybe.empty() + .flattenStreamAsObservable(f) + .test() + .assertResult(); + + verify(f, never()).apply(any()); + } + + @Test + public void error() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Maybe.error(new TestException()) + .flattenStreamAsObservable(f) + .test() + .assertFailure(TestException.class); + + verify(f, never()).apply(any()); + } + + @Test + public void mapperCrash() { + Maybe.just(1) + .flattenStreamAsObservable(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(Maybe.never().flattenStreamAsObservable(Stream::of)); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeMaybeToObservable(m -> m.flattenStreamAsObservable(Stream::of)); + } + + @Test + public void fusedEmpty() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of()) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void fusedJust() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of(v)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1); + } + + @Test + public void fusedMany() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3); + } + + @Test + public void fusedManyRejected() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.SYNC); + + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3); + } + + @Test + public void fusedStreamAvailableLater() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + MaybeSubject ms = MaybeSubject.create(); + + ms + .flattenStreamAsObservable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertEmpty(); + + ms.onSuccess(1); + + to + .assertResult(1, 2, 3); + } + + @Test + public void fused() throws Throwable { + AtomicReference> qdr = new AtomicReference<>(); + + MaybeSubject ms = MaybeSubject.create(); + + ms + .flattenStreamAsObservable(Stream::of) + .subscribe(new Observer() { + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + + @Override + @SuppressWarnings("unchecked") + public void onSubscribe(Disposable d) { + qdr.set((QueueDisposable)d); + } + }); + + QueueDisposable qd = qdr.get(); + + assertEquals(QueueFuseable.ASYNC, qd.requestFusion(QueueFuseable.ASYNC)); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + ms.onSuccess(1); + + assertFalse(qd.isEmpty()); + assertEquals(1, qd.poll().intValue()); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + qd.dispose(); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + } + + @Test + public void fused2() throws Throwable { + AtomicReference> qdr = new AtomicReference<>(); + + MaybeSubject ms = MaybeSubject.create(); + + ms + .flattenStreamAsObservable(v -> Stream.of(v, v + 1)) + .subscribe(new Observer() { + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + + @Override + @SuppressWarnings("unchecked") + public void onSubscribe(Disposable d) { + qdr.set((QueueDisposable)d); + } + }); + + QueueDisposable qd = qdr.get(); + + assertEquals(QueueFuseable.ASYNC, qd.requestFusion(QueueFuseable.ASYNC)); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + ms.onSuccess(1); + + assertFalse(qd.isEmpty()); + assertEquals(1, qd.poll().intValue()); + + assertFalse(qd.isEmpty()); + assertEquals(2, qd.poll().intValue()); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + qd.dispose(); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + } + + @Test + public void streamCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Maybe.just(1) + .flattenStreamAsObservable(v -> Stream.of(v).onClose(() -> { throw new TestException(); })) + .test() + .assertResult(1); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void hasNextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + throw new TestException(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Maybe.just(1) + .flattenStreamAsObservable(v -> stream) + .test() + .assertFailure(TestException.class, 1); + } + + @Test + public void nextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + throw new TestException(); + } + }); + + Maybe.just(1) + .flattenStreamAsObservable(v -> stream) + .test() + .assertFailure(TestException.class); + } + + @Test + public void cancelAfterHasNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestObserver to = new TestObserver<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + to.dispose(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Maybe.just(1) + .flattenStreamAsObservable(v -> stream) + .subscribeWith(to) + .assertValuesOnly(1); + } + + @Test + public void cancelAfterNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestObserver to = new TestObserver<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + to.dispose(); + return 1; + } + }); + + Maybe.just(1) + .flattenStreamAsObservable(v -> stream) + .subscribeWith(to) + .assertEmpty(); + } + + @Test + public void cancelSuccessRace() { + for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { + MaybeSubject ms = MaybeSubject.create(); + + TestObserver to = new TestObserver<>(); + + ms.flattenStreamAsObservable(Stream::of) + .subscribe(to); + + Runnable r1 = () -> ms.onSuccess(1); + Runnable r2 = () -> to.dispose(); + + TestHelper.race(r1, r2); + } + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTckTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTckTest.java new file mode 100644 index 0000000000..509429d4af --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTckTest.java @@ -0,0 +1,40 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import java.util.stream.*; + +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.tck.BaseTck; + +@Test +public class SingleFlattenStreamAsFlowableTckTest extends BaseTck { + + @Override + public Publisher createPublisher(final long elements) { + return + Single.just(1).flattenStreamAsFlowable(v -> IntStream.range(0, (int)elements).boxed()) + ; + } + + @Override + public Publisher createFailedPublisher() { + Stream stream = Stream.of(1); + stream.forEach(v -> { }); + return Single.just(1).flattenStreamAsFlowable(v -> stream); + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTest.java new file mode 100644 index 0000000000..72dc8fb239 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsFlowableTest.java @@ -0,0 +1,440 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.*; + +import org.junit.Test; +import org.reactivestreams.Subscription; + +import io.reactivex.rxjava3.annotations.NonNull; +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.internal.subscriptions.BooleanSubscription; +import io.reactivex.rxjava3.subjects.SingleSubject; +import io.reactivex.rxjava3.subscribers.TestSubscriber; +import io.reactivex.rxjava3.testsupport.*; + +public class SingleFlattenStreamAsFlowableTest extends RxJavaTest { + + @Test + public void successJust() { + Single.just(1) + .flattenStreamAsFlowable(Stream::of) + .test() + .assertResult(1); + } + + @Test + public void successEmpty() { + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of()) + .test() + .assertResult(); + } + + @Test + public void successMany() { + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of(2, 3, 4, 5, 6)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void successManyTake() { + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of(2, 3, 4, 5, 6)) + .take(3) + .test() + .assertResult(2, 3, 4); + } + + @Test + public void error() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Single.error(new TestException()) + .flattenStreamAsFlowable(f) + .test() + .assertFailure(TestException.class); + + verify(f, never()).apply(any()); + } + + @Test + public void mapperCrash() { + Single.just(1) + .flattenStreamAsFlowable(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(Single.never().flattenStreamAsFlowable(Stream::of)); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeSingleToFlowable(m -> m.flattenStreamAsFlowable(Stream::of)); + } + + @Test + public void badRequest() { + TestHelper.assertBadRequestReported(SingleSubject.create().flattenStreamAsFlowable(Stream::of)); + } + + @Test + public void fusedEmpty() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of()) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void fusedJust() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1); + } + + @Test + public void fusedMany() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3); + } + + @Test + public void fusedManyRejected() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.SYNC); + + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3); + } + + @Test + public void manyBackpressured() { + Single.just(1) + .flattenStreamAsFlowable(v -> IntStream.rangeClosed(1, 5).boxed()) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(1, 2) + .requestMore(2) + .assertValuesOnly(1, 2, 3, 4) + .requestMore(1) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void manyBackpressured2() { + Single.just(1) + .flattenStreamAsFlowable(v -> IntStream.rangeClosed(1, 5).boxed()) + .rebatchRequests(1) + .test(0L) + .assertEmpty() + .requestMore(2) + .assertValuesOnly(1, 2) + .requestMore(2) + .assertValuesOnly(1, 2, 3, 4) + .requestMore(1) + .assertResult(1, 2, 3, 4, 5); + } + + @Test + public void fusedStreamAvailableLater() { + TestSubscriberEx ts = new TestSubscriberEx<>(); + ts.setInitialFusionMode(QueueFuseable.ANY); + + SingleSubject ss = SingleSubject.create(); + + ss + .flattenStreamAsFlowable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(ts); + + ts.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertEmpty(); + + ss.onSuccess(1); + + ts + .assertResult(1, 2, 3); + } + + @Test + public void fused() throws Throwable { + AtomicReference> qsr = new AtomicReference<>(); + + SingleSubject ss = SingleSubject.create(); + + ss + .flattenStreamAsFlowable(Stream::of) + .subscribe(new FlowableSubscriber() { + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + + @Override + @SuppressWarnings("unchecked") + public void onSubscribe(@NonNull Subscription s) { + qsr.set((QueueSubscription)s); + } + }); + + QueueSubscription qs = qsr.get(); + + assertEquals(QueueFuseable.ASYNC, qs.requestFusion(QueueFuseable.ASYNC)); + + assertTrue(qs.isEmpty()); + assertNull(qs.poll()); + + ss.onSuccess(1); + + assertFalse(qs.isEmpty()); + assertEquals(1, qs.poll().intValue()); + + assertTrue(qs.isEmpty()); + assertNull(qs.poll()); + + qs.cancel(); + + assertTrue(qs.isEmpty()); + assertNull(qs.poll()); + } + + @Test + public void requestOneByOne() { + TestSubscriber ts = new TestSubscriber<>(); + + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of(1, 2, 3, 4, 5)) + .subscribe(new FlowableSubscriber() { + + Subscription upstream; + + @Override + public void onSubscribe(@NonNull Subscription s) { + ts.onSubscribe(new BooleanSubscription()); + upstream = s; + s.request(1); + } + + @Override + public void onNext(Integer t) { + ts.onNext(t); + upstream.request(1); + } + + @Override + public void onError(Throwable t) { + ts.onError(t); + } + + @Override + public void onComplete() { + ts.onComplete(); + } + }); + + ts.assertResult(1, 2, 3, 4, 5); + } + + @Test + public void streamCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Single.just(1) + .flattenStreamAsFlowable(v -> Stream.of(v).onClose(() -> { throw new TestException(); })) + .test() + .assertResult(1); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void hasNextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + throw new TestException(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Single.just(1) + .flattenStreamAsFlowable(v -> stream) + .test() + .assertFailure(TestException.class, 1); + } + + @Test + public void nextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + throw new TestException(); + } + }); + + Single.just(1) + .flattenStreamAsFlowable(v -> stream) + .test() + .assertFailure(TestException.class); + } + + @Test + public void cancelAfterHasNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestSubscriber ts = new TestSubscriber<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + ts.cancel(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Single.just(1) + .flattenStreamAsFlowable(v -> stream) + .subscribeWith(ts) + .assertValuesOnly(1); + } + + @Test + public void cancelAfterNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestSubscriber ts = new TestSubscriber<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + ts.cancel(); + return 1; + } + }); + + Single.just(1) + .flattenStreamAsFlowable(v -> stream) + .subscribeWith(ts) + .assertEmpty(); + } + + @Test + public void requestSuccessRace() { + for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { + SingleSubject ss = SingleSubject.create(); + + TestSubscriber ts = new TestSubscriber<>(0L); + + ss.flattenStreamAsFlowable(Stream::of) + .subscribe(ts); + + Runnable r1 = () -> ss.onSuccess(1); + Runnable r2 = () -> ts.request(1); + + TestHelper.race(r1, r2); + + ts.assertResult(1); + } + } +} diff --git a/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservableTest.java b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservableTest.java new file mode 100644 index 0000000000..2b3a624d50 --- /dev/null +++ b/src/test/java/io/reactivex/rxjava3/internal/jdk8/SingleFlattenStreamAsObservableTest.java @@ -0,0 +1,418 @@ +/** + * Copyright (c) 2016-present, RxJava Contributors. + * + * 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 io.reactivex.rxjava3.internal.jdk8; + +import static org.junit.Assert.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Stream; + +import org.junit.Test; + +import io.reactivex.rxjava3.core.*; +import io.reactivex.rxjava3.disposables.Disposable; +import io.reactivex.rxjava3.exceptions.TestException; +import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.internal.fuseable.*; +import io.reactivex.rxjava3.observers.TestObserver; +import io.reactivex.rxjava3.subjects.SingleSubject; +import io.reactivex.rxjava3.testsupport.*; + +public class SingleFlattenStreamAsObservableTest extends RxJavaTest { + + @Test + public void successJust() { + Single.just(1) + .flattenStreamAsObservable(Stream::of) + .test() + .assertResult(1); + } + + @Test + public void successEmpty() { + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of()) + .test() + .assertResult(); + } + + @Test + public void successMany() { + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of(2, 3, 4, 5, 6)) + .test() + .assertResult(2, 3, 4, 5, 6); + } + + @Test + public void successManyTake() { + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of(2, 3, 4, 5, 6)) + .take(3) + .test() + .assertResult(2, 3, 4); + } + + @Test + public void error() throws Throwable { + @SuppressWarnings("unchecked") + Function> f = mock(Function.class); + + Single.error(new TestException()) + .flattenStreamAsObservable(f) + .test() + .assertFailure(TestException.class); + + verify(f, never()).apply(any()); + } + + @Test + public void mapperCrash() { + Single.just(1) + .flattenStreamAsObservable(v -> { throw new TestException(); }) + .test() + .assertFailure(TestException.class); + } + + @Test + public void dispose() { + TestHelper.checkDisposed(Single.never().flattenStreamAsObservable(Stream::of)); + } + + @Test + public void doubleOnSubscribe() { + TestHelper.checkDoubleOnSubscribeSingleToObservable(m -> m.flattenStreamAsObservable(Stream::of)); + } + + @Test + public void fusedEmpty() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of()) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(); + } + + @Test + public void fusedJust() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of(v)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1); + } + + @Test + public void fusedMany() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertResult(1, 2, 3); + } + + @Test + public void fusedManyRejected() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.SYNC); + + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.NONE) + .assertResult(1, 2, 3); + } + + @Test + public void fusedStreamAvailableLater() { + TestObserverEx to = new TestObserverEx<>(); + to.setInitialFusionMode(QueueFuseable.ANY); + + SingleSubject ss = SingleSubject.create(); + + ss + .flattenStreamAsObservable(v -> Stream.of(v, v + 1, v + 2)) + .subscribe(to); + + to.assertFuseable() + .assertFusionMode(QueueFuseable.ASYNC) + .assertEmpty(); + + ss.onSuccess(1); + + to + .assertResult(1, 2, 3); + } + + @Test + public void fused() throws Throwable { + AtomicReference> qdr = new AtomicReference<>(); + + SingleSubject ss = SingleSubject.create(); + + ss + .flattenStreamAsObservable(Stream::of) + .subscribe(new Observer() { + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + + @Override + @SuppressWarnings("unchecked") + public void onSubscribe(Disposable d) { + qdr.set((QueueDisposable)d); + } + }); + + QueueDisposable qd = qdr.get(); + + assertEquals(QueueFuseable.ASYNC, qd.requestFusion(QueueFuseable.ASYNC)); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + ss.onSuccess(1); + + assertFalse(qd.isEmpty()); + assertEquals(1, qd.poll().intValue()); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + qd.dispose(); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + } + + @Test + public void fused2() throws Throwable { + AtomicReference> qdr = new AtomicReference<>(); + + SingleSubject ss = SingleSubject.create(); + + ss + .flattenStreamAsObservable(v -> Stream.of(v, v + 1)) + .subscribe(new Observer() { + + @Override + public void onNext(Integer t) { + } + + @Override + public void onError(Throwable t) { + } + + @Override + public void onComplete() { + } + + @Override + @SuppressWarnings("unchecked") + public void onSubscribe(Disposable d) { + qdr.set((QueueDisposable)d); + } + }); + + QueueDisposable qd = qdr.get(); + + assertEquals(QueueFuseable.ASYNC, qd.requestFusion(QueueFuseable.ASYNC)); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + ss.onSuccess(1); + + assertFalse(qd.isEmpty()); + assertEquals(1, qd.poll().intValue()); + + assertFalse(qd.isEmpty()); + assertEquals(2, qd.poll().intValue()); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + + qd.dispose(); + + assertTrue(qd.isEmpty()); + assertNull(qd.poll()); + } + + @Test + public void streamCloseCrash() throws Throwable { + TestHelper.withErrorTracking(errors -> { + Single.just(1) + .flattenStreamAsObservable(v -> Stream.of(v).onClose(() -> { throw new TestException(); })) + .test() + .assertResult(1); + + TestHelper.assertUndeliverable(errors, 0, TestException.class); + }); + } + + @Test + public void hasNextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + throw new TestException(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Single.just(1) + .flattenStreamAsObservable(v -> stream) + .test() + .assertFailure(TestException.class, 1); + } + + @Test + public void nextThrowsInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + throw new TestException(); + } + }); + + Single.just(1) + .flattenStreamAsObservable(v -> stream) + .test() + .assertFailure(TestException.class); + } + + @Test + public void cancelAfterHasNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestObserver to = new TestObserver<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + int count; + + @Override + public boolean hasNext() { + if (count++ > 0) { + to.dispose(); + } + return true; + } + + @Override + public Integer next() { + return 1; + } + }); + + Single.just(1) + .flattenStreamAsObservable(v -> stream) + .subscribeWith(to) + .assertValuesOnly(1); + } + + @Test + public void cancelAfterNextInDrain() { + @SuppressWarnings("unchecked") + Stream stream = mock(Stream.class); + + TestObserver to = new TestObserver<>(); + + when(stream.iterator()).thenReturn(new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + to.dispose(); + return 1; + } + }); + + Single.just(1) + .flattenStreamAsObservable(v -> stream) + .subscribeWith(to) + .assertEmpty(); + } + + @Test + public void cancelSuccessRace() { + for (int i = 0; i < TestHelper.RACE_LONG_LOOPS; i++) { + SingleSubject ss = SingleSubject.create(); + + TestObserver to = new TestObserver<>(); + + ss.flattenStreamAsObservable(Stream::of) + .subscribe(to); + + Runnable r1 = () -> ss.onSuccess(1); + Runnable r2 = () -> to.dispose(); + + TestHelper.race(r1, r2); + } + } +} diff --git a/src/test/java/io/reactivex/rxjava3/validators/JavadocForAnnotations.java b/src/test/java/io/reactivex/rxjava3/validators/JavadocForAnnotations.java index 2f16d05c14..1411295126 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/JavadocForAnnotations.java +++ b/src/test/java/io/reactivex/rxjava3/validators/JavadocForAnnotations.java @@ -156,9 +156,9 @@ static final void scanForBadMethod(StringBuilder sourceCode, String annotation, ; int lc = lineNumber(sourceCode, idx); - e.append(" at io.reactivex.").append(baseClassName) - .append(" (").append(baseClassName).append(".java:") - .append(lc).append(")").append("\r\n\r\n"); + e.append(" at io.reactivex.rxjava3.core.").append(baseClassName) + .append(".method(").append(baseClassName).append(".java:") + .append(lc).append(")").append("\r\n"); } } } diff --git a/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java b/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java index 4a4baaf8ae..fc0d1db9cf 100644 --- a/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java +++ b/src/test/java/io/reactivex/rxjava3/validators/JavadocWording.java @@ -139,8 +139,11 @@ public void maybeDocRefersToMaybeTypes() throws Exception { int idx = m.javadoc.indexOf("Flowable", jdx); if (idx >= 0) { if (!m.signature.contains("Flowable")) { - e.append("java.lang.RuntimeException: Maybe doc mentions Flowable but not in the signature\r\n at io.reactivex.rxjava3.core.") - .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + Pattern p = Pattern.compile("@see\\s+#[A-Za-z0-9 _.,()]*Flowable"); + if (!p.matcher(m.javadoc).find()) { + e.append("java.lang.RuntimeException: Maybe doc mentions Flowable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + } } jdx = idx + 6; } else { @@ -180,8 +183,11 @@ public void maybeDocRefersToMaybeTypes() throws Exception { int idx = m.javadoc.indexOf("Observable", jdx); if (idx >= 0) { if (!m.signature.contains("Observable")) { - e.append("java.lang.RuntimeException: Maybe doc mentions Observable but not in the signature\r\n at io.reactivex.rxjava3.core.") - .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + Pattern p = Pattern.compile("@see\\s+#[A-Za-z0-9 _.,()]*Observable"); + if (!p.matcher(m.javadoc).find()) { + e.append("java.lang.RuntimeException: Maybe doc mentions Observable but not in the signature\r\n at io.reactivex.rxjava3.core.") + .append("Maybe.method(Maybe.java:").append(m.javadocLine + lineNumber(m.javadoc, idx) - 1).append(")\r\n\r\n"); + } } jdx = idx + 6; } else { From 641e8015494e8eae9732d4925b9a5962229e6954 Mon Sep 17 00:00:00 2001 From: David Karnok Date: Tue, 31 Dec 2019 10:39:25 +0100 Subject: [PATCH 032/665] 3.x: Prettify Flowable javadoc + validator (#6806) --- .../io/reactivex/rxjava3/core/Flowable.java | 6394 ++++++++--------- .../validators/JavadocCodesAndLinks.java | 375 + .../rxjava3/validators/JavadocWording.java | 113 + 3 files changed, 3684 insertions(+), 3198 deletions(-) create mode 100644 src/test/java/io/reactivex/rxjava3/validators/JavadocCodesAndLinks.java diff --git a/src/main/java/io/reactivex/rxjava3/core/Flowable.java b/src/main/java/io/reactivex/rxjava3/core/Flowable.java index 97785c1f4c..e38004b214 100644 --- a/src/main/java/io/reactivex/rxjava3/core/Flowable.java +++ b/src/main/java/io/reactivex/rxjava3/core/Flowable.java @@ -38,14 +38,14 @@ import io.reactivex.rxjava3.subscribers.*; /** - * The Flowable class that implements the Reactive Streams + * The {@code Flowable} class that implements the Reactive Streams {@link Publisher} * Pattern and offers factory methods, intermediate operators and the ability to consume reactive dataflows. *

- * Reactive Streams operates with {@link Publisher}s which {@code Flowable} extends. Many operators + * Reactive Streams operates with {@code Publisher}s which {@code Flowable} extends. Many operators * therefore accept general {@code Publisher}s directly and allow direct interoperation with other - * Reactive Streams implementations. + * Reactive Streams implementations. *

- * The Flowable hosts the default buffer size of 128 elements for operators, accessible via {@link #bufferSize()}, + * The {@code Flowable} hosts the default buffer size of 128 elements for operators, accessible via {@link #bufferSize()}, * that can be overridden globally via the system parameter {@code rx3.buffer-size}. Most operators, however, have * overloads that allow setting their internal buffer size explicitly. *

@@ -62,7 +62,7 @@ * Unlike the {@code Observable.subscribe()} of version 1.x, {@link #subscribe(Subscriber)} does not allow external cancellation * of a subscription and the {@link Subscriber} instance is expected to expose such capability if needed. *

- * Flowables support backpressure and require {@link Subscriber}s to signal demand via {@link Subscription#request(long)}. + * {@code Flowable}s support backpressure and require {@code Subscriber}s to signal demand via {@link Subscription#request(long)}. *

* Example: *


@@ -90,13 +90,13 @@
  * d.dispose();
  * 
*

- * The Reactive Streams specification is relatively strict when defining interactions between {@code Publisher}s and {@code Subscriber}s, so much so + * The Reactive Streams specification is relatively strict when defining interactions between {@code Publisher}s and {@code Subscriber}s, so much so * that there is a significant performance penalty due certain timing requirements and the need to prepare for invalid * request amounts via {@link Subscription#request(long)}. * Therefore, RxJava has introduced the {@link FlowableSubscriber} interface that indicates the consumer can be driven with relaxed rules. * All RxJava operators are implemented with these relaxed rules in mind. - * If the subscribing {@code Subscriber} does not implement this interface, for example, due to it being from another Reactive Streams compliant - * library, the Flowable will automatically apply a compliance wrapper around it. + * If the subscribing {@code Subscriber} does not implement this interface, for example, due to it being from another Reactive Streams compliant + * library, the {@code Flowable} will automatically apply a compliance wrapper around it. *

* {@code Flowable} is an abstract class, but it is not advised to implement sources and custom operators by extending the class directly due * to the large amounts of Reactive Streams @@ -142,11 +142,10 @@ * has to be explicitly expressed via operators such as {@link #subscribeOn(Scheduler)}, {@link #observeOn(Scheduler)} and {@link #parallel()}. In general, * operators featuring a {@link Scheduler} parameter are introducing this type of asynchrony into the flow. *

- * For more information see the ReactiveX - * documentation. + * For more information see the ReactiveX documentation. * * @param - * the type of the items emitted by the Flowable + * the type of the items emitted by the {@code Flowable} * @see Observable * @see ParallelFlowable * @see io.reactivex.rxjava3.subscribers.DisposableSubscriber @@ -159,7 +158,7 @@ public abstract class Flowable implements Publisher { } /** - * Mirrors the one Publisher in an Iterable of several Publishers that first either emits an item or sends + * Mirrors the one {@link Publisher} in an {@link Iterable} of several {@code Publisher}s that first either emits an item or sends * a termination notification. *

* @@ -173,9 +172,9 @@ public abstract class Flowable implements Publisher { * * @param the common element type * @param sources - * an Iterable of Publishers sources competing to react first. A subscription to each Publisher will - * occur in the same order as in this Iterable. - * @return a Flowable that emits the same sequence as whichever of the source Publishers first + * an {@code Iterable} of {@code Publisher}s sources competing to react first. A subscription to each {@code Publisher} will + * occur in the same order as in this {@code Iterable}. + * @return a {@code Flowable} that emits the same sequence as whichever of the source {@code Publisher}s first * emitted an item or sent a termination notification * @see ReactiveX operators documentation: Amb */ @@ -189,7 +188,7 @@ public static Flowable amb(@NonNull Iterable * @@ -203,9 +202,9 @@ public static Flowable amb(@NonNull Iterable the common element type * @param sources - * an array of Publisher sources competing to react first. A subscription to each Publisher will - * occur in the same order as in this Iterable. - * @return a Flowable that emits the same sequence as whichever of the source Publishers first + * an array of {@code Publisher} sources competing to react first. A subscription to each {@code Publisher} will + * occur in the same order as in this array. + * @return a {@code Flowable} that emits the same sequence as whichever of the source {@code Publisher}s first * emitted an item or sent a termination notification * @see ReactiveX operators documentation: Amb */ @@ -229,7 +228,7 @@ public static Flowable ambArray(Publisher... sources) { /** * Returns the default internal buffer size used by most async operators. *

The value can be overridden via system parameter {@code rx3.buffer-size} - * before the Flowable class is loaded. + * before the {@code Flowable} class is loaded. * @return the default internal buffer size. */ @CheckReturnValue @@ -238,26 +237,26 @@ public static int bufferSize() { } /** - * Combines a collection of source Publishers by emitting an item that aggregates the latest values of each of - * the source Publishers each time an item is received from any of the source Publishers, where this + * Combines a collection of source {@link Publisher}s by emitting an item that aggregates the latest values of each of + * the source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided array of source Publishers is empty, the resulting sequence completes immediately without emitting + * If the provided array of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting * any items and without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatestArray} does not operate by default on a particular {@link Scheduler}.
*
@@ -267,11 +266,11 @@ public static int bufferSize() { * @param * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -283,26 +282,26 @@ public static int bufferSize() { } /** - * Combines a collection of source Publishers by emitting an item that aggregates the latest values of each of - * the source Publishers each time an item is received from any of the source Publishers, where this + * Combines a collection of source {@link Publisher}s by emitting an item that aggregates the latest values of each of + * the source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided array of source Publishers is empty, the resulting sequence completes immediately without emitting + * If the provided array of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting * any items and without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatestArray} does not operate by default on a particular {@link Scheduler}.
*
@@ -312,13 +311,13 @@ public static int bufferSize() { * @param * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers + * the aggregation function used to combine the items emitted by the source {@code Publisher}s * @param bufferSize - * the internal buffer size and prefetch amount applied to every source Flowable - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the internal buffer size and prefetch amount applied to every source {@code Flowable} + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -336,26 +335,26 @@ public static Flowable combineLatestArray(@NonNull Publisher * Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided iterable of source Publishers is empty, the resulting sequence completes immediately without emitting + * If the provided iterable of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting * any items and without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
*
@@ -365,11 +364,11 @@ public static Flowable combineLatestArray(@NonNull Publisher * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -382,26 +381,26 @@ public static Flowable combineLatestArray(@NonNull Publisher * Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided iterable of source Publishers is empty, the resulting sequence completes immediately without emitting any items and + * If the provided iterable of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting any items and * without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
*
@@ -411,13 +410,13 @@ public static Flowable combineLatestArray(@NonNull Publisher * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers + * the aggregation function used to combine the items emitted by the source {@code Publisher}s * @param bufferSize - * the internal buffer size and prefetch amount applied to every source Flowable - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the internal buffer size and prefetch amount applied to every source {@code Flowable} + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -433,26 +432,26 @@ public static Flowable combineLatest(@NonNull Iterable * Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided array of source Publishers is empty, the resulting sequence completes immediately without emitting + * If the provided array of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting * any items and without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatestDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -462,11 +461,11 @@ public static Flowable combineLatest(@NonNull Iterable * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -479,27 +478,27 @@ public static Flowable combineLatest(@NonNull Iterable * Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided array of source Publishers is empty, the resulting sequence completes immediately without emitting + * If the provided array of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting * any items and without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatestDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -509,13 +508,13 @@ public static Flowable combineLatest(@NonNull Iterable * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers + * the aggregation function used to combine the items emitted by the source {@code Publisher}s * @param bufferSize - * the internal buffer size and prefetch amount applied to every source Flowable - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the internal buffer size and prefetch amount applied to every source {@code Flowable} + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -534,27 +533,27 @@ public static Flowable combineLatest(@NonNull Iterable * Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided iterable of source Publishers is empty, the resulting sequence completes immediately without emitting + * If the provided iterable of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting * any items and without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatestDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -564,11 +563,11 @@ public static Flowable combineLatest(@NonNull Iterable * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -581,27 +580,27 @@ public static Flowable combineLatest(@NonNull Iterable * Note on method signature: since Java doesn't allow creating a generic array with {@code new T[]}, the * implementation of this operator has to create an {@code Object[]} instead. Unfortunately, a - * {@code Function} passed to the method would trigger a {@code ClassCastException}. + * {@code Function} passed to the method would trigger a {@link ClassCastException}. *

* If any of the sources never produces an item but only terminates (normally or with an error), the * resulting sequence terminates immediately (normally or with all the errors accumulated until that point). * If that input source is also synchronous, other sources after it will not be subscribed to. *

- * If the provided iterable of source Publishers is empty, the resulting sequence completes immediately without emitting + * If the provided iterable of source {@code Publisher}s is empty, the resulting sequence completes immediately without emitting * any items and without any calls to the combiner function. * *

*
Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatestDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -611,13 +610,13 @@ public static Flowable combineLatest(@NonNull Iterable * the result type * @param sources - * the collection of source Publishers + * the collection of source {@code Publisher}s * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers + * the aggregation function used to combine the items emitted by the source {@code Publisher}s * @param bufferSize - * the internal buffer size and prefetch amount applied to every source Flowable - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the internal buffer size and prefetch amount applied to every source {@code Flowable} + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SchedulerSupport(SchedulerSupport.NONE) @@ -633,8 +632,8 @@ public static Flowable combineLatestDelayError(@NonNull Iterable * If any of the sources never produces an item but only terminates (normally or with an error), the @@ -646,7 +645,7 @@ public static Flowable combineLatestDelayError(@NonNull IterableBackpressure: *
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -655,13 +654,13 @@ public static Flowable combineLatestDelayError(@NonNull Iterable the element type of the second source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -679,8 +678,8 @@ public static Flowable combineLatest( } /** - * Combines three source Publishers by emitting an item that aggregates the latest values of each of the - * source Publishers each time an item is received from any of the source Publishers, where this + * Combines three source {@link Publisher}s by emitting an item that aggregates the latest values of each of the + * source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* If any of the sources never produces an item but only terminates (normally or with an error), the @@ -692,7 +691,7 @@ public static Flowable combineLatest( *

Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -702,15 +701,15 @@ public static Flowable combineLatest( * @param the element type of the third source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param source3 - * the third source Publisher + * the third source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -730,8 +729,8 @@ public static Flowable combineLatest( } /** - * Combines four source Publishers by emitting an item that aggregates the latest values of each of the - * source Publishers each time an item is received from any of the source Publishers, where this + * Combines four source {@link Publisher}s by emitting an item that aggregates the latest values of each of the + * source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* If any of the sources never produces an item but only terminates (normally or with an error), the @@ -743,7 +742,7 @@ public static Flowable combineLatest( *

Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -754,17 +753,17 @@ public static Flowable combineLatest( * @param the element type of the fourth source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param source3 - * the third source Publisher + * the third source {@code Publisher} * @param source4 - * the fourth source Publisher + * the fourth source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -785,8 +784,8 @@ public static Flowable combineLatest( } /** - * Combines five source Publishers by emitting an item that aggregates the latest values of each of the - * source Publishers each time an item is received from any of the source Publishers, where this + * Combines five source {@link Publisher}s by emitting an item that aggregates the latest values of each of the + * source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* If any of the sources never produces an item but only terminates (normally or with an error), the @@ -798,7 +797,7 @@ public static Flowable combineLatest( *

Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -810,19 +809,19 @@ public static Flowable combineLatest( * @param the element type of the fifth source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param source3 - * the third source Publisher + * the third source {@code Publisher} * @param source4 - * the fourth source Publisher + * the fourth source {@code Publisher} * @param source5 - * the fifth source Publisher + * the fifth source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -845,8 +844,8 @@ public static Flowable combineLatest( } /** - * Combines six source Publishers by emitting an item that aggregates the latest values of each of the - * source Publishers each time an item is received from any of the source Publishers, where this + * Combines six source {@link Publisher}s by emitting an item that aggregates the latest values of each of the + * source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* If any of the sources never produces an item but only terminates (normally or with an error), the @@ -858,7 +857,7 @@ public static Flowable combineLatest( *

Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -871,21 +870,21 @@ public static Flowable combineLatest( * @param the element type of the sixth source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param source3 - * the third source Publisher + * the third source {@code Publisher} * @param source4 - * the fourth source Publisher + * the fourth source {@code Publisher} * @param source5 - * the fifth source Publisher + * the fifth source {@code Publisher} * @param source6 - * the sixth source Publisher + * the sixth source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -909,8 +908,8 @@ public static Flowable combineLatest( } /** - * Combines seven source Publishers by emitting an item that aggregates the latest values of each of the - * source Publishers each time an item is received from any of the source Publishers, where this + * Combines seven source {@link Publisher}s by emitting an item that aggregates the latest values of each of the + * source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* If any of the sources never produces an item but only terminates (normally or with an error), the @@ -922,7 +921,7 @@ public static Flowable combineLatest( *

Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -936,23 +935,23 @@ public static Flowable combineLatest( * @param the element type of the seventh source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param source3 - * the third source Publisher + * the third source {@code Publisher} * @param source4 - * the fourth source Publisher + * the fourth source {@code Publisher} * @param source5 - * the fifth source Publisher + * the fifth source {@code Publisher} * @param source6 - * the sixth source Publisher + * the sixth source {@code Publisher} * @param source7 - * the seventh source Publisher + * the seventh source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -978,8 +977,8 @@ public static Flowable combineLatest( } /** - * Combines eight source Publishers by emitting an item that aggregates the latest values of each of the - * source Publishers each time an item is received from any of the source Publishers, where this + * Combines eight source {@link Publisher}s by emitting an item that aggregates the latest values of each of the + * source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* If any of the sources never produces an item but only terminates (normally or with an error), the @@ -991,7 +990,7 @@ public static Flowable combineLatest( *

Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -1006,25 +1005,25 @@ public static Flowable combineLatest( * @param the element type of the eighth source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param source3 - * the third source Publisher + * the third source {@code Publisher} * @param source4 - * the fourth source Publisher + * the fourth source {@code Publisher} * @param source5 - * the fifth source Publisher + * the fifth source {@code Publisher} * @param source6 - * the sixth source Publisher + * the sixth source {@code Publisher} * @param source7 - * the seventh source Publisher + * the seventh source {@code Publisher} * @param source8 - * the eighth source Publisher + * the eighth source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -1051,8 +1050,8 @@ public static Flowable combineLatest( } /** - * Combines nine source Publishers by emitting an item that aggregates the latest values of each of the - * source Publishers each time an item is received from any of the source Publishers, where this + * Combines nine source {@link Publisher}s by emitting an item that aggregates the latest values of each of the + * source {@code Publisher}s each time an item is received from any of the source {@code Publisher}s, where this * aggregation is defined by a specified function. *

* If any of the sources never produces an item but only terminates (normally or with an error), the @@ -1064,7 +1063,7 @@ public static Flowable combineLatest( *

Backpressure:
*
The returned {@code Publisher} honors backpressure from downstream. The source {@code Publisher}s * are requested in a bounded manner, however, their backpressure is not enforced (the operator won't signal - * {@code MissingBackpressureException}) and may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * {@link MissingBackpressureException}) and may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code combineLatest} does not operate by default on a particular {@link Scheduler}.
* @@ -1080,27 +1079,27 @@ public static Flowable combineLatest( * @param the element type of the ninth source * @param the combined output type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * the second source Publisher + * the second source {@code Publisher} * @param source3 - * the third source Publisher + * the third source {@code Publisher} * @param source4 - * the fourth source Publisher + * the fourth source {@code Publisher} * @param source5 - * the fifth source Publisher + * the fifth source {@code Publisher} * @param source6 - * the sixth source Publisher + * the sixth source {@code Publisher} * @param source7 - * the seventh source Publisher + * the seventh source {@code Publisher} * @param source8 - * the eighth source Publisher + * the eighth source {@code Publisher} * @param source9 - * the ninth source Publisher + * the ninth source {@code Publisher} * @param combiner - * the aggregation function used to combine the items emitted by the source Publishers - * @return a Flowable that emits items that are the result of combining the items emitted by the source - * Publishers by means of the given aggregation function + * the aggregation function used to combine the items emitted by the source {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of combining the items emitted by the source + * {@code Publisher}s by means of the given aggregation function * @see ReactiveX operators documentation: CombineLatest */ @SuppressWarnings("unchecked") @@ -1129,7 +1128,7 @@ public static Flowable combineLatest( } /** - * Concatenates elements of each Publisher provided via an Iterable sequence into a single sequence + * Concatenates elements of each {@link Publisher} provided via an {@link Iterable} sequence into a single sequence * of elements without interleaving them. *

* @@ -1138,13 +1137,13 @@ public static Flowable combineLatest( *

The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, it may throw an - * {@code IllegalStateException} when the source {@code Publisher} completes.
+ * {@link IllegalStateException} when the source {@code Publisher} completes. *
Scheduler:
*
{@code concat} does not operate by default on a particular {@link Scheduler}.
* * @param the common value type of the sources - * @param sources the Iterable sequence of Publishers - * @return the new Flowable instance + * @param sources the {@code Iterable} sequence of {@code Publisher}s + * @return the new {@code Flowable} instance */ @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @@ -1158,25 +1157,25 @@ public static Flowable concat(@NonNull Iterable * *
*
Backpressure:
*
The operator honors backpressure from downstream. Both the outer and inner {@code Publisher} * sources are expected to honor backpressure as well. If the outer violates this, a - * {@code MissingBackpressureException} is signaled. If any of the inner {@code Publisher}s violates - * this, it may throw an {@code IllegalStateException} when an inner {@code Publisher} completes.
+ * {@link MissingBackpressureException} is signaled. If any of the inner {@code Publisher}s violates + * this, it may throw an {@link IllegalStateException} when an inner {@code Publisher} completes. *
Scheduler:
*
{@code concat} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * a Publisher that emits Publishers - * @return a Flowable that emits items all of the items emitted by the Publishers emitted by - * {@code Publishers}, one after the other, without interleaving them + * a {@code Publisher} that emits {@code Publisher}s + * @return a {@code Flowable} that emits items all of the items emitted by the {@code Publisher}s emitted by + * {@code Publisher}s, one after the other, without interleaving them * @see ReactiveX operators documentation: Concat */ @CheckReturnValue @@ -1188,27 +1187,27 @@ public static Flowable concat(@NonNull Publisher * *
*
Backpressure:
*
The operator honors backpressure from downstream. Both the outer and inner {@code Publisher} * sources are expected to honor backpressure as well. If the outer violates this, a - * {@code MissingBackpressureException} is signaled. If any of the inner {@code Publisher}s violates - * this, it may throw an {@code IllegalStateException} when an inner {@code Publisher} completes.
+ * {@link MissingBackpressureException} is signaled. If any of the inner {@code Publisher}s violates + * this, it may throw an {@link IllegalStateException} when an inner {@code Publisher} completes. *
Scheduler:
*
{@code concat} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * a Publisher that emits Publishers + * a {@code Publisher} that emits {@code Publisher}s * @param prefetch - * the number of Publishers to prefetch from the sources sequence. - * @return a Flowable that emits items all of the items emitted by the Publishers emitted by - * {@code Publishers}, one after the other, without interleaving them + * the number of {@code Publisher}s to prefetch from the sources sequence. + * @return a {@code Flowable} that emits items all of the items emitted by the {@code Publisher}s emitted by + * {@code Publisher}s, one after the other, without interleaving them * @see ReactiveX operators documentation: Concat */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -1221,7 +1220,7 @@ public static Flowable concat(@NonNull Publisher * @@ -1230,17 +1229,17 @@ public static Flowable concat(@NonNull PublisherThe operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, it may throw an - * {@code IllegalStateException} when the source {@code Publisher} completes. + * {@link IllegalStateException} when the source {@code Publisher} completes. *
Scheduler:
*
{@code concat} does not operate by default on a particular {@link Scheduler}.
* * * @param the common element base type * @param source1 - * a Publisher to be concatenated + * a {@code Publisher} to be concatenated * @param source2 - * a Publisher to be concatenated - * @return a Flowable that emits items emitted by the two source Publishers, one after the other, + * a {@code Publisher} to be concatenated + * @return a {@code Flowable} that emits items emitted by the two source {@code Publisher}s, one after the other, * without interleaving them * @see ReactiveX operators documentation: Concat */ @@ -1255,7 +1254,7 @@ public static Flowable concat(@NonNull Publisher source1, @N } /** - * Returns a Flowable that emits the items emitted by three Publishers, one after the other, without + * Returns a {@code Flowable} that emits the items emitted by three {@link Publisher}s, one after the other, without * interleaving them. *

* @@ -1264,19 +1263,19 @@ public static Flowable concat(@NonNull Publisher source1, @N *

The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, it may throw an - * {@code IllegalStateException} when the source {@code Publisher} completes.
+ * {@link IllegalStateException} when the source {@code Publisher} completes. *
Scheduler:
*
{@code concat} does not operate by default on a particular {@link Scheduler}.
* * * @param the common element base type * @param source1 - * a Publisher to be concatenated + * a {@code Publisher} to be concatenated * @param source2 - * a Publisher to be concatenated + * a {@code Publisher} to be concatenated * @param source3 - * a Publisher to be concatenated - * @return a Flowable that emits items emitted by the three source Publishers, one after the other, + * a {@code Publisher} to be concatenated + * @return a {@code Flowable} that emits items emitted by the three source {@code Publisher}s, one after the other, * without interleaving them * @see ReactiveX operators documentation: Concat */ @@ -1294,7 +1293,7 @@ public static Flowable concat( } /** - * Returns a Flowable that emits the items emitted by four Publishers, one after the other, without + * Returns a {@code Flowable} that emits the items emitted by four {@link Publisher}s, one after the other, without * interleaving them. *

* @@ -1303,21 +1302,21 @@ public static Flowable concat( *

The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, it may throw an - * {@code IllegalStateException} when the source {@code Publisher} completes.
+ * {@link IllegalStateException} when the source {@code Publisher} completes. *
Scheduler:
*
{@code concat} does not operate by default on a particular {@link Scheduler}.
* * * @param the common element base type * @param source1 - * a Publisher to be concatenated + * a {@code Publisher} to be concatenated * @param source2 - * a Publisher to be concatenated + * a {@code Publisher} to be concatenated * @param source3 - * a Publisher to be concatenated + * a {@code Publisher} to be concatenated * @param source4 - * a Publisher to be concatenated - * @return a Flowable that emits items emitted by the four source Publishers, one after the other, + * a {@code Publisher} to be concatenated + * @return a {@code Flowable} that emits items emitted by the four source {@code Publisher}s, one after the other, * without interleaving them * @see ReactiveX operators documentation: Concat */ @@ -1336,9 +1335,9 @@ public static Flowable concat( } /** - * Concatenates a variable number of Publisher sources. + * Concatenates a variable number of {@link Publisher} sources. *

- * Note: named this way because of overload conflict with concat(Publisher<Publisher>). + * Note: named this way because of overload conflict with {@code concat(Publisher>}). *

* *

@@ -1346,14 +1345,14 @@ public static Flowable concat( *
The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, it may throw an - * {@code IllegalStateException} when the source {@code Publisher} completes.
+ * {@link IllegalStateException} when the source {@code Publisher} completes. *
Scheduler:
*
{@code concatArray} does not operate by default on a particular {@link Scheduler}.
*
- * @param sources the array of sources + * @param sources the array of source {@code Publisher}s * @param the common base value type - * @return the new Publisher instance - * @throws NullPointerException if sources is null + * @return the new {@code Publisher} instance + * @throws NullPointerException if sources is {@code null} */ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @@ -1371,7 +1370,7 @@ public static Flowable concatArray(@NonNull Publisher... sou } /** - * Concatenates a variable number of Publisher sources and delays errors from any of them + * Concatenates a variable number of {@link Publisher} sources and delays errors from any of them * till all terminate. *

* @@ -1380,14 +1379,14 @@ public static Flowable concatArray(@NonNull Publisher... sou *

The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, it may throw an - * {@code IllegalStateException} when the source {@code Publisher} completes.
+ * {@link IllegalStateException} when the source {@code Publisher} completes. *
Scheduler:
*
{@code concatArrayDelayError} does not operate by default on a particular {@link Scheduler}.
* - * @param sources the array of sources + * @param sources the array of source {@code Publisher}s * @param the common base value type - * @return the new Flowable instance - * @throws NullPointerException if sources is null + * @return the new {@code Flowable} instance + * @throws NullPointerException if sources is {@code null} */ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @@ -1405,25 +1404,25 @@ public static Flowable concatArrayDelayError(@NonNull Publisher * *

* Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them * in order, each one after the previous one completes. *

*
Backpressure:
*
The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, the operator will signal a - * {@code MissingBackpressureException}.
+ * {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
* @param the value type - * @param sources an array of Publishers that need to be eagerly concatenated - * @return the new Publisher instance with the specified concatenation behavior + * @param sources an array of {@code Publisher}s that need to be eagerly concatenated + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -1436,28 +1435,28 @@ public static Flowable concatArrayEager(@NonNull Publisher.. } /** - * Concatenates an array of Publishers eagerly into a single stream of values. + * Concatenates an array of {@link Publisher}s eagerly into a single stream of values. *

* *

* Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them * in order, each one after the previous one completes. *

*
Backpressure:
*
The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, the operator will signal a - * {@code MissingBackpressureException}.
+ * {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
* @param the value type - * @param sources an array of Publishers that need to be eagerly concatenated + * @param sources an array of {@code Publisher}s that need to be eagerly concatenated * @param maxConcurrency the maximum number of concurrent subscriptions at a time, {@link Integer#MAX_VALUE} * is interpreted as an indication to subscribe to all sources at once - * @param prefetch the number of elements to prefetch from each Publisher source - * @return the new Publisher instance with the specified concatenation behavior + * @param prefetch the number of elements to prefetch from each {@code Publisher} source + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -1487,13 +1486,13 @@ public static Flowable concatArrayEager(int maxConcurrency, int prefetch, *
The operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, the operator will signal a - * {@code MissingBackpressureException}.
+ * {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
* * @param the value type * @param sources an array of {@code Publisher}s that need to be eagerly concatenated - * @return the new Flowable instance with the specified concatenation behavior + * @return the new {@code Flowable} instance with the specified concatenation behavior * @since 2.2.1 - experimental */ @CheckReturnValue @@ -1519,7 +1518,7 @@ public static Flowable concatArrayEagerDelayError(@NonNull PublisherThe operator honors backpressure from downstream. The {@code Publisher} * sources are expected to honor backpressure as well. * If any of the source {@code Publisher}s violate this, the operator will signal a - * {@code MissingBackpressureException}. + * {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
* @@ -1528,7 +1527,7 @@ public static Flowable concatArrayEagerDelayError(@NonNull Publisher Flowable concatArrayEagerDelayError(int maxConcurrency, int } /** - * Concatenates the Iterable sequence of Publishers into a single sequence by subscribing to each Publisher, - * one after the other, one at a time and delays any errors till the all inner Publishers terminate. + * Concatenates the {@link Iterable} sequence of {@link Publisher}s into a single sequence by subscribing to each {@code Publisher}, + * one after the other, one at a time and delays any errors till the all inner {@code Publisher}s terminate. * *
*
Backpressure:
*
The operator honors backpressure from downstream. Both the outer and inner {@code Publisher} * sources are expected to honor backpressure as well. If the outer violates this, a - * {@code MissingBackpressureException} is signaled. If any of the inner {@code Publisher}s violates - * this, it may throw an {@code IllegalStateException} when an inner {@code Publisher} completes.
+ * {@link MissingBackpressureException} is signaled. If any of the inner {@code Publisher}s violates + * this, it may throw an {@link IllegalStateException} when an inner {@code Publisher} completes. *
Scheduler:
*
{@code concatDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type - * @param sources the Iterable sequence of Publishers - * @return the new Publisher with the concatenating behavior + * @param sources the {@code Iterable} sequence of {@code Publisher}s + * @return the new {@code Publisher} with the concatenating behavior */ @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @@ -1570,8 +1569,8 @@ public static Flowable concatDelayError(@NonNull Iterable *
Backpressure:
@@ -1581,8 +1580,8 @@ public static Flowable concatDelayError(@NonNull Iterable * * @param the common element base type - * @param sources the Publisher sequence of Publishers - * @return the new Publisher with the concatenating behavior + * @param sources the {@code Publisher} sequence of {@code Publisher}s + * @return the new {@code Publisher} with the concatenating behavior */ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @@ -1593,8 +1592,8 @@ public static Flowable concatDelayError(@NonNull Publisher *
Backpressure:
@@ -1604,11 +1603,11 @@ public static Flowable concatDelayError(@NonNull Publisher * * @param the common element base type - * @param sources the Publisher sequence of Publishers - * @param prefetch the number of elements to prefetch from the outer Publisher - * @param tillTheEnd if true exceptions from the outer and all inner Publishers are delayed to the end - * if false, exception from the outer Publisher is delayed till the current Publisher terminates - * @return the new Publisher with the concatenating behavior + * @param sources the {@code Publisher} sequence of {@code Publisher}s + * @param prefetch the number of elements to prefetch from the outer {@code Publisher} + * @param tillTheEnd if {@code true}, exceptions from the outer and all inner {@code Publisher}s are delayed to the end + * if {@code false}, exception from the outer {@code Publisher} is delayed till the current {@code Publisher} terminates + * @return the new {@code Publisher} with the concatenating behavior */ @SuppressWarnings({ "unchecked", "rawtypes" }) @CheckReturnValue @@ -1620,22 +1619,22 @@ public static Flowable concatDelayError(@NonNull Publisher * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * emitted source Publishers as they are observed. The operator buffers the values emitted by these - * Publishers and then drains them in order, each one after the previous one completes. + * emitted source {@code Publisher}s as they are observed. The operator buffers the values emitted by these + * {@code Publisher}s and then drains them in order, each one after the previous one completes. *
*
Backpressure:
- *
Backpressure is honored towards the downstream and both the outer and inner Publishers are + *
Backpressure is honored towards the downstream and both the outer and inner {@code Publisher}s are * expected to support backpressure. Violating this assumption, the operator will - * signal {@code MissingBackpressureException}.
+ * signal {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
* @param the value type - * @param sources a sequence of Publishers that need to be eagerly concatenated - * @return the new Publisher instance with the specified concatenation behavior + * @param sources a sequence of {@code Publisher}s that need to be eagerly concatenated + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -1647,25 +1646,25 @@ public static Flowable concatEager(@NonNull Publisher * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * emitted source Publishers as they are observed. The operator buffers the values emitted by these - * Publishers and then drains them in order, each one after the previous one completes. + * emitted source {@code Publisher}s as they are observed. The operator buffers the values emitted by these + * {@code Publisher}s and then drains them in order, each one after the previous one completes. *
*
Backpressure:
- *
Backpressure is honored towards the downstream and both the outer and inner Publishers are + *
Backpressure is honored towards the downstream and both the outer and inner {@code Publisher}s are * expected to support backpressure. Violating this assumption, the operator will - * signal {@code MissingBackpressureException}.
+ * signal {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
* @param the value type - * @param sources a sequence of Publishers that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrently running inner Publishers; {@link Integer#MAX_VALUE} - * is interpreted as all inner Publishers can be active at the same time - * @param prefetch the number of elements to prefetch from each inner Publisher source - * @return the new Publisher instance with the specified concatenation behavior + * @param sources a sequence of {@code Publisher}s that need to be eagerly concatenated + * @param maxConcurrency the maximum number of concurrently running inner {@code Publisher}s; {@link Integer#MAX_VALUE} + * is interpreted as all inner {@code Publisher}s can be active at the same time + * @param prefetch the number of elements to prefetch from each inner {@code Publisher} source + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -1681,22 +1680,22 @@ public static Flowable concatEager(@NonNull Publisher * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them * in order, each one after the previous one completes. *
*
Backpressure:
- *
Backpressure is honored towards the downstream and the inner Publishers are + *
Backpressure is honored towards the downstream and the inner {@code Publisher}s are * expected to support backpressure. Violating this assumption, the operator will - * signal {@code MissingBackpressureException}.
+ * signal {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
* @param the value type - * @param sources a sequence of Publishers that need to be eagerly concatenated - * @return the new Publisher instance with the specified concatenation behavior + * @param sources a sequence of {@code Publisher}s that need to be eagerly concatenated + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -1708,25 +1707,25 @@ public static Flowable concatEager(@NonNull Iterable * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them * in order, each one after the previous one completes. *
*
Backpressure:
- *
Backpressure is honored towards the downstream and both the outer and inner Publishers are + *
Backpressure is honored towards the downstream and both the outer and inner {@code Publisher}s are * expected to support backpressure. Violating this assumption, the operator will - * signal {@code MissingBackpressureException}.
+ * signal {@link MissingBackpressureException}. *
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
* @param the value type - * @param sources a sequence of Publishers that need to be eagerly concatenated - * @param maxConcurrency the maximum number of concurrently running inner Publishers; {@link Integer#MAX_VALUE} - * is interpreted as all inner Publishers can be active at the same time - * @param prefetch the number of elements to prefetch from each inner Publisher source - * @return the new Publisher instance with the specified concatenation behavior + * @param sources a sequence of {@code Publisher}s that need to be eagerly concatenated + * @param maxConcurrency the maximum number of concurrently running inner {@code Publisher}s; {@link Integer#MAX_VALUE} + * is interpreted as all inner {@code Publisher}s can be active at the same time + * @param prefetch the number of elements to prefetch from each inner {@code Publisher} source + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -1742,7 +1741,7 @@ public static Flowable concatEager(@NonNull Iterable * Example: @@ -1773,10 +1772,11 @@ public static Flowable concatEager(@NonNull Iterable - * You should call the FlowableEmitter onNext, onError and onComplete methods in a serialized fashion. The + * You should call the {@link FlowableEmitter#onNext(Object)}, {@link FlowableEmitter#onError(Throwable)} + * and {@link FlowableEmitter#onComplete()} methods in a serialized fashion. The * rest of its methods are thread-safe. *
*
Backpressure:
@@ -1786,9 +1786,9 @@ public static Flowable concatEager(@NonNull Iterable * * @param the element type - * @param source the emitter that is called when a Subscriber subscribes to the returned {@code Flowable} - * @param mode the backpressure mode to apply if the downstream Subscriber doesn't request (fast) enough - * @return the new Flowable instance + * @param source the emitter that is called when a {@code Subscriber} subscribes to the returned {@code Flowable} + * @param mode the backpressure mode to apply if the downstream {@code Subscriber} doesn't request (fast) enough + * @return the new {@code Flowable} instance * @see FlowableOnSubscribe * @see BackpressureStrategy * @see Cancellable @@ -1804,14 +1804,14 @@ public static Flowable create(@NonNull FlowableOnSubscribe source, @No } /** - * Returns a Flowable that calls a Publisher factory to create a Publisher for each new Subscriber - * that subscribes. That is, for each subscriber, the actual Publisher that subscriber observes is + * Returns a {@code Flowable} that calls a {@link Publisher} factory to create a {@code Publisher} for each new {@link Subscriber} + * that subscribes. That is, for each subscriber, the actual {@code Publisher} that subscriber observes is * determined by the factory function. *

* *

- * The defer Subscriber allows you to defer or delay emitting items from a Publisher until such time as a - * Subscriber subscribes to the Publisher. This allows a {@link Subscriber} to easily obtain updates or a + * The defer {@code Subscriber} allows you to defer or delay emitting items from a {@code Publisher} until such time as a + * {@code Subscriber} subscribes to the {@code Publisher}. This allows a {@code Subscriber} to easily obtain updates or a * refreshed version of the sequence. *

*
Backpressure:
@@ -1822,12 +1822,12 @@ public static Flowable create(@NonNull FlowableOnSubscribe source, @No *
* * @param supplier - * the Publisher factory function to invoke for each {@link Subscriber} that subscribes to the - * resulting Publisher + * the {@code Publisher} factory function to invoke for each {@code Subscriber} that subscribes to the + * resulting {@code Publisher} * @param - * the type of the items emitted by the Publisher - * @return a Flowable whose {@link Subscriber}s' subscriptions trigger an invocation of the given - * Publisher factory function + * the type of the items emitted by the {@code Publisher} + * @return a {@code Flowable} whose {@code Subscriber}s' subscriptions trigger an invocation of the given + * {@code Publisher} factory function * @see ReactiveX operators documentation: Defer */ @CheckReturnValue @@ -1840,7 +1840,7 @@ public static Flowable defer(@NonNull Supplier * @@ -1852,9 +1852,9 @@ public static Flowable defer(@NonNull Supplier * * @param - * the type of the items (ostensibly) emitted by the Publisher - * @return a Flowable that emits no items to the {@link Subscriber} but immediately invokes the - * {@link Subscriber}'s {@link Subscriber#onComplete() onComplete} method + * the type of the items (ostensibly) emitted by the {@link Publisher} + * @return a {@code Flowable} that emits no items to the {@code Subscriber} but immediately invokes the + * {@code Subscriber}'s {@link Subscriber#onComplete() onComplete} method * @see ReactiveX operators documentation: Empty */ @CheckReturnValue @@ -1867,8 +1867,8 @@ public static Flowable empty() { } /** - * Returns a Flowable that invokes a {@link Subscriber}'s {@link Subscriber#onError onError} method when the - * Subscriber subscribes to it. + * Returns a {@code Flowable} that invokes a {@link Subscriber}'s {@link Subscriber#onError onError} method when the + * {@code Subscriber} subscribes to it. *

* *

@@ -1879,11 +1879,11 @@ public static Flowable empty() { *
* * @param supplier - * a Supplier factory to return a Throwable for each individual Subscriber + * a {@link Supplier} factory to return a {@link Throwable} for each individual {@code Subscriber} * @param - * the type of the items (ostensibly) emitted by the Publisher - * @return a Flowable that invokes the {@link Subscriber}'s {@link Subscriber#onError onError} method when - * the Subscriber subscribes to it + * the type of the items (ostensibly) emitted by the {@link Publisher} + * @return a {@code Flowable} that invokes the {@code Subscriber}'s {@link Subscriber#onError onError} method when + * the {@code Subscriber} subscribes to it * @see ReactiveX operators documentation: Throw */ @CheckReturnValue @@ -1896,8 +1896,8 @@ public static Flowable error(@NonNull Supplier suppl } /** - * Returns a Flowable that invokes a {@link Subscriber}'s {@link Subscriber#onError onError} method when the - * Subscriber subscribes to it. + * Returns a {@code Flowable} that invokes a {@link Subscriber}'s {@link Subscriber#onError onError} method when the + * {@code Subscriber} subscribes to it. *

* *

@@ -1908,11 +1908,11 @@ public static Flowable error(@NonNull Supplier suppl *
* * @param throwable - * the particular Throwable to pass to {@link Subscriber#onError onError} + * the particular {@link Throwable} to pass to {@link Subscriber#onError onError} * @param - * the type of the items (ostensibly) emitted by the Publisher - * @return a Flowable that invokes the {@link Subscriber}'s {@link Subscriber#onError onError} method when - * the Subscriber subscribes to it + * the type of the items (ostensibly) emitted by the {@link Publisher} + * @return a {@code Flowable} that invokes the {@code Subscriber}'s {@link Subscriber#onError onError} method when + * the {@code Subscriber} subscribes to it * @see ReactiveX operators documentation: Throw */ @CheckReturnValue @@ -1925,7 +1925,7 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Converts an Array into a Publisher that emits the items in the Array. + * Converts an Array into a {@link Publisher} that emits the items in the Array. *

* *

@@ -1939,8 +1939,8 @@ public static Flowable error(@NonNull Throwable throwable) { * @param items * the array of elements * @param - * the type of items in the Array and the type of items to be emitted by the resulting Publisher - * @return a Flowable that emits each item in the source Array + * the type of items in the Array and the type of items to be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits each item in the source Array * @see ReactiveX operators documentation: From */ @CheckReturnValue @@ -1960,13 +1960,13 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Returns a Flowable that, when a Subscriber subscribes to it, invokes a function you specify and then + * Returns a {@code Flowable} that, when a {@link Subscriber} subscribes to it, invokes a function you specify and then * emits the value returned from that function. *

* *

- * This allows you to defer the execution of the function you specify until a Subscriber subscribes to the - * Publisher. That is to say, it makes the function "lazy." + * This allows you to defer the execution of the function you specify until a {@code Subscriber} subscribes to the + * {@link Publisher}. That is to say, it makes the function "lazy." *

*
Backpressure:
*
The operator honors backpressure from downstream.
@@ -1983,10 +1983,10 @@ public static Flowable error(@NonNull Throwable throwable) { * * @param supplier * a function, the execution of which should be deferred; {@code fromCallable} will invoke this - * function only when a Subscriber subscribes to the Publisher that {@code fromCallable} returns + * function only when a {@code Subscriber} subscribes to the {@code Publisher} that {@code fromCallable} returns * @param - * the type of the item emitted by the Publisher - * @return a Flowable whose {@link Subscriber}s' subscriptions trigger an invocation of the given function + * the type of the item emitted by the {@code Publisher} + * @return a {@code Flowable} whose {@code Subscriber}s' subscriptions trigger an invocation of the given function * @see #defer(Supplier) * @see #fromSupplier(Supplier) * @since 2.0 @@ -2001,21 +2001,21 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Converts a {@link Future} into a Publisher. + * Converts a {@link Future} into a {@link Publisher}. *

* *

- * You can convert any object that supports the {@link Future} interface into a Publisher that emits the + * You can convert any object that supports the {@code Future} interface into a {@code Publisher} that emits the * return value of the {@link Future#get} method of that object by passing the object into the {@code from} * method. *

- * Important note: This Publisher is blocking on the thread it gets subscribed on; you cannot cancel it. + * Important note: This {@code Publisher} is blocking on the thread it gets subscribed on; you cannot cancel it. *

* Also note that this operator will consume a {@link CompletionStage}-based {@code Future} subclass (such as * {@link CompletableFuture}) in a blocking manner as well. Use the {@link #fromCompletionStage(CompletionStage)} * operator to convert and consume such sources in a non-blocking fashion instead. *

- * Unlike 1.x, canceling the Flowable won't cancel the future. If necessary, one can use composition to achieve the + * Unlike 1.x, canceling the {@code Flowable} won't cancel the future. If necessary, one can use composition to achieve the * cancellation effect: {@code futurePublisher.doOnCancel(() -> future.cancel(true));}. *

*
Backpressure:
@@ -2025,11 +2025,11 @@ public static Flowable error(@NonNull Throwable throwable) { *
* * @param future - * the source {@link Future} + * the source {@code Future} * @param - * the type of object that the {@link Future} returns, and also the type of item to be emitted by - * the resulting Publisher - * @return a Flowable that emits the item from the source {@link Future} + * the type of object that the {@code Future} returns, and also the type of item to be emitted by + * the resulting {@code Publisher} + * @return a {@code Flowable} that emits the item from the source {@code Future} * @see ReactiveX operators documentation: From * @see #fromCompletionStage(CompletionStage) */ @@ -2043,18 +2043,18 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Converts a {@link Future} into a Publisher, with a timeout on the Future. + * Converts a {@link Future} into a {@link Publisher}, with a timeout on the {@code Future}. *

* *

- * You can convert any object that supports the {@link Future} interface into a Publisher that emits the + * You can convert any object that supports the {@code Future} interface into a {@code Publisher} that emits the * return value of the {@link Future#get} method of that object by passing the object into the {@code fromFuture} * method. *

- * Unlike 1.x, canceling the Flowable won't cancel the future. If necessary, one can use composition to achieve the + * Unlike 1.x, canceling the {@code Flowable} won't cancel the future. If necessary, one can use composition to achieve the * cancellation effect: {@code futurePublisher.doOnCancel(() -> future.cancel(true));}. *

- * Important note: This Publisher is blocking on the thread it gets subscribed on; you cannot cancel it. + * Important note: This {@code Publisher} is blocking on the thread it gets subscribed on; you cannot cancel it. *

* Also note that this operator will consume a {@link CompletionStage}-based {@code Future} subclass (such as * {@link CompletableFuture}) in a blocking manner as well. Use the {@link #fromCompletionStage(CompletionStage)} @@ -2067,15 +2067,15 @@ public static Flowable error(@NonNull Throwable throwable) { *

* * @param future - * the source {@link Future} + * the source {@code Future} * @param timeout * the maximum time to wait before calling {@code get} * @param unit * the {@link TimeUnit} of the {@code timeout} argument * @param - * the type of object that the {@link Future} returns, and also the type of item to be emitted by - * the resulting Publisher - * @return a Flowable that emits the item from the source {@link Future} + * the type of object that the {@code Future} returns, and also the type of item to be emitted by + * the resulting {@code Publisher} + * @return a {@code Flowable} that emits the item from the source {@code Future} * @see ReactiveX operators documentation: From * @see #fromCompletionStage(CompletionStage) */ @@ -2090,18 +2090,18 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Converts a {@link Future} into a Publisher, with a timeout on the Future. + * Converts a {@link Future} into a {@link Publisher}, with a timeout on the {@code Future}. *

* *

- * You can convert any object that supports the {@link Future} interface into a Publisher that emits the + * You can convert any object that supports the {@code Future} interface into a {@code Publisher} that emits the * return value of the {@link Future#get} method of that object by passing the object into the {@code from} * method. *

- * Unlike 1.x, canceling the Flowable won't cancel the future. If necessary, one can use composition to achieve the + * Unlike 1.x, canceling the {@code Flowable} won't cancel the future. If necessary, one can use composition to achieve the * cancellation effect: {@code futurePublisher.doOnCancel(() -> future.cancel(true));}. *

- * Important note: This Publisher is blocking; you cannot cancel it. + * Important note: This {@code Publisher} is blocking; you cannot cancel it. *

* Also note that this operator will consume a {@link CompletionStage}-based {@code Future} subclass (such as * {@link CompletableFuture}) in a blocking manner as well. Use the {@link #fromCompletionStage(CompletionStage)} @@ -2114,18 +2114,18 @@ public static Flowable error(@NonNull Throwable throwable) { *

* * @param future - * the source {@link Future} + * the source {@code Future} * @param timeout * the maximum time to wait before calling {@code get} * @param unit * the {@link TimeUnit} of the {@code timeout} argument * @param scheduler - * the {@link Scheduler} to wait for the Future on. Use a Scheduler such as - * {@link Schedulers#io()} that can block and wait on the Future + * the {@code Scheduler} to wait for the {@code Future} on. Use a {@code Scheduler} such as + * {@link Schedulers#io()} that can block and wait on the {@code Future} * @param - * the type of object that the {@link Future} returns, and also the type of item to be emitted by - * the resulting Publisher - * @return a Flowable that emits the item from the source {@link Future} + * the type of object that the {@code Future} returns, and also the type of item to be emitted by + * the resulting {@code Publisher} + * @return a {@code Flowable} that emits the item from the source {@code Future} * @see ReactiveX operators documentation: From * @see #fromCompletionStage(CompletionStage) */ @@ -2140,32 +2140,32 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Converts a {@link Future}, operating on a specified {@link Scheduler}, into a Publisher. + * Converts a {@link Future}, operating on a specified {@link Scheduler}, into a {@link Publisher}. *

* *

- * You can convert any object that supports the {@link Future} interface into a Publisher that emits the + * You can convert any object that supports the {@code Future} interface into a {@code Publisher} that emits the * return value of the {@link Future#get} method of that object by passing the object into the {@code from} * method. *

- * Unlike 1.x, canceling the Flowable won't cancel the future. If necessary, one can use composition to achieve the + * Unlike 1.x, canceling the {@code Flowable} won't cancel the future. If necessary, one can use composition to achieve the * cancellation effect: {@code futurePublisher.doOnCancel(() -> future.cancel(true));}. *

*
Backpressure:
*
The operator honors backpressure from downstream.
*
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param future - * the source {@link Future} + * the source {@code Future} * @param scheduler - * the {@link Scheduler} to wait for the Future on. Use a Scheduler such as - * {@link Schedulers#io()} that can block and wait on the Future + * the {@code Scheduler} to wait for the {@code Future} on. Use a {@code Scheduler} such as + * {@link Schedulers#io()} that can block and wait on the {@code Future} * @param - * the type of object that the {@link Future} returns, and also the type of item to be emitted by - * the resulting Publisher - * @return a Flowable that emits the item from the source {@link Future} + * the type of object that the {@code Future} returns, and also the type of item to be emitted by + * the resulting {@code Publisher} + * @return a {@code Flowable} that emits the item from the source {@code Future} * @see ReactiveX operators documentation: From */ @SuppressWarnings({ "unchecked" }) @@ -2179,7 +2179,7 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Converts an {@link Iterable} sequence into a Publisher that emits the items in the sequence. + * Converts an {@link Iterable} sequence into a {@link Publisher} that emits the items in the sequence. *

* *

@@ -2191,11 +2191,11 @@ public static Flowable error(@NonNull Throwable throwable) { *
* * @param source - * the source {@link Iterable} sequence + * the source {@code Iterable} sequence * @param - * the type of items in the {@link Iterable} sequence and the type of items to be emitted by the - * resulting Publisher - * @return a Flowable that emits each item in the source {@link Iterable} sequence + * the type of items in the {@code Iterable} sequence and the type of items to be emitted by the + * resulting {@code Publisher} + * @return a {@code Flowable} that emits each item in the source {@code Iterable} sequence * @see ReactiveX operators documentation: From * @see #fromStream(Stream) */ @@ -2209,17 +2209,17 @@ public static Flowable error(@NonNull Throwable throwable) { } /** - * Converts an arbitrary Reactive Streams Publisher into a Flowable if not already a - * Flowable. + * Converts an arbitrary Reactive Streams {@link Publisher} into a {@code Flowable} if not already a + * {@code Flowable}. *

- * The {@link Publisher} must follow the + * The {@code Publisher} must follow the * Reactive-Streams specification. * Violating the specification may result in undefined behavior. *

* If possible, use {@link #create(FlowableOnSubscribe, BackpressureStrategy)} to create a * source-like {@code Flowable} instead. *

- * Note that even though {@link Publisher} appears to be a functional interface, it + * Note that even though {@code Publisher} appears to be a functional interface, it * is not recommended to implement it through a lambda as the specification requires * state management that is not achievable with a stateless lambda. *

@@ -2230,9 +2230,9 @@ public static Flowable error(@NonNull Throwable throwable) { *
{@code fromPublisher} does not operate by default on a particular {@link Scheduler}.
*
* @param the value type of the flow - * @param source the Publisher to convert - * @return the new Flowable instance - * @throws NullPointerException if the {@code source} {@code Publisher} is null + * @param source the {@code Publisher} to convert + * @return the new {@code Flowable} instance + * @throws NullPointerException if the {@code source} {@code Publisher} is {@code null} * @see #create(FlowableOnSubscribe, BackpressureStrategy) */ @CheckReturnValue @@ -2250,13 +2250,13 @@ public static Flowable fromPublisher(@NonNull Publisher sour } /** - * Returns a Flowable that, when a Subscriber subscribes to it, invokes a supplier function you specify and then + * Returns a {@code Flowable} that, when a {@link Subscriber} subscribes to it, invokes a supplier function you specify and then * emits the value returned from that function. *

* *

- * This allows you to defer the execution of the function you specify until a Subscriber subscribes to the - * Publisher. That is to say, it makes the function "lazy." + * This allows you to defer the execution of the function you specify until a {@code Subscriber} subscribes to the + * {@link Publisher}. That is to say, it makes the function "lazy." *

*
Backpressure:
*
The operator honors backpressure from downstream.
@@ -2273,10 +2273,10 @@ public static Flowable fromPublisher(@NonNull Publisher sour * * @param supplier * a function, the execution of which should be deferred; {@code fromSupplier} will invoke this - * function only when a Subscriber subscribes to the Publisher that {@code fromSupplier} returns + * function only when a {@code Subscriber} subscribes to the {@code Publisher} that {@code fromSupplier} returns * @param - * the type of the item emitted by the Publisher - * @return a Flowable whose {@link Subscriber}s' subscriptions trigger an invocation of the given function + * the type of the item emitted by the {@code Publisher} + * @return a {@code Flowable} whose {@code Subscriber}s' subscriptions trigger an invocation of the given function * @see #defer(Supplier) * @see #fromCallable(Callable) * @since 3.0.0 @@ -2305,11 +2305,11 @@ public static Flowable fromPublisher(@NonNull Publisher sour *
* * @param the generated value type - * @param generator the Consumer called whenever a particular downstream Subscriber has + * @param generator the {@link Consumer} called whenever a particular downstream {@link Subscriber} has * requested a value. The callback then should call {@code onNext}, {@code onError} or * {@code onComplete} to signal a value or a terminal event. Signaling multiple {@code onNext} - * in a call will make the operator signal {@code IllegalStateException}. - * @return the new Flowable instance + * in a call will make the operator signal {@link IllegalStateException}. + * @return the new {@code Flowable} instance */ @CheckReturnValue @NonNull @@ -2336,14 +2336,14 @@ public static Flowable generate(@NonNull Consumer<@NonNull Emitter> ge *
{@code generate} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the type of the per-Subscriber state + * @param the type of the per-{@link Subscriber} state * @param the generated value type - * @param initialState the Supplier to generate the initial state for each Subscriber - * @param generator the Consumer called with the current state whenever a particular downstream Subscriber has + * @param initialState the {@link Supplier} to generate the initial state for each {@code Subscriber} + * @param generator the {@link Consumer} called with the current state whenever a particular downstream {@code Subscriber} has * requested a value. The callback then should call {@code onNext}, {@code onError} or * {@code onComplete} to signal a value or a terminal event. Signaling multiple {@code onNext} - * in a call will make the operator signal {@code IllegalStateException}. - * @return the new Flowable instance + * in a call will make the operator signal {@link IllegalStateException}. + * @return the new {@code Flowable} instance */ @CheckReturnValue @NonNull @@ -2369,16 +2369,16 @@ public static Flowable generate(@NonNull Supplier initialState, @No *
{@code generate} does not operate by default on a particular {@link Scheduler}.
* * - * @param the type of the per-Subscriber state + * @param the type of the per-{@link Subscriber} state * @param the generated value type - * @param initialState the Supplier to generate the initial state for each Subscriber - * @param generator the Consumer called with the current state whenever a particular downstream Subscriber has + * @param initialState the {@link Supplier} to generate the initial state for each {@code Subscriber} + * @param generator the {@link Consumer} called with the current state whenever a particular downstream {@code Subscriber} has * requested a value. The callback then should call {@code onNext}, {@code onError} or * {@code onComplete} to signal a value or a terminal event. Signaling multiple {@code onNext} - * in a call will make the operator signal {@code IllegalStateException}. - * @param disposeState the Consumer that is called with the current state when the generator + * in a call will make the operator signal {@link IllegalStateException}. + * @param disposeState the {@code Consumer} that is called with the current state when the generator * terminates the sequence or it gets canceled - * @return the new Flowable instance + * @return the new {@code Flowable} instance */ @CheckReturnValue @NonNull @@ -2404,15 +2404,15 @@ public static Flowable generate(@NonNull Supplier initialState, @No *
{@code generate} does not operate by default on a particular {@link Scheduler}.
* * - * @param the type of the per-Subscriber state + * @param the type of the per-{@link Subscriber} state * @param the generated value type - * @param initialState the Supplier to generate the initial state for each Subscriber - * @param generator the Function called with the current state whenever a particular downstream Subscriber has + * @param initialState the {@link Supplier} to generate the initial state for each {@code Subscriber} + * @param generator the {@link Function} called with the current state whenever a particular downstream {@code Subscriber} has * requested a value. The callback then should call {@code onNext}, {@code onError} or * {@code onComplete} to signal a value or a terminal event and should return a (new) state for * the next invocation. Signaling multiple {@code onNext} - * in a call will make the operator signal {@code IllegalStateException}. - * @return the new Flowable instance + * in a call will make the operator signal {@link IllegalStateException}. + * @return the new {@code Flowable} instance */ @CheckReturnValue @BackpressureSupport(BackpressureKind.FULL) @@ -2436,17 +2436,17 @@ public static Flowable generate(@NonNull Supplier initialState, @No *
{@code generate} does not operate by default on a particular {@link Scheduler}.
* * - * @param the type of the per-Subscriber state + * @param the type of the per-{@link Subscriber} state * @param the generated value type - * @param initialState the Supplier to generate the initial state for each Subscriber - * @param generator the Function called with the current state whenever a particular downstream Subscriber has + * @param initialState the {@link Supplier} to generate the initial state for each {@code Subscriber} + * @param generator the {@link Function} called with the current state whenever a particular downstream {@code Subscriber} has * requested a value. The callback then should call {@code onNext}, {@code onError} or * {@code onComplete} to signal a value or a terminal event and should return a (new) state for * the next invocation. Signaling multiple {@code onNext} - * in a call will make the operator signal {@code IllegalStateException}. - * @param disposeState the Consumer that is called with the current state when the generator + * in a call will make the operator signal {@link IllegalStateException}. + * @param disposeState the {@link Consumer} that is called with the current state when the generator * terminates the sequence or it gets canceled - * @return the new Flowable instance + * @return the new {@code Flowable} instance */ @CheckReturnValue @NonNull @@ -2460,15 +2460,15 @@ public static Flowable generate(@NonNull Supplier initialState, @No } /** - * Returns a Flowable that emits a {@code 0L} after the {@code initialDelay} and ever-increasing numbers + * Returns a {@code Flowable} that emits a {@code 0L} after the {@code initialDelay} and ever-increasing numbers * after each {@code period} of time thereafter. *

* *

*
Backpressure:
*
The operator generates values based on time and ignores downstream backpressure which - * may lead to {@code MissingBackpressureException} at some point in the chain. - * Consumers should consider applying one of the {@code onBackpressureXXX} operators as well.
+ * may lead to {@link MissingBackpressureException} at some point in the chain. + * Downstream consumers should consider applying one of the {@code onBackpressureXXX} operators as well. *
Scheduler:
*
{@code interval} operates by default on the {@code computation} {@link Scheduler}.
*
@@ -2479,7 +2479,7 @@ public static Flowable generate(@NonNull Supplier initialState, @No * the period of time between emissions of the subsequent numbers * @param unit * the time unit for both {@code initialDelay} and {@code period} - * @return a Flowable that emits a 0L after the {@code initialDelay} and ever-increasing numbers after + * @return a {@code Flowable} that emits a 0L after the {@code initialDelay} and ever-increasing numbers after * each {@code period} of time thereafter * @see ReactiveX operators documentation: Interval * @since 1.0.12 @@ -2493,17 +2493,17 @@ public static Flowable interval(long initialDelay, long period, @NonNull T } /** - * Returns a Flowable that emits a {@code 0L} after the {@code initialDelay} and ever-increasing numbers + * Returns a {@code Flowable} that emits a {@code 0L} after the {@code initialDelay} and ever-increasing numbers * after each {@code period} of time thereafter, on a specified {@link Scheduler}. *

* *

*
Backpressure:
*
The operator generates values based on time and ignores downstream backpressure which - * may lead to {@code MissingBackpressureException} at some point in the chain. - * Consumers should consider applying one of the {@code onBackpressureXXX} operators as well.
+ * may lead to {@link MissingBackpressureException} at some point in the chain. + * Downstream consumers should consider applying one of the {@code onBackpressureXXX} operators as well. *
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param initialDelay @@ -2513,9 +2513,9 @@ public static Flowable interval(long initialDelay, long period, @NonNull T * @param unit * the time unit for both {@code initialDelay} and {@code period} * @param scheduler - * the Scheduler on which the waiting happens and items are emitted - * @return a Flowable that emits a 0L after the {@code initialDelay} and ever-increasing numbers after - * each {@code period} of time thereafter, while running on the given Scheduler + * the {@code Scheduler} on which the waiting happens and items are emitted + * @return a {@code Flowable} that emits a 0L after the {@code initialDelay} and ever-increasing numbers after + * each {@code period} of time thereafter, while running on the given {@code Scheduler} * @see ReactiveX operators documentation: Interval * @since 1.0.12 */ @@ -2530,12 +2530,12 @@ public static Flowable interval(long initialDelay, long period, @NonNull T } /** - * Returns a Flowable that emits a sequential number every specified interval of time. + * Returns a {@code Flowable} that emits a sequential number every specified interval of time. *

* *

*
Backpressure:
- *
The operator signals a {@code MissingBackpressureException} if the downstream + *
The operator signals a {@link MissingBackpressureException} if the downstream * is not ready to receive the next value.
*
Scheduler:
*
{@code interval} operates by default on the {@code computation} {@link Scheduler}.
@@ -2545,7 +2545,7 @@ public static Flowable interval(long initialDelay, long period, @NonNull T * the period size in time units (see below) * @param unit * time units to use for the interval size - * @return a Flowable that emits a sequential number each time interval + * @return a {@code Flowable} that emits a sequential number each time interval * @see ReactiveX operators documentation: Interval */ @CheckReturnValue @@ -2557,17 +2557,17 @@ public static Flowable interval(long period, @NonNull TimeUnit unit) { } /** - * Returns a Flowable that emits a sequential number every specified interval of time, on a - * specified Scheduler. + * Returns a {@code Flowable} that emits a sequential number every specified interval of time, on a + * specified {@link Scheduler}. *

* *

*
Backpressure:
*
The operator generates values based on time and ignores downstream backpressure which - * may lead to {@code MissingBackpressureException} at some point in the chain. - * Consumers should consider applying one of the {@code onBackpressureXXX} operators as well.
+ * may lead to {@link MissingBackpressureException} at some point in the chain. + * Downstream consumers should consider applying one of the {@code onBackpressureXXX} operators as well. *
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param period @@ -2575,8 +2575,8 @@ public static Flowable interval(long period, @NonNull TimeUnit unit) { * @param unit * time units to use for the interval size * @param scheduler - * the Scheduler to use for scheduling the items - * @return a Flowable that emits a sequential number each time interval + * the {@code Scheduler} to use for scheduling the items + * @return a {@code Flowable} that emits a sequential number each time interval * @see ReactiveX operators documentation: Interval */ @CheckReturnValue @@ -2590,19 +2590,19 @@ public static Flowable interval(long period, @NonNull TimeUnit unit, @NonN /** * Signals a range of long values, the first after some initial delay and the rest periodically after. *

- * The sequence completes immediately after the last value (start + count - 1) has been reached. + * The sequence completes immediately after the last value {@code (start + count - 1)} has been reached. *

*
Backpressure:
- *
The operator signals a {@code MissingBackpressureException} if the downstream can't keep up.
+ *
The operator signals a {@link MissingBackpressureException} if the downstream can't keep up.
*
Scheduler:
*
{@code intervalRange} by default operates on the {@link Schedulers#computation() computation} {@link Scheduler}.
*
* @param start that start value of the range - * @param count the number of values to emit in total, if zero, the operator emits an onComplete after the initial delay. + * @param count the number of values to emit in total, if zero, the operator emits an {@code onComplete} after the initial delay. * @param initialDelay the initial delay before signaling the first value (the start) * @param period the period between subsequent values - * @param unit the unit of measure of the initialDelay and period amounts - * @return the new Flowable instance + * @param unit the unit of measure of the {@code initialDelay} and {@code period} amounts + * @return the new {@code Flowable} instance */ @CheckReturnValue @BackpressureSupport(BackpressureKind.ERROR) @@ -2618,17 +2618,17 @@ public static Flowable intervalRange(long start, long count, long initialD * The sequence completes immediately after the last value (start + count - 1) has been reached. *
*
Backpressure:
- *
The operator signals a {@code MissingBackpressureException} if the downstream can't keep up.
+ *
The operator signals a {@link MissingBackpressureException} if the downstream can't keep up.
*
Scheduler:
*
you provide the {@link Scheduler}.
*
* @param start that start value of the range - * @param count the number of values to emit in total, if zero, the operator emits an onComplete after the initial delay. + * @param count the number of values to emit in total, if zero, the operator emits an {@code onComplete} after the initial delay. * @param initialDelay the initial delay before signaling the first value (the start) * @param period the period between subsequent values - * @param unit the unit of measure of the initialDelay and period amounts - * @param scheduler the target scheduler where the values and terminal signals will be emitted - * @return the new Flowable instance + * @param unit the unit of measure of the {@code initialDelay} and {@code period} amounts + * @param scheduler the target {@code Scheduler} where the values and terminal signals will be emitted + * @return the new {@code Flowable} instance */ @CheckReturnValue @NonNull @@ -2653,12 +2653,12 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Returns a Flowable that signals the given (constant reference) item and then completes. + * Returns a {@code Flowable} that signals the given (constant reference) item and then completes. *

* *

* Note that the item is taken and re-emitted as is and not computed by any means by {@code just}. Use {@link #fromCallable(Callable)} - * to generate a single item on demand (when {@code Subscriber}s subscribe to it). + * to generate a single item on demand (when {@link Subscriber}s subscribe to it). *

* See the multi-parameter overloads of {@code just} to emit more than one (constant reference) items one after the other. * Use {@link #fromArray(Object...)} to emit an arbitrary number of items that are known upfront. @@ -2675,7 +2675,7 @@ public static Flowable intervalRange(long start, long count, long initialD * the item to emit * @param * the type of that item - * @return a Flowable that emits {@code value} as a single item and then completes + * @return a {@code Flowable} that emits {@code value} as a single item and then completes * @see ReactiveX operators documentation: Just * @see #just(Object, Object) * @see #fromCallable(Callable) @@ -2692,7 +2692,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts two items into a Publisher that emits those items. + * Converts two items into a {@link Publisher} that emits those items. *

* *

@@ -2708,7 +2708,7 @@ public static Flowable intervalRange(long start, long count, long initialD * second item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -2723,7 +2723,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts three items into a Publisher that emits those items. + * Converts three items into a {@link Publisher} that emits those items. *

* *

@@ -2741,7 +2741,7 @@ public static Flowable intervalRange(long start, long count, long initialD * third item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -2757,7 +2757,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts four items into a Publisher that emits those items. + * Converts four items into a {@link Publisher} that emits those items. *

* *

@@ -2777,7 +2777,7 @@ public static Flowable intervalRange(long start, long count, long initialD * fourth item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -2794,7 +2794,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts five items into a Publisher that emits those items. + * Converts five items into a {@link Publisher} that emits those items. *

* *

@@ -2816,7 +2816,7 @@ public static Flowable intervalRange(long start, long count, long initialD * fifth item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -2834,7 +2834,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts six items into a Publisher that emits those items. + * Converts six items into a {@link Publisher} that emits those items. *

* *

@@ -2858,7 +2858,7 @@ public static Flowable intervalRange(long start, long count, long initialD * sixth item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -2877,7 +2877,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts seven items into a Publisher that emits those items. + * Converts seven items into a {@link Publisher} that emits those items. *

* *

@@ -2903,7 +2903,7 @@ public static Flowable intervalRange(long start, long count, long initialD * seventh item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -2923,7 +2923,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts eight items into a Publisher that emits those items. + * Converts eight items into a {@link Publisher} that emits those items. *

* *

@@ -2951,7 +2951,7 @@ public static Flowable intervalRange(long start, long count, long initialD * eighth item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -2972,7 +2972,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts nine items into a Publisher that emits those items. + * Converts nine items into a {@link Publisher} that emits those items. *

* *

@@ -3002,7 +3002,7 @@ public static Flowable intervalRange(long start, long count, long initialD * ninth item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -3024,7 +3024,7 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Converts ten items into a Publisher that emits those items. + * Converts ten items into a {@link Publisher} that emits those items. *

* *

@@ -3056,7 +3056,7 @@ public static Flowable intervalRange(long start, long count, long initialD * tenth item * @param * the type of these items - * @return a Flowable that emits each item + * @return a {@code Flowable} that emits each item * @see ReactiveX operators documentation: Just */ @CheckReturnValue @@ -3079,27 +3079,27 @@ public static Flowable intervalRange(long start, long count, long initialD } /** - * Flattens an Iterable of Publishers into one Publisher, without any transformation, while limiting the - * number of concurrent subscriptions to these Publishers. + * Flattens an {@link Iterable} of {@link Publisher}s into one {@code Publisher}, without any transformation, while limiting the + * number of concurrent subscriptions to these {@code Publisher}s. *

* *

- * You can combine the items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine the items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Iterable, int, int)} to merge sources and terminate only when all source {@code Publisher}s @@ -3109,13 +3109,13 @@ public static Flowable intervalRange(long start, long count, long initialD * * @param the common element base type * @param sources - * the Iterable of Publishers + * the {@code Iterable} of {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param bufferSize - * the number of items to prefetch from each inner Publisher - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the number of items to prefetch from each inner {@code Publisher} + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s in the {@code Iterable} * @throws IllegalArgumentException * if {@code maxConcurrency} is less than or equal to 0 * @see ReactiveX operators documentation: Merge @@ -3131,27 +3131,27 @@ public static Flowable merge(@NonNull Iterable * *

- * You can combine the items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine the items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code mergeArray} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeArrayDelayError(int, int, Publisher[])} to merge sources and terminate only when all source {@code Publisher}s @@ -3161,13 +3161,13 @@ public static Flowable merge(@NonNull Iterable the common element base type * @param sources - * the array of Publishers + * the array of {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param bufferSize - * the number of items to prefetch from each inner Publisher - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the number of items to prefetch from each inner {@code Publisher} + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s * @throws IllegalArgumentException * if {@code maxConcurrency} is less than or equal to 0 * @see ReactiveX operators documentation: Merge @@ -3184,26 +3184,26 @@ public static Flowable mergeArray(int maxConcurrency, int bufferSize, @No } /** - * Flattens an Iterable of Publishers into one Publisher, without any transformation. + * Flattens an {@link Iterable} of {@link Publisher}s into one {@code Publisher}, without any transformation. *

* *

- * You can combine the items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine the items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Iterable)} to merge sources and terminate only when all source {@code Publisher}s @@ -3213,9 +3213,9 @@ public static Flowable mergeArray(int maxConcurrency, int bufferSize, @No * * @param the common element base type * @param sources - * the Iterable of Publishers - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the {@code Iterable} of {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s in the {@code Iterable} * @see ReactiveX operators documentation: Merge * @see #mergeDelayError(Iterable) */ @@ -3229,27 +3229,27 @@ public static Flowable merge(@NonNull Iterable * *

- * You can combine the items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine the items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Iterable, int)} to merge sources and terminate only when all source {@code Publisher}s @@ -3259,11 +3259,11 @@ public static Flowable merge(@NonNull Iterable the common element base type * @param sources - * the Iterable of Publishers + * the {@code Iterable} of {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the maximum number of {@code Publisher}s that may be subscribed to concurrently + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s in the {@code Iterable} * @throws IllegalArgumentException * if {@code maxConcurrency} is less than or equal to 0 * @see ReactiveX operators documentation: Merge @@ -3279,28 +3279,28 @@ public static Flowable merge(@NonNull Iterable * *

- * You can combine the items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine the items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The outer {@code Publisher} is consumed * in unbounded mode (i.e., no backpressure is applied to it). The inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Publisher)} to merge sources and terminate only when all source {@code Publisher}s @@ -3310,9 +3310,9 @@ public static Flowable merge(@NonNull Iterable the common element base type * @param sources - * a Publisher that emits Publishers - * @return a Flowable that emits items that are the result of flattening the Publishers emitted by the - * {@code source} Publisher + * a {@code Publisher} that emits {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of flattening the {@code Publisher}s emitted by the + * {@code source} {@code Publisher} * @see ReactiveX operators documentation: Merge * @see #mergeDelayError(Publisher) */ @@ -3325,28 +3325,28 @@ public static Flowable merge(@NonNull Publisher * *

- * You can combine the items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine the items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. Both the outer and inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Publisher, int)} to merge sources and terminate only when all source {@code Publisher}s @@ -3356,11 +3356,11 @@ public static Flowable merge(@NonNull Publisher the common element base type * @param sources - * a Publisher that emits Publishers + * a {@code Publisher} that emits {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently - * @return a Flowable that emits items that are the result of flattening the Publishers emitted by the - * {@code source} Publisher + * the maximum number of {@code Publisher}s that may be subscribed to concurrently + * @return a {@code Flowable} that emits items that are the result of flattening the {@code Publisher}s emitted by the + * {@code source} {@code Publisher} * @throws IllegalArgumentException * if {@code maxConcurrency} is less than or equal to 0 * @see ReactiveX operators documentation: Merge @@ -3377,26 +3377,26 @@ public static Flowable merge(@NonNull Publisher * *

- * You can combine items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code mergeArray} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeArrayDelayError(Publisher...)} to merge sources and terminate only when all source {@code Publisher}s @@ -3406,8 +3406,8 @@ public static Flowable merge(@NonNull Publisher the common element base type * @param sources - * the array of Publishers - * @return a Flowable that emits all of the items emitted by the Publishers in the Array + * the array of {@code Publisher}s + * @return a {@code Flowable} that emits all of the items emitted by the {@code Publisher}s * @see ReactiveX operators documentation: Merge * @see #mergeArrayDelayError(Publisher...) */ @@ -3422,26 +3422,26 @@ public static Flowable mergeArray(@NonNull Publisher... sour } /** - * Flattens two Publishers into a single Publisher, without any transformation. + * Flattens two {@link Publisher}s into a single {@code Publisher}, without any transformation. *

* *

- * You can combine items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Publisher, Publisher)} to merge sources and terminate only when all source {@code Publisher}s @@ -3451,10 +3451,10 @@ public static Flowable mergeArray(@NonNull Publisher... sour * * @param the common element base type * @param source1 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source2 - * a Publisher to be merged - * @return a Flowable that emits all of the items emitted by the source Publishers + * a {@code Publisher} to be merged + * @return a {@code Flowable} that emits all of the items emitted by the source {@code Publisher}s * @see ReactiveX operators documentation: Merge * @see #mergeDelayError(Publisher, Publisher) */ @@ -3470,26 +3470,26 @@ public static Flowable merge(@NonNull Publisher source1, @No } /** - * Flattens three Publishers into a single Publisher, without any transformation. + * Flattens three {@link Publisher}s into a single {@code Publisher}, without any transformation. *

* *

- * You can combine items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Publisher, Publisher, Publisher)} to merge sources and terminate only when all source {@code Publisher}s @@ -3499,12 +3499,12 @@ public static Flowable merge(@NonNull Publisher source1, @No * * @param the common element base type * @param source1 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source2 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source3 - * a Publisher to be merged - * @return a Flowable that emits all of the items emitted by the source Publishers + * a {@code Publisher} to be merged + * @return a {@code Flowable} that emits all of the items emitted by the source {@code Publisher}s * @see ReactiveX operators documentation: Merge * @see #mergeDelayError(Publisher, Publisher, Publisher) */ @@ -3521,26 +3521,26 @@ public static Flowable merge(@NonNull Publisher source1, @No } /** - * Flattens four Publishers into a single Publisher, without any transformation. + * Flattens four {@link Publisher}s into a single {@code Publisher}, without any transformation. *

* *

- * You can combine items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code merge} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code merge} does not operate by default on a particular {@link Scheduler}.
*
Error handling:
- *
If any of the source {@code Publisher}s signal a {@code Throwable} via {@code onError}, the resulting + *
If any of the source {@code Publisher}s signal a {@link Throwable} via {@code onError}, the resulting * {@code Flowable} terminates with that {@code Throwable} and all other source {@code Publisher}s are canceled. * If more than one {@code Publisher} signals an error, the resulting {@code Flowable} may terminate with the * first one's error or, depending on the concurrency of the sources, may terminate with a - * {@code CompositeException} containing two or more of the various error signals. + * {@link CompositeException} containing two or more of the various error signals. * {@code Throwable}s that didn't make into the composite will be sent (individually) to the global error handler via - * {@link RxJavaPlugins#onError(Throwable)} method as {@code UndeliverableException} errors. Similarly, {@code Throwable}s + * {@link RxJavaPlugins#onError(Throwable)} method as {@link UndeliverableException} errors. Similarly, {@code Throwable}s * signaled by source(s) after the returned {@code Flowable} has been canceled or terminated with a * (composite) error will be sent to the same global error handler. * Use {@link #mergeDelayError(Publisher, Publisher, Publisher, Publisher)} to merge sources and terminate only when all source {@code Publisher}s @@ -3550,14 +3550,14 @@ public static Flowable merge(@NonNull Publisher source1, @No * * @param the common element base type * @param source1 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source2 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source3 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source4 - * a Publisher to be merged - * @return a Flowable that emits all of the items emitted by the source Publishers + * a {@code Publisher} to be merged + * @return a {@code Flowable} that emits all of the items emitted by the source {@code Publisher}s * @see ReactiveX operators documentation: Merge * @see #mergeDelayError(Publisher, Publisher, Publisher, Publisher) */ @@ -3577,31 +3577,31 @@ public static Flowable merge( } /** - * Flattens an Iterable of Publishers into one Publisher, in a way that allows a Subscriber to receive all - * successfully emitted items from each of the source Publishers without being interrupted by an error + * Flattens an {@link Iterable} of {@link Publisher}s into one {@code Publisher}, in a way that allows a {@link Subscriber} to receive all + * successfully emitted items from each of the source {@code Publisher}s without being interrupted by an error * notification from one of them. *

- * This behaves like {@link #merge(Publisher)} except that if any of the merged Publishers notify of an + * This behaves like {@link #merge(Publisher)} except that if any of the merged {@code Publisher}s notify of an * error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from propagating that - * error notification until all of the merged Publishers have finished emitting items. + * error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. All inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}.
*
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * the Iterable of Publishers - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the {@code Iterable} of {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s in the {@code Iterable} * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3614,35 +3614,35 @@ public static Flowable mergeDelayError(@NonNull Iterable - * This behaves like {@link #merge(Publisher)} except that if any of the merged Publishers notify of an + * This behaves like {@link #merge(Publisher)} except that if any of the merged {@code Publisher}s notify of an * error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from propagating that - * error notification until all of the merged Publishers have finished emitting items. + * error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. All inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * the Iterable of Publishers + * the {@code Iterable} of {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param bufferSize - * the number of items to prefetch from each inner Publisher - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the number of items to prefetch from each inner {@code Publisher} + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s in the {@code Iterable} * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3655,35 +3655,35 @@ public static Flowable mergeDelayError(@NonNull Iterable - * This behaves like {@link #merge(Publisher)} except that if any of the merged Publishers notify of an + * This behaves like {@link #merge(Publisher)} except that if any of the merged {@code Publisher}s notify of an * error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from propagating that - * error notification until all of the merged Publishers have finished emitting items. + * error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. All source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeArrayDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * the array of Publishers + * the array of {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param bufferSize - * the number of items to prefetch from each inner Publisher - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the number of items to prefetch from each inner {@code Publisher} + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3697,33 +3697,33 @@ public static Flowable mergeArrayDelayError(int maxConcurrency, int buffe } /** - * Flattens an Iterable of Publishers into one Publisher, in a way that allows a Subscriber to receive all - * successfully emitted items from each of the source Publishers without being interrupted by an error - * notification from one of them, while limiting the number of concurrent subscriptions to these Publishers. + * Flattens an {@link Iterable} of {@link Publisher}s into one {@code Publisher}, in a way that allows a {@link Subscriber} to receive all + * successfully emitted items from each of the source {@code Publisher}s without being interrupted by an error + * notification from one of them, while limiting the number of concurrent subscriptions to these {@code Publisher}s. *

- * This behaves like {@link #merge(Publisher)} except that if any of the merged Publishers notify of an + * This behaves like {@link #merge(Publisher)} except that if any of the merged {@code Publisher}s notify of an * error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from propagating that - * error notification until all of the merged Publishers have finished emitting items. + * error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. All inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * the Iterable of Publishers + * the {@code Iterable} of {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the maximum number of {@code Publisher}s that may be subscribed to concurrently + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s in the {@code Iterable} * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3736,32 +3736,32 @@ public static Flowable mergeDelayError(@NonNull Iterable - * This behaves like {@link #merge(Publisher)} except that if any of the merged Publishers notify of an + * This behaves like {@link #merge(Publisher)} except that if any of the merged {@code Publisher}s notify of an * error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from propagating that - * error notification until all of the merged Publishers have finished emitting items. + * error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. The outer {@code Publisher} is consumed * in unbounded mode (i.e., no backpressure is applied to it). The inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * a Publisher that emits Publishers - * @return a Flowable that emits all of the items emitted by the Publishers emitted by the - * {@code source} Publisher + * a {@code Publisher} that emits {@code Publisher}s + * @return a {@code Flowable} that emits all of the items emitted by the {@code Publisher}s emitted by the + * {@code source} {@code Publisher} * @see ReactiveX operators documentation: Merge */ @CheckReturnValue @@ -3773,34 +3773,34 @@ public static Flowable mergeDelayError(@NonNull Publisher - * This behaves like {@link #merge(Publisher)} except that if any of the merged Publishers notify of an + * This behaves like {@link #merge(Publisher)} except that if any of the merged {@code Publisher}s notify of an * error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from propagating that - * error notification until all of the merged Publishers have finished emitting items. + * error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. Both the outer and inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * a Publisher that emits Publishers + * a {@code Publisher} that emits {@code Publisher}s * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently - * @return a Flowable that emits all of the items emitted by the Publishers emitted by the - * {@code source} Publisher + * the maximum number of {@code Publisher}s that may be subscribed to concurrently + * @return a {@code Flowable} that emits all of the items emitted by the {@code Publisher}s emitted by the + * {@code source} {@code Publisher} * @see ReactiveX operators documentation: Merge * @since 2.0 */ @@ -3814,31 +3814,31 @@ public static Flowable mergeDelayError(@NonNull Publisher - * This behaves like {@link #merge(Publisher)} except that if any of the merged Publishers notify of an + * This behaves like {@link #merge(Publisher)} except that if any of the merged {@code Publisher}s notify of an * error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from propagating that - * error notification until all of the merged Publishers have finished emitting items. + * error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. Both the outer and inner {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeArrayDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param sources - * the Iterable of Publishers - * @return a Flowable that emits items that are the result of flattening the items emitted by the - * Publishers in the Iterable + * the array of {@code Publisher}s + * @return a {@code Flowable} that emits items that are the result of flattening the items emitted by the + * {@code Publisher}s * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3852,32 +3852,32 @@ public static Flowable mergeArrayDelayError(@NonNull Publisher - * This behaves like {@link #merge(Publisher, Publisher)} except that if any of the merged Publishers + * This behaves like {@link #merge(Publisher, Publisher)} except that if any of the merged {@code Publisher}s * notify of an error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain from - * propagating that error notification until all of the merged Publishers have finished emitting items. + * propagating that error notification until all of the merged {@code Publisher}s have finished emitting items. *

* *

- * Even if both merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if both merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param source1 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source2 - * a Publisher to be merged - * @return a Flowable that emits all of the items that are emitted by the two source Publishers + * a {@code Publisher} to be merged + * @return a {@code Flowable} that emits all of the items that are emitted by the two source {@code Publisher}s * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3892,35 +3892,35 @@ public static Flowable mergeDelayError(@NonNull Publisher so } /** - * Flattens three Publishers into one Publisher, in a way that allows a Subscriber to receive all - * successfully emitted items from all of the source Publishers without being interrupted by an error + * Flattens three {@link Publisher}s into one {@code Publisher}, in a way that allows a {@link Subscriber} to receive all + * successfully emitted items from all of the source {@code Publisher}s without being interrupted by an error * notification from one of them. *

* This behaves like {@link #merge(Publisher, Publisher, Publisher)} except that if any of the merged - * Publishers notify of an error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain - * from propagating that error notification until all of the merged Publishers have finished emitting + * {@code Publisher}s notify of an error via {@link Subscriber#onError onError}, {@code mergeDelayError} will refrain + * from propagating that error notification until all of the merged {@code Publisher}s have finished emitting * items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param source1 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source2 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source3 - * a Publisher to be merged - * @return a Flowable that emits all of the items that are emitted by the source Publishers + * a {@code Publisher} to be merged + * @return a {@code Flowable} that emits all of the items that are emitted by the source {@code Publisher}s * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3936,37 +3936,37 @@ public static Flowable mergeDelayError(@NonNull Publisher so } /** - * Flattens four Publishers into one Publisher, in a way that allows a Subscriber to receive all - * successfully emitted items from all of the source Publishers without being interrupted by an error + * Flattens four {@link Publisher}s into one {@code Publisher}, in a way that allows a {@link Subscriber} to receive all + * successfully emitted items from all of the source {@code Publisher}s without being interrupted by an error * notification from one of them. *

* This behaves like {@link #merge(Publisher, Publisher, Publisher, Publisher)} except that if any of - * the merged Publishers notify of an error via {@link Subscriber#onError onError}, {@code mergeDelayError} - * will refrain from propagating that error notification until all of the merged Publishers have finished + * the merged {@code Publisher}s notify of an error via {@link Subscriber#onError onError}, {@code mergeDelayError} + * will refrain from propagating that error notification until all of the merged {@code Publisher}s have finished * emitting items. *

* *

- * Even if multiple merged Publishers send {@code onError} notifications, {@code mergeDelayError} will only - * invoke the {@code onError} method of its Subscribers once. + * Even if multiple merged {@code Publisher}s send {@code onError} notifications, {@code mergeDelayError} will only + * invoke the {@code onError} method of its {@code Subscriber}s once. *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the common element base type * @param source1 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source2 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source3 - * a Publisher to be merged + * a {@code Publisher} to be merged * @param source4 - * a Publisher to be merged - * @return a Flowable that emits all of the items that are emitted by the source Publishers + * a {@code Publisher} to be merged + * @return a {@code Flowable} that emits all of the items that are emitted by the source {@code Publisher}s * @see ReactiveX operators documentation: Merge */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -3985,11 +3985,11 @@ public static Flowable mergeDelayError( } /** - * Returns a Flowable that never sends any items or notifications to a {@link Subscriber}. + * Returns a {@code Flowable} that never sends any items or notifications to a {@link Subscriber}. *

* *

- * This Publisher is useful primarily for testing purposes. + * This {@link Publisher} is useful primarily for testing purposes. *

*
Backpressure:
*
This source doesn't produce any elements and effectively ignores downstream backpressure.
@@ -3998,8 +3998,8 @@ public static Flowable mergeDelayError( *
* * @param - * the type of items (not) emitted by the Publisher - * @return a Flowable that never emits any items or sends any notifications to a {@link Subscriber} + * the type of items (not) emitted by the {@code Publisher} + * @return a {@code Flowable} that never emits any items or sends any notifications to a {@code Subscriber} * @see ReactiveX operators documentation: Never */ @CheckReturnValue @@ -4012,7 +4012,7 @@ public static Flowable never() { } /** - * Returns a Flowable that emits a sequence of Integers within a specified range. + * Returns a {@code Flowable} that emits a sequence of {@link Integer}s within a specified range. *

* *

@@ -4023,10 +4023,10 @@ public static Flowable never() { *
* * @param start - * the value of the first Integer in the sequence + * the value of the first {@code Integer} in the sequence * @param count - * the number of sequential Integers to generate - * @return a Flowable that emits a range of sequential Integers + * the number of sequential {@code Integer}s to generate + * @return a {@code Flowable} that emits a range of sequential {@code Integer}s * @throws IllegalArgumentException * if {@code count} is less than zero, or if {@code start} + {@code count} − 1 exceeds * {@link Integer#MAX_VALUE} @@ -4053,7 +4053,7 @@ public static Flowable range(int start, int count) { } /** - * Returns a Flowable that emits a sequence of Longs within a specified range. + * Returns a {@code Flowable} that emits a sequence of {@link Long}s within a specified range. *

* *

@@ -4064,10 +4064,10 @@ public static Flowable range(int start, int count) { *
* * @param start - * the value of the first Long in the sequence + * the value of the first {@code Long} in the sequence * @param count - * the number of sequential Longs to generate - * @return a Flowable that emits a range of sequential Longs + * the number of sequential {@code Long}s to generate + * @return a {@code Flowable} that emits a range of sequential {@code Long}s * @throws IllegalArgumentException * if {@code count} is less than zero, or if {@code start} + {@code count} − 1 exceeds * {@link Long#MAX_VALUE} @@ -4099,25 +4099,25 @@ public static Flowable rangeLong(long start, long count) { } /** - * Returns a Single that emits a Boolean value that indicates whether two Publisher sequences are the - * same by comparing the items emitted by each Publisher pairwise. + * Returns a {@link Single} that emits a {@link Boolean} value that indicates whether two {@link Publisher} sequences are the + * same by comparing the items emitted by each {@code Publisher} pairwise. *

* *

*
Backpressure:
*
This operator honors downstream backpressure and expects both of its sources - * to honor backpressure as well. If violated, the operator will emit a MissingBackpressureException.
+ * to honor backpressure as well. If violated, the operator will emit a {@link MissingBackpressureException}. *
Scheduler:
*
{@code sequenceEqual} does not operate by default on a particular {@link Scheduler}.
*
* * @param source1 - * the first Publisher to compare + * the first {@code Publisher} to compare * @param source2 - * the second Publisher to compare + * the second {@code Publisher} to compare * @param - * the type of items emitted by each Publisher - * @return a Flowable that emits a Boolean value that indicates whether the two sequences are the same + * the type of items emitted by each {@code Publisher} + * @return a {@code Single} that emits a {@code Boolean} value that indicates whether the two sequences are the same * @see ReactiveX operators documentation: SequenceEqual */ @CheckReturnValue @@ -4129,28 +4129,28 @@ public static Single sequenceEqual(@NonNull Publisher } /** - * Returns a Single that emits a Boolean value that indicates whether two Publisher sequences are the - * same by comparing the items emitted by each Publisher pairwise based on the results of a specified + * Returns a {@link Single} that emits a {@link Boolean} value that indicates whether two {@link Publisher} sequences are the + * same by comparing the items emitted by each {@code Publisher} pairwise based on the results of a specified * equality function. *

* *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator signals a {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator signals a {@link MissingBackpressureException}. *
Scheduler:
*
{@code sequenceEqual} does not operate by default on a particular {@link Scheduler}.
*
* * @param source1 - * the first Publisher to compare + * the first {@code Publisher} to compare * @param source2 - * the second Publisher to compare + * the second {@code Publisher} to compare * @param isEqual - * a function used to compare items emitted by each Publisher + * a function used to compare items emitted by each {@code Publisher} * @param - * the type of items emitted by each Publisher - * @return a Single that emits a Boolean value that indicates whether the two Publisher sequences + * the type of items emitted by each {@code Publisher} + * @return a {@code Single} that emits a {@code Boolean} value that indicates whether the two {@code Publisher} sequences * are the same according to the specified function * @see ReactiveX operators documentation: SequenceEqual */ @@ -4164,30 +4164,30 @@ public static Single sequenceEqual(@NonNull Publisher } /** - * Returns a Single that emits a Boolean value that indicates whether two Publisher sequences are the - * same by comparing the items emitted by each Publisher pairwise based on the results of a specified + * Returns a {@link Single} that emits a {@link Boolean} value that indicates whether two {@link Publisher} sequences are the + * same by comparing the items emitted by each {@code Publisher} pairwise based on the results of a specified * equality function. *

* *

*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s are expected to honor - * backpressure; if violated, the operator signals a {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator signals a {@link MissingBackpressureException}. *
Scheduler:
*
{@code sequenceEqual} does not operate by default on a particular {@link Scheduler}.
*
* * @param source1 - * the first Publisher to compare + * the first {@code Publisher} to compare * @param source2 - * the second Publisher to compare + * the second {@code Publisher} to compare * @param isEqual - * a function used to compare items emitted by each Publisher + * a function used to compare items emitted by each {@code Publisher} * @param bufferSize - * the number of items to prefetch from the first and second source Publisher + * the number of items to prefetch from the first and second source {@code Publisher} * @param - * the type of items emitted by each Publisher - * @return a Single that emits a Boolean value that indicates whether the two Publisher sequences + * the type of items emitted by each {@code Publisher} + * @return a {@code Single} that emits a {@code Boolean} value that indicates whether the two {@code Publisher} sequences * are the same according to the specified function * @see ReactiveX operators documentation: SequenceEqual */ @@ -4205,27 +4205,27 @@ public static Single sequenceEqual(@NonNull Publisher } /** - * Returns a Single that emits a Boolean value that indicates whether two Publisher sequences are the - * same by comparing the items emitted by each Publisher pairwise. + * Returns a {@link Single} that emits a {@link Boolean} value that indicates whether two {@link Publisher} sequences are the + * same by comparing the items emitted by each {@code Publisher} pairwise. *

* *

*
Backpressure:
*
This operator honors downstream backpressure and expects both of its sources - * to honor backpressure as well. If violated, the operator will emit a MissingBackpressureException.
+ * to honor backpressure as well. If violated, the operator will emit a {@link MissingBackpressureException}. *
Scheduler:
*
{@code sequenceEqual} does not operate by default on a particular {@link Scheduler}.
*
* * @param source1 - * the first Publisher to compare + * the first {@code Publisher} to compare * @param source2 - * the second Publisher to compare + * the second {@code Publisher} to compare * @param bufferSize - * the number of items to prefetch from the first and second source Publisher + * the number of items to prefetch from the first and second source {@code Publisher} * @param - * the type of items emitted by each Publisher - * @return a Single that emits a Boolean value that indicates whether the two sequences are the same + * the type of items emitted by each {@code Publisher} + * @return a {@code Single} that emits a {@code Boolean} value that indicates whether the two sequences are the same * @see ReactiveX operators documentation: SequenceEqual */ @CheckReturnValue @@ -4237,35 +4237,35 @@ public static Single sequenceEqual(@NonNull Publisher } /** - * Converts a Publisher that emits Publishers into a Publisher that emits the items emitted by the - * most recently emitted of those Publishers. + * Converts a {@link Publisher} that emits {@code Publisher}s into a {@code Publisher} that emits the items emitted by the + * most recently emitted of those {@code Publisher}s. *

* *

- * {@code switchOnNext} subscribes to a Publisher that emits Publishers. Each time it observes one of - * these emitted Publishers, the Publisher returned by {@code switchOnNext} begins emitting the items - * emitted by that Publisher. When a new Publisher is emitted, {@code switchOnNext} stops emitting items - * from the earlier-emitted Publisher and begins emitting items from the new one. + * {@code switchOnNext} subscribes to a {@code Publisher} that emits {@code Publisher}s. Each time it observes one of + * these emitted {@code Publisher}s, the {@code Publisher} returned by {@code switchOnNext} begins emitting the items + * emitted by that {@code Publisher}. When a new {@code Publisher} is emitted, {@code switchOnNext} stops emitting items + * from the earlier-emitted {@code Publisher} and begins emitting items from the new one. *

- * The resulting Publisher completes if both the outer Publisher and the last inner Publisher, if any, complete. - * If the outer Publisher signals an onError, the inner Publisher is canceled and the error delivered in-sequence. + * The resulting {@code Publisher} completes if both the outer {@code Publisher} and the last inner {@code Publisher}, if any, complete. + * If the outer {@code Publisher} signals an {@code onError}, the inner {@code Publisher} is canceled and the error delivered in-sequence. *

*
Backpressure:
*
The operator honors backpressure from downstream. The outer {@code Publisher} is consumed in an * unbounded manner (i.e., without backpressure) and the inner {@code Publisher}s are expected to honor - * backpressure but it is not enforced; the operator won't signal a {@code MissingBackpressureException} - * but the violation may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * backpressure but it is not enforced; the operator won't signal a {@link MissingBackpressureException} + * but the violation may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code switchOnNext} does not operate by default on a particular {@link Scheduler}.
*
* * @param the item type * @param sources - * the source Publisher that emits Publishers + * the source {@code Publisher} that emits {@code Publisher}s * @param bufferSize - * the number of items to prefetch from the inner Publishers - * @return a Flowable that emits the items emitted by the Publisher most recently emitted by the source - * Publisher + * the number of items to prefetch from the inner {@code Publisher}s + * @return a {@code Flowable} that emits the items emitted by the {@code Publisher} most recently emitted by the source + * {@code Publisher} * @see ReactiveX operators documentation: Switch */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -4278,33 +4278,33 @@ public static Flowable switchOnNext(@NonNull Publisher * *

- * {@code switchOnNext} subscribes to a Publisher that emits Publishers. Each time it observes one of - * these emitted Publishers, the Publisher returned by {@code switchOnNext} begins emitting the items - * emitted by that Publisher. When a new Publisher is emitted, {@code switchOnNext} stops emitting items - * from the earlier-emitted Publisher and begins emitting items from the new one. + * {@code switchOnNext} subscribes to a {@code Publisher} that emits {@code Publisher}s. Each time it observes one of + * these emitted {@code Publisher}s, the {@code Publisher} returned by {@code switchOnNext} begins emitting the items + * emitted by that {@code Publisher}. When a new {@code Publisher} is emitted, {@code switchOnNext} stops emitting items + * from the earlier-emitted {@code Publisher} and begins emitting items from the new one. *

- * The resulting Publisher completes if both the outer Publisher and the last inner Publisher, if any, complete. - * If the outer Publisher signals an onError, the inner Publisher is canceled and the error delivered in-sequence. + * The resulting {@code Publisher} completes if both the outer {@code Publisher} and the last inner {@code Publisher}, if any, complete. + * If the outer {@code Publisher} signals an {@code onError}, the inner {@code Publisher} is canceled and the error delivered in-sequence. *

*
Backpressure:
*
The operator honors backpressure from downstream. The outer {@code Publisher} is consumed in an * unbounded manner (i.e., without backpressure) and the inner {@code Publisher}s are expected to honor - * backpressure but it is not enforced; the operator won't signal a {@code MissingBackpressureException} - * but the violation may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * backpressure but it is not enforced; the operator won't signal a {@link MissingBackpressureException} + * but the violation may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code switchOnNext} does not operate by default on a particular {@link Scheduler}.
*
* * @param the item type * @param sources - * the source Publisher that emits Publishers - * @return a Flowable that emits the items emitted by the Publisher most recently emitted by the source - * Publisher + * the source {@code Publisher} that emits {@code Publisher}s + * @return a {@code Flowable} that emits the items emitted by the {@code Publisher} most recently emitted by the source + * {@code Publisher} * @see ReactiveX operators documentation: Switch */ @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -4317,34 +4317,34 @@ public static Flowable switchOnNext(@NonNull Publisher * *

- * {@code switchOnNext} subscribes to a Publisher that emits Publishers. Each time it observes one of - * these emitted Publishers, the Publisher returned by {@code switchOnNext} begins emitting the items - * emitted by that Publisher. When a new Publisher is emitted, {@code switchOnNext} stops emitting items - * from the earlier-emitted Publisher and begins emitting items from the new one. + * {@code switchOnNext} subscribes to a {@code Publisher} that emits {@code Publisher}s. Each time it observes one of + * these emitted {@code Publisher}s, the {@code Publisher} returned by {@code switchOnNext} begins emitting the items + * emitted by that {@code Publisher}. When a new {@code Publisher} is emitted, {@code switchOnNext} stops emitting items + * from the earlier-emitted {@code Publisher} and begins emitting items from the new one. *

- * The resulting Publisher completes if both the main Publisher and the last inner Publisher, if any, complete. - * If the main Publisher signals an onError, the termination of the last inner Publisher will emit that error as is - * or wrapped into a CompositeException along with the other possible errors the former inner Publishers signaled. + * The resulting {@code Publisher} completes if both the main {@code Publisher} and the last inner {@code Publisher}, if any, complete. + * If the main {@code Publisher} signals an {@code onError}, the termination of the last inner {@code Publisher} will emit that error as is + * or wrapped into a {@link CompositeException} along with the other possible errors the former inner {@code Publisher}s signaled. *

*
Backpressure:
*
The operator honors backpressure from downstream. The outer {@code Publisher} is consumed in an * unbounded manner (i.e., without backpressure) and the inner {@code Publisher}s are expected to honor - * backpressure but it is not enforced; the operator won't signal a {@code MissingBackpressureException} - * but the violation may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * backpressure but it is not enforced; the operator won't signal a {@link MissingBackpressureException} + * but the violation may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code switchOnNextDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the item type * @param sources - * the source Publisher that emits Publishers - * @return a Flowable that emits the items emitted by the Publisher most recently emitted by the source - * Publisher + * the source {@code Publisher} that emits {@code Publisher}s + * @return a {@code Flowable} that emits the items emitted by the {@code Publisher} most recently emitted by the source + * {@code Publisher} * @see ReactiveX operators documentation: Switch * @since 2.0 */ @@ -4357,36 +4357,36 @@ public static Flowable switchOnNextDelayError(@NonNull Publisher * *

- * {@code switchOnNext} subscribes to a Publisher that emits Publishers. Each time it observes one of - * these emitted Publishers, the Publisher returned by {@code switchOnNext} begins emitting the items - * emitted by that Publisher. When a new Publisher is emitted, {@code switchOnNext} stops emitting items - * from the earlier-emitted Publisher and begins emitting items from the new one. + * {@code switchOnNext} subscribes to a {@code Publisher} that emits {@code Publisher}s. Each time it observes one of + * these emitted {@code Publisher}s, the {@code Publisher} returned by {@code switchOnNext} begins emitting the items + * emitted by that {@code Publisher}. When a new {@code Publisher} is emitted, {@code switchOnNext} stops emitting items + * from the earlier-emitted {@code Publisher} and begins emitting items from the new one. *

- * The resulting Publisher completes if both the main Publisher and the last inner Publisher, if any, complete. - * If the main Publisher signals an onError, the termination of the last inner Publisher will emit that error as is - * or wrapped into a CompositeException along with the other possible errors the former inner Publishers signaled. + * The resulting {@code Publisher} completes if both the main {@code Publisher} and the last inner {@code Publisher}, if any, complete. + * If the main {@code Publisher} signals an {@code onError}, the termination of the last inner {@code Publisher} will emit that error as is + * or wrapped into a {@link CompositeException} along with the other possible errors the former inner {@code Publisher}s signaled. *

*
Backpressure:
*
The operator honors backpressure from downstream. The outer {@code Publisher} is consumed in an * unbounded manner (i.e., without backpressure) and the inner {@code Publisher}s are expected to honor - * backpressure but it is not enforced; the operator won't signal a {@code MissingBackpressureException} - * but the violation may lead to {@code OutOfMemoryError} due to internal buffer bloat.
+ * backpressure but it is not enforced; the operator won't signal a {@link MissingBackpressureException} + * but the violation may lead to {@link OutOfMemoryError} due to internal buffer bloat. *
Scheduler:
*
{@code switchOnNextDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the item type * @param sources - * the source Publisher that emits Publishers + * the source {@code Publisher} that emits {@code Publisher}s * @param prefetch - * the number of items to prefetch from the inner Publishers - * @return a Flowable that emits the items emitted by the Publisher most recently emitted by the source - * Publisher + * the number of items to prefetch from the inner {@code Publisher}s + * @return a {@code Flowable} that emits the items emitted by the {@code Publisher} most recently emitted by the source + * {@code Publisher} * @see ReactiveX operators documentation: Switch * @since 2.0 */ @@ -4399,7 +4399,7 @@ public static Flowable switchOnNextDelayError(@NonNull Publisher * *
@@ -4414,7 +4414,7 @@ public static Flowable switchOnNextDelayError(@NonNull PublisherReactiveX operators documentation: Timer */ @CheckReturnValue @@ -4426,7 +4426,7 @@ public static Flowable timer(long delay, @NonNull TimeUnit unit) { } /** - * Returns a Flowable that emits {@code 0L} after a specified delay, on a specified Scheduler, and then + * Returns a {@code Flowable} that emits {@code 0L} after a specified delay, on a specified {@link Scheduler}, and then * completes. *

* @@ -4435,7 +4435,7 @@ public static Flowable timer(long delay, @NonNull TimeUnit unit) { *

This operator does not support backpressure as it uses time. If the downstream needs a slower rate * it should slow the timer or use something like {@link #onBackpressureDrop}.
*
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param delay @@ -4443,8 +4443,8 @@ public static Flowable timer(long delay, @NonNull TimeUnit unit) { * @param unit * time units to use for {@code delay} * @param scheduler - * the {@link Scheduler} to use for scheduling the item - * @return a Flowable that emits {@code 0L} after a specified delay, on a specified Scheduler, and then + * the {@code Scheduler} to use for scheduling the item + * @return a {@code Flowable} that emits {@code 0L} after a specified delay, on a specified {@code Scheduler}, and then * completes * @see ReactiveX operators documentation: Timer */ @@ -4460,19 +4460,19 @@ public static Flowable timer(long delay, @NonNull TimeUnit unit, @NonNull } /** - * Create a Flowable by wrapping a Publisher which has to be implemented according - * to the Reactive Streams specification by handling backpressure and - * cancellation correctly; no safeguards are provided by the Flowable itself. + * Create a {@code Flowable} by wrapping a {@link Publisher} which has to be implemented according + * to the Reactive Streams specification by handling backpressure and + * cancellation correctly; no safeguards are provided by the {@code Flowable} itself. *
*
Backpressure:
*
This operator is a pass-through for backpressure and the behavior is determined by the - * provided Publisher implementation.
+ * provided {@code Publisher} implementation. *
Scheduler:
*
{@code unsafeCreate} by default doesn't operate on any particular {@link Scheduler}.
*
* @param the value type emitted - * @param onSubscribe the Publisher instance to wrap - * @return the new Flowable instance + * @param onSubscribe the {@code Publisher} instance to wrap + * @return the new {@code Flowable} instance * @throws IllegalArgumentException if {@code onSubscribe} is a subclass of {@code Flowable}; such * instances don't need conversion and is possibly a port remnant from 1.x or one should use {@link #hide()} * instead. @@ -4490,26 +4490,26 @@ public static Flowable unsafeCreate(@NonNull Publisher onSubscribe) { } /** - * Constructs a Publisher that creates a dependent resource object which is disposed of on cancellation. + * Constructs a {@link Publisher} that creates a dependent resource object which is disposed of on cancellation. *

* *

*
Backpressure:
*
The operator is a pass-through for backpressure and otherwise depends on the - * backpressure support of the Publisher returned by the {@code resourceFactory}.
+ * backpressure support of the {@code Publisher} returned by the {@code resourceFactory}. *
Scheduler:
*
{@code using} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the element type of the generated Publisher + * @param the element type of the generated {@code Publisher} * @param the type of the resource associated with the output sequence * @param resourceSupplier - * the factory function to create a resource object that depends on the Publisher + * the factory function to create a resource object that depends on the {@code Publisher} * @param sourceSupplier - * the factory function to create a Publisher + * the factory function to create a {@code Publisher} * @param resourceDisposer * the function that will dispose of the resource - * @return the Publisher whose lifetime controls the lifetime of the dependent resource object + * @return the {@code Publisher} whose lifetime controls the lifetime of the dependent resource object * @see ReactiveX operators documentation: Using */ @CheckReturnValue @@ -4524,27 +4524,27 @@ public static Flowable using( } /** - * Constructs a Publisher that creates a dependent resource object which is disposed of just before + * Constructs a {@link Publisher} that creates a dependent resource object which is disposed of just before * termination if you have set {@code disposeEagerly} to {@code true} and cancellation does not occur * before termination. Otherwise, resource disposal will occur on cancellation. Eager disposal is - * particularly appropriate for a synchronous Publisher that reuses resources. {@code disposeAction} will + * particularly appropriate for a synchronous {@code Publisher} that reuses resources. {@code disposeAction} will * only be called once per subscription. *

* *

*
Backpressure:
*
The operator is a pass-through for backpressure and otherwise depends on the - * backpressure support of the Publisher returned by the {@code resourceFactory}.
+ * backpressure support of the {@code Publisher} returned by the {@code resourceFactory}. *
Scheduler:
*
{@code using} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the element type of the generated Publisher + * @param the element type of the generated {@code Publisher} * @param the type of the resource associated with the output sequence * @param resourceSupplier - * the factory function to create a resource object that depends on the Publisher + * the factory function to create a resource object that depends on the {@code Publisher} * @param sourceSupplier - * the factory function to create a Publisher + * the factory function to create a {@code Publisher} * @param resourceDisposer * the function that will dispose of the resource * @param eager @@ -4552,7 +4552,7 @@ public static Flowable using( * or just before the emission of a terminal event ({@code onComplete} or {@code onError}). * If {@code false} the resource disposal will happen either on a {@code cancel()} call after the upstream is disposed * or just after the emission of a terminal event ({@code onComplete} or {@code onError}). - * @return the Publisher whose lifetime controls the lifetime of the dependent resource object + * @return the {@code Publisher} whose lifetime controls the lifetime of the dependent resource object * @see ReactiveX operators documentation: Using * @since 2.0 */ @@ -4572,16 +4572,16 @@ public static Flowable using( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * items emitted, in sequence, by an Iterable of other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * items emitted, in sequence, by an {@link Iterable} of other {@link Publisher}s. *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher - * will be the result of the function applied to the first item emitted by each of the source Publishers; - * the second item emitted by the new Publisher will be the result of the function applied to the second - * item emitted by each of those Publishers; and so forth. + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} + * will be the result of the function applied to the first item emitted by each of the source {@code Publisher}s; + * the second item emitted by the new {@code Publisher} will be the result of the function applied to the second + * item emitted by each of those {@code Publisher}s; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@code onNext} as many times as - * the number of {@code onNext} invocations of the source Publisher that emits the fewest items. + * the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if * one of the sources is shorter than the rest while canceling the other sources. Therefore, it @@ -4599,7 +4599,7 @@ public static Flowable using( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -4608,11 +4608,11 @@ public static Flowable using( * @param the common value type * @param the zipped result type * @param sources - * an Iterable of source Publishers + * an {@code Iterable} of source {@code Publisher}s * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -4626,16 +4626,16 @@ public static Flowable zip(@NonNull Iterable - * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher - * will be the result of the function applied to the first item emitted by each of the source Publishers; - * the second item emitted by the new Publisher will be the result of the function applied to the second - * item emitted by each of those Publishers; and so forth. + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} + * will be the result of the function applied to the first item emitted by each of the source {@code Publisher}s; + * the second item emitted by the new {@code Publisher} will be the result of the function applied to the second + * item emitted by each of those {@code Publisher}s; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@code onNext} as many times as - * the number of {@code onNext} invocations of the source Publisher that emits the fewest items. + * the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if * one of the sources is shorter than the rest while canceling the other sources. Therefore, it @@ -4653,7 +4653,7 @@ public static Flowable zip(@NonNull Iterable *

Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -4661,17 +4661,17 @@ public static Flowable zip(@NonNull Iterable the common source value type * @param the zipped result type - * @return a Flowable that emits the zipped results + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -4688,18 +4688,18 @@ public static Flowable zip(@NonNull Iterable * *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} * will be the result of the function applied to the first item emitted by {@code o1} and the first item - * emitted by {@code o2}; the second item emitted by the new Publisher will be the result of the function + * emitted by {@code o2}; the second item emitted by the new {@code Publisher} will be the result of the function * applied to the second item emitted by {@code o1} and the second item emitted by {@code o2}; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -4716,7 +4716,7 @@ public static Flowable zip(@NonNull Iterable *

Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -4726,13 +4726,13 @@ public static Flowable zip(@NonNull Iterable the value type of the second source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results - * in an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results + * in an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -4749,18 +4749,18 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * two items emitted, in sequence, by two other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * two items emitted, in sequence, by two other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} * will be the result of the function applied to the first item emitted by {@code o1} and the first item - * emitted by {@code o2}; the second item emitted by the new Publisher will be the result of the function + * emitted by {@code o2}; the second item emitted by the new {@code Publisher} will be the result of the function * applied to the second item emitted by {@code o1} and the second item emitted by {@code o2}; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -4777,7 +4777,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -4787,14 +4787,14 @@ public static Flowable zip( * @param the value type of the second source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results - * in an item that will be emitted by the resulting Publisher - * @param delayError delay errors from any of the source Publishers till the other terminates - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results + * in an item that will be emitted by the resulting {@code Publisher} + * @param delayError delay errors from any of the source {@code Publisher}s till the other terminates + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -4811,18 +4811,18 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * two items emitted, in sequence, by two other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * two items emitted, in sequence, by two other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} * will be the result of the function applied to the first item emitted by {@code o1} and the first item - * emitted by {@code o2}; the second item emitted by the new Publisher will be the result of the function + * emitted by {@code o2}; the second item emitted by the new {@code Publisher} will be the result of the function * applied to the second item emitted by {@code o1} and the second item emitted by {@code o2}; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -4839,7 +4839,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -4849,15 +4849,15 @@ public static Flowable zip( * @param the value type of the second source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results - * in an item that will be emitted by the resulting Publisher - * @param delayError delay errors from any of the source Publishers till the other terminates - * @param bufferSize the number of elements to prefetch from each source Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results + * in an item that will be emitted by the resulting {@code Publisher} + * @param delayError delay errors from any of the source {@code Publisher}s till the other terminates + * @param bufferSize the number of elements to prefetch from each source {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -4874,19 +4874,19 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * three items emitted, in sequence, by three other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * three items emitted, in sequence, by three other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} * will be the result of the function applied to the first item emitted by {@code o1}, the first item * emitted by {@code o2}, and the first item emitted by {@code o3}; the second item emitted by the new - * Publisher will be the result of the function applied to the second item emitted by {@code o1}, the + * {@code Publisher} will be the result of the function applied to the second item emitted by {@code o1}, the * second item emitted by {@code o2}, and the second item emitted by {@code o3}; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -4903,7 +4903,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -4914,15 +4914,15 @@ public static Flowable zip( * @param the value type of the third source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param source3 - * a third source Publisher + * a third source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -4940,19 +4940,19 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * four items emitted, in sequence, by four other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * four items emitted, in sequence, by four other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} * will be the result of the function applied to the first item emitted by {@code o1}, the first item * emitted by {@code o2}, the first item emitted by {@code o3}, and the first item emitted by {@code 04}; - * the second item emitted by the new Publisher will be the result of the function applied to the second - * item emitted by each of those Publishers; and so forth. + * the second item emitted by the new {@code Publisher} will be the result of the function applied to the second + * item emitted by each of those {@code Publisher}s; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -4969,7 +4969,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -4981,17 +4981,17 @@ public static Flowable zip( * @param the value type of the fourth source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param source3 - * a third source Publisher + * a third source {@code Publisher} * @param source4 - * a fourth source Publisher + * a fourth source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -5011,19 +5011,19 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * five items emitted, in sequence, by five other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * five items emitted, in sequence, by five other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} * will be the result of the function applied to the first item emitted by {@code o1}, the first item * emitted by {@code o2}, the first item emitted by {@code o3}, the first item emitted by {@code o4}, and - * the first item emitted by {@code o5}; the second item emitted by the new Publisher will be the result of - * the function applied to the second item emitted by each of those Publishers; and so forth. + * the first item emitted by {@code o5}; the second item emitted by the new {@code Publisher} will be the result of + * the function applied to the second item emitted by each of those {@code Publisher}s; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -5040,7 +5040,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -5053,19 +5053,19 @@ public static Flowable zip( * @param the value type of the fifth source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param source3 - * a third source Publisher + * a third source {@code Publisher} * @param source4 - * a fourth source Publisher + * a fourth source {@code Publisher} * @param source5 - * a fifth source Publisher + * a fifth source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -5086,18 +5086,18 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * six items emitted, in sequence, by six other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * six items emitted, in sequence, by six other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher - * will be the result of the function applied to the first item emitted by each source Publisher, the - * second item emitted by the new Publisher will be the result of the function applied to the second item - * emitted by each of those Publishers, and so forth. + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} + * will be the result of the function applied to the first item emitted by each source {@code Publisher}, the + * second item emitted by the new {@code Publisher} will be the result of the function applied to the second item + * emitted by each of those {@code Publisher}s, and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -5114,7 +5114,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -5128,21 +5128,21 @@ public static Flowable zip( * @param the value type of the sixth source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param source3 - * a third source Publisher + * a third source {@code Publisher} * @param source4 - * a fourth source Publisher + * a fourth source {@code Publisher} * @param source5 - * a fifth source Publisher + * a fifth source {@code Publisher} * @param source6 - * a sixth source Publisher + * a sixth source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -5164,18 +5164,18 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * seven items emitted, in sequence, by seven other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * seven items emitted, in sequence, by seven other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher - * will be the result of the function applied to the first item emitted by each source Publisher, the - * second item emitted by the new Publisher will be the result of the function applied to the second item - * emitted by each of those Publishers, and so forth. + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} + * will be the result of the function applied to the first item emitted by each source {@code Publisher}, the + * second item emitted by the new {@code Publisher} will be the result of the function applied to the second item + * emitted by each of those {@code Publisher}s, and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -5192,7 +5192,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -5207,23 +5207,23 @@ public static Flowable zip( * @param the value type of the seventh source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param source3 - * a third source Publisher + * a third source {@code Publisher} * @param source4 - * a fourth source Publisher + * a fourth source {@code Publisher} * @param source5 - * a fifth source Publisher + * a fifth source {@code Publisher} * @param source6 - * a sixth source Publisher + * a sixth source {@code Publisher} * @param source7 - * a seventh source Publisher + * a seventh source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -5247,18 +5247,18 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * eight items emitted, in sequence, by eight other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * eight items emitted, in sequence, by eight other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher - * will be the result of the function applied to the first item emitted by each source Publisher, the - * second item emitted by the new Publisher will be the result of the function applied to the second item - * emitted by each of those Publishers, and so forth. + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} + * will be the result of the function applied to the first item emitted by each source {@code Publisher}, the + * second item emitted by the new {@code Publisher} will be the result of the function applied to the second item + * emitted by each of those {@code Publisher}s, and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -5275,7 +5275,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -5291,25 +5291,25 @@ public static Flowable zip( * @param the value type of the eighth source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param source3 - * a third source Publisher + * a third source {@code Publisher} * @param source4 - * a fourth source Publisher + * a fourth source {@code Publisher} * @param source5 - * a fifth source Publisher + * a fifth source {@code Publisher} * @param source6 - * a sixth source Publisher + * a sixth source {@code Publisher} * @param source7 - * a seventh source Publisher + * a seventh source {@code Publisher} * @param source8 - * an eighth source Publisher + * an eighth source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -5334,18 +5334,18 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * nine items emitted, in sequence, by nine other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * nine items emitted, in sequence, by nine other {@link Publisher}s. *

* *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher - * will be the result of the function applied to the first item emitted by each source Publisher, the - * second item emitted by the new Publisher will be the result of the function applied to the second item - * emitted by each of those Publishers, and so forth. + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} + * will be the result of the function applied to the first item emitted by each source {@code Publisher}, the + * second item emitted by the new {@code Publisher} will be the result of the function applied to the second item + * emitted by each of those {@code Publisher}s, and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@link Subscriber#onNext onNext} - * as many times as the number of {@code onNext} invocations of the source Publisher that emits the fewest + * as many times as the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest * items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if @@ -5362,7 +5362,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zip} does not operate by default on a particular {@link Scheduler}.
@@ -5379,27 +5379,27 @@ public static Flowable zip( * @param the value type of the ninth source * @param the zipped result type * @param source1 - * the first source Publisher + * the first source {@code Publisher} * @param source2 - * a second source Publisher + * a second source {@code Publisher} * @param source3 - * a third source Publisher + * a third source {@code Publisher} * @param source4 - * a fourth source Publisher + * a fourth source {@code Publisher} * @param source5 - * a fifth source Publisher + * a fifth source {@code Publisher} * @param source6 - * a sixth source Publisher + * a sixth source {@code Publisher} * @param source7 - * a seventh source Publisher + * a seventh source {@code Publisher} * @param source8 - * an eighth source Publisher + * an eighth source {@code Publisher} * @param source9 - * a ninth source Publisher + * a ninth source {@code Publisher} * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher - * @return a Flowable that emits the zipped results + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -5426,16 +5426,16 @@ public static Flowable zip( } /** - * Returns a Flowable that emits the results of a specified combiner function applied to combinations of - * items emitted, in sequence, by an array of other Publishers. + * Returns a {@code Flowable} that emits the results of a specified combiner function applied to combinations of + * items emitted, in sequence, by an array of other {@link Publisher}s. *

- * {@code zip} applies this function in strict sequence, so the first item emitted by the new Publisher - * will be the result of the function applied to the first item emitted by each of the source Publishers; - * the second item emitted by the new Publisher will be the result of the function applied to the second - * item emitted by each of those Publishers; and so forth. + * {@code zip} applies this function in strict sequence, so the first item emitted by the new {@code Publisher} + * will be the result of the function applied to the first item emitted by each of the source {@code Publisher}s; + * the second item emitted by the new {@code Publisher} will be the result of the function applied to the second + * item emitted by each of those {@code Publisher}s; and so forth. *

* The resulting {@code Publisher} returned from {@code zip} will invoke {@code onNext} as many times as - * the number of {@code onNext} invocations of the source Publisher that emits the fewest items. + * the number of {@code onNext} invocations of the source {@code Publisher} that emits the fewest items. *

* The operator subscribes to its sources in the order they are specified and completes eagerly if * one of the sources is shorter than the rest while canceling the other sources. Therefore, it @@ -5454,7 +5454,7 @@ public static Flowable zip( *

*
Backpressure:
*
The operator expects backpressure from the sources and honors backpressure from the downstream. - * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in MissingBackpressureException, use + * (I.e., zipping with {@link #interval(long, TimeUnit)} may result in {@link MissingBackpressureException}, use * one of the {@code onBackpressureX} to handle similar, backpressure-ignoring sources.
*
Scheduler:
*
{@code zipArray} does not operate by default on a particular {@link Scheduler}.
@@ -5463,15 +5463,15 @@ public static Flowable zip( * @param the common element type * @param the result type * @param sources - * an array of source Publishers + * an array of source {@code Publisher}s * @param zipper - * a function that, when applied to an item emitted by each of the source Publishers, results in - * an item that will be emitted by the resulting Publisher + * a function that, when applied to an item emitted by each of the source {@code Publisher}s, results in + * an item that will be emitted by the resulting {@code Publisher} * @param delayError - * delay errors signaled by any of the source Publisher until all Publishers terminate + * delay errors signaled by any of the source {@code Publisher} until all {@code Publisher}s terminate * @param bufferSize - * the number of elements to prefetch from each source Publisher - * @return a Flowable that emits the zipped results + * the number of elements to prefetch from each source {@code Publisher} + * @return a {@code Flowable} that emits the zipped results * @see ReactiveX operators documentation: Zip */ @CheckReturnValue @@ -5494,8 +5494,8 @@ public static Flowable zipArray(@NonNull Function * *
@@ -5507,8 +5507,8 @@ public static Flowable zipArray(@NonNull Function * * @param predicate - * a function that evaluates an item and returns a Boolean - * @return a Single that emits {@code true} if all items emitted by the source Publisher satisfy the + * a function that evaluates an item and returns a {@code Boolean} + * @return a {@code Single} that emits {@code true} if all items emitted by the source {@code Publisher} satisfy the * predicate; otherwise, {@code false} * @see ReactiveX operators documentation: All */ @@ -5522,7 +5522,7 @@ public final Single all(@NonNull Predicate predicate) { } /** - * Mirrors the Publisher (current or provided) that first either emits an item or sends a termination + * Mirrors the {@link Publisher} (current or provided) that first either emits an item or sends a termination * notification. *

* @@ -5535,9 +5535,9 @@ public final Single all(@NonNull Predicate predicate) { *

* * @param other - * a Publisher competing to react first. A subscription to this provided Publisher will occur after subscribing - * to the current Publisher. - * @return a Flowable that emits the same sequence as whichever of the source Publishers first + * a {@code Publisher} competing to react first. A subscription to this provided {@code Publisher} will occur after subscribing + * to the current {@code Publisher}. + * @return a {@code Flowable} that emits the same sequence as whichever of the source {@code Publisher}s first * emitted an item or sent a termination notification * @see ReactiveX operators documentation: Amb */ @@ -5551,9 +5551,9 @@ public final Flowable ambWith(@NonNull Publisher other) { } /** - * Returns a Single that emits {@code true} if any item emitted by the source Publisher satisfies a + * Returns a {@link Single} that emits {@code true} if any item emitted by the source {@link Publisher} satisfies a * specified condition, otherwise {@code false}. Note: this always emits {@code false} if the - * source Publisher is empty. + * source {@code Publisher} is empty. *

* *

@@ -5568,9 +5568,9 @@ public final Flowable ambWith(@NonNull Publisher other) { *

* * @param predicate - * the condition to test items emitted by the source Publisher - * @return a Single that emits a Boolean that indicates whether any item emitted by the source - * Publisher satisfies the {@code predicate} + * the condition to test items emitted by the source {@code Publisher} + * @return a {@code Single} that emits a {@link Boolean} that indicates whether any item emitted by the source + * {@code Publisher} satisfies the {@code predicate} * @see ReactiveX operators documentation: Contains */ @CheckReturnValue @@ -5584,7 +5584,7 @@ public final Single any(@NonNull Predicate predicate) { /** * Returns the first item emitted by this {@code Flowable}, or throws - * {@code NoSuchElementException} if it emits no items. + * {@link NoSuchElementException} if it emits no items. *
*
Backpressure:
*
The operator consumes the source {@code Flowable} in an unbounded manner @@ -5650,7 +5650,7 @@ public final T blockingFirst(@NonNull T defaultItem) { /** * Consumes the upstream {@code Flowable} in a blocking fashion and invokes the given - * {@code Consumer} with each upstream item on the current thread until the + * {@link Consumer} with each upstream item on the current thread until the * upstream terminates. *

* @@ -5674,10 +5674,10 @@ public final T blockingFirst(@NonNull T defaultItem) { *

* * @param onNext - * the {@link Consumer} to invoke for each item emitted by the {@code Flowable} + * the {@code Consumer} to invoke for each item emitted by the {@code Flowable} * @throws RuntimeException - * if an error occurs; {@link Error}s and {@link RuntimeException}s are rethrown - * as they are, checked {@link Exception}s are wrapped into {@code RuntimeException}s + * if an error occurs; {@code Error}s and {@code RuntimeException}s are rethrown + * as they are, checked {@code Exception}s are wrapped into {@code RuntimeException}s * @see ReactiveX documentation: Subscribe * @see #subscribe(Consumer) * @see #blockingForEach(Consumer, int) @@ -5690,7 +5690,7 @@ public final void blockingForEach(@NonNull Consumer onNext) { /** * Consumes the upstream {@code Flowable} in a blocking fashion and invokes the given - * {@code Consumer} with each upstream item on the current thread until the + * {@link Consumer} with each upstream item on the current thread until the * upstream terminates. *

* @@ -5714,12 +5714,12 @@ public final void blockingForEach(@NonNull Consumer onNext) { *

* * @param onNext - * the {@link Consumer} to invoke for each item emitted by the {@code Flowable} + * the {@code Consumer} to invoke for each item emitted by the {@code Flowable} * @param bufferSize * the number of items to prefetch upfront, then 75% of it after 75% received * @throws RuntimeException - * if an error occurs; {@link Error}s and {@link RuntimeException}s are rethrown - * as they are, checked {@link Exception}s are wrapped into {@code RuntimeException}s + * if an error occurs; {@code Error}s and {@code RuntimeException}s are rethrown + * as they are, checked {@code Exception}s are wrapped into {@code RuntimeException}s * @see ReactiveX documentation: Subscribe * @see #subscribe(Consumer) */ @@ -5745,12 +5745,12 @@ public final void blockingForEach(@NonNull Consumer onNext, int buffe *
*
Backpressure:
*
The operator expects the upstream to honor backpressure otherwise the returned - * Iterable's iterator will throw a {@code MissingBackpressureException}.
+ * {@code Iterable}'s iterator will throw a {@link MissingBackpressureException}. *
Scheduler:
*
{@code blockingIterable} does not operate by default on a particular {@link Scheduler}.
*
* - * @return an {@link Iterable} version of this {@code Flowable} + * @return an {@code Iterable} version of this {@code Flowable} * @see ReactiveX documentation: To */ @CheckReturnValue @@ -5768,14 +5768,14 @@ public final Iterable blockingIterable() { *
*
Backpressure:
*
The operator expects the upstream to honor backpressure otherwise the returned - * Iterable's iterator will throw a {@code MissingBackpressureException}. + * {@code Iterable}'s iterator will throw a {@link MissingBackpressureException}. *
*
Scheduler:
*
{@code blockingIterable} does not operate by default on a particular {@link Scheduler}.
*
* - * @param bufferSize the number of items to prefetch from the current Flowable - * @return an {@link Iterable} version of this {@code Flowable} + * @param bufferSize the number of items to prefetch from the current {@code Flowable} + * @return an {@code Iterable} version of this {@code Flowable} * @see ReactiveX documentation: To */ @CheckReturnValue @@ -5789,7 +5789,7 @@ public final Iterable blockingIterable(int bufferSize) { /** * Returns the last item emitted by this {@code Flowable}, or throws - * {@code NoSuchElementException} if this {@code Flowable} emits no items. + * {@link NoSuchElementException} if this {@code Flowable} emits no items. *

* *

@@ -5874,7 +5874,7 @@ public final T blockingLast(@NonNull T defaultItem) { *
{@code blockingLatest} does not operate by default on a particular {@link Scheduler}.
*
* - * @return an Iterable that always returns the latest item emitted by this {@code Flowable} + * @return an {@code Iterable} that always returns the latest item emitted by this {@code Flowable} * @see ReactiveX documentation: First */ @CheckReturnValue @@ -5899,9 +5899,9 @@ public final Iterable blockingLatest() { *
* * @param initialItem - * the initial item that the {@link Iterable} sequence will yield if this + * the initial item that the {@code Iterable} sequence will yield if this * {@code Flowable} has not yet emitted an item - * @return an {@link Iterable} that on each iteration returns the item that this {@code Flowable} + * @return an {@code Iterable} that on each iteration returns the item that this {@code Flowable} * has most recently emitted * @see ReactiveX documentation: First */ @@ -5926,8 +5926,8 @@ public final Iterable blockingMostRecent(@NonNull T initialItem) { *
{@code blockingNext} does not operate by default on a particular {@link Scheduler}.
*
* - * @return an {@link Iterable} that blocks upon each iteration until this {@code Flowable} emits - * a new item, whereupon the Iterable returns that item + * @return an {@code Iterable} that blocks upon each iteration until this {@code Flowable} emits + * a new item, whereupon the {@code Iterable} returns that item * @see ReactiveX documentation: TakeLast */ @CheckReturnValue @@ -5940,7 +5940,7 @@ public final Iterable blockingNext() { /** * If this {@code Flowable} completes after emitting a single item, return that item, otherwise - * throw a {@code NoSuchElementException}. + * throw a {@link NoSuchElementException}. *

* *

@@ -5968,7 +5968,7 @@ public final T blockingSingle() { /** * If this {@code Flowable} completes after emitting a single item, return that item; if it emits - * more than one item, throw an {@code IllegalArgumentException}; if it emits no items, return a default + * more than one item, throw an {@link IllegalArgumentException}; if it emits no items, return a default * value. *

* @@ -6003,8 +6003,8 @@ public final T blockingSingle(@NonNull T defaultItem) { *

* *

- * If the {@link Flowable} emits more than one item, {@link java.util.concurrent.Future} will receive an - * {@link java.lang.IndexOutOfBoundsException}. If the {@link Flowable} is empty, {@link java.util.concurrent.Future} + * If the {@code Flowable} emits more than one item, {@link java.util.concurrent.Future} will receive an + * {@link java.lang.IndexOutOfBoundsException}. If the {@code Flowable} is empty, {@link java.util.concurrent.Future} * will receive a {@link java.util.NoSuchElementException}. The {@code Flowable} source has to terminate in order * for the returned {@code Future} to terminate as well. *

@@ -6017,7 +6017,7 @@ public final T blockingSingle(@NonNull T defaultItem) { *

{@code toFuture} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a {@link Future} that expects a single item to be emitted by this {@code Flowable} + * @return a {@code Future} that expects a single item to be emitted by this {@code Flowable} * @see ReactiveX documentation: To */ @CheckReturnValue @@ -6029,7 +6029,7 @@ public final Future toFuture() { } /** - * Runs the source Flowable to a terminal event, ignoring any values and rethrowing any exception. + * Runs the source {@code Flowable} to a terminal event, ignoring any values and rethrowing any exception. *

* Note that calling this method will block the caller thread until the upstream terminates * normally or with an error. Therefore, calling this method from special threads such as the @@ -6055,9 +6055,9 @@ public final void blockingSubscribe() { /** * Subscribes to the source and calls the given callbacks on the current thread. *

- * If the Flowable emits an error, it is wrapped into an + * If the {@code Flowable} emits an error, it is wrapped into an * {@link io.reactivex.rxjava3.exceptions.OnErrorNotImplementedException OnErrorNotImplementedException} - * and routed to the RxJavaPlugins.onError handler. + * and routed to the {@link RxJavaPlugins#onError(Throwable)} handler. * Using the overloads {@link #blockingSubscribe(Consumer, Consumer)} * or {@link #blockingSubscribe(Consumer, Consumer, Action)} instead is recommended. *

@@ -6085,9 +6085,9 @@ public final void blockingSubscribe(@NonNull Consumer onNext) { /** * Subscribes to the source and calls the given callbacks on the current thread. *

- * If the Flowable emits an error, it is wrapped into an + * If the {@code Flowable} emits an error, it is wrapped into an * {@link io.reactivex.rxjava3.exceptions.OnErrorNotImplementedException OnErrorNotImplementedException} - * and routed to the RxJavaPlugins.onError handler. + * and routed to the {@link RxJavaPlugins#onError(Throwable)} handler. * Using the overloads {@link #blockingSubscribe(Consumer, Consumer)} * or {@link #blockingSubscribe(Consumer, Consumer, Action)} instead is recommended. *

@@ -6241,17 +6241,17 @@ public final void blockingSubscribe(@NonNull Subscriber subscriber) { } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits connected, non-overlapping buffers, each containing {@code count} items. When the source - * Publisher completes, the resulting Publisher emits the current buffer and propagates the notification from the - * source Publisher. Note that if the source Publisher issues an onError notification the event is passed on + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits connected, non-overlapping buffers, each containing {@code count} items. When the source + * {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer and propagates the notification from the + * source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} notification the event is passed on * immediately without first emitting the buffer it is in the process of assembling. *

* *

*
Backpressure:
*
The operator honors backpressure from downstream and expects the source {@code Publisher} to honor it as - * well, although not enforced; violation may lead to {@code MissingBackpressureException} somewhere + * well, although not enforced; violation may lead to {@link MissingBackpressureException} somewhere * downstream.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
@@ -6259,8 +6259,8 @@ public final void blockingSubscribe(@NonNull Subscriber subscriber) { * * @param count * the maximum number of items in each buffer before it should be emitted - * @return a Flowable that emits connected, non-overlapping buffers, each containing at most - * {@code count} items from the source Publisher + * @return a {@code Flowable} that emits connected, non-overlapping buffers, each containing at most + * {@code count} items from the source {@code Publisher} * @see ReactiveX operators documentation: Buffer */ @CheckReturnValue @@ -6272,17 +6272,17 @@ public final Flowable> buffer(int count) { } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits buffers every {@code skip} items, each containing {@code count} items. When the source - * Publisher completes, the resulting Publisher emits the current buffer and propagates the notification from the - * source Publisher. Note that if the source Publisher issues an onError notification the event is passed on + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits buffers every {@code skip} items, each containing {@code count} items. When the source + * {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer and propagates the notification from the + * source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} notification the event is passed on * immediately without first emitting the buffer it is in the process of assembling. *

* *

*
Backpressure:
*
The operator honors backpressure from downstream and expects the source {@code Publisher} to honor it as - * well, although not enforced; violation may lead to {@code MissingBackpressureException} somewhere + * well, although not enforced; violation may lead to {@link MissingBackpressureException} somewhere * downstream.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
@@ -6291,10 +6291,10 @@ public final Flowable> buffer(int count) { * @param count * the maximum size of each buffer before it should be emitted * @param skip - * how many items emitted by the source Publisher should be skipped before starting a new + * how many items emitted by the source {@code Publisher} should be skipped before starting a new * buffer. Note that when {@code skip} and {@code count} are equal, this is the same operation as * {@link #buffer(int)}. - * @return a Flowable that emits buffers for every {@code skip} item from the source Publisher and + * @return a {@code Flowable} that emits buffers for every {@code skip} item from the source {@code Publisher} and * containing at most {@code count} items * @see ReactiveX operators documentation: Buffer */ @@ -6307,17 +6307,17 @@ public final Flowable> buffer(int count, int skip) { } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits buffers every {@code skip} items, each containing {@code count} items. When the source - * Publisher completes, the resulting Publisher emits the current buffer and propagates the notification from the - * source Publisher. Note that if the source Publisher issues an onError notification the event is passed on + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits buffers every {@code skip} items, each containing {@code count} items. When the source + * {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer and propagates the notification from the + * source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} notification the event is passed on * immediately without first emitting the buffer it is in the process of assembling. *

* *

*
Backpressure:
*
The operator honors backpressure from downstream and expects the source {@code Publisher} to honor it as - * well, although not enforced; violation may lead to {@code MissingBackpressureException} somewhere + * well, although not enforced; violation may lead to {@link MissingBackpressureException} somewhere * downstream.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
@@ -6327,13 +6327,13 @@ public final Flowable> buffer(int count, int skip) { * @param count * the maximum size of each buffer before it should be emitted * @param skip - * how many items emitted by the source Publisher should be skipped before starting a new + * how many items emitted by the source {@code Publisher} should be skipped before starting a new * buffer. Note that when {@code skip} and {@code count} are equal, this is the same operation as * {@link #buffer(int)}. * @param bufferSupplier * a factory function that returns an instance of the collection subclass to be used and returned * as the buffer - * @return a Flowable that emits buffers for every {@code skip} item from the source Publisher and + * @return a {@code Flowable} that emits buffers for every {@code skip} item from the source {@code Publisher} and * containing at most {@code count} items * @see ReactiveX operators documentation: Buffer */ @@ -6349,17 +6349,17 @@ public final > Flowable buffer(int count, int } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits connected, non-overlapping buffers, each containing {@code count} items. When the source - * Publisher completes, the resulting Publisher emits the current buffer and propagates the notification from the - * source Publisher. Note that if the source Publisher issues an onError notification the event is passed on + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits connected, non-overlapping buffers, each containing {@code count} items. When the source + * {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer and propagates the notification from the + * source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} notification the event is passed on * immediately without first emitting the buffer it is in the process of assembling. *

* *

*
Backpressure:
*
The operator honors backpressure from downstream and expects the source {@code Publisher} to honor it as - * well, although not enforced; violation may lead to {@code MissingBackpressureException} somewhere + * well, although not enforced; violation may lead to {@link MissingBackpressureException} somewhere * downstream.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
@@ -6371,8 +6371,8 @@ public final > Flowable buffer(int count, int * @param bufferSupplier * a factory function that returns an instance of the collection subclass to be used and returned * as the buffer - * @return a Flowable that emits connected, non-overlapping buffers, each containing at most - * {@code count} items from the source Publisher + * @return a {@code Flowable} that emits connected, non-overlapping buffers, each containing at most + * {@code count} items from the source {@code Publisher} * @see ReactiveX operators documentation: Buffer */ @CheckReturnValue @@ -6384,11 +6384,11 @@ public final > Flowable buffer(int count, @No } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher starts a new buffer periodically, as determined by the {@code timeskip} argument. It emits + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} starts a new buffer periodically, as determined by the {@code timeskip} argument. It emits * each buffer after a fixed timespan, specified by the {@code timespan} argument. When the source - * Publisher completes, the resulting Publisher emits the current buffer and propagates the notification from the - * source Publisher. Note that if the source Publisher issues an onError notification the event is passed on + * {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer and propagates the notification from the + * source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} notification the event is passed on * immediately without first emitting the buffer it is in the process of assembling. *

* @@ -6406,7 +6406,7 @@ public final > Flowable buffer(int count, @No * the period of time after which a new buffer will be created * @param unit * the unit of time that applies to the {@code timespan} and {@code timeskip} arguments - * @return a Flowable that emits new buffers of items emitted by the source Publisher periodically after + * @return a {@code Flowable} that emits new buffers of items emitted by the source {@code Publisher} periodically after * a fixed timespan has elapsed * @see ReactiveX operators documentation: Buffer */ @@ -6419,11 +6419,11 @@ public final Flowable> buffer(long timespan, long timeskip, @NonNull Tim } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher starts a new buffer periodically, as determined by the {@code timeskip} argument, and on the + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} starts a new buffer periodically, as determined by the {@code timeskip} argument, and on the * specified {@code scheduler}. It emits each buffer after a fixed timespan, specified by the - * {@code timespan} argument. When the source Publisher completes, the resulting Publisher emits the current buffer - * and propagates the notification from the source Publisher. Note that if the source Publisher issues an onError + * {@code timespan} argument. When the source {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer + * and propagates the notification from the source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} * notification the event is passed on immediately without first emitting the buffer it is in the process of * assembling. *

@@ -6443,8 +6443,8 @@ public final Flowable> buffer(long timespan, long timeskip, @NonNull Tim * @param unit * the unit of time that applies to the {@code timespan} and {@code timeskip} arguments * @param scheduler - * the {@link Scheduler} to use when determining the end and start of a buffer - * @return a Flowable that emits new buffers of items emitted by the source Publisher periodically after + * the {@code Scheduler} to use when determining the end and start of a buffer + * @return a {@code Flowable} that emits new buffers of items emitted by the source {@code Publisher} periodically after * a fixed timespan has elapsed * @see ReactiveX operators documentation: Buffer */ @@ -6457,11 +6457,11 @@ public final Flowable> buffer(long timespan, long timeskip, @NonNull Tim } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher starts a new buffer periodically, as determined by the {@code timeskip} argument, and on the + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} starts a new buffer periodically, as determined by the {@code timeskip} argument, and on the * specified {@code scheduler}. It emits each buffer after a fixed timespan, specified by the - * {@code timespan} argument. When the source Publisher completes, the resulting Publisher emits the current buffer - * and propagates the notification from the source Publisher. Note that if the source Publisher issues an onError + * {@code timespan} argument. When the source {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer + * and propagates the notification from the source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} * notification the event is passed on immediately without first emitting the buffer it is in the process of * assembling. *

@@ -6482,11 +6482,11 @@ public final Flowable> buffer(long timespan, long timeskip, @NonNull Tim * @param unit * the unit of time that applies to the {@code timespan} and {@code timeskip} arguments * @param scheduler - * the {@link Scheduler} to use when determining the end and start of a buffer + * the {@code Scheduler} to use when determining the end and start of a buffer * @param bufferSupplier * a factory function that returns an instance of the collection subclass to be used and returned * as the buffer - * @return a Flowable that emits new buffers of items emitted by the source Publisher periodically after + * @return a {@code Flowable} that emits new buffers of items emitted by the source {@code Publisher} periodically after * a fixed timespan has elapsed * @see ReactiveX operators documentation: Buffer */ @@ -6503,10 +6503,10 @@ public final > Flowable buffer(long timespan, } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits connected, non-overlapping buffers, each of a fixed duration specified by the - * {@code timespan} argument. When the source Publisher completes, the resulting Publisher emits the current buffer - * and propagates the notification from the source Publisher. Note that if the source Publisher issues an onError + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits connected, non-overlapping buffers, each of a fixed duration specified by the + * {@code timespan} argument. When the source {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer + * and propagates the notification from the source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} * notification the event is passed on immediately without first emitting the buffer it is in the process of * assembling. *

@@ -6524,8 +6524,8 @@ public final > Flowable buffer(long timespan, * buffer * @param unit * the unit of time that applies to the {@code timespan} argument - * @return a Flowable that emits connected, non-overlapping buffers of items emitted by the source - * Publisher within a fixed duration + * @return a {@code Flowable} that emits connected, non-overlapping buffers of items emitted by the source + * {@code Publisher} within a fixed duration * @see ReactiveX operators documentation: Buffer */ @CheckReturnValue @@ -6537,11 +6537,11 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit) { } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits connected, non-overlapping buffers, each of a fixed duration specified by the + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits connected, non-overlapping buffers, each of a fixed duration specified by the * {@code timespan} argument or a maximum size specified by the {@code count} argument (whichever is reached - * first). When the source Publisher completes, the resulting Publisher emits the current buffer and propagates the - * notification from the source Publisher. Note that if the source Publisher issues an onError notification the event + * first). When the source {@code Publisher} completes, the resulting {@code Publisher} emits the current buffer and propagates the + * notification from the source {@code Publisher}. Note that if the source {@code Publisher} issues an {@code onError} notification the event * is passed on immediately without first emitting the buffer it is in the process of assembling. *

* @@ -6560,8 +6560,8 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit) { * the unit of time which applies to the {@code timespan} argument * @param count * the maximum size of each buffer before it is emitted - * @return a Flowable that emits connected, non-overlapping buffers of items emitted by the source - * Publisher, after a fixed duration or when the buffer reaches maximum capacity (whichever occurs + * @return a {@code Flowable} that emits connected, non-overlapping buffers of items emitted by the source + * {@code Publisher}, after a fixed duration or when the buffer reaches maximum capacity (whichever occurs * first) * @see ReactiveX operators documentation: Buffer */ @@ -6574,12 +6574,12 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, int } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits connected, non-overlapping buffers, each of a fixed duration specified by the + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits connected, non-overlapping buffers, each of a fixed duration specified by the * {@code timespan} argument as measured on the specified {@code scheduler}, or a maximum size specified by - * the {@code count} argument (whichever is reached first). When the source Publisher completes, the resulting - * Publisher emits the current buffer and propagates the notification from the source Publisher. Note that if the - * source Publisher issues an onError notification the event is passed on immediately without first emitting the + * the {@code count} argument (whichever is reached first). When the source {@code Publisher} completes, the resulting + * {@code Publisher} emits the current buffer and propagates the notification from the source {@code Publisher}. Note that if the + * source {@code Publisher} issues an {@code onError} notification the event is passed on immediately without first emitting the * buffer it is in the process of assembling. *

* @@ -6597,11 +6597,11 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, int * @param unit * the unit of time which applies to the {@code timespan} argument * @param scheduler - * the {@link Scheduler} to use when determining the end and start of a buffer + * the {@code Scheduler} to use when determining the end and start of a buffer * @param count * the maximum size of each buffer before it is emitted - * @return a Flowable that emits connected, non-overlapping buffers of items emitted by the source - * Publisher after a fixed duration or when the buffer reaches maximum capacity (whichever occurs + * @return a {@code Flowable} that emits connected, non-overlapping buffers of items emitted by the source + * {@code Publisher} after a fixed duration or when the buffer reaches maximum capacity (whichever occurs * first) * @see ReactiveX operators documentation: Buffer */ @@ -6614,12 +6614,12 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @No } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits connected, non-overlapping buffers, each of a fixed duration specified by the + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits connected, non-overlapping buffers, each of a fixed duration specified by the * {@code timespan} argument as measured on the specified {@code scheduler}, or a maximum size specified by - * the {@code count} argument (whichever is reached first). When the source Publisher completes, the resulting - * Publisher emits the current buffer and propagates the notification from the source Publisher. Note that if the - * source Publisher issues an onError notification the event is passed on immediately without first emitting the + * the {@code count} argument (whichever is reached first). When the source {@code Publisher} completes, the resulting + * {@code Publisher} emits the current buffer and propagates the notification from the source {@code Publisher}. Note that if the + * source {@code Publisher} issues an {@code onError} notification the event is passed on immediately without first emitting the * buffer it is in the process of assembling. *

* @@ -6638,16 +6638,16 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @No * @param unit * the unit of time which applies to the {@code timespan} argument * @param scheduler - * the {@link Scheduler} to use when determining the end and start of a buffer + * the {@code Scheduler} to use when determining the end and start of a buffer * @param count * the maximum size of each buffer before it is emitted * @param bufferSupplier * a factory function that returns an instance of the collection subclass to be used and returned * as the buffer - * @param restartTimerOnMaxSize if true the time window is restarted when the max capacity of the current buffer + * @param restartTimerOnMaxSize if {@code true}, the time window is restarted when the max capacity of the current buffer * is reached - * @return a Flowable that emits connected, non-overlapping buffers of items emitted by the source - * Publisher after a fixed duration or when the buffer reaches maximum capacity (whichever occurs + * @return a {@code Flowable} that emits connected, non-overlapping buffers of items emitted by the source + * {@code Publisher} after a fixed duration or when the buffer reaches maximum capacity (whichever occurs * first) * @see ReactiveX operators documentation: Buffer */ @@ -6668,11 +6668,11 @@ public final > Flowable buffer( } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits connected, non-overlapping buffers, each of a fixed duration specified by the - * {@code timespan} argument and on the specified {@code scheduler}. When the source Publisher completes, the - * resulting Publisher emits the current buffer and propagates the notification from the source Publisher. Note that - * if the source Publisher issues an onError notification the event is passed on immediately without first emitting + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits connected, non-overlapping buffers, each of a fixed duration specified by the + * {@code timespan} argument and on the specified {@code scheduler}. When the source {@code Publisher} completes, the + * resulting {@code Publisher} emits the current buffer and propagates the notification from the source {@code Publisher}. Note that + * if the source {@code Publisher} issues an {@code onError} notification the event is passed on immediately without first emitting * the buffer it is in the process of assembling. *

* @@ -6690,9 +6690,9 @@ public final > Flowable buffer( * @param unit * the unit of time which applies to the {@code timespan} argument * @param scheduler - * the {@link Scheduler} to use when determining the end and start of a buffer - * @return a Flowable that emits connected, non-overlapping buffers of items emitted by the source - * Publisher within a fixed duration + * the {@code Scheduler} to use when determining the end and start of a buffer + * @return a {@code Flowable} that emits connected, non-overlapping buffers of items emitted by the source + * {@code Publisher} within a fixed duration * @see ReactiveX operators documentation: Buffer */ @CheckReturnValue @@ -6704,30 +6704,30 @@ public final Flowable> buffer(long timespan, @NonNull TimeUnit unit, @No } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits buffers that it creates when the specified {@code openingIndicator} Publisher emits an - * item, and closes when the Publisher returned from {@code closingIndicator} emits an item. If any of the source - * Publisher, {@code openingIndicator} or {@code closingIndicator} issues an onError notification the event is passed + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits buffers that it creates when the specified {@code openingIndicator} {@code Publisher} emits an + * item, and closes when the {@code Publisher} returned from {@code closingIndicator} emits an item. If any of the source + * {@code Publisher}, {@code openingIndicator} or {@code closingIndicator} issues an {@code onError} notification the event is passed * on immediately without first emitting the buffer it is in the process of assembling. *

* *

*
Backpressure:
- *
This operator does not support backpressure as it is instead controlled by the given Publishers and + *
This operator does not support backpressure as it is instead controlled by the given {@code Publisher}s and * buffers data. It requests {@link Long#MAX_VALUE} upstream and does not obey downstream requests.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the element type of the buffer-opening Publisher - * @param the element type of the individual buffer-closing Publishers + * @param the element type of the buffer-opening {@code Publisher} + * @param the element type of the individual buffer-closing {@code Publisher}s * @param openingIndicator - * the Publisher that, when it emits an item, causes a new buffer to be created + * the {@code Publisher} that, when it emits an item, causes a new buffer to be created * @param closingIndicator - * the {@link Function} that is used to produce a Publisher for every buffer created. When this - * Publisher emits an item, the associated buffer is emitted. - * @return a Flowable that emits buffers, containing items from the source Publisher, that are created - * and closed when the specified Publishers emit items + * the {@link Function} that is used to produce a {@code Publisher} for every buffer created. When this + * {@code Publisher} emits an item, the associated buffer is emitted. + * @return a {@code Flowable} that emits buffers, containing items from the source {@code Publisher}, that are created + * and closed when the specified {@code Publisher}s emit items * @see ReactiveX operators documentation: Buffer */ @CheckReturnValue @@ -6741,34 +6741,34 @@ public final Flowable> buffer( } /** - * Returns a Flowable that emits buffers of items it collects from the source Publisher. The resulting - * Publisher emits buffers that it creates when the specified {@code openingIndicator} Publisher emits an - * item, and closes when the Publisher returned from {@code closingIndicator} emits an item. If any of the source - * Publisher, {@code openingIndicator} or {@code closingIndicator} issues an onError notification the event is passed + * Returns a {@code Flowable} that emits buffers of items it collects from the source {@link Publisher}. The resulting + * {@code Publisher} emits buffers that it creates when the specified {@code openingIndicator} {@code Publisher} emits an + * item, and closes when the {@code Publisher} returned from {@code closingIndicator} emits an item. If any of the source + * {@code Publisher}, {@code openingIndicator} or {@code closingIndicator} issues an {@code onError} notification the event is passed * on immediately without first emitting the buffer it is in the process of assembling. *

* *

*
Backpressure:
- *
This operator does not support backpressure as it is instead controlled by the given Publishers and + *
This operator does not support backpressure as it is instead controlled by the given {@code Publisher}s and * buffers data. It requests {@link Long#MAX_VALUE} upstream and does not obey downstream requests.
*
Scheduler:
*
This version of {@code buffer} does not operate by default on a particular {@link Scheduler}.
*
* * @param the collection subclass type to buffer into - * @param the element type of the buffer-opening Publisher - * @param the element type of the individual buffer-closing Publishers + * @param the element type of the buffer-opening {@code Publisher} + * @param the element type of the individual buffer-closing {@code Publisher}s * @param openingIndicator - * the Publisher that, when it emits an item, causes a new buffer to be created + * the {@code Publisher} that, when it emits an item, causes a new buffer to be created * @param closingIndicator - * the {@link Function} that is used to produce a Publisher for every buffer created. When this - * Publisher emits an item, the associated buffer is emitted. + * the {@link Function} that is used to produce a {@code Publisher} for every buffer created. When this + * {@code Publisher} emits an item, the associated buffer is emitted. * @param bufferSupplier * a factory function that returns an instance of the collection subclass to be used and returned * as the buffer - * @return a Flowable that emits buffers, containing items from the source Publisher, that are created - * and closed when the specified Publishers emit items + * @return a {@code Flowable} that emits buffers, containing items from the source {@code Publisher}, that are created + * and closed when the specified {@code Publisher}s emit items * @see ReactiveX operators documentation: Buffer */ @CheckReturnValue @@ -6786,13 +6786,13 @@ public final > Flowable b } /** - * Returns a Flowable that emits non-overlapping buffered items from the source Publisher each time the - * specified boundary Publisher emits an item. + * Returns a {@code Flowable} that emits non-overlapping buffered items from the source {@link Publisher} each time the + * specified boundary {@code Publisher} emits an item. *

* *

- * Completion of either the source or the boundary Publisher causes the returned Publisher to emit the - * latest buffer and complete. If either the source Publisher or the boundary Publisher issues an onError notification + * Completion of either the source or the boundary {@code Publisher} causes the returned {@code Publisher} to emit the + * latest buffer and complete. If either the source {@code Publisher} or the boundary {@code Publisher} issues an {@code onError} notification * the event is passed on immediately without first emitting the buffer it is in the process of assembling. *

*
Backpressure:
@@ -6806,8 +6806,8 @@ public final > Flowable b * @param * the boundary value type (ignored) * @param boundaryIndicator - * the boundary Publisher - * @return a Flowable that emits buffered items from the source Publisher when the boundary Publisher + * the boundary {@code Publisher} + * @return a {@code Flowable} that emits buffered items from the source {@code Publisher} when the boundary {@code Publisher} * emits an item * @see #buffer(Publisher, int) * @see ReactiveX operators documentation: Buffer @@ -6821,13 +6821,13 @@ public final Flowable> buffer(@NonNull Publisher boundaryIndicato } /** - * Returns a Flowable that emits non-overlapping buffered items from the source Publisher each time the - * specified boundary Publisher emits an item. + * Returns a {@code Flowable} that emits non-overlapping buffered items from the source {@link Publisher} each time the + * specified boundary {@code Publisher} emits an item. *

* *

- * Completion of either the source or the boundary Publisher causes the returned Publisher to emit the - * latest buffer and complete. If either the source Publisher or the boundary Publisher issues an onError notification + * Completion of either the source or the boundary {@code Publisher} causes the returned {@code Publisher} to emit the + * latest buffer and complete. If either the source {@code Publisher} or the boundary {@code Publisher} issues an {@code onError} notification * the event is passed on immediately without first emitting the buffer it is in the process of assembling. *

*
Backpressure:
@@ -6841,10 +6841,10 @@ public final Flowable> buffer(@NonNull Publisher boundaryIndicato * @param * the boundary value type (ignored) * @param boundaryIndicator - * the boundary Publisher + * the boundary {@code Publisher} * @param initialCapacity * the initial capacity of each buffer chunk - * @return a Flowable that emits buffered items from the source Publisher when the boundary Publisher + * @return a {@code Flowable} that emits buffered items from the source {@code Publisher} when the boundary {@code Publisher} * emits an item * @see ReactiveX operators documentation: Buffer * @see #buffer(Publisher) @@ -6859,13 +6859,13 @@ public final Flowable> buffer(@NonNull Publisher boundaryIndicato } /** - * Returns a Flowable that emits non-overlapping buffered items from the source Publisher each time the - * specified boundary Publisher emits an item. + * Returns a {@code Flowable} that emits non-overlapping buffered items from the source {@link Publisher} each time the + * specified boundary {@code Publisher} emits an item. *

* *

- * Completion of either the source or the boundary Publisher causes the returned Publisher to emit the - * latest buffer and complete. If either the source Publisher or the boundary Publisher issues an onError notification + * Completion of either the source or the boundary {@code Publisher} causes the returned {@code Publisher} to emit the + * latest buffer and complete. If either the source {@code Publisher} or the boundary {@code Publisher} issues an {@code onError} notification * the event is passed on immediately without first emitting the buffer it is in the process of assembling. *

*
Backpressure:
@@ -6880,11 +6880,11 @@ public final Flowable> buffer(@NonNull Publisher boundaryIndicato * @param * the boundary value type (ignored) * @param boundaryIndicator - * the boundary Publisher + * the boundary {@code Publisher} * @param bufferSupplier * a factory function that returns an instance of the collection subclass to be used and returned * as the buffer - * @return a Flowable that emits buffered items from the source Publisher when the boundary Publisher + * @return a {@code Flowable} that emits buffered items from the source {@code Publisher} when the boundary {@code Publisher} * emits an item * @see #buffer(Publisher, int) * @see ReactiveX operators documentation: Buffer @@ -6900,23 +6900,23 @@ public final > Flowable buffer(@NonNull Pu } /** - * Returns a Flowable that subscribes to this Publisher lazily, caches all of its events + * Returns a {@code Flowable} that subscribes to this {@link Publisher} lazily, caches all of its events * and replays them, in the same order as received, to all the downstream subscribers. *

* *

- * This is useful when you want a Publisher to cache responses and you can't control the + * This is useful when you want a {@code Publisher} to cache responses and you can't control the * subscribe/cancel behavior of all the {@link Subscriber}s. *

* The operator subscribes only when the first downstream subscriber subscribes and maintains - * a single subscription towards this Publisher. In contrast, the operator family of {@link #replay()} + * a single subscription towards this {@code Publisher}. In contrast, the operator family of {@link #replay()} * that return a {@link ConnectableFlowable} require an explicit call to {@link ConnectableFlowable#connect()}. *

* Note: You sacrifice the ability to cancel the origin when you use the {@code cache} - * Subscriber so be careful not to use this Subscriber on Publishers that emit an infinite or very large number + * {@code Subscriber} so be careful not to use this {@code Subscriber} on {@code Publisher}s that emit an infinite or very large number * of items that will use up memory. - * A possible workaround is to apply `takeUntil` with a predicate or - * another source before (and perhaps after) the application of cache(). + * A possible workaround is to apply {@link #takeUntil(Publisher)} with a predicate or + * another source before (and perhaps after) the application of {@code cache()}. *


      * AtomicBoolean shouldStop = new AtomicBoolean();
      *
@@ -6940,13 +6940,13 @@ public final > Flowable buffer(@NonNull Pu
      * 
*
*
Backpressure:
- *
The operator consumes this Publisher in an unbounded fashion but respects the backpressure - * of each downstream Subscriber individually.
+ *
The operator consumes this {@code Publisher} in an unbounded fashion but respects the backpressure + * of each downstream {@code Subscriber} individually.
*
Scheduler:
*
{@code cache} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a Flowable that, when first subscribed to, caches all of its items and notifications for the + * @return a {@code Flowable} that, when first subscribed to, caches all of its items and notifications for the * benefit of subsequent subscribers * @see ReactiveX operators documentation: Replay */ @@ -6959,23 +6959,23 @@ public final Flowable cache() { } /** - * Returns a Flowable that subscribes to this Publisher lazily, caches all of its events + * Returns a {@code Flowable} that subscribes to this {@link Publisher} lazily, caches all of its events * and replays them, in the same order as received, to all the downstream subscribers. *

* *

- * This is useful when you want a Publisher to cache responses and you can't control the + * This is useful when you want a {@code Publisher} to cache responses and you can't control the * subscribe/cancel behavior of all the {@link Subscriber}s. *

* The operator subscribes only when the first downstream subscriber subscribes and maintains - * a single subscription towards this Publisher. In contrast, the operator family of {@link #replay()} + * a single subscription towards this {@code Publisher}. In contrast, the operator family of {@link #replay()} * that return a {@link ConnectableFlowable} require an explicit call to {@link ConnectableFlowable#connect()}. *

* Note: You sacrifice the ability to cancel the origin when you use the {@code cache} - * Subscriber so be careful not to use this Subscriber on Publishers that emit an infinite or very large number + * {@code Subscriber} so be careful not to use this {@code Subscriber} on {@code Publisher}s that emit an infinite or very large number * of items that will use up memory. - * A possible workaround is to apply `takeUntil` with a predicate or - * another source before (and perhaps after) the application of cache(). + * A possible workaround is to apply {@link #takeUntil(Publisher)} with a predicate or + * another source before (and perhaps after) the application of {@code cacheWithInitialCapacity()}. *


      * AtomicBoolean shouldStop = new AtomicBoolean();
      *
@@ -6999,8 +6999,8 @@ public final Flowable cache() {
      * 
*
*
Backpressure:
- *
The operator consumes this Publisher in an unbounded fashion but respects the backpressure - * of each downstream Subscriber individually.
+ *
The operator consumes this {@code Publisher} in an unbounded fashion but respects the backpressure + * of each downstream {@code Subscriber} individually.
*
Scheduler:
*
{@code cacheWithInitialCapacity} does not operate by default on a particular {@link Scheduler}.
*
@@ -7009,7 +7009,7 @@ public final Flowable cache() { * {@link #replay(int)} in combination with {@link ConnectableFlowable#autoConnect()} or similar. * * @param initialCapacity hint for number of items to cache (for optimizing underlying data structure) - * @return a Flowable that, when first subscribed to, caches all of its items and notifications for the + * @return a {@code Flowable} that, when first subscribed to, caches all of its items and notifications for the * benefit of subsequent subscribers * @see ReactiveX operators documentation: Replay */ @@ -7023,7 +7023,7 @@ public final Flowable cacheWithInitialCapacity(int initialCapacity) { } /** - * Returns a Flowable that emits the items emitted by the source Publisher, converted to the specified + * Returns a {@code Flowable} that emits the items emitted by the source {@link Publisher}, converted to the specified * type. *

* @@ -7037,9 +7037,9 @@ public final Flowable cacheWithInitialCapacity(int initialCapacity) { * * @param the output value type cast to * @param clazz - * the target class type that {@code cast} will cast the items emitted by the source Publisher - * into before emitting them from the resulting Publisher - * @return a Flowable that emits each item from the source Publisher after converting it to the + * the target class type that {@code cast} will cast the items emitted by the source {@code Publisher} + * into before emitting them from the resulting {@code Publisher} + * @return a {@code Flowable} that emits each item from the source {@code Publisher} after converting it to the * specified type * @see ReactiveX operators documentation: Map */ @@ -7053,8 +7053,8 @@ public final Flowable cast(@NonNull Class clazz) { } /** - * Collects items emitted by the finite source Publisher into a single mutable data structure and returns - * a Single that emits this structure. + * Collects items emitted by the finite source {@link Publisher} into a single mutable data structure and returns + * a {@link Single} that emits this structure. *

* *

@@ -7062,7 +7062,7 @@ public final Flowable cast(@NonNull Class clazz) { *

* Note that this operator requires the upstream to signal {@code onComplete} for the accumulator object to * be emitted. Sources that are infinite and never complete will never emit anything through this - * operator and an infinite source may lead to a fatal {@code OutOfMemoryError}. + * operator and an infinite source may lead to a fatal {@link OutOfMemoryError}. *

*
Backpressure:
*
This operator does not support backpressure because by intent it will receive all values and reduce @@ -7077,7 +7077,7 @@ public final Flowable cast(@NonNull Class clazz) { * @param collector * a function that accepts the {@code state} and an emitted item, and modifies {@code state} * accordingly - * @return a Single that emits the result of collecting the values emitted by the source Publisher + * @return a {@code Single} that emits the result of collecting the values emitted by the source {@code Publisher} * into a single mutable data structure * @see ReactiveX operators documentation: Reduce * @see #collect(Collector) @@ -7093,8 +7093,8 @@ public final Single collect(@NonNull Supplier initialItemSup } /** - * Collects items emitted by the finite source Publisher into a single mutable data structure and returns - * a Single that emits this structure. + * Collects items emitted by the finite source {@link Publisher} into a single mutable data structure and returns + * a {@link Single} that emits this structure. *

* *

@@ -7102,7 +7102,7 @@ public final Single collect(@NonNull Supplier initialItemSup *

* Note that this operator requires the upstream to signal {@code onComplete} for the accumulator object to * be emitted. Sources that are infinite and never complete will never emit anything through this - * operator and an infinite source may lead to a fatal {@code OutOfMemoryError}. + * operator and an infinite source may lead to a fatal {@link OutOfMemoryError}. *

*
Backpressure:
*
This operator does not support backpressure because by intent it will receive all values and reduce @@ -7117,7 +7117,7 @@ public final Single collect(@NonNull Supplier initialItemSup * @param collector * a function that accepts the {@code state} and an emitted item, and modifies {@code state} * accordingly - * @return a Single that emits the result of collecting the values emitted by the source Publisher + * @return a {@code Single} that emits the result of collecting the values emitted by the source {@code Publisher} * into a single mutable data structure * @see ReactiveX operators documentation: Reduce */ @@ -7131,13 +7131,13 @@ public final Single collect(@NonNull Supplier initialItemSup } /** - * Transform a Publisher by applying a particular Transformer function to it. + * Transform a {@link Publisher} by applying a particular Transformer function to it. *

- * This method operates on the Publisher itself whereas {@link #lift} operates on the Publisher's - * Subscribers or Subscribers. + * This method operates on the {@code Publisher} itself whereas {@link #lift} operates on the {@code Publisher}'s + * {@link Subscriber}s. *

* If the operator you are creating is designed to act on the individual items emitted by a source - * Publisher, use {@link #lift}. If your operator is designed to transform the source Publisher as a whole + * {@code Publisher}, use {@link #lift}. If your operator is designed to transform the source {@code Publisher} as a whole * (for instance, by applying a particular set of existing RxJava operators to it) use {@code compose}. *

*
Backpressure:
@@ -7147,9 +7147,9 @@ public final Single collect(@NonNull Supplier initialItemSup *
{@code compose} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the output Publisher - * @param composer implements the function that transforms the source Publisher - * @return the source Publisher, transformed by the transformer function + * @param the value type of the output {@code Publisher} + * @param composer implements the function that transforms the source {@code Publisher} + * @return the source {@code Publisher}, transformed by the transformer function * @see RxJava wiki: Implementing Your Own Operators */ @SuppressWarnings("unchecked") @@ -7162,9 +7162,9 @@ public final Flowable compose(@NonNull FlowableTransformer * *

@@ -7175,19 +7175,19 @@ public final Flowable compose(@NonNull FlowableTransformerBackpressure: *

The operator honors backpressure from downstream. Both this and the inner {@code Publisher}s are * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor - * backpressure, that may throw an {@code IllegalStateException} when that + * signal a {@link MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor + * backpressure, that may throw an {@link IllegalStateException} when that * {@code Publisher} completes.
*
Scheduler:
*
{@code concatMap} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the type of the inner Publisher sources and thus the output type + * @param the type of the inner {@code Publisher} sources and thus the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and concatenating the Publishers obtained from this transformation + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and concatenating the {@code Publisher}s obtained from this transformation * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -7199,9 +7199,9 @@ public final Flowable concatMap(@NonNull Function * *

@@ -7212,21 +7212,21 @@ public final Flowable concatMap(@NonNull FunctionBackpressure: *

The operator honors backpressure from downstream. Both this and the inner {@code Publisher}s are * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor - * backpressure, that may throw an {@code IllegalStateException} when that + * signal a {@link MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor + * backpressure, that may throw an {@link IllegalStateException} when that * {@code Publisher} completes.
*
Scheduler:
*
{@code concatMap} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the type of the inner Publisher sources and thus the output type + * @param the type of the inner {@code Publisher} sources and thus the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} * @param prefetch - * the number of elements to prefetch from the current Flowable - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and concatenating the Publishers obtained from this transformation + * the number of elements to prefetch from the current {@code Flowable} + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and concatenating the {@code Publisher}s obtained from this transformation * @see ReactiveX operators documentation: FlatMap * @see #concatMap(Function, int, Scheduler) */ @@ -7249,9 +7249,9 @@ public final Flowable concatMap(@NonNull Function * *

@@ -7261,23 +7261,23 @@ public final Flowable concatMap(@NonNull FunctionBackpressure: *

The operator honors backpressure from downstream. Both this and the inner {@code Publisher}s are * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor - * backpressure, that may throw an {@code IllegalStateException} when that + * signal a {@link MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor + * backpressure, that may throw an {@link IllegalStateException} when that * {@code Publisher} completes.
*
Scheduler:
*
{@code concatMap} executes the given {@code mapper} function on the provided {@link Scheduler}.
*
* - * @param the type of the inner Publisher sources and thus the output type + * @param the type of the inner {@code Publisher} sources and thus the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} * @param prefetch - * the number of elements to prefetch from the current Flowable + * the number of elements to prefetch from the current {@code Flowable} * @param scheduler * the scheduler where the {@code mapper} function will be executed - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and concatenating the Publishers obtained from this transformation + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and concatenating the {@code Publisher}s obtained from this transformation * @see ReactiveX operators documentation: FlatMap * @since 3.0.0 * @see #concatMap(Function, int) @@ -7302,7 +7302,7 @@ public final Flowable concatMap(@NonNull Function *
Backpressure:
*
The operator expects the upstream to support backpressure. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapCompletable} does not operate by default on a particular {@link Scheduler}.
*
@@ -7310,7 +7310,7 @@ public final Flowable concatMap(@NonNull Function *
Backpressure:
*
The operator expects the upstream to support backpressure. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapCompletable} does not operate by default on a particular {@link Scheduler}.
*
@@ -7342,7 +7342,7 @@ public final Completable concatMapCompletable(@NonNull Function *
Backpressure:
*
The operator expects the upstream to support backpressure. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapCompletableDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -7373,7 +7373,7 @@ public final Completable concatMapCompletable(@NonNull Function *
Backpressure:
*
The operator expects the upstream to support backpressure. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapCompletableDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -7408,7 +7408,7 @@ public final Completable concatMapCompletableDelayError(@NonNull Function *
Backpressure:
*
The operator expects the upstream to support backpressure. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapCompletableDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -7447,7 +7447,7 @@ public final Completable concatMapCompletableDelayError(@NonNull Function * Note that there is no guarantee where the given {@code mapper} function will be executed; it could be on the subscribing thread, @@ -7474,16 +7474,16 @@ public final Completable concatMapCompletableDelayError(@NonNull FunctionBackpressure: *
The operator honors backpressure from downstream. Both this and the inner {@code Publisher}s are * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor - * backpressure, that may throw an {@code IllegalStateException} when that + * signal a {@link MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor + * backpressure, that may throw an {@link IllegalStateException} when that * {@code Publisher} completes.
*
Scheduler:
*
{@code concatMapDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the result value type - * @param mapper the function that maps the items of this Publisher into the inner Publishers. - * @return the new Publisher instance with the concatenation behavior + * @param mapper the function that maps the items of this {@code Publisher} into the inner {@code Publisher}s. + * @return the new {@code Publisher} instance with the concatenation behavior * @see #concatMapDelayError(Function, boolean, int, Scheduler) */ @CheckReturnValue @@ -7495,9 +7495,9 @@ public final Flowable concatMapDelayError(@NonNull Function * Note that there is no guarantee where the given {@code mapper} function will be executed; it could be on the subscribing thread, @@ -7508,21 +7508,21 @@ public final Flowable concatMapDelayError(@NonNull FunctionBackpressure: *
The operator honors backpressure from downstream. Both this and the inner {@code Publisher}s are * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor - * backpressure, that may throw an {@code IllegalStateException} when that + * signal a {@link MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor + * backpressure, that may throw an {@link IllegalStateException} when that * {@code Publisher} completes.
*
Scheduler:
*
{@code concatMapDelayError} does not operate by default on a particular {@link Scheduler}.
*
* * @param the result value type - * @param mapper the function that maps the items of this Publisher into the inner Publishers. + * @param mapper the function that maps the items of this {@code Publisher} into the inner {@code Publisher}s. * @param tillTheEnd - * if true, all errors from the outer and inner Publisher sources are delayed until the end, - * if false, an error from the main source is signaled when the current Publisher source terminates + * if {@code true}, all errors from the outer and inner {@code Publisher} sources are delayed until the end, + * if {@code false}, an error from the main source is signaled when the current {@code Publisher} source terminates * @param prefetch - * the number of elements to prefetch from the current Flowable - * @return the new Publisher instance with the concatenation behavior + * the number of elements to prefetch from the current {@code Flowable} + * @return the new {@code Publisher} instance with the concatenation behavior * @see #concatMapDelayError(Function, boolean, int, Scheduler) */ @CheckReturnValue @@ -7545,10 +7545,10 @@ public final Flowable concatMapDelayError(@NonNull Function * The difference between {@link #concatMapDelayError(Function, boolean, int)} and this operator is that this operator guarantees the {@code mapper} * function is executed on the specified scheduler. @@ -7557,23 +7557,23 @@ public final Flowable concatMapDelayError(@NonNull FunctionBackpressure: *
The operator honors backpressure from downstream. Both this and the inner {@code Publisher}s are * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor - * backpressure, that may throw an {@code IllegalStateException} when that + * signal a {@link MissingBackpressureException}. If any of the inner {@code Publisher}s doesn't honor + * backpressure, that may throw an {@link IllegalStateException} when that * {@code Publisher} completes.
*
Scheduler:
*
{@code concatMapDelayError} executes the given {@code mapper} function on the provided {@link Scheduler}.
*
* * @param the result value type - * @param mapper the function that maps the items of this Publisher into the inner Publishers. + * @param mapper the function that maps the items of this {@code Publisher} into the inner {@code Publisher}s. * @param tillTheEnd - * if true, all errors from the outer and inner Publisher sources are delayed until the end, - * if false, an error from the main source is signaled when the current Publisher source terminates + * if {@code true}, all errors from the outer and inner {@code Publisher} sources are delayed until the end, + * if {@code false}, an error from the main source is signaled when the current {@code Publisher} source terminates * @param prefetch - * the number of elements to prefetch from the current Flowable + * the number of elements to prefetch from the current {@code Flowable} * @param scheduler * the scheduler where the {@code mapper} function will be executed - * @return the new Publisher instance with the concatenation behavior + * @return the new {@code Publisher} instance with the concatenation behavior * @see #concatMapDelayError(Function, boolean, int) * @since 3.0.0 */ @@ -7590,11 +7590,11 @@ public final Flowable concatMapDelayError(@NonNull Function * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them in + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them in * order, each one after the previous one completes. *
*
Backpressure:
@@ -7604,9 +7604,9 @@ public final Flowable concatMapDelayError(@NonNull FunctionThis 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 Publishers that will be + * @param mapper the function that maps a sequence of values into a sequence of {@code Publisher}s that will be * eagerly concatenated - * @return the new Publisher instance with the specified concatenation behavior + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -7618,11 +7618,11 @@ public final Flowable concatMapEager(@NonNull Function * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them in + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them in * order, each one after the previous one completes. *
*
Backpressure:
@@ -7632,11 +7632,11 @@ public final Flowable concatMapEager(@NonNull FunctionThis 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 Publishers that will be + * @param mapper the function that maps a sequence of values into a sequence of {@code Publisher}s that will be * eagerly concatenated - * @param maxConcurrency the maximum number of concurrent subscribed Publishers - * @param prefetch hints about the number of expected values from each inner Publisher, must be positive - * @return the new Publisher instance with the specified concatenation behavior + * @param maxConcurrency the maximum number of concurrent subscribed {@code Publisher}s + * @param prefetch hints about the number of expected values from each inner {@code Publisher}, must be positive + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -7652,11 +7652,11 @@ public final Flowable concatMapEager(@NonNull Function * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them in + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them in * order, each one after the previous one completes. *
*
Backpressure:
@@ -7666,12 +7666,12 @@ public final Flowable concatMapEager(@NonNull FunctionThis 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 Publishers that will be + * @param mapper the function that maps a sequence of values into a sequence of {@code Publisher}s that will be * eagerly concatenated * @param tillTheEnd - * if true, all errors from the outer and inner Publisher sources are delayed until the end, - * if false, an error from the main source is signaled when the current Publisher source terminates - * @return the new Publisher instance with the specified concatenation behavior + * if {@code true}, all errors from the outer and inner {@code Publisher} sources are delayed until the end, + * if {@code false}, an error from the main source is signaled when the current {@code Publisher} source terminates + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -7684,11 +7684,11 @@ public final Flowable concatMapEagerDelayError(@NonNull Function * Eager concatenation means that once a subscriber subscribes, this operator subscribes to all of the - * source Publishers. The operator buffers the values emitted by these Publishers and then drains them in + * source {@code Publisher}s. The operator buffers the values emitted by these {@code Publisher}s and then drains them in * order, each one after the previous one completes. *
*
Backpressure:
@@ -7698,16 +7698,16 @@ public final Flowable concatMapEagerDelayError(@NonNull FunctionThis 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 Publishers that will be + * @param mapper the function that maps a sequence of values into a sequence of {@code Publisher}s that will be * eagerly concatenated * @param tillTheEnd - * if true, exceptions from the current Flowable and all the inner Publishers are delayed until - * all of them terminate, if false, exception from the current Flowable is delayed until the - * currently running Publisher terminates - * @param maxConcurrency the maximum number of concurrent subscribed Publishers + * if {@code true}, exceptions from the current {@code Flowable} and all the inner {@code Publisher}s are delayed until + * all of them terminate, if {@code false}, exception from the current {@code Flowable} is delayed until the + * currently running {@code Publisher} terminates + * @param maxConcurrency the maximum number of concurrent subscribed {@code Publisher}s * @param prefetch - * the number of elements to prefetch from each source Publisher - * @return the new Publisher instance with the specified concatenation behavior + * the number of elements to prefetch from each source {@code Publisher} + * @return the new {@code Publisher} instance with the specified concatenation behavior * @since 2.0 */ @CheckReturnValue @@ -7723,25 +7723,25 @@ public final Flowable concatMapEagerDelayError(@NonNull Function *
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s is * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapIterable} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of item emitted by the resulting Publisher + * the type of item emitted by the resulting {@code Publisher} * @param mapper - * a function that returns an Iterable sequence of values for when given an item emitted by the - * source Publisher - * @return a Flowable that emits the results of concatenating the items emitted by the source Publisher with - * the values in the Iterables corresponding to those items, as generated by {@code collectionSelector} + * a function that returns an {@code Iterable} sequence of values for when given an item emitted by the + * source {@code Publisher} + * @return a {@code Flowable} that emits the results of concatenating the items emitted by the source {@code Publisher} with + * the values in the {@code Iterable}s corresponding to those items, as generated by {@code collectionSelector} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -7753,27 +7753,27 @@ public final Flowable concatMapIterable(@NonNull Function *
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s is * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapIterable} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of item emitted by the resulting Publisher + * the type of item emitted by the resulting {@code Publisher} * @param mapper - * a function that returns an Iterable sequence of values for when given an item emitted by the - * source Publisher + * a function that returns an {@code Iterable} sequence of values for when given an item emitted by the + * source {@code Publisher} * @param prefetch - * the number of elements to prefetch from the current Flowable - * @return a Flowable that emits the results of concatenating the items emitted by the source Publisher with - * the values in the Iterables corresponding to those items, as generated by {@code collectionSelector} + * the number of elements to prefetch from the current {@code Flowable} + * @return a {@code Flowable} that emits the results of concatenating the items emitted by the source {@code Publisher} with + * the values in the {@code Iterable}s corresponding to those items, as generated by {@code collectionSelector} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -7796,7 +7796,7 @@ public final Flowable concatMapIterable(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapMaybe} does not operate by default on a particular {@link Scheduler}.
*
@@ -7805,7 +7805,7 @@ public final Flowable concatMapIterable(@NonNull Function Flowable concatMapMaybe(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapMaybe} does not operate by default on a particular {@link Scheduler}.
*
@@ -7841,7 +7841,7 @@ public final Flowable concatMapMaybe(@NonNull Function Flowable concatMapMaybe(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapMaybeDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -7875,7 +7875,7 @@ public final Flowable concatMapMaybe(@NonNull Function Flowable concatMapMaybeDelayError(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapMaybeDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -7913,7 +7913,7 @@ public final Flowable concatMapMaybeDelayError(@NonNull Function Flowable concatMapMaybeDelayError(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapMaybeDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -7955,7 +7955,7 @@ public final Flowable concatMapMaybeDelayError(@NonNull Function Flowable concatMapMaybeDelayError(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapSingle} does not operate by default on a particular {@link Scheduler}.
*
@@ -7988,7 +7988,7 @@ public final Flowable concatMapMaybeDelayError(@NonNull Function Flowable concatMapSingle(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapSingle} does not operate by default on a particular {@link Scheduler}.
*
@@ -8024,7 +8024,7 @@ public final Flowable concatMapSingle(@NonNull Function Flowable concatMapSingle(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapSingleDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -8058,7 +8058,7 @@ public final Flowable concatMapSingle(@NonNull Function Flowable concatMapSingleDelayError(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapSingleDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -8096,7 +8096,7 @@ public final Flowable concatMapSingleDelayError(@NonNull Function Flowable concatMapSingleDelayError(@NonNull FunctionBackpressure: *
The operator expects the upstream to support backpressure and honors * the backpressure from downstream. If this {@code Flowable} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code concatMapSingleDelayError} does not operate by default on a particular {@link Scheduler}.
*
@@ -8138,7 +8138,7 @@ public final Flowable concatMapSingleDelayError(@NonNull Function Flowable concatMapSingleDelayError(@NonNull Function * @@ -8161,14 +8161,14 @@ public final Flowable concatMapSingleDelayError(@NonNull FunctionBackpressure: *
The operator honors backpressure from downstream. Both this and the {@code other} {@code Publisher}s * are expected to honor backpressure as well. If any of then violates this rule, it may throw an - * {@code IllegalStateException} when the source {@code Publisher} completes.
+ * {@link IllegalStateException} when the source {@code Publisher} completes. *
Scheduler:
*
{@code concatWith} does not operate by default on a particular {@link Scheduler}.
*
* * @param other - * a Publisher to be concatenated after the current - * @return a Flowable that emits items emitted by the two source Publishers, one after the other, + * a {@code Publisher} to be concatenated after the current + * @return a {@code Flowable} that emits items emitted by the two source {@code Publisher}s, one after the other, * without interleaving them * @see ReactiveX operators documentation: Concat */ @@ -8194,8 +8194,8 @@ public final Flowable concatWith(@NonNull Publisher other) { *
{@code concatWith} does not operate by default on a particular {@link Scheduler}.
*
*

History: 2.1.10 - experimental - * @param other the SingleSource whose signal should be emitted after this {@code Flowable} completes normally. - * @return the new Flowable instance + * @param other the {@code SingleSource} whose signal should be emitted after this {@code Flowable} completes normally. + * @return the new {@code Flowable} instance * @since 2.2 */ @CheckReturnValue @@ -8220,8 +8220,8 @@ public final Flowable concatWith(@NonNull SingleSource other) { *

{@code concatWith} does not operate by default on a particular {@link Scheduler}.
*
*

History: 2.1.10 - experimental - * @param other the MaybeSource whose signal should be emitted after this Flowable completes normally. - * @return the new Flowable instance + * @param other the {@code MaybeSource} whose signal should be emitted after this {@code Flowable} completes normally. + * @return the new {@code Flowable} instance * @since 2.2 */ @CheckReturnValue @@ -8240,16 +8240,16 @@ public final Flowable concatWith(@NonNull MaybeSource other) { * *

*
Backpressure:
- *
The operator does not interfere with backpressure between the current Flowable and the + *
The operator does not interfere with backpressure between the current {@code Flowable} and the * downstream consumer (i.e., acts as pass-through). When the operator switches to the - * {@code Completable}, backpressure is no longer present because {@code Completable} doesn't + * {@link Completable}, backpressure is no longer present because {@code Completable} doesn't * have items to apply backpressure to.
*
Scheduler:
*
{@code concatWith} does not operate by default on a particular {@link Scheduler}.
*
*

History: 2.1.10 - experimental * @param other the {@code CompletableSource} to subscribe to once the current {@code Flowable} completes normally - * @return the new Flowable instance + * @return the new {@code Flowable} instance * @since 2.2 */ @CheckReturnValue @@ -8262,7 +8262,7 @@ public final Flowable concatWith(@NonNull CompletableSource other) { } /** - * Returns a Single that emits a Boolean that indicates whether the source Publisher emitted a + * Returns a {@link Single} that emits a {@link Boolean} that indicates whether the source {@link Publisher} emitted a * specified item. *

* @@ -8275,9 +8275,9 @@ public final Flowable concatWith(@NonNull CompletableSource other) { * * * @param item - * the item to search for in the emissions from the source Publisher - * @return a Flowable that emits {@code true} if the specified item is emitted by the source Publisher, - * or {@code false} if the source Publisher completes without emitting that item + * the item to search for in the emissions from the source {@code Publisher} + * @return a {@code Single} that emits {@code true} if the specified item is emitted by the source {@code Publisher}, + * or {@code false} if the source {@code Publisher} completes without emitting that item * @see ReactiveX operators documentation: Contains */ @CheckReturnValue @@ -8290,8 +8290,8 @@ public final Single contains(@NonNull Object item) { } /** - * Returns a Single that counts the total number of items emitted by the source Publisher and emits - * this count as a 64-bit Long. + * Returns a {@link Single} that counts the total number of items emitted by the source {@link Publisher} and emits + * this count as a 64-bit {@link Long}. *

* *

@@ -8302,8 +8302,8 @@ public final Single contains(@NonNull Object item) { *
{@code count} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a Single that emits a single item: the number of items emitted by the source Publisher as a - * 64-bit Long item + * @return a {@code Single} that emits a single item: the number of items emitted by the source {@code Publisher} as a + * 64-bit {@code Long} item * @see ReactiveX operators documentation: Count */ @CheckReturnValue @@ -8315,8 +8315,8 @@ public final Single count() { } /** - * Returns a Flowable that mirrors the source Publisher, except that it drops items emitted by the - * source Publisher that are followed by another item within a computed debounce duration. + * Returns a {@code Flowable} that mirrors the source {@link Publisher}, except that it drops items emitted by the + * source {@code Publisher} that are followed by another item within a computed debounce duration. *

* *

@@ -8339,7 +8339,7 @@ public final Single count() { * the debounce value type (ignored) * @param debounceIndicator * function to retrieve a sequence that indicates the throttle duration for each item - * @return a Flowable that omits items emitted by the source Publisher that are followed by another item + * @return a {@code Flowable} that omits items emitted by the source {@code Publisher} that are followed by another item * within a computed debounce duration * @see ReactiveX operators documentation: Debounce * @see RxJava wiki: Backpressure @@ -8354,16 +8354,16 @@ public final Flowable debounce(@NonNull Function - * Note: If items keep being emitted by the source Publisher faster than the timeout then no items - * will be emitted by the resulting Publisher. + * Note: If items keep being emitted by the source {@code Publisher} faster than the timeout then no items + * will be emitted by the resulting {@code Publisher}. *

* *

- * Delivery of the item after the grace period happens on the {@code computation} {@code Scheduler}'s + * Delivery of the item after the grace period happens on the {@code computation} {@link Scheduler}'s * {@code Worker} which if takes too long, a newer item may arrive from the upstream, causing the * {@code Worker}'s task to get disposed, which may also interrupt any downstream blocking operation * (yielding an {@code InterruptedException}). It is recommended processing items @@ -8373,16 +8373,16 @@ public final Flowable debounce(@NonNull FunctionBackpressure: *

This operator does not support backpressure as it uses time to control data flow.
*
Scheduler:
- *
{@code debounce} operates by default on the {@code computation} {@link Scheduler}.
+ *
{@code debounce} operates by default on the {@code computation} {@code Scheduler}.
* * * @param timeout * the length of the window of time that must pass after the emission of an item from the source - * Publisher in which that Publisher emits no items in order for the item to be emitted by the - * resulting Publisher + * {@code Publisher} in which that {@code Publisher} emits no items in order for the item to be emitted by the + * resulting {@code Publisher} * @param unit * the unit of time for the specified {@code timeout} - * @return a Flowable that filters out items from the source Publisher that are too quickly followed by + * @return a {@code Flowable} that filters out items from the source {@code Publisher} that are too quickly followed by * newer items * @see ReactiveX operators documentation: Debounce * @see RxJava wiki: Backpressure @@ -8397,12 +8397,12 @@ public final Flowable debounce(long timeout, @NonNull TimeUnit unit) { } /** - * Returns a Flowable that mirrors the source Publisher, except that it drops items emitted by the - * source Publisher that are followed by newer items before a timeout value expires on a specified - * Scheduler. The timer resets on each emission. + * Returns a {@code Flowable} that mirrors the source {@link Publisher}, except that it drops items emitted by the + * source {@code Publisher} that are followed by newer items before a timeout value expires on a specified + * {@link Scheduler}. The timer resets on each emission. *

- * Note: If items keep being emitted by the source Publisher faster than the timeout then no items - * will be emitted by the resulting Publisher. + * Note: If items keep being emitted by the source {@code Publisher} faster than the timeout then no items + * will be emitted by the resulting {@code Publisher}. *

* *

@@ -8416,18 +8416,18 @@ public final Flowable debounce(long timeout, @NonNull TimeUnit unit) { *

Backpressure:
*
This operator does not support backpressure as it uses time to control data flow.
*
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
* * * @param timeout - * the time each item has to be "the most recent" of those emitted by the source Publisher to + * the time each item has to be "the most recent" of those emitted by the source {@code Publisher} to * ensure that it's not dropped * @param unit * the unit of time for the specified {@code timeout} * @param scheduler - * the {@link Scheduler} to use internally to manage the timers that handle the timeout for each + * the {@code Scheduler} to use internally to manage the timers that handle the timeout for each * item - * @return a Flowable that filters out items from the source Publisher that are too quickly followed by + * @return a {@code Flowable} that filters out items from the source {@code Publisher} that are too quickly followed by * newer items * @see ReactiveX operators documentation: Debounce * @see RxJava wiki: Backpressure @@ -8444,24 +8444,24 @@ public final Flowable debounce(long timeout, @NonNull TimeUnit unit, @NonNull } /** - * Returns a Flowable that emits the items emitted by the source Publisher or a specified default item - * if the source Publisher is empty. + * Returns a {@code Flowable} that emits the items emitted by the source {@link Publisher} or a specified default item + * if the source {@code Publisher} is empty. *

* *

*
Backpressure:
*
If the source {@code Publisher} is empty, this operator is guaranteed to honor backpressure from downstream. * If the source {@code Publisher} is non-empty, it is expected to honor backpressure as well; if the rule is violated, - * a {@code MissingBackpressureException} may get signaled somewhere downstream. + * a {@link MissingBackpressureException} may get signaled somewhere downstream. *
*
Scheduler:
*
{@code defaultIfEmpty} does not operate by default on a particular {@link Scheduler}.
*
* * @param defaultItem - * the item to emit if the source Publisher emits no items - * @return a Flowable that emits either the specified default item if the source Publisher emits no - * items, or the items emitted by the source Publisher + * the item to emit if the source {@code Publisher} emits no items + * @return a {@code Flowable} that emits either the specified default item if the source {@code Publisher} emits no + * items, or the items emitted by the source {@code Publisher} * @see ReactiveX operators documentation: DefaultIfEmpty */ @CheckReturnValue @@ -8474,13 +8474,13 @@ public final Flowable defaultIfEmpty(@NonNull T defaultItem) { } /** - * Returns a Flowable that delays the emissions of the source Publisher via another Publisher on a + * Returns a {@code Flowable} that delays the emissions of the source {@link Publisher} via another {@code Publisher} on a * per-item basis. *

* *

- * Note: the resulting Publisher will immediately propagate any {@code onError} notification - * from the source Publisher. + * Note: the resulting {@code Publisher} will immediately propagate any {@code onError} notification + * from the source {@code Publisher}. *

*
Backpressure:
*
The operator doesn't interfere with the backpressure behavior which is determined by the source {@code Publisher}. @@ -8493,10 +8493,10 @@ public final Flowable defaultIfEmpty(@NonNull T defaultItem) { * @param * the item delay value type (ignored) * @param itemDelayIndicator - * a function that returns a Publisher for each item emitted by the source Publisher, which is - * then used to delay the emission of that item by the resulting Publisher until the Publisher + * a function that returns a {@code Publisher} for each item emitted by the source {@code Publisher}, which is + * then used to delay the emission of that item by the resulting {@code Publisher} until the {@code Publisher} * returned from {@code itemDelay} emits an item - * @return a Flowable that delays the emissions of the source Publisher via another Publisher on a + * @return a {@code Flowable} that delays the emissions of the source {@code Publisher} via another {@code Publisher} on a * per-item basis * @see ReactiveX operators documentation: Delay */ @@ -8510,8 +8510,8 @@ public final Flowable delay(@NonNull Function * *
@@ -8525,7 +8525,7 @@ public final Flowable delay(@NonNull FunctionReactiveX operators documentation: Delay */ @CheckReturnValue @@ -8537,8 +8537,8 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit) { } /** - * Returns a Flowable that emits the items emitted by the source Publisher shifted forward in time by a - * specified delay. If {@code delayError} is true, error notifications will also be delayed. + * Returns a {@code Flowable} that emits the items emitted by the source {@link Publisher} shifted forward in time by a + * specified delay. If {@code delayError} is {@code true}, error notifications will also be delayed. *

* *

@@ -8553,9 +8553,9 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit) { * @param unit * the {@link TimeUnit} in which {@code period} is defined * @param delayError - * if true, the upstream exception is signaled with the given delay, after all preceding normal elements, - * if false, the upstream exception is signaled immediately - * @return the source Publisher shifted in time by the specified delay + * if {@code true}, the upstream exception is signaled with the given delay, after all preceding normal elements, + * if {@code false}, the upstream exception is signaled immediately + * @return the source {@code Publisher} shifted in time by the specified delay * @see ReactiveX operators documentation: Delay */ @CheckReturnValue @@ -8567,8 +8567,8 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit, boolean delay } /** - * Returns a Flowable that emits the items emitted by the source Publisher shifted forward in time by a - * specified delay. Error notifications from the source Publisher are not delayed. + * Returns a {@code Flowable} that emits the items emitted by the source {@link Publisher} shifted forward in time by a + * specified delay. The {@code onError} notification from the source {@code Publisher} is not delayed. *

* *

@@ -8583,8 +8583,8 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit, boolean delay * @param unit * the time unit of {@code delay} * @param scheduler - * the {@link Scheduler} to use for delaying - * @return the source Publisher shifted in time by the specified delay + * the {@code Scheduler} to use for delaying + * @return the source {@code Publisher} shifted in time by the specified delay * @see ReactiveX operators documentation: Delay */ @CheckReturnValue @@ -8596,8 +8596,8 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit, @NonNull Sche } /** - * Returns a Flowable that emits the items emitted by the source Publisher shifted forward in time by a - * specified delay. If {@code delayError} is true, error notifications will also be delayed. + * Returns a {@code Flowable} that emits the items emitted by the source {@link Publisher} shifted forward in time by a + * specified delay. If {@code delayError} is {@code true}, error notifications will also be delayed. *

* *

@@ -8612,11 +8612,11 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit, @NonNull Sche * @param unit * the time unit of {@code delay} * @param scheduler - * the {@link Scheduler} to use for delaying + * the {@code Scheduler} to use for delaying * @param delayError - * if true, the upstream exception is signaled with the given delay, after all preceding normal elements, - * if false, the upstream exception is signaled immediately - * @return the source Publisher shifted in time by the specified delay + * if {@code true}, the upstream exception is signaled with the given delay, after all preceding normal elements, + * if {@code false}, the upstream exception is signaled immediately + * @return the source {@code Publisher} shifted in time by the specified delay * @see ReactiveX operators documentation: Delay */ @CheckReturnValue @@ -8631,13 +8631,13 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit, @NonNull Sche } /** - * Returns a Flowable that delays the subscription to and emissions from the source Publisher via another - * Publisher on a per-item basis. + * Returns a {@code Flowable} that delays the subscription to and emissions from the source {@link Publisher} via another + * {@code Publisher} on a per-item basis. *

* *

- * Note: the resulting Publisher will immediately propagate any {@code onError} notification - * from the source Publisher. + * Note: the resulting {@code Publisher} will immediately propagate any {@code onError} notification + * from the source {@code Publisher}. *

*
Backpressure:
*
The operator doesn't interfere with the backpressure behavior which is determined by the source {@code Publisher}. @@ -8652,14 +8652,14 @@ public final Flowable delay(long delay, @NonNull TimeUnit unit, @NonNull Sche * @param * the item delay value type (ignored) * @param subscriptionIndicator - * a function that returns a Publisher that triggers the subscription to the source Publisher + * a function that returns a {@code Publisher} that triggers the subscription to the source {@code Publisher} * once it emits any item * @param itemDelayIndicator - * a function that returns a Publisher for each item emitted by the source Publisher, which is - * then used to delay the emission of that item by the resulting Publisher until the Publisher + * a function that returns a {@code Publisher} for each item emitted by the source {@code Publisher}, which is + * then used to delay the emission of that item by the resulting {@code Publisher} until the {@code Publisher} * returned from {@code itemDelay} emits an item - * @return a Flowable that delays the subscription and emissions of the source Publisher via another - * Publisher on a per-item basis + * @return a {@code Flowable} that delays the subscription and emissions of the source {@code Publisher} via another + * {@code Publisher} on a per-item basis * @see ReactiveX operators documentation: Delay */ @CheckReturnValue @@ -8672,21 +8672,21 @@ public final Flowable delay(@NonNull Publisher subscriptionIndicato } /** - * Returns a Flowable that delays the subscription to this Publisher - * until the other Publisher emits an element or completes normally. + * Returns a {@code Flowable} that delays the subscription to this {@link Publisher} + * until the other {@code Publisher} emits an element or completes normally. *
*
Backpressure:
- *
The operator forwards the backpressure requests to this Publisher once - * the subscription happens and requests {@link Long#MAX_VALUE} from the other Publisher
+ *
The operator forwards the backpressure requests to this {@code Publisher} once + * the subscription happens and requests {@link Long#MAX_VALUE} from the other {@code Publisher}
*
Scheduler:
*
This method does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the other Publisher, irrelevant - * @param subscriptionIndicator the other Publisher that should trigger the subscription - * to this Publisher. - * @return a Flowable that delays the subscription to this Publisher - * until the other Publisher emits an element or completes normally. + * @param the value type of the other {@code Publisher}, irrelevant + * @param subscriptionIndicator the other {@code Publisher} that should trigger the subscription + * to this {@code Publisher}. + * @return a {@code Flowable} that delays the subscription to this {@code Publisher} + * until the other {@code Publisher} emits an element or completes normally. * @since 2.0 */ @CheckReturnValue @@ -8699,7 +8699,7 @@ public final Flowable delaySubscription(@NonNull Publisher subscriptio } /** - * Returns a Flowable that delays the subscription to the source Publisher by a given amount of time. + * Returns a {@code Flowable} that delays the subscription to the source {@link Publisher} by a given amount of time. *

* *

@@ -8713,7 +8713,7 @@ public final Flowable delaySubscription(@NonNull Publisher subscriptio * the time to delay the subscription * @param unit * the time unit of {@code delay} - * @return a Flowable that delays the subscription to the source Publisher by the given amount + * @return a {@code Flowable} that delays the subscription to the source {@code Publisher} by the given amount * @see ReactiveX operators documentation: Delay */ @CheckReturnValue @@ -8725,15 +8725,15 @@ public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit) { } /** - * Returns a Flowable that delays the subscription to the source Publisher by a given amount of time, - * both waiting and subscribing on a given Scheduler. + * Returns a {@code Flowable} that delays the subscription to the source {@link Publisher} by a given amount of time, + * both waiting and subscribing on a given {@link Scheduler}. *

* *

*
Backpressure:
*
The operator doesn't interfere with the backpressure behavior which is determined by the source {@code Publisher}.
*
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param delay @@ -8741,9 +8741,9 @@ public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit) { * @param unit * the time unit of {@code delay} * @param scheduler - * the Scheduler on which the waiting and subscription will happen - * @return a Flowable that delays the subscription to the source Publisher by a given - * amount, waiting and subscribing on the given Scheduler + * the {@code Scheduler} on which the waiting and subscription will happen + * @return a {@code Flowable} that delays the subscription to the source {@code Publisher} by a given + * amount, waiting and subscribing on the given {@code Scheduler} * @see ReactiveX operators documentation: Delay */ @CheckReturnValue @@ -8755,9 +8755,9 @@ public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit, @ } /** - * Returns a Flowable that reverses the effect of {@link #materialize materialize} by transforming the + * Returns a {@code Flowable} that reverses the effect of {@link #materialize materialize} by transforming the * {@link Notification} objects extracted from the source items via a selector function - * into their respective {@code Subscriber} signal types. + * into their respective {@link Subscriber} signal types. *

* *

@@ -8769,7 +8769,7 @@ public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit, @ *

* When the upstream signals an {@link Notification#createOnError(Throwable) onError} or * {@link Notification#createOnComplete() onComplete} item, the - * returned Flowable cancels of the flow and terminates with that type of terminal event: + * returned {@code Flowable} cancels of the flow and terminates with that type of terminal event: *


      * Flowable.just(createOnNext(1), createOnComplete(), createOnNext(2))
      * .doOnCancel(() -> System.out.println("Canceled!"));
@@ -8789,7 +8789,7 @@ public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit, @
      * with a {@link #never()} source.
      * 
*
Backpressure:
- *
The operator doesn't interfere with backpressure which is determined by the source {@code Publisher}'s + *
The operator doesn't interfere with backpressure which is determined by the source {@link Publisher}'s * backpressure behavior.
*
Scheduler:
*
{@code dematerialize} does not operate by default on a particular {@link Scheduler}.
@@ -8797,10 +8797,10 @@ public final Flowable delaySubscription(long delay, @NonNull TimeUnit unit, @ *

History: 2.2.4 - experimental * * @param the output value type - * @param selector function that returns the upstream item and should return a Notification to signal + * @param selector function that returns the upstream item and should return a {@code Notification} to signal * the corresponding {@code Subscriber} event to the downstream. - * @return a Flowable that emits the items and notifications embedded in the {@link Notification} objects - * selected from the items emitted by the source Flowable + * @return a {@code Flowable} that emits the items and notifications embedded in the {@code Notification} objects + * selected from the items emitted by the source {@code Flowable} * @see ReactiveX operators documentation: Dematerialize * @since 3.0.0 */ @@ -8814,7 +8814,7 @@ public final Flowable dematerialize(@NonNull Function<@NonNull ? super T, } /** - * Returns a Flowable that emits all items emitted by the source Publisher that are distinct + * Returns a {@code Flowable} that emits all items emitted by the source {@link Publisher} that are distinct * based on {@link Object#equals(Object)} comparison. *

* @@ -8822,13 +8822,13 @@ public final Flowable dematerialize(@NonNull Function<@NonNull ? super T, * It is recommended the elements' class {@code T} in the flow overrides the default {@code Object.equals()} and {@link Object#hashCode()} to provide * a meaningful comparison between items as the default Java implementation only considers reference equivalence. *

- * By default, {@code distinct()} uses an internal {@link java.util.HashSet} per Subscriber to remember + * By default, {@code distinct()} uses an internal {@link java.util.HashSet} per {@link Subscriber} to remember * previously seen items and uses {@link java.util.Set#add(Object)} returning {@code false} as the * indicator for duplicates. *

- * Note that this internal {@code HashSet} may grow unbounded as items won't be removed from it by + * Note that this internal {@link HashSet} may grow unbounded as items won't be removed from it by * the operator. Therefore, using very long or infinite upstream (with very distinct elements) may lead - * to {@code OutOfMemoryError}. + * to {@link OutOfMemoryError}. *

* Customizing the retention policy can happen only by providing a custom {@link java.util.Collection} implementation * to the {@link #distinct(Function, Supplier)} overload. @@ -8840,7 +8840,7 @@ public final Flowable dematerialize(@NonNull Function<@NonNull ? super T, *

{@code distinct} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a Flowable that emits only those items emitted by the source Publisher that are distinct from + * @return a {@code Flowable} that emits only those items emitted by the source {@code Publisher} that are distinct from * each other * @see ReactiveX operators documentation: Distinct * @see #distinct(Function) @@ -8856,7 +8856,7 @@ public final Flowable distinct() { } /** - * Returns a Flowable that emits all items emitted by the source Publisher that are distinct according + * Returns a {@code Flowable} that emits all items emitted by the source {@link Publisher} that are distinct according * to a key selector function and based on {@link Object#equals(Object)} comparison of the objects * returned by the key selector function. *

@@ -8865,13 +8865,13 @@ public final Flowable distinct() { * It is recommended the keys' class {@code K} overrides the default {@code Object.equals()} and {@link Object#hashCode()} to provide * a meaningful comparison between the key objects as the default Java implementation only considers reference equivalence. *

- * By default, {@code distinct()} uses an internal {@link java.util.HashSet} per Subscriber to remember + * By default, {@code distinct()} uses an internal {@link java.util.HashSet} per {@link Subscriber} to remember * previously seen keys and uses {@link java.util.Set#add(Object)} returning {@code false} as the * indicator for duplicates. *

- * Note that this internal {@code HashSet} may grow unbounded as keys won't be removed from it by + * Note that this internal {@link HashSet} may grow unbounded as keys won't be removed from it by * the operator. Therefore, using very long or infinite upstream (with very distinct keys) may lead - * to {@code OutOfMemoryError}. + * to {@link OutOfMemoryError}. *

* Customizing the retention policy can happen only by providing a custom {@link java.util.Collection} implementation * to the {@link #distinct(Function, Supplier)} overload. @@ -8887,7 +8887,7 @@ public final Flowable distinct() { * @param keySelector * a function that projects an emitted item to a key value that is used to decide whether an item * is distinct from another one or not - * @return a Flowable that emits those items emitted by the source Publisher that have distinct keys + * @return a {@code Flowable} that emits those items emitted by the source {@code Publisher} that have distinct keys * @see ReactiveX operators documentation: Distinct * @see #distinct(Function, Supplier) */ @@ -8900,7 +8900,7 @@ public final Flowable distinct(@NonNull Function keySelecto } /** - * Returns a Flowable that emits all items emitted by the source Publisher that are distinct according + * Returns a {@code Flowable} that emits all items emitted by the source {@link Publisher} that are distinct according * to a key selector function and based on {@link Object#equals(Object)} comparison of the objects * returned by the key selector function. *

@@ -8921,9 +8921,9 @@ public final Flowable distinct(@NonNull Function keySelecto * a function that projects an emitted item to a key value that is used to decide whether an item * is distinct from another one or not * @param collectionSupplier - * function called for each individual Subscriber to return a Collection subtype for holding the extracted + * function called for each individual {@link Subscriber} to return a {@link Collection} subtype for holding the extracted * keys and whose add() method's return indicates uniqueness. - * @return a Flowable that emits those items emitted by the source Publisher that have distinct keys + * @return a {@code Flowable} that emits those items emitted by the source {@code Publisher} that have distinct keys * @see ReactiveX operators documentation: Distinct */ @CheckReturnValue @@ -8938,7 +8938,7 @@ public final Flowable distinct(@NonNull Function keySelecto } /** - * Returns a Flowable that emits all items emitted by the source Publisher that are distinct from their + * Returns a {@code Flowable} that emits all items emitted by the source {@link Publisher} that are distinct from their * immediate predecessors based on {@link Object#equals(Object)} comparison. *

* @@ -8954,7 +8954,7 @@ public final Flowable distinct(@NonNull Function keySelecto *

* Note that if element type {@code T} in the flow is mutable, the comparison of the previous and current * item may yield unexpected results if the items are mutated externally. Common cases are mutable - * {@code CharSequence}s or {@code List}s where the objects will actually have the same + * {@link CharSequence}s or {@link List}s where the objects will actually have the same * references when they are modified and {@code distinctUntilChanged} will evaluate subsequent items as same. * To avoid such situation, it is recommended that mutable data is converted to an immutable one, * for example using {@code map(CharSequence::toString)} or {@code map(list -> Collections.unmodifiableList(new ArrayList<>(list)))}. @@ -8966,7 +8966,7 @@ public final Flowable distinct(@NonNull Function keySelecto *

{@code distinctUntilChanged} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a Flowable that emits those items from the source Publisher that are distinct from their + * @return a {@code Flowable} that emits those items from the source {@code Publisher} that are distinct from their * immediate predecessors * @see ReactiveX operators documentation: Distinct * @see #distinctUntilChanged(BiPredicate) @@ -8980,7 +8980,7 @@ public final Flowable distinctUntilChanged() { } /** - * Returns a Flowable that emits all items emitted by the source Publisher that are distinct from their + * Returns a {@code Flowable} that emits all items emitted by the source {@link Publisher} that are distinct from their * immediate predecessors, according to a key selector function and based on {@link Object#equals(Object)} comparison * of those objects returned by the key selector function. *

@@ -8998,7 +8998,7 @@ public final Flowable distinctUntilChanged() { *

* Note that if element type {@code T} in the flow is mutable, the comparison of the previous and current * item may yield unexpected results if the items are mutated externally. Common cases are mutable - * {@code CharSequence}s or {@code List}s where the objects will actually have the same + * {@link CharSequence}s or {@link List}s where the objects will actually have the same * references when they are modified and {@code distinctUntilChanged} will evaluate subsequent items as same. * To avoid such situation, it is recommended that mutable data is converted to an immutable one, * for example using {@code map(CharSequence::toString)} or {@code map(list -> Collections.unmodifiableList(new ArrayList<>(list)))}. @@ -9014,7 +9014,7 @@ public final Flowable distinctUntilChanged() { * @param keySelector * a function that projects an emitted item to a key value that is used to decide whether an item * is distinct from another one or not - * @return a Flowable that emits those items from the source Publisher whose keys are distinct from + * @return a {@code Flowable} that emits those items from the source {@code Publisher} whose keys are distinct from * those of their immediate predecessors * @see ReactiveX operators documentation: Distinct */ @@ -9028,7 +9028,7 @@ public final Flowable distinctUntilChanged(@NonNull Function * @@ -9038,7 +9038,7 @@ public final Flowable distinctUntilChanged(@NonNull Function * Note that if element type {@code T} in the flow is mutable, the comparison of the previous and current * item may yield unexpected results if the items are mutated externally. Common cases are mutable - * {@code CharSequence}s or {@code List}s where the objects will actually have the same + * {@link CharSequence}s or {@link List}s where the objects will actually have the same * references when they are modified and {@code distinctUntilChanged} will evaluate subsequent items as same. * To avoid such situation, it is recommended that mutable data is converted to an immutable one, * for example using {@code map(CharSequence::toString)} or {@code map(list -> Collections.unmodifiableList(new ArrayList<>(list)))}. @@ -9051,8 +9051,8 @@ public final Flowable distinctUntilChanged(@NonNull Function * * @param comparer the function that receives the previous item and the current item and is - * expected to return true if the two are equal, thus skipping the current value. - * @return a Flowable that emits those items from the source Publisher that are distinct from their + * expected to return {@code true} if the two are equal, thus skipping the current value. + * @return a {@code Flowable} that emits those items from the source {@code Publisher} that are distinct from their * immediate predecessors * @see ReactiveX operators documentation: Distinct * @since 2.0 @@ -9067,7 +9067,7 @@ public final Flowable distinctUntilChanged(@NonNull BiPredicateIn case of a race between a terminal event and a cancellation, the provided {@code onFinally} action * is executed once per subscription. @@ -9075,17 +9075,17 @@ public final Flowable distinctUntilChanged(@NonNull BiPredicate *

Backpressure:
- *
The operator doesn't interfere with backpressure which is determined by the source {@code Publisher}'s backpressure + *
The operator doesn't interfere with backpressure which is determined by the source {@link Publisher}'s backpressure * behavior.
*
Scheduler:
*
{@code doFinally} does not operate by default on a particular {@link Scheduler}.
*
Operator-fusion:
- *
This operator supports normal and conditional Subscribers as well as boundary-limited + *
This operator supports normal and conditional {@link Subscriber}s as well as boundary-limited * synchronous or asynchronous queue-fusion.
*
*

History: 2.0.1 - experimental - * @param onFinally the action called when this Flowable terminates or gets canceled - * @return the new Flowable instance + * @param onFinally the action called when this {@code Flowable} terminates or gets canceled + * @return the new {@code Flowable} instance * @since 2.1 */ @CheckReturnValue @@ -9103,17 +9103,17 @@ public final Flowable doFinally(@NonNull Action onFinally) { * should be thread-safe. *

*
Backpressure:
- *
The operator doesn't interfere with backpressure which is determined by the source {@code Publisher}'s backpressure + *
The operator doesn't interfere with backpressure which is determined by the source {@link Publisher}'s backpressure * behavior.
*
Scheduler:
*
{@code doAfterNext} does not operate by default on a particular {@link Scheduler}.
*
Operator-fusion:
- *
This operator supports normal and conditional Subscribers as well as boundary-limited + *
This operator supports normal and conditional {@link Subscriber}s as well as boundary-limited * synchronous or asynchronous queue-fusion.
*
*

History: 2.0.1 - experimental - * @param onAfterNext the Consumer that will be called after emitting an item from upstream to the downstream - * @return the new Flowable instance + * @param onAfterNext the {@link Consumer} that will be called after emitting an item from upstream to the downstream + * @return the new {@code Flowable} instance * @since 2.1 */ @CheckReturnValue @@ -9126,7 +9126,7 @@ public final Flowable doAfterNext(@NonNull Consumer onAfterNext) { } /** - * Registers an {@link Action} to be called when this Publisher invokes either + * Registers an {@link Action} to be called when this {@link Publisher} invokes either * {@link Subscriber#onComplete onComplete} or {@link Subscriber#onError onError}. *

* @@ -9139,9 +9139,9 @@ public final Flowable doAfterNext(@NonNull Consumer onAfterNext) { *

* * @param onAfterTerminate - * an {@link Action} to be invoked when the source Publisher finishes - * @return a Flowable that emits the same items as the source Publisher, then invokes the - * {@link Action} + * an {@code Action} to be invoked when the source {@code Publisher} finishes + * @return a {@code Flowable} that emits the same items as the source {@code Publisher}, then invokes the + * {@code Action} * @see ReactiveX operators documentation: Do * @see #doOnTerminate(Action) */ @@ -9155,17 +9155,15 @@ public final Flowable doAfterTerminate(@NonNull Action onAfterTerminate) { } /** - * Calls the cancel {@code Action} if the downstream cancels the sequence. + * Calls the cancel {@link Action} if the downstream cancels the sequence. + *

+ * *

* The action is shared between subscriptions and thus may be called concurrently from multiple * threads; the action must be thread-safe. *

* If the action throws a runtime exception, that exception is rethrown by the {@code onCancel()} call, - * sometimes as a {@code CompositeException} if there were multiple exceptions along the way. - *

- * Note that terminal events trigger the action unless the {@code Publisher} is subscribed to via {@code unsafeSubscribe()}. - *

- * + * sometimes as a {@link CompositeException} if there were multiple exceptions along the way. *

*
Backpressure:
*
{@code doOnCancel} does not interact with backpressure requests or value delivery; backpressure @@ -9175,8 +9173,8 @@ public final Flowable doAfterTerminate(@NonNull Action onAfterTerminate) { *
* * @param onCancel - * the action that gets called when the source {@code Publisher}'s Subscription is canceled - * @return the source {@code Publisher} modified so as to call this Action when appropriate + * the action that gets called when the source {@link Publisher}'s {@link Subscription} is canceled + * @return the source {@code Publisher} modified so as to call this {@code Action} when appropriate * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9188,7 +9186,7 @@ public final Flowable doOnCancel(@NonNull Action onCancel) { } /** - * Modifies the source Publisher so that it invokes an action when it calls {@code onComplete}. + * Modifies the source {@link Publisher} so that it invokes an action when it calls {@code onComplete}. *

* *

@@ -9200,8 +9198,8 @@ public final Flowable doOnCancel(@NonNull Action onCancel) { *
* * @param onComplete - * the action to invoke when the source Publisher calls {@code onComplete} - * @return the source Publisher with the side-effecting behavior applied + * the action to invoke when the source {@code Publisher} calls {@code onComplete} + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9220,13 +9218,13 @@ public final Flowable doOnComplete(@NonNull Action onComplete) { * *
*
Backpressure:
- *
The operator doesn't interfere with backpressure which is determined by the source {@code Publisher}'s + *
The operator doesn't interfere with backpressure which is determined by the source {@link Publisher}'s * backpressure behavior.
*
Scheduler:
*
{@code doOnEach} does not operate by default on a particular {@link Scheduler}.
*
* - * @return the source Publisher with the side-effecting behavior applied + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9243,7 +9241,7 @@ private Flowable doOnEach(@NonNull Consumer onNext, @NonNull Consu } /** - * Modifies the source Publisher so that it invokes an action for each item it emits. + * Modifies the source {@link Publisher} so that it invokes an action for each item it emits. *

* *

@@ -9255,8 +9253,8 @@ private Flowable doOnEach(@NonNull Consumer onNext, @NonNull Consu *
* * @param onNotification - * the action to invoke for each item emitted by the source Publisher - * @return the source Publisher with the side-effecting behavior applied + * the action to invoke for each item emitted by the source {@code Publisher} + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9274,11 +9272,11 @@ public final Flowable doOnEach(@NonNull Consumer<@NonNull ? super Notificatio } /** - * Modifies the source Publisher so that it notifies a Subscriber for each item and terminal event it emits. + * Modifies the source {@link Publisher} so that it notifies a {@link Subscriber} for each item and terminal event it emits. *

- * In case the {@code onError} of the supplied Subscriber throws, the downstream will receive a composite + * In case the {@code onError} of the supplied {@code Subscriber} throws, the downstream will receive a composite * exception containing the original exception and the exception thrown by {@code onError}. If either the - * {@code onNext} or the {@code onComplete} method of the supplied Subscriber throws, the downstream will be + * {@code onNext} or the {@code onComplete} method of the supplied {@code Subscriber} throws, the downstream will be * terminated and will receive this thrown exception. *

* @@ -9291,9 +9289,9 @@ public final Flowable doOnEach(@NonNull Consumer<@NonNull ? super Notificatio *

* * @param subscriber - * the Subscriber to be notified about onNext, onError and onComplete events on its - * respective methods before the actual downstream Subscriber gets notified. - * @return the source Publisher with the side-effecting behavior applied + * the {@code Subscriber} to be notified about {@code onNext}, {@code onError} and {@code onComplete} events on its + * respective methods before the actual downstream {@code Subscriber} gets notified. + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9310,7 +9308,7 @@ public final Flowable doOnEach(@NonNull Subscriber subscriber) { } /** - * Modifies the source Publisher so that it invokes an action if it calls {@code onError}. + * Modifies the source {@link Publisher} so that it invokes an action if it calls {@code onError}. *

* In case the {@code onError} action throws, the downstream will receive a composite exception containing * the original exception and the exception thrown by {@code onError}. @@ -9325,8 +9323,8 @@ public final Flowable doOnEach(@NonNull Subscriber subscriber) { *

* * @param onError - * the action to invoke if the source Publisher calls {@code onError} - * @return the source Publisher with the side-effecting behavior applied + * the action to invoke if the source {@code Publisher} calls {@code onError} + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9339,25 +9337,25 @@ public final Flowable doOnError(@NonNull Consumer onError) } /** - * Calls the appropriate onXXX method (shared between all Subscribers) for the lifecycle events of + * Calls the appropriate {@code onXXX} method (shared between all {@link Subscriber}s) for the lifecycle events of * the sequence (subscription, cancellation, requesting). *

* *

*
Backpressure:
- *
The operator doesn't interfere with backpressure which is determined by the source {@code Publisher}'s + *
The operator doesn't interfere with backpressure which is determined by the source {@link Publisher}'s * backpressure behavior.
*
Scheduler:
*
{@code doOnLifecycle} does not operate by default on a particular {@link Scheduler}.
*
* * @param onSubscribe - * a Consumer called with the Subscription sent via Subscriber.onSubscribe() + * a {@link Consumer} called with the {@link Subscription} sent via {@link Subscriber#onSubscribe(Subscription)} * @param onRequest - * a LongConsumer called with the request amount sent via Subscription.request() + * a {@link LongConsumer} called with the request amount sent via {@link Subscription#request(long)} * @param onCancel - * called when the downstream cancels the Subscription via cancel() - * @return the source Publisher with the side-effecting behavior applied + * called when the downstream cancels the {@code Subscription} via {@link Subscription#cancel()} + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9373,7 +9371,7 @@ public final Flowable doOnLifecycle(@NonNull Consumer o } /** - * Modifies the source Publisher so that it invokes an action when it calls {@code onNext}. + * Modifies the source {@link Publisher} so that it invokes an action when it calls {@code onNext}. *

* *

@@ -9385,8 +9383,8 @@ public final Flowable doOnLifecycle(@NonNull Consumer o *
* * @param onNext - * the action to invoke when the source Publisher calls {@code onNext} - * @return the source Publisher with the side-effecting behavior applied + * the action to invoke when the source {@code Publisher} calls {@code onNext} + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9399,7 +9397,7 @@ public final Flowable doOnNext(@NonNull Consumer onNext) { } /** - * Modifies the source {@code Publisher} so that it invokes the given action when it receives a + * Modifies the source {@link Publisher} 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 @@ -9413,9 +9411,9 @@ public final Flowable doOnNext(@NonNull Consumer onNext) { *

* * @param onRequest - * the action that gets called when a Subscriber requests items from this + * the action that gets called when a {@link Subscriber} requests items from this * {@code Publisher} - * @return the source {@code Publisher} modified so as to call this Action when appropriate + * @return the source {@code Publisher} modified so as to call this {@link Action} when appropriate * @see ReactiveX operators * documentation: Do * @since 2.0 @@ -9429,7 +9427,7 @@ public final Flowable doOnRequest(@NonNull LongConsumer onRequest) { } /** - * Modifies the source {@code Publisher} so that it invokes the given action when it is subscribed from + * Modifies the source {@link Publisher} so that it invokes the given action when it is subscribed from * its subscribers. Each subscription will result in an invocation of the given action except when the * source {@code Publisher} is reference counted, in which case the source {@code Publisher} will invoke * the given action for the first subscription. @@ -9444,8 +9442,8 @@ public final Flowable doOnRequest(@NonNull LongConsumer onRequest) { *
* * @param onSubscribe - * the Consumer that gets called when a Subscriber subscribes to the current {@code Flowable} - * @return the source {@code Publisher} modified so as to call this Consumer when appropriate + * the {@link Consumer} that gets called when a {@link Subscriber} subscribes to the current {@code Flowable} + * @return the source {@code Publisher} modified so as to call this {@code Consumer} when appropriate * @see ReactiveX operators documentation: Do */ @CheckReturnValue @@ -9457,7 +9455,7 @@ public final Flowable doOnSubscribe(@NonNull Consumer o } /** - * Modifies the source Publisher so that it invokes an action when it calls {@code onComplete} or + * Modifies the source {@link Publisher} so that it invokes an action when it calls {@code onComplete} or * {@code onError}. *

* @@ -9473,8 +9471,8 @@ public final Flowable doOnSubscribe(@NonNull Consumer o * * * @param onTerminate - * the action to invoke when the source Publisher calls {@code onComplete} or {@code onError} - * @return the source Publisher with the side-effecting behavior applied + * the action to invoke when the source {@code Publisher} calls {@code onComplete} or {@code onError} + * @return the source {@code Publisher} with the side-effecting behavior applied * @see ReactiveX operators documentation: Do * @see #doAfterTerminate(Action) */ @@ -9488,21 +9486,21 @@ public final Flowable doOnTerminate(@NonNull Action onTerminate) { } /** - * Returns a Maybe that emits the single item at a specified index in a sequence of emissions from - * this Flowable or completes if this Flowable sequence has fewer elements than index. + * Returns a {@link Maybe} that emits the single item at a specified index in a sequence of emissions from + * this {@code Flowable} or completes if this {@code Flowable} sequence has fewer elements than index. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in a bounded manner.
+ *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in a bounded manner.
*
Scheduler:
*
{@code elementAt} does not operate by default on a particular {@link Scheduler}.
*
* * @param index * the zero-based index of the item to retrieve - * @return a Maybe that emits a single item: the item at the specified position in the sequence of - * those emitted by the source Publisher + * @return a {@code Maybe} that emits a single item: the item at the specified position in the sequence of + * those emitted by the source {@code Publisher} * @see ReactiveX operators documentation: ElementAt */ @CheckReturnValue @@ -9517,13 +9515,13 @@ public final Maybe elementAt(long index) { } /** - * Returns a Single that emits the item found at a specified index in a sequence of emissions from - * this Flowable, or a default item if that index is out of range. + * Returns a {@link Single} that emits the item found at a specified index in a sequence of emissions from + * this {@code Flowable}, or a default item if that index is out of range. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in a bounded manner.
+ *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in a bounded manner.
*
Scheduler:
*
{@code elementAt} does not operate by default on a particular {@link Scheduler}.
*
@@ -9532,8 +9530,8 @@ public final Maybe elementAt(long index) { * the zero-based index of the item to retrieve * @param defaultItem * the default item - * @return a Single that emits the item at the specified position in the sequence emitted by the source - * Publisher, or the default item if that index is outside the bounds of the source sequence + * @return a {@code Single} that emits the item at the specified position in the sequence emitted by the source + * {@code Publisher}, or the default item if that index is outside the bounds of the source sequence * @throws IndexOutOfBoundsException * if {@code index} is less than 0 * @see ReactiveX operators documentation: ElementAt @@ -9551,21 +9549,21 @@ public final Single elementAt(long index, @NonNull T defaultItem) { } /** - * Returns a Single that emits the item found at a specified index in a sequence of emissions from - * this Flowable or signals a {@link NoSuchElementException} if this Flowable has fewer elements than index. + * Returns a {@link Single} that emits the item found at a specified index in a sequence of emissions from + * this {@code Flowable} or signals a {@link NoSuchElementException} if this {@code Flowable} has fewer elements than index. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in a bounded manner.
+ *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in a bounded manner.
*
Scheduler:
*
{@code elementAtOrError} does not operate by default on a particular {@link Scheduler}.
*
* * @param index * the zero-based index of the item to retrieve - * @return a Single that emits the item at the specified position in the sequence emitted by the source - * Publisher, or the default item if that index is outside the bounds of the source sequence + * @return a {@code Single} that emits the item at the specified position in the sequence emitted by the source + * {@code Publisher}, or the default item if that index is outside the bounds of the source sequence * @throws IndexOutOfBoundsException * if {@code index} is less than 0 * @see ReactiveX operators documentation: ElementAt @@ -9582,7 +9580,7 @@ public final Single elementAtOrError(long index) { } /** - * Filters items emitted by a Publisher by only emitting those that satisfy a specified predicate. + * Filters items emitted by a {@link Publisher} by only emitting those that satisfy a specified predicate. *

* *

@@ -9594,9 +9592,9 @@ public final Single elementAtOrError(long index) { *
* * @param predicate - * a function that evaluates each item emitted by the source Publisher, returning {@code true} + * a function that evaluates each item emitted by the source {@code Publisher}, returning {@code true} * if it passes the filter - * @return a Flowable that emits only those items emitted by the source Publisher that the filter + * @return a {@code Flowable} that emits only those items emitted by the source {@code Publisher} that the filter * evaluates as {@code true} * @see ReactiveX operators documentation: Filter */ @@ -9610,18 +9608,18 @@ public final Flowable filter(@NonNull Predicate predicate) { } /** - * Returns a Maybe that emits only the very first item emitted by this Flowable or - * completes if this Flowable is empty. + * Returns a {@link Maybe} that emits only the very first item emitted by this {@code Flowable} or + * completes if this {@code Flowable} is empty. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in a bounded manner.
+ *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in a bounded manner.
*
Scheduler:
*
{@code firstElement} does not operate by default on a particular {@link Scheduler}.
*
* - * @return the new Maybe instance + * @return the new {@code Maybe} instance * @see ReactiveX operators documentation: First */ @CheckReturnValue @@ -9633,21 +9631,21 @@ public final Maybe firstElement() { } /** - * Returns a Single that emits only the very first item emitted by this Flowable, or a default - * item if this Flowable completes without emitting anything. + * Returns a {@link Single} that emits only the very first item emitted by this {@code Flowable}, or a default + * item if this {@code Flowable} completes without emitting anything. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in a bounded manner.
+ *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in a bounded manner.
*
Scheduler:
*
{@code first} does not operate by default on a particular {@link Scheduler}.
*
* * @param defaultItem - * the default item to emit if the source Publisher doesn't emit anything - * @return a Single that emits only the very first item from the source, or a default item if the - * source Publisher completes without emitting any items + * the default item to emit if the source {@code Publisher} doesn't emit anything + * @return a {@code Single} that emits only the very first item from the source, or a default item if the + * source {@code Publisher} completes without emitting any items * @see ReactiveX operators documentation: First */ @CheckReturnValue @@ -9659,18 +9657,18 @@ public final Single first(@NonNull T defaultItem) { } /** - * Returns a Single that emits only the very first item emitted by this Flowable or - * signals a {@link NoSuchElementException} if this Flowable is empty. + * Returns a {@link Single} that emits only the very first item emitted by this {@code Flowable} or + * signals a {@link NoSuchElementException} if this {@code Flowable} is empty. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in a bounded manner.
+ *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in a bounded manner.
*
Scheduler:
*
{@code firstOrError} does not operate by default on a particular {@link Scheduler}.
*
* - * @return the new Single instance + * @return the new {@code Single} instance * @see ReactiveX operators documentation: First */ @CheckReturnValue @@ -9682,27 +9680,27 @@ public final Single firstOrError() { } /** - * Returns a Flowable that emits items based on applying a function that you supply to each item emitted - * by the source Publisher, where that function returns a Publisher, and then merging those resulting - * Publishers and emitting the results of this merger. + * Returns a {@code Flowable} that emits items based on applying a function that you supply to each item emitted + * by the source {@link Publisher}, where that function returns a {@code Publisher}, and then merging those resulting + * {@code Publisher}s and emitting the results of this merger. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@link #bufferSize()} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the inner Publishers and the output type + * @param the value type of the inner {@code Publisher}s and the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and merging the results of the Publishers obtained from this + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and merging the results of the {@code Publisher}s obtained from this * transformation * @see ReactiveX operators documentation: FlatMap */ @@ -9715,30 +9713,30 @@ public final Flowable flatMap(@NonNull Function * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@link #bufferSize()} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the inner Publishers and the output type + * @param the value type of the inner {@code Publisher}s and the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} * @param delayErrors - * if true, exceptions from the current Flowable and all inner Publishers are delayed until all of them terminate - * if false, the first one signaling an exception will terminate the whole sequence immediately - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and merging the results of the Publishers obtained from this + * if {@code true}, exceptions from the current {@code Flowable} and all inner {@code Publisher}s are delayed until all of them terminate + * if {@code false}, the first one signaling an exception will terminate the whole sequence immediately + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and merging the results of the {@code Publisher}s obtained from this * transformation * @see ReactiveX operators documentation: FlatMap */ @@ -9751,30 +9749,30 @@ public final Flowable flatMap(@NonNull Function --> * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@code maxConcurrency} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the inner Publishers and the output type + * @param the value type of the inner {@code Publisher}s and the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and merging the results of the Publishers obtained from this + * the maximum number of {@code Publisher}s that may be subscribed to concurrently + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and merging the results of the {@code Publisher}s obtained from this * transformation * @see ReactiveX operators documentation: FlatMap * @since 2.0 @@ -9788,33 +9786,33 @@ public final Flowable flatMap(@NonNull Function --> * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@code maxConcurrency} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the inner Publishers and the output type + * @param the value type of the inner {@code Publisher}s and the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param delayErrors - * if true, exceptions from the current Flowable and all inner Publishers are delayed until all of them terminate - * if false, the first one signaling an exception will terminate the whole sequence immediately - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and merging the results of the Publishers obtained from this + * if {@code true}, exceptions from the current {@code Flowable} and all inner {@code Publisher}s are delayed until all of them terminate + * if {@code false}, the first one signaling an exception will terminate the whole sequence immediately + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and merging the results of the {@code Publisher}s obtained from this * transformation * @see ReactiveX operators documentation: FlatMap * @since 2.0 @@ -9828,35 +9826,35 @@ public final Flowable flatMap(@NonNull Function --> * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@code maxConcurrency} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the inner Publishers and the output type + * @param the value type of the inner {@code Publisher}s and the output type * @param mapper - * a function that, when applied to an item emitted by the source Publisher, returns a - * Publisher + * a function that, when applied to an item emitted by the source {@code Publisher}, returns a + * {@code Publisher} * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param delayErrors - * if true, exceptions from the current Flowable and all inner Publishers are delayed until all of them terminate - * if false, the first one signaling an exception will terminate the whole sequence immediately + * if {@code true}, exceptions from the current {@code Flowable} and all inner {@code Publisher}s are delayed until all of them terminate + * if {@code false}, the first one signaling an exception will terminate the whole sequence immediately * @param bufferSize - * the number of elements to prefetch from each inner Publisher - * @return a Flowable that emits the result of applying the transformation function to each item emitted - * by the source Publisher and merging the results of the Publishers obtained from this + * the number of elements to prefetch from each inner {@code Publisher} + * @return a {@code Flowable} that emits the result of applying the transformation function to each item emitted + * by the source {@code Publisher} and merging the results of the {@code Publisher}s obtained from this * transformation * @see ReactiveX operators documentation: FlatMap * @since 2.0 @@ -9882,16 +9880,16 @@ public final Flowable flatMap(@NonNull Function * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@link #bufferSize()} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
@@ -9899,15 +9897,15 @@ public final Flowable flatMap(@NonNull Function * the result type * @param onNextMapper - * a function that returns a Publisher to merge for each item emitted by the source Publisher + * a function that returns a {@code Publisher} to merge for each item emitted by the source {@code Publisher} * @param onErrorMapper - * a function that returns a Publisher to merge for an onError notification from the source - * Publisher + * a function that returns a {@code Publisher} to merge for an {@code onError} notification from the source + * {@code Publisher} * @param onCompleteSupplier - * a function that returns a Publisher to merge for an onComplete notification from the source - * Publisher - * @return a Flowable that emits the results of merging the Publishers returned from applying the - * specified functions to the emissions and notifications of the source Publisher + * a function that returns a {@code Publisher} to merge for an {@code onComplete} notification from the source + * {@code Publisher} + * @return a {@code Flowable} that emits the results of merging the {@code Publisher}s returned from applying the + * specified functions to the emissions and notifications of the source {@code Publisher} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -9925,17 +9923,17 @@ public final Flowable flatMap( } /** - * Returns a Flowable that applies a function to each item emitted or notification raised by the source - * Publisher and then flattens the Publishers returned from these functions and emits the resulting items, - * while limiting the maximum number of concurrent subscriptions to these Publishers. + * Returns a {@code Flowable} that applies a function to each item emitted or notification raised by the source + * {@link Publisher} and then flattens the {@code Publisher}s returned from these functions and emits the resulting items, + * while limiting the maximum number of concurrent subscriptions to these {@code Publisher}s. * * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@code maxConcurrency} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
@@ -9943,17 +9941,17 @@ public final Flowable flatMap( * @param * the result type * @param onNextMapper - * a function that returns a Publisher to merge for each item emitted by the source Publisher + * a function that returns a {@code Publisher} to merge for each item emitted by the source {@code Publisher} * @param onErrorMapper - * a function that returns a Publisher to merge for an onError notification from the source - * Publisher + * a function that returns a {@code Publisher} to merge for an {@code onError} notification from the source + * {@code Publisher} * @param onCompleteSupplier - * a function that returns a Publisher to merge for an onComplete notification from the source - * Publisher + * a function that returns a {@code Publisher} to merge for an {@code onComplete} notification from the source + * {@code Publisher} * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently - * @return a Flowable that emits the results of merging the Publishers returned from applying the - * specified functions to the emissions and notifications of the source Publisher + * the maximum number of {@code Publisher}s that may be subscribed to concurrently + * @return a {@code Flowable} that emits the results of merging the {@code Publisher}s returned from applying the + * specified functions to the emissions and notifications of the source {@code Publisher} * @see ReactiveX operators documentation: FlatMap * @since 2.0 */ @@ -9974,31 +9972,31 @@ public final Flowable flatMap( } /** - * Returns a Flowable that emits the results of a specified function to the pair of values emitted by the - * source Publisher and a specified collection Publisher. + * Returns a {@code Flowable} that emits the results of a specified function to the pair of values emitted by the + * source {@link Publisher} and a specified collection {@code Publisher}. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@code maxConcurrency} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of items emitted by the inner Publishers + * the type of items emitted by the inner {@code Publisher}s * @param * the type of items emitted by the combiner function * @param mapper - * a function that returns a Publisher for each item emitted by the source Publisher + * a function that returns a {@code Publisher} for each item emitted by the source {@code Publisher} * @param combiner - * a function that combines one item emitted by each of the source and collection Publishers and - * returns an item to be emitted by the resulting Publisher - * @return a Flowable that emits the results of applying a function to a pair of values emitted by the - * source Publisher and the collection Publisher + * a function that combines one item emitted by each of the source and collection {@code Publisher}s and + * returns an item to be emitted by the resulting {@code Publisher} + * @return a {@code Flowable} that emits the results of applying a function to a pair of values emitted by the + * source {@code Publisher} and the collection {@code Publisher} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -10011,34 +10009,34 @@ public final Flowable flatMap(@NonNull Function * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@link #bufferSize()} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of items emitted by the inner Publishers + * the type of items emitted by the inner {@code Publisher}s * @param * the type of items emitted by the combiner functions * @param mapper - * a function that returns a Publisher for each item emitted by the source Publisher + * a function that returns a {@code Publisher} for each item emitted by the source {@code Publisher} * @param combiner - * a function that combines one item emitted by each of the source and collection Publishers and - * returns an item to be emitted by the resulting Publisher + * a function that combines one item emitted by each of the source and collection {@code Publisher}s and + * returns an item to be emitted by the resulting {@code Publisher} * @param delayErrors - * if true, exceptions from the current Flowable and all inner Publishers are delayed until all of them terminate - * if false, the first one signaling an exception will terminate the whole sequence immediately - * @return a Flowable that emits the results of applying a function to a pair of values emitted by the - * source Publisher and the collection Publisher + * if {@code true}, exceptions from the current {@code Flowable} and all inner {@code Publisher}s are delayed until all of them terminate + * if {@code false}, the first one signaling an exception will terminate the whole sequence immediately + * @return a {@code Flowable} that emits the results of applying a function to a pair of values emitted by the + * source {@code Publisher} and the collection {@code Publisher} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -10051,37 +10049,37 @@ public final Flowable flatMap(@NonNull Function --> * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@code maxConcurrency} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of items emitted by the inner Publishers + * the type of items emitted by the inner {@code Publisher}s * @param * the type of items emitted by the combiner function * @param mapper - * a function that returns a Publisher for each item emitted by the source Publisher + * a function that returns a {@code Publisher} for each item emitted by the source {@code Publisher} * @param combiner - * a function that combines one item emitted by each of the source and collection Publishers and - * returns an item to be emitted by the resulting Publisher + * a function that combines one item emitted by each of the source and collection {@code Publisher}s and + * returns an item to be emitted by the resulting {@code Publisher} * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param delayErrors - * if true, exceptions from the current Flowable and all inner Publishers are delayed until all of them terminate - * if false, the first one signaling an exception will terminate the whole sequence immediately - * @return a Flowable that emits the results of applying a function to a pair of values emitted by the - * source Publisher and the collection Publisher + * if {@code true}, exceptions from the current {@code Flowable} and all inner {@code Publisher}s are delayed until all of them terminate + * if {@code false}, the first one signaling an exception will terminate the whole sequence immediately + * @return a {@code Flowable} that emits the results of applying a function to a pair of values emitted by the + * source {@code Publisher} and the collection {@code Publisher} * @see ReactiveX operators documentation: FlatMap * @since 2.0 */ @@ -10095,39 +10093,39 @@ public final Flowable flatMap(@NonNull Function --> * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@code maxConcurrency} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of items emitted by the inner Publishers + * the type of items emitted by the inner {@code Publisher}s * @param * the type of items emitted by the combiner function * @param mapper - * a function that returns a Publisher for each item emitted by the source Publisher + * a function that returns a {@code Publisher} for each item emitted by the source {@code Publisher} * @param combiner - * a function that combines one item emitted by each of the source and collection Publishers and - * returns an item to be emitted by the resulting Publisher + * a function that combines one item emitted by each of the source and collection {@code Publisher}s and + * returns an item to be emitted by the resulting {@code Publisher} * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently + * the maximum number of {@code Publisher}s that may be subscribed to concurrently * @param delayErrors - * if true, exceptions from the current Flowable and all inner Publishers are delayed until all of them terminate - * if false, the first one signaling an exception will terminate the whole sequence immediately + * if {@code true}, exceptions from the current {@code Flowable} and all inner {@code Publisher}s are delayed until all of them terminate + * if {@code false}, the first one signaling an exception will terminate the whole sequence immediately * @param bufferSize - * the number of elements to prefetch from the inner Publishers. - * @return a Flowable that emits the results of applying a function to a pair of values emitted by the - * source Publisher and the collection Publisher + * the number of elements to prefetch from the inner {@code Publisher}s. + * @return a {@code Flowable} that emits the results of applying a function to a pair of values emitted by the + * source {@code Publisher} and the collection {@code Publisher} * @see ReactiveX operators documentation: FlatMap * @since 2.0 */ @@ -10145,34 +10143,34 @@ public final Flowable flatMap(@NonNull Function --> * *
*
Backpressure:
- *
The operator honors backpressure from downstream. The upstream Flowable is consumed + *
The operator honors backpressure from downstream. The upstream {@code Flowable} is consumed * in a bounded manner (up to {@link #bufferSize()} outstanding request amount for items). * The inner {@code Publisher}s are expected to honor backpressure; if violated, - * the operator may signal {@code MissingBackpressureException}.
+ * the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMap} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of items emitted by the inner Publishers + * the type of items emitted by the inner {@code Publisher}s * @param * the type of items emitted by the combiner function * @param mapper - * a function that returns a Publisher for each item emitted by the source Publisher + * a function that returns a {@code Publisher} for each item emitted by the source {@code Publisher} * @param combiner - * a function that combines one item emitted by each of the source and collection Publishers and - * returns an item to be emitted by the resulting Publisher + * a function that combines one item emitted by each of the source and collection {@code Publisher}s and + * returns an item to be emitted by the resulting {@code Publisher} * @param maxConcurrency - * the maximum number of Publishers that may be subscribed to concurrently - * @return a Flowable that emits the results of applying a function to a pair of values emitted by the - * source Publisher and the collection Publisher + * the maximum number of {@code Publisher}s that may be subscribed to concurrently + * @return a {@code Flowable} that emits the results of applying a function to a pair of values emitted by the + * source {@code Publisher} and the collection {@code Publisher} * @see ReactiveX operators documentation: FlatMap * @since 2.0 */ @@ -10186,16 +10184,16 @@ public final Flowable flatMap(@NonNull Function *
Backpressure:
*
The operator consumes the upstream in an unbounded manner.
*
Scheduler:
*
{@code flatMapCompletable} does not operate by default on a particular {@link Scheduler}.
* - * @param mapper the function that received each source value and transforms them into CompletableSources. - * @return the new Completable instance + * @param mapper the function that received each source value and transforms them into {@code CompletableSource}s. + * @return the new {@link Completable} instance */ @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @@ -10206,8 +10204,8 @@ public final Completable flatMapCompletable(@NonNull Function *
Backpressure:
*
If {@code maxConcurrency == }{@link Integer#MAX_VALUE} the operator consumes the upstream in an unbounded manner. @@ -10216,11 +10214,11 @@ public final Completable flatMapCompletable(@NonNull FunctionScheduler: *
{@code flatMapCompletable} does not operate by default on a particular {@link Scheduler}.
* - * @param mapper the function that received each source value and transforms them into CompletableSources. - * @param delayErrors if true errors from the upstream and inner CompletableSources are delayed until each of them + * @param mapper the function that received each source value and transforms them into {@code CompletableSource}s. + * @param delayErrors if {@code true}, errors from the upstream and inner {@code CompletableSource}s are delayed until each of them * terminates. - * @param maxConcurrency the maximum number of active subscriptions to the CompletableSources. - * @return the new Completable instance + * @param maxConcurrency the maximum number of active subscriptions to the {@code CompletableSource}s. + * @return the new {@link Completable} instance */ @CheckReturnValue @NonNull @@ -10233,26 +10231,26 @@ public final Completable flatMapCompletable(@NonNull Function * *
*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s is * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMapIterable} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of item emitted by the resulting Iterable + * the type of item emitted by the resulting {@code Iterable} * @param mapper - * a function that returns an Iterable sequence of values for when given an item emitted by the - * source Publisher - * @return a Flowable that emits the results of merging the items emitted by the source Publisher with - * the values in the Iterables corresponding to those items, as generated by {@code collectionSelector} + * a function that returns an {@code Iterable} sequence of values for when given an item emitted by the + * source {@code Publisher} + * @return a {@code Flowable} that emits the results of merging the items emitted by the source {@code Publisher} with + * the values in the {@code Iterable}s corresponding to those items, as generated by {@code collectionSelector} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -10264,28 +10262,28 @@ public final Flowable flatMapIterable(@NonNull Function * *
*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s is * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMapIterable} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the type of item emitted by the resulting Iterable + * the type of item emitted by the resulting {@code Iterable} * @param mapper - * a function that returns an Iterable sequence of values for when given an item emitted by the - * source Publisher + * a function that returns an {@code Iterable} sequence of values for when given an item emitted by the + * source {@code Publisher} * @param bufferSize - * the number of elements to prefetch from the current Flowable - * @return a Flowable that emits the results of merging the items emitted by the source Publisher with - * the values in the Iterables corresponding to those items, as generated by {@code collectionSelector} + * the number of elements to prefetch from the current {@code Flowable} + * @return a {@code Flowable} that emits the results of merging the items emitted by the source {@code Publisher} with + * the values in the {@code Iterable}s corresponding to those items, as generated by {@code collectionSelector} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -10299,8 +10297,8 @@ public final Flowable flatMapIterable(@NonNull Function * *
@@ -10314,15 +10312,15 @@ public final Flowable flatMapIterable(@NonNull Function * the collection element type * @param - * the type of item emitted by the resulting Iterable + * the type of item emitted by the resulting {@code Iterable} * @param mapper - * a function that returns an Iterable sequence of values for each item emitted by the source - * Publisher + * a function that returns an {@code Iterable} sequence of values for each item emitted by the source + * {@code Publisher} * @param resultSelector - * a function that returns an item based on the item emitted by the source Publisher and the - * Iterable returned for that item by the {@code collectionSelector} - * @return a Flowable that emits the items returned by {@code resultSelector} for each item in the source - * Publisher + * a function that returns an item based on the item emitted by the source {@code Publisher} and the + * {@code Iterable} returned for that item by the {@code collectionSelector} + * @return a {@code Flowable} that emits the items returned by {@code resultSelector} for each item in the source + * {@code Publisher} * @see ReactiveX operators documentation: FlatMap */ @CheckReturnValue @@ -10337,34 +10335,34 @@ public final Flowable flatMapIterable(@NonNull Function * *
*
Backpressure:
*
The operator honors backpressure from downstream. The source {@code Publisher}s is * expected to honor backpressure as well. If the source {@code Publisher} violates the rule, the operator will - * signal a {@code MissingBackpressureException}.
+ * signal a {@link MissingBackpressureException}. *
Scheduler:
*
{@code flatMapIterable} does not operate by default on a particular {@link Scheduler}.
*
* * @param - * the element type of the inner Iterable sequences + * the element type of the inner {@code Iterable} sequences * @param - * the type of item emitted by the resulting Publisher + * the type of item emitted by the resulting {@code Publisher} * @param mapper - * a function that returns an Iterable sequence of values for when given an item emitted by the - * source Publisher + * a function that returns an {@code Iterable} sequence of values for when given an item emitted by the + * source {@code Publisher} * @param resultSelector - * a function that returns an item based on the item emitted by the source Publisher and the - * Iterable returned for that item by the {@code collectionSelector} + * a function that returns an item based on the item emitted by the source {@code Publisher} and the + * {@code Iterable} returned for that item by the {@code collectionSelector} * @param prefetch - * the number of elements to prefetch from the current Flowable - * @return a Flowable that emits the results of merging the items emitted by the source Publisher with - * the values in the Iterables corresponding to those items, as generated by {@code collectionSelector} + * the number of elements to prefetch from the current {@code Flowable} + * @return a {@code Flowable} that emits the results of merging the items emitted by the source {@code Publisher} with + * the values in the {@code Iterable}s corresponding to those items, as generated by {@code collectionSelector} * @see ReactiveX operators documentation: FlatMap * @since 2.0 */ @@ -10380,8 +10378,8 @@ public final Flowable flatMapIterable(@NonNull Function *
Backpressure:
*
The operator consumes the upstream in an unbounded manner.
@@ -10389,8 +10387,8 @@ public final Flowable flatMapIterable(@NonNull Function{@code flatMapMaybe} does not operate by default on a particular {@link Scheduler}. *
* @param the result value type - * @param mapper the function that received each source value and transforms them into MaybeSources. - * @return the new Flowable instance + * @param mapper the function that received each source value and transforms them into {@code MaybeSource}s. + * @return the new {@code Flowable} instance */ @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @@ -10401,9 +10399,9 @@ public final Flowable flatMapMaybe(@NonNull Function *
Backpressure:
*
If {@code maxConcurrency == }{@link Integer#MAX_VALUE} the operator consumes the upstream in an unbounded manner. @@ -10413,11 +10411,11 @@ public final Flowable flatMapMaybe(@NonNull Function{@code flatMapMaybe} does not operate by default on a particular {@link Scheduler}.
* * @param the result value type - * @param mapper the function that received each source value and transforms them into MaybeSources. - * @param delayErrors if true errors from the upstream and inner MaybeSources are delayed until each of them + * @param mapper the function that received each source value and transforms them into {@code MaybeSource}s. + * @param delayErrors if {@code true}, errors from the upstream and inner {@code MaybeSource}s are delayed until each of them * terminates. - * @param maxConcurrency the maximum number of active subscriptions to the MaybeSources. - * @return the new Flowable instance + * @param maxConcurrency the maximum number of active subscriptions to the {@code MaybeSource}s. + * @return the new {@code Flowable} instance */ @CheckReturnValue @NonNull @@ -10430,8 +10428,8 @@ public final Flowable flatMapMaybe(@NonNull Function *
Backpressure:
*
The operator consumes the upstream in an unbounded manner.
@@ -10439,8 +10437,8 @@ public final Flowable flatMapMaybe(@NonNull Function{@code flatMapSingle} does not operate by default on a particular {@link Scheduler}. * * @param the result value type - * @param mapper the function that received each source value and transforms them into SingleSources. - * @return the new Flowable instance + * @param mapper the function that received each source value and transforms them into {@code SingleSource}s. + * @return the new {@code Flowable} instance */ @CheckReturnValue @BackpressureSupport(BackpressureKind.UNBOUNDED_IN) @@ -10451,9 +10449,9 @@ public final Flowable flatMapSingle(@NonNull Function *
Backpressure:
*
If {@code maxConcurrency == }{@link Integer#MAX_VALUE} the operator consumes the upstream in an unbounded manner. @@ -10463,11 +10461,11 @@ public final Flowable flatMapSingle(@NonNull Function{@code flatMapSingle} does not operate by default on a particular {@link Scheduler}.
* * @param the result value type - * @param mapper the function that received each source value and transforms them into SingleSources. - * @param delayErrors if true errors from the upstream and inner SingleSources are delayed until each of them + * @param mapper the function that received each source value and transforms them into {@code SingleSource}s. + * @param delayErrors if {@code true}, errors from the upstream and inner {@code SingleSources} are delayed until each of them * terminates. - * @param maxConcurrency the maximum number of active subscriptions to the SingleSources. - * @return the new Flowable instance + * @param maxConcurrency the maximum number of active subscriptions to the {@code SingleSource}s. + * @return the new {@code Flowable} instance */ @CheckReturnValue @NonNull @@ -10494,9 +10492,9 @@ public final Flowable flatMapSingle(@NonNull FunctionReactiveX operators documentation: Subscribe */ @CheckReturnValue @@ -10509,11 +10507,11 @@ public final Disposable forEach(@NonNull Consumer onNext) { /** * Subscribes to the {@link Publisher} and receives notifications for each element until the - * onNext Predicate returns false. + * {@code onNext} Predicate returns {@code false}. *

- * If the Flowable emits an error, it is wrapped into an + * If the {@code Flowable} emits an error, it is wrapped into an * {@link io.reactivex.rxjava3.exceptions.OnErrorNotImplementedException OnErrorNotImplementedException} - * and routed to the RxJavaPlugins.onError handler. + * and routed to the {@link RxJavaPlugins#onError(Throwable)} handler. *

*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner (i.e., no @@ -10527,7 +10525,7 @@ public final Disposable forEach(@NonNull Consumer onNext) { * @return * a {@link Disposable} that allows canceling an asynchronous sequence * @throws NullPointerException - * if {@code onNext} is null + * if {@code onNext} is {@code null} * @see ReactiveX operators documentation: Subscribe */ @CheckReturnValue @@ -10540,7 +10538,7 @@ public final Disposable forEachWhile(@NonNull Predicate onNext) { /** * Subscribes to the {@link Publisher} and receives notifications for each element and error events until the - * onNext Predicate returns false. + * {@code onNext} Predicate returns {@code false}. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner (i.e., no @@ -10556,8 +10554,8 @@ public final Disposable forEachWhile(@NonNull Predicate onNext) { * @return * a {@link Disposable} that allows canceling an asynchronous sequence * @throws NullPointerException - * if {@code onNext} is null, or - * if {@code onError} is null + * if {@code onNext} is {@code null}, or + * if {@code onError} is {@code null} * @see ReactiveX operators documentation: Subscribe */ @CheckReturnValue @@ -10570,7 +10568,7 @@ public final Disposable forEachWhile(@NonNull Predicate onNext, @NonN /** * Subscribes to the {@link Publisher} and receives notifications for each element and the terminal events until the - * onNext Predicate returns false. + * {@code onNext} Predicate returns {@code false}. *
*
Backpressure:
*
The operator consumes the source {@code Publisher} in an unbounded manner (i.e., no @@ -10588,9 +10586,9 @@ public final Disposable forEachWhile(@NonNull Predicate onNext, @NonN * @return * a {@link Disposable} that allows canceling an asynchronous sequence * @throws NullPointerException - * if {@code onNext} is null, or - * if {@code onError} is null, or - * if {@code onComplete} is null + * if {@code onNext} is {@code null}, or + * if {@code onError} is {@code null}, or + * if {@code onComplete} is {@code null} * @see ReactiveX operators documentation: Subscribe */ @CheckReturnValue @@ -10609,20 +10607,20 @@ public final Disposable forEachWhile(@NonNull Predicate onNext, @NonN } /** - * Groups the items emitted by a {@code Publisher} according to a specified criterion, and emits these - * grouped items as {@link GroupedFlowable}s. The emitted {@code GroupedPublisher} allows only a single + * Groups the items emitted by a {@link Publisher} according to a specified criterion, and emits these + * grouped items as {@link GroupedFlowable}s. The emitted {@code GroupedFlowable} allows only a single * {@link Subscriber} during its lifetime and if this {@code Subscriber} cancels before the * source terminates, the next emission by the source having the same key will trigger a new - * {@code GroupedPublisher} emission. + * {@code GroupedFlowable} emission. *

* *

- * Note: A {@link GroupedFlowable} will cache the items it is to emit until such time as it + * Note: A {@code GroupedFlowable} will cache the items it is to emit until such time as it * is subscribed to. For this reason, in order to avoid memory leaks, you should not simply ignore those - * {@code GroupedPublisher}s that do not concern you. Instead, you can signal to them that they may + * {@code GroupedFlowable}s that do not concern you. Instead, you can signal to them that they may * discard their buffers by applying an operator like {@link #ignoreElements} to them. *

- * Note that the {@link GroupedFlowable}s should be subscribed to as soon as possible, otherwise, + * Note that the {@code GroupedFlowable}s should be subscribed to as soon as possible, otherwise, * the unconsumed groups may starve other groups due to the internal backpressure * coordination of the {@code groupBy} operator. Such hangs can be usually avoided by using * {@link #flatMap(Function, int)} or {@link #concatMapEager(Function, int, int)} and overriding the default maximum concurrency @@ -10652,8 +10650,8 @@ public final Disposable forEachWhile(@NonNull Predicate onNext, @NonN * a function that extracts the key for each item * @param * the key type - * @return a {@code Publisher} that emits {@link GroupedFlowable}s, each of which corresponds to a - * unique key value and each of which emits those items from the source Publisher that share that + * @return a {@code Publisher} that emits {@code GroupedFlowable}s, each of which corresponds to a + * unique key value and each of which emits those items from the source {@code Publisher} that share that * key value * @see ReactiveX operators documentation: GroupBy * @see #groupBy(Function, boolean) @@ -10668,20 +10666,20 @@ public final Flowable> groupBy(@NonNull Function * *

- * Note: A {@link GroupedFlowable} will cache the items it is to emit until such time as it + * Note: A {@code GroupedFlowable} will cache the items it is to emit until such time as it * is subscribed to. For this reason, in order to avoid memory leaks, you should not simply ignore those - * {@code GroupedPublisher}s that do not concern you. Instead, you can signal to them that they may + * {@code GroupedFlowable}s that do not concern you. Instead, you can signal to them that they may * discard their buffers by applying an operator like {@link #ignoreElements} to them. *

- * Note that the {@link GroupedFlowable}s should be subscribed to as soon as possible, otherwise, + * Note that the {@code GroupedFlowable}s should be subscribed to as soon as possible, otherwise, * the unconsumed groups may starve other groups due to the internal backpressure * coordination of the {@code groupBy} operator. Such hangs can be usually avoided by using * {@link #flatMap(Function, int)} or {@link #concatMapEager(Function, int, int)} and overriding the default maximum concurrency @@ -10712,10 +10710,10 @@ public final Flowable> groupBy(@NonNull Function * the key type * @param delayError - * if true, the exception from the current Flowable is delayed in each group until that specific group emitted - * the normal values; if false, the exception bypasses values in the groups and is reported immediately. - * @return a {@code Publisher} that emits {@link GroupedFlowable}s, each of which corresponds to a - * unique key value and each of which emits those items from the source Publisher that share that + * if {@code true}, the exception from the current {@code Flowable} is delayed in each group until that specific group emitted + * the normal values; if {@code false}, the exception bypasses values in the groups and is reported immediately. + * @return a {@code Publisher} that emits {@code GroupedFlowable}s, each of which corresponds to a + * unique key value and each of which emits those items from the source {@code Publisher} that share that * key value * @see ReactiveX operators documentation: GroupBy */ @@ -10728,20 +10726,20 @@ public final Flowable> groupBy(@NonNull Function * *

- * Note: A {@link GroupedFlowable} will cache the items it is to emit until such time as it + * Note: A {@code GroupedFlowable} will cache the items it is to emit until such time as it * is subscribed to. For this reason, in order to avoid memory leaks, you should not simply ignore those - * {@code GroupedPublisher}s that do not concern you. Instead, you can signal to them that they may + * {@code GroupedFlowable}s that do not concern you. Instead, you can signal to them that they may * discard their buffers by applying an operator like {@link #ignoreElements} to them. *

- * Note that the {@link GroupedFlowable}s should be subscribed to as soon as possible, otherwise, + * Note that the {@code GroupedFlowable}s should be subscribed to as soon as possible, otherwise, * the unconsumed groups may starve other groups due to the internal backpressure * coordination of the {@code groupBy} operator. Such hangs can be usually avoided by using * {@link #flatMap(Function, int)} or {@link #concatMapEager(Function, int, int)} and overriding the default maximum concurrency @@ -10775,8 +10773,8 @@ public final Flowable> groupBy(@NonNull Function * the element type - * @return a {@code Publisher} that emits {@link GroupedFlowable}s, each of which corresponds to a - * unique key value and each of which emits those items from the source Publisher that share that + * @return a {@code Publisher} that emits {@code GroupedFlowable}s, each of which corresponds to a + * unique key value and each of which emits those items from the source {@code Publisher} that share that * key value * @see ReactiveX operators documentation: GroupBy * @see #groupBy(Function, Function, boolean) @@ -10793,20 +10791,20 @@ public final Flowable> groupBy(@NonNull Function * *

- * Note: A {@link GroupedFlowable} will cache the items it is to emit until such time as it + * Note: A {@code GroupedFlowable} will cache the items it is to emit until such time as it * is subscribed to. For this reason, in order to avoid memory leaks, you should not simply ignore those - * {@code GroupedPublisher}s that do not concern you. Instead, you can signal to them that they may + * {@code GroupedFlowable}s that do not concern you. Instead, you can signal to them that they may * discard their buffers by applying an operator like {@link #ignoreElements} to them. *

- * Note that the {@link GroupedFlowable}s should be subscribed to as soon as possible, otherwise, + * Note that the {@code GroupedFlowable}s should be subscribed to as soon as possible, otherwise, * the unconsumed groups may starve other groups due to the internal backpressure * coordination of the {@code groupBy} operator. Such hangs can be usually avoided by using * {@link #flatMap(Function, int)} or {@link #concatMapEager(Function, int, int)} and overriding the default maximum concurrency @@ -10841,10 +10839,10 @@ public final Flowable> groupBy(@NonNull Function * the element type * @param delayError - * if true, the exception from the current Flowable is delayed in each group until that specific group emitted - * the normal values; if false, the exception bypasses values in the groups and is reported immediately. - * @return a {@code Publisher} that emits {@link GroupedFlowable}s, each of which corresponds to a - * unique key value and each of which emits those items from the source Publisher that share that + * if {@code true}, the exception from the current {@code Flowable} is delayed in each group until that specific group emitted + * the normal values; if {@code false}, the exception bypasses values in the groups and is reported immediately. + * @return a {@code Publisher} that emits {@code GroupedFlowable}s, each of which corresponds to a + * unique key value and each of which emits those items from the source {@code Publisher} that share that * key value * @see ReactiveX operators documentation: GroupBy * @see #groupBy(Function, Function, boolean, int) @@ -10859,20 +10857,20 @@ public final Flowable> groupBy(@NonNull Function * *

- * Note: A {@link GroupedFlowable} will cache the items it is to emit until such time as it + * Note: A {@code GroupedFlowable} will cache the items it is to emit until such time as it * is subscribed to. For this reason, in order to avoid memory leaks, you should not simply ignore those - * {@code GroupedPublisher}s that do not concern you. Instead, you can signal to them that they may + * {@code GroupedFlowable}s that do not concern you. Instead, you can signal to them that they may * discard their buffers by applying an operator like {@link #ignoreElements} to them. *

- * Note that the {@link GroupedFlowable}s should be subscribed to as soon as possible, otherwise, + * Note that the {@code GroupedFlowable}s should be subscribed to as soon as possible, otherwise, * the unconsumed groups may starve other groups due to the internal backpressure * coordination of the {@code groupBy} operator. Such hangs can be usually avoided by using * {@link #flatMap(Function, int)} or {@link #concatMapEager(Function, int, int)} and overriding the default maximum concurrency @@ -10903,16 +10901,16 @@ public final Flowable> groupBy(@NonNull Function * the key type * @param * the element type - * @return a {@code Publisher} that emits {@link GroupedFlowable}s, each of which corresponds to a - * unique key value and each of which emits those items from the source Publisher that share that + * @return a {@code Publisher} that emits {@code GroupedFlowable}s, each of which corresponds to a + * unique key value and each of which emits those items from the source {@code Publisher} that share that * key value * @see ReactiveX operators documentation: GroupBy */ @@ -10931,20 +10929,20 @@ public final Flowable> groupBy(@NonNull Function} with the entry value (not the key!) when an entry in this * map has been evicted. The next source emission will bring about the completion of the evicted - * {@link GroupedFlowable}s and the arrival of an item with the same key as a completed {@link GroupedFlowable} - * will prompt the creation and emission of a new {@link GroupedFlowable} with that key. + * {@code GroupedFlowable}s and the arrival of an item with the same key as a completed {@code GroupedFlowable} + * will prompt the creation and emission of a new {@code GroupedFlowable} with that key. * *

A use case for specifying an {@code evictingMapFactory} is where the source is infinite and fast and * over time the number of keys grows enough to be a concern in terms of the memory footprint of the - * internal hash map containing the {@link GroupedFlowable}s. + * internal hash map containing the {@code GroupedFlowable}s. * *

The map created by an {@code evictingMapFactory} must be thread-safe. * @@ -10969,7 +10967,7 @@ public final Flowable> groupBy(@NonNull Function Flowable> groupBy(@NonNull Function * *

- * Note: A {@link GroupedFlowable} will cache the items it is to emit until such time as it + * Note: A {@code GroupedFlowable} will cache the items it is to emit until such time as it * is subscribed to. For this reason, in order to avoid memory leaks, you should not simply ignore those * {@code GroupedFlowable}s that do not concern you. Instead, you can signal to them that they may * discard their buffers by applying an operator like {@link #ignoreElements} to them. *

- * Note that the {@link GroupedFlowable}s should be subscribed to as soon as possible, otherwise, + * Note that the {@code GroupedFlowable}s should be subscribed to as soon as possible, otherwise, * the unconsumed groups may starve other groups due to the internal backpressure * coordination of the {@code groupBy} operator. Such hangs can be usually avoided by using * {@link #flatMap(Function, int)} or {@link #concatMapEager(Function, int, int)} and overriding the default maximum concurrency @@ -11016,22 +11014,22 @@ public final Flowable> groupBy(@NonNull Function} with the entry value (not the key!) when * an entry in this map has been evicted. The next source emission will bring about the - * completion of the evicted {@link GroupedFlowable}s. See example above. + * completion of the evicted {@code GroupedFlowable}s. See example above. * @param * the key type * @param * the element type - * @return a {@code Publisher} that emits {@link GroupedFlowable}s, each of which corresponds to a - * unique key value and each of which emits those items from the source Publisher that share that + * @return a {@code Publisher} that emits {@code GroupedFlowable}s, each of which corresponds to a + * unique key value and each of which emits those items from the source {@code Publisher} that share that * key value * @see ReactiveX operators documentation: GroupBy * @@ -11054,10 +11052,10 @@ public final Flowable> groupBy(@NonNull Function * There are no guarantees in what order the items get combined when multiple - * items from one or both source Publishers overlap. + * items from one or both source {@code Publisher}s overlap. *

* *

@@ -11068,22 +11066,22 @@ public final Flowable> groupBy(@NonNull Function{@code groupJoin} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the right Publisher source - * @param the element type of the left duration Publishers - * @param the element type of the right duration Publishers + * @param the value type of the right {@code Publisher} source + * @param the element type of the left duration {@code Publisher}s + * @param the element type of the right duration {@code Publisher}s * @param the result type * @param other - * the other Publisher to correlate items from the source Publisher with + * the other {@code Publisher} to correlate items from the source {@code Publisher} with * @param leftEnd - * a function that returns a Publisher whose emissions indicate the duration of the values of - * the source Publisher + * a function that returns a {@code Publisher} whose emissions indicate the duration of the values of + * the source {@code Publisher} * @param rightEnd - * a function that returns a Publisher whose emissions indicate the duration of the values of - * the {@code right} Publisher + * a function that returns a {@code Publisher} whose emissions indicate the duration of the values of + * the {@code right} {@code Publisher} * @param resultSelector - * a function that takes an item emitted by each Publisher and returns the value to be emitted - * by the resulting Publisher - * @return a Flowable that emits items based on combining those items emitted by the source Publishers + * a function that takes an item emitted by each {@code Publisher} and returns the value to be emitted + * by the resulting {@code Publisher} + * @return a {@code Flowable} that emits items based on combining those items emitted by the source {@code Publisher}s * whose durations overlap * @see ReactiveX operators documentation: Join */ @@ -11105,7 +11103,7 @@ public final Flowable groupJoin( } /** - * Hides the identity of this Flowable and its Subscription. + * Hides the identity of this {@code Flowable} and its {@link Subscription}. *

Allows hiding extra features such as {@link Processor}'s * {@link Subscriber} methods or preventing certain identity-based * optimizations (fusion). @@ -11116,7 +11114,7 @@ public final Flowable groupJoin( *

Scheduler:
*
{@code hide} does not operate by default on a particular {@link Scheduler}.
*
- * @return the new Flowable instance + * @return the new {@code Flowable} instance * * @since 2.0 */ @@ -11129,7 +11127,7 @@ public final Flowable hide() { } /** - * Ignores all items emitted by the source Publisher and only calls {@code onComplete} or {@code onError}. + * Ignores all items emitted by the source {@link Publisher} and only calls {@code onComplete} or {@code onError}. *

* *

@@ -11140,8 +11138,8 @@ public final Flowable hide() { *
{@code ignoreElements} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a Completable that only calls {@code onComplete} or {@code onError}, based on which one is - * called by the source Publisher + * @return a {@link Completable} that only calls {@code onComplete} or {@code onError}, based on which one is + * called by the source {@code Publisher} * @see ReactiveX operators documentation: IgnoreElements */ @CheckReturnValue @@ -11153,9 +11151,9 @@ public final Completable ignoreElements() { } /** - * Returns a Single that emits {@code true} if the source Publisher is empty, otherwise {@code false}. + * Returns a {@link Single} that emits {@code true} if the source {@link Publisher} is empty, otherwise {@code false}. *

- * In Rx.Net this is negated as the {@code any} Subscriber but we renamed this in RxJava to better match Java + * In Rx.Net this is negated as the {@code any} {@link Subscriber} but we renamed this in RxJava to better match Java * naming idioms. *

* @@ -11167,7 +11165,7 @@ public final Completable ignoreElements() { *

{@code isEmpty} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a Flowable that emits a Boolean + * @return a {@code Single} that emits a {@link Boolean} * @see ReactiveX operators documentation: Contains */ @CheckReturnValue @@ -11179,10 +11177,10 @@ public final Single isEmpty() { } /** - * Correlates the items emitted by two Publishers based on overlapping durations. + * Correlates the items emitted by two {@link Publisher}s based on overlapping durations. *

* There are no guarantees in what order the items get combined when multiple - * items from one or both source Publishers overlap. + * items from one or both source {@code Publisher}s overlap. *

* *

@@ -11193,22 +11191,22 @@ public final Single isEmpty() { *
{@code join} does not operate by default on a particular {@link Scheduler}.
*
* - * @param the value type of the right Publisher source - * @param the element type of the left duration Publishers - * @param the element type of the right duration Publishers + * @param the value type of the right {@code Publisher} source + * @param the element type of the left duration {@code Publisher}s + * @param the element type of the right duration {@code Publisher}s * @param the result type * @param other - * the second Publisher to join items from + * the second {@code Publisher} to join items from * @param leftEnd - * a function to select a duration for each item emitted by the source Publisher, used to + * a function to select a duration for each item emitted by the source {@code Publisher}, used to * determine overlap * @param rightEnd - * a function to select a duration for each item emitted by the {@code right} Publisher, used to + * a function to select a duration for each item emitted by the {@code right} {@code Publisher}, used to * determine overlap * @param resultSelector - * a function that computes an item to be emitted by the resulting Publisher for any two - * overlapping items emitted by the two Publishers - * @return a Flowable that emits items correlating to items emitted by the source Publishers that have + * a function that computes an item to be emitted by the resulting {@code Publisher} for any two + * overlapping items emitted by the two {@code Publisher}s + * @return a {@code Flowable} that emits items correlating to items emitted by the source {@code Publisher}s that have * overlapping durations * @see ReactiveX operators documentation: Join */ @@ -11230,19 +11228,19 @@ public final Flowable join( } /** - * Returns a Maybe that emits the last item emitted by this Flowable or completes if - * this Flowable is empty. + * Returns a {@link Maybe} that emits the last item emitted by this {@code Flowable} or completes if + * this {@code Flowable} is empty. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in an + *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in an * unbounded manner (i.e., without applying backpressure).
*
Scheduler:
*
{@code lastElement} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a new Maybe instance + * @return a new {@code Maybe} instance * @see ReactiveX operators documentation: Last */ @CheckReturnValue @@ -11254,21 +11252,21 @@ public final Maybe lastElement() { } /** - * Returns a Single that emits only the last item emitted by this Flowable, or a default item - * if this Flowable completes without emitting any items. + * Returns a {@link Single} that emits only the last item emitted by this {@code Flowable}, or a default item + * if this {@code Flowable} completes without emitting any items. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in an + *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in an * unbounded manner (i.e., without applying backpressure).
*
Scheduler:
*
{@code last} does not operate by default on a particular {@link Scheduler}.
*
* * @param defaultItem - * the default item to emit if the source Publisher is empty - * @return the new Single instance + * the default item to emit if the source {@code Publisher} is empty + * @return the new {@code Single} instance * @see ReactiveX operators documentation: Last */ @CheckReturnValue @@ -11281,19 +11279,19 @@ public final Single last(@NonNull T defaultItem) { } /** - * Returns a Single that emits only the last item emitted by this Flowable or signals - * a {@link NoSuchElementException} if this Flowable is empty. + * Returns a {@link Single} that emits only the last item emitted by this {@code Flowable} or signals + * a {@link NoSuchElementException} if this {@code Flowable} is empty. *

* *

*
Backpressure:
- *
The operator honors backpressure from downstream and consumes the source {@code Publisher} in an + *
The operator honors backpressure from downstream and consumes the source {@link Publisher} in an * unbounded manner (i.e., without applying backpressure).
*
Scheduler:
*
{@code lastOrError} does not operate by default on a particular {@link Scheduler}.
*
* - * @return the new Single instance + * @return the new {@code Single} instance * @see ReactiveX operators documentation: Last */ @CheckReturnValue @@ -11309,7 +11307,7 @@ public final Single lastOrError() { * other standard composition methods first; * Returns a {@code Flowable} which, when subscribed to, invokes the {@link FlowableOperator#apply(Subscriber) apply(Subscriber)} method * of the provided {@link FlowableOperator} for each individual downstream {@link Subscriber} and allows the - * insertion of a custom operator by accessing the downstream's {@link Subscriber} during this subscription phase + * insertion of a custom operator by accessing the downstream's {@code Subscriber} during this subscription phase * and providing a new {@code Subscriber}, containing the custom operator's intended business logic, that will be * used in the subscription process going further upstream. *

@@ -11426,27 +11424,27 @@ public final Single lastOrError() { * class and creating a {@link FlowableTransformer} with it is recommended. *

* Note also that it is not possible to stop the subscription phase in {@code lift()} as the {@code apply()} method - * requires a non-null {@code Subscriber} instance to be returned, which is then unconditionally subscribed to + * requires a non-{@code null} {@code Subscriber} instance to be returned, which is then unconditionally subscribed to * the upstream {@code Flowable}. For example, if the operator decided there is no reason to subscribe to the * upstream source because of some optimization possibility or a failure to prepare the operator, it still has to - * return a {@code Subscriber} that should immediately cancel the upstream's {@code Subscription} in its + * return a {@code Subscriber} that should immediately cancel the upstream's {@link Subscription} in its * {@code onSubscribe} method. Again, using a {@code FlowableTransformer} and extending the {@code Flowable} is * a better option as {@link #subscribeActual} can decide to not subscribe to its upstream after all. *

*
Backpressure:
- *
The {@code Subscriber} instance returned by the {@link FlowableOperator} is responsible to be - * backpressure-aware or document the fact that the consumer of the returned {@code Publisher} has to apply one of + *
The {@code Subscriber} instance returned by the {@code FlowableOperator} is responsible to be + * backpressure-aware or document the fact that the consumer of the returned {@link Publisher} has to apply one of * the {@code onBackpressureXXX} operators.
*
Scheduler:
*
{@code lift} does not operate by default on a particular {@link Scheduler}, however, the - * {@link FlowableOperator} may use a {@code Scheduler} to support its own asynchronous behavior.
+ * {@code FlowableOperator} may use a {@code Scheduler} to support its own asynchronous behavior. *
* * @param the output value type - * @param lifter the {@link FlowableOperator} that receives the downstream's {@code Subscriber} and should return + * @param lifter the {@code FlowableOperator} that receives the downstream's {@code Subscriber} and should return * a {@code Subscriber} with custom behavior to be used as the consumer for the current * {@code Flowable}. - * @return the new Flowable instance + * @return the new {@code Flowable} instance * @see RxJava wiki: Writing operators * @see #compose(FlowableTransformer) */ @@ -11460,7 +11458,7 @@ public final Flowable lift(@NonNull FlowableOperator * @@ -11474,8 +11472,8 @@ public final Flowable lift(@NonNull FlowableOperator the output type * @param mapper - * a function to apply to each item emitted by the Publisher - * @return a Flowable that emits the items from the source Publisher, transformed by the specified + * a function to apply to each item emitted by the {@code Publisher} + * @return a {@code Flowable} that emits the items from the source {@code Publisher}, transformed by the specified * function * @see ReactiveX operators documentation: Map * @see #mapOptional(Function) @@ -11490,20 +11488,20 @@ public final Flowable lift(@NonNull FlowableOperatorand notifications from the source - * Publisher into emissions marked with their original types within {@link Notification} objects. + * Returns a {@code Flowable} that represents all of the emissions and notifications from the source + * {@link Publisher} into emissions marked with their original types within {@link Notification} objects. *

* *

*
Backpressure:
*
The operator honors backpressure from downstream and expects it from the source {@code Publisher}. - * If this expectation is violated, the operator may throw an {@code IllegalStateException}.
+ * If this expectation is violated, the operator may throw an {@link IllegalStateException}. *
Scheduler:
*
{@code materialize} does not operate by default on a particular {@link Scheduler}.
*
* - * @return a Flowable that emits items that are the result of materializing the items and notifications - * of the source Publisher + * @return a {@code Flowable} that emits items that are the result of materializing the items and notifications + * of the source {@code Publisher} * @see ReactiveX operators documentation: Materialize * @see #dematerialize(Function) */ @@ -11516,23 +11514,23 @@ public final Flowable> materialize() { } /** - * Flattens this and another Publisher into a single Publisher, without any transformation. + * Flattens this and another {@link Publisher} into a single {@code Publisher}, without any transformation. *

* *

- * You can combine items emitted by multiple Publishers so that they appear as a single Publisher, by + * You can combine items emitted by multiple {@code Publisher}s so that they appear as a single {@code Publisher}, by * using the {@code mergeWith} method. *

*
Backpressure:
*
The operator honors backpressure from downstream. This and the other {@code Publisher}s are expected to honor - * backpressure; if violated, the operator may signal {@code MissingBackpressureException}.
+ * backpressure; if violated, the operator may signal {@link MissingBackpressureException}. *
Scheduler:
*
{@code mergeWith} does not operate by default on a particular {@link Scheduler}.
*
* * @param other - * a Publisher to be merged - * @return a Flowable that emits all of the items emitted by the source Publishers + * a {@code Publisher} to be merged + * @return a {@code Flowable} that emits all of the items emitted by the source {@code Publisher}s * @see ReactiveX operators documentation: Merge */ @CheckReturnValue @@ -11545,7 +11543,7 @@ public final Flowable mergeWith(@NonNull Publisher other) { } /** - * Merges the sequence of items of this Flowable with the success value of the other SingleSource. + * Merges the sequence of items of this {@code Flowable} with the success value of the other {@link SingleSource}. *

* *

@@ -11560,7 +11558,7 @@ public final Flowable mergeWith(@NonNull Publisher other) { * *

History: 2.1.10 - experimental * @param other the {@code SingleSource} whose success value to merge with - * @return the new Flowable instance + * @return the new {@code Flowable} instance * @since 2.2 */ @CheckReturnValue @@ -11573,8 +11571,8 @@ public final Flowable mergeWith(@NonNull SingleSource other) { } /** - * Merges the sequence of items of this Flowable with the success value of the other MaybeSource - * or waits for both to complete normally if the MaybeSource is empty. + * Merges the sequence of items of this {@code Flowable} with the success value of the other {@link MaybeSource} + * or waits for both to complete normally if the {@code MaybeSource} is empty. *

* *

@@ -11589,7 +11587,7 @@ public final Flowable mergeWith(@NonNull SingleSource other) { * *

History: 2.1.10 - experimental * @param other the {@code MaybeSource} which provides a success value to merge with or completes - * @return the new Flowable instance + * @return the new {@code Flowable} instance * @since 2.2 */ @CheckReturnValue @@ -11602,20 +11600,20 @@ public final Flowable mergeWith(@NonNull MaybeSource other) { } /** - * Relays the items of this Flowable and completes only when the other CompletableSource completes + * Relays the items of this {@code Flowable} and completes only when the other {@link CompletableSource} completes * as well. *

* *

*
Backpressure:
- *
The operator doesn't interfere with backpressure which is determined by the source {@code Publisher}'s backpressure + *
The operator doesn't interfere with backpressure which is determined by the source {@link Publisher}'s backpressure * behavior.
*
Scheduler:
*
{@code mergeWith} does not operate by default on a particular {@link Scheduler}.
*
*

History: 2.1.10 - experimental * @param other the {@code CompletableSource} to await for completion - * @return the new Flowable instance + * @return the new {@code Flowable} instance * @since 2.2 */ @CheckReturnValue @@ -11628,22 +11626,22 @@ public final Flowable mergeWith(@NonNull CompletableSource other) { } /** - * Modifies a Publisher to perform its emissions and notifications on a specified {@link Scheduler}, + * Modifies a {@link Publisher} to perform its emissions and notifications on a specified {@link Scheduler}, * asynchronously with a bounded buffer of {@link #bufferSize()} slots. * - *

Note that onError notifications will cut ahead of onNext notifications on the emission thread if Scheduler is truly + *

Note that {@code onError} notifications will cut ahead of {@code onNext} notifications on the emission thread if {@code Scheduler} is truly * asynchronous. If strict event ordering is required, consider using the {@link #observeOn(Scheduler, boolean)} overload. *

* *

- * This operator keeps emitting as many signals as it can on the given Scheduler's Worker thread, + * This operator keeps emitting as many signals as it can on the given {@code Scheduler}'s Worker thread, * which may result in a longer than expected occupation of this thread. In other terms, * it does not allow per-signal fairness in case the worker runs on a shared underlying thread. * If such fairness and signal/work interleaving is preferred, use the delay operator with zero time instead. *

*
Backpressure:
*
This operator honors backpressure from downstream and expects it from the source {@code Publisher}. Violating this - * expectation will lead to {@code MissingBackpressureException}. This is the most common operator where the exception + * expectation will lead to {@link MissingBackpressureException}. This is the most common operator where the exception * pops up; look for sources up the chain that don't support backpressure, * such as {@link #interval(long, TimeUnit)}, {@link #timer(long, TimeUnit)}, * {@link io.reactivex.rxjava3.processors.PublishProcessor PublishProcessor} or @@ -11656,13 +11654,13 @@ public final Flowable mergeWith(@NonNull CompletableSource other) { * {@link #delay(long, TimeUnit, Scheduler)} with zero time instead. *
*
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param scheduler - * the {@link Scheduler} to notify {@link Subscriber}s on - * @return the source Publisher modified so that its {@link Subscriber}s are notified on the specified - * {@link Scheduler} + * the {@code Scheduler} to notify {@link Subscriber}s on + * @return the source {@code Publisher} modified so that its {@code Subscriber}s are notified on the specified + * {@code Scheduler} * @see ReactiveX operators documentation: ObserveOn * @see RxJava Threading Examples * @see #subscribeOn @@ -11679,19 +11677,19 @@ public final Flowable observeOn(@NonNull Scheduler scheduler) { } /** - * Modifies a Publisher to perform its emissions and notifications on a specified {@link Scheduler}, - * asynchronously with a bounded buffer and optionally delays onError notifications. + * Modifies a {@link Publisher} to perform its emissions and notifications on a specified {@link Scheduler}, + * asynchronously with a bounded buffer and optionally delays {@code onError} notifications. *

* *

- * This operator keeps emitting as many signals as it can on the given Scheduler's Worker thread, + * This operator keeps emitting as many signals as it can on the given {@code Scheduler}'s Worker thread, * which may result in a longer than expected occupation of this thread. In other terms, * it does not allow per-signal fairness in case the worker runs on a shared underlying thread. * If such fairness and signal/work interleaving is preferred, use the delay operator with zero time instead. *

*
Backpressure:
*
This operator honors backpressure from downstream and expects it from the source {@code Publisher}. Violating this - * expectation will lead to {@code MissingBackpressureException}. This is the most common operator where the exception + * expectation will lead to {@link MissingBackpressureException}. This is the most common operator where the exception * pops up; look for sources up the chain that don't support backpressure, * such as {@link #interval(long, TimeUnit)}, {@link #timer(long, TimeUnit)}, * {@link io.reactivex.rxjava3.processors.PublishProcessor PublishProcessor} or @@ -11704,17 +11702,17 @@ public final Flowable observeOn(@NonNull Scheduler scheduler) { * {@link #delay(long, TimeUnit, Scheduler, boolean)} with zero time instead. *
*
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param scheduler - * the {@link Scheduler} to notify {@link Subscriber}s on + * the {@code Scheduler} to notify {@link Subscriber}s on * @param delayError - * indicates if the onError notification may not cut ahead of onNext notification on the other side of the - * scheduling boundary. If true a sequence ending in onError will be replayed in the same order as was received + * indicates if the {@code onError} notification may not cut ahead of {@code onNext} notification on the other side of the + * scheduling boundary. If {@code true} a sequence ending in {@code onError} will be replayed in the same order as was received * from upstream - * @return the source Publisher modified so that its {@link Subscriber}s are notified on the specified - * {@link Scheduler} + * @return the source {@code Publisher} modified so that its {@code Subscriber}s are notified on the specified + * {@code Scheduler} * @see ReactiveX operators documentation: ObserveOn * @see RxJava Threading Examples * @see #subscribeOn @@ -11731,19 +11729,19 @@ public final Flowable observeOn(@NonNull Scheduler scheduler, boolean delayEr } /** - * Modifies a Publisher to perform its emissions and notifications on a specified {@link Scheduler}, - * asynchronously with a bounded buffer of configurable size and optionally delays onError notifications. + * Modifies a {@link Publisher} to perform its emissions and notifications on a specified {@link Scheduler}, + * asynchronously with a bounded buffer of configurable size and optionally delays {@code onError} notifications. *

* *

- * This operator keeps emitting as many signals as it can on the given Scheduler's Worker thread, + * This operator keeps emitting as many signals as it can on the given {@code Scheduler}'s Worker thread, * which may result in a longer than expected occupation of this thread. In other terms, * it does not allow per-signal fairness in case the worker runs on a shared underlying thread. * If such fairness and signal/work interleaving is preferred, use the delay operator with zero time instead. *

*
Backpressure:
*
This operator honors backpressure from downstream and expects it from the source {@code Publisher}. Violating this - * expectation will lead to {@code MissingBackpressureException}. This is the most common operator where the exception + * expectation will lead to {@link MissingBackpressureException}. This is the most common operator where the exception * pops up; look for sources up the chain that don't support backpressure, * such as {@link #interval(long, TimeUnit)}, {@link #timer(long, TimeUnit)}, * {@link io.reactivex.rxjava3.processors.PublishProcessor PublishProcessor} or @@ -11756,18 +11754,18 @@ public final Flowable observeOn(@NonNull Scheduler scheduler, boolean delayEr * {@link #delay(long, TimeUnit, Scheduler, boolean)} with zero time instead. *
*
Scheduler:
- *
You specify which {@link Scheduler} this operator will use.
+ *
You specify which {@code Scheduler} this operator will use.
*
* * @param scheduler - * the {@link Scheduler} to notify {@link Subscriber}s on + * the {@code Scheduler} to notify {@link Subscriber}s on * @param delayError - * indicates if the onError notification may not cut ahead of onNext notification on the other side of the - * scheduling boundary. If true a sequence ending in onError will be replayed in the same order as was received + * indicates if the {@code onError} notification may not cut ahead of {@code onNext} notification on the other side of the + * scheduling boundary. If {@code true} a sequence ending in {@code onError} will be replayed in the same order as was received * from upstream * @param bufferSize the size of the buffer. - * @return the source Publisher modified so that its {@link Subscriber}s are notified on the specified - * {@link Scheduler} + * @return the source {@code Publisher} modified so that its {@code Subscriber}s are notified on the specified + * {@code Scheduler} * @see ReactiveX operators documentation: ObserveOn * @see RxJava Threading Examples * @see #subscribeOn @@ -11786,7 +11784,7 @@ public final Flowable observeOn(@NonNull Scheduler scheduler, boolean delayEr } /** - * Filters the items emitted by a Publisher, only emitting those of the specified type. + * Filters the items emitted by a {@link Publisher}, only emitting those of the specified type. *

* *

@@ -11799,8 +11797,8 @@ public final Flowable observeOn(@NonNull Scheduler scheduler, boolean delayEr * * @param the output type * @param clazz - * the class type to filter the items emitted by the source Publisher - * @return a Flowable that emits items from the source Publisher of type {@code clazz} + * the class type to filter the items emitted by the source {@code Publisher} + * @return a {@code Flowable} that emits items from the source {@code Publisher} of type {@code clazz} * @see ReactiveX operators documentation: Filter */ @CheckReturnValue @@ -11813,7 +11811,7 @@ public final Flowable ofType(@NonNull Class clazz) { } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer these + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer these * items indefinitely until they can be emitted. *

* @@ -11825,7 +11823,7 @@ public final Flowable ofType(@NonNull Class clazz) { *

{@code onBackpressureBuffer} does not operate by default on a particular {@link Scheduler}.
*
* - * @return the source Publisher modified to buffer items to the extent system resources allow + * @return the source {@code Publisher} modified to buffer items to the extent system resources allow * @see ReactiveX operators documentation: backpressure operators */ @CheckReturnValue @@ -11837,7 +11835,7 @@ public final Flowable onBackpressureBuffer() { } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer these + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer these * items indefinitely until they can be emitted. *

* @@ -11849,10 +11847,10 @@ public final Flowable onBackpressureBuffer() { *

{@code onBackpressureBuffer} does not operate by default on a particular {@link Scheduler}.
* * @param delayError - * if true, an exception from the current Flowable is delayed until all buffered elements have been - * consumed by the downstream; if false, an exception is immediately signaled to the downstream, skipping + * if {@code true}, an exception from the current {@code Flowable} is delayed until all buffered elements have been + * consumed by the downstream; if {@code false}, an exception is immediately signaled to the downstream, skipping * any buffered element - * @return the source Publisher modified to buffer items to the extent system resources allow + * @return the source {@code Publisher} modified to buffer items to the extent system resources allow * @see ReactiveX operators documentation: backpressure operators */ @CheckReturnValue @@ -11864,8 +11862,8 @@ public final Flowable onBackpressureBuffer(boolean delayError) { } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer up to - * a given amount of items until they can be emitted. The resulting Publisher will signal + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer up to + * a given amount of items until they can be emitted. The resulting {@code Publisher} will signal * a {@code BufferOverflowException} via {@code onError} as soon as the buffer's capacity is exceeded, dropping all undelivered * items, and canceling the source. *

@@ -11892,8 +11890,8 @@ public final Flowable onBackpressureBuffer(int capacity) { } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer up to - * a given amount of items until they can be emitted. The resulting Publisher will signal + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer up to + * a given amount of items until they can be emitted. The resulting {@code Publisher} will signal * a {@code BufferOverflowException} via {@code onError} as soon as the buffer's capacity is exceeded, dropping all undelivered * items, and canceling the source. *

@@ -11908,8 +11906,8 @@ public final Flowable onBackpressureBuffer(int capacity) { * * @param capacity number of slots available in the buffer. * @param delayError - * if true, an exception from the current Flowable is delayed until all buffered elements have been - * consumed by the downstream; if false, an exception is immediately signaled to the downstream, skipping + * if {@code true}, an exception from the current {@code Flowable} is delayed until all buffered elements have been + * consumed by the downstream; if {@code false}, an exception is immediately signaled to the downstream, skipping * any buffered element * @return the source {@code Publisher} modified to buffer items up to the given capacity. * @see ReactiveX operators documentation: backpressure operators @@ -11924,8 +11922,8 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError) } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer up to - * a given amount of items until they can be emitted. The resulting Publisher will signal + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer up to + * a given amount of items until they can be emitted. The resulting {@code Publisher} will signal * a {@code BufferOverflowException} via {@code onError} as soon as the buffer's capacity is exceeded, dropping all undelivered * items, and canceling the source. *

@@ -11940,11 +11938,11 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError) * * @param capacity number of slots available in the buffer. * @param delayError - * if true, an exception from the current Flowable is delayed until all buffered elements have been - * consumed by the downstream; if false, an exception is immediately signaled to the downstream, skipping + * if {@code true}, an exception from the current {@code Flowable} is delayed until all buffered elements have been + * consumed by the downstream; if {@code false}, an exception is immediately signaled to the downstream, skipping * any buffered element * @param unbounded - * if true, the capacity value is interpreted as the internal "island" size of the unbounded buffer + * if {@code true}, the capacity value is interpreted as the internal "island" size of the unbounded buffer * @return the source {@code Publisher} modified to buffer items up to the given capacity. * @see ReactiveX operators documentation: backpressure operators * @since 1.1.0 @@ -11959,8 +11957,8 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError, } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer up to - * a given amount of items until they can be emitted. The resulting Publisher will signal + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer up to + * a given amount of items until they can be emitted. The resulting {@code Publisher} will signal * a {@code BufferOverflowException} via {@code onError} as soon as the buffer's capacity is exceeded, dropping all undelivered * items, canceling the source, and notifying the producer with {@code onOverflow}. *

@@ -11975,11 +11973,11 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError, * * @param capacity number of slots available in the buffer. * @param delayError - * if true, an exception from the current Flowable is delayed until all buffered elements have been - * consumed by the downstream; if false, an exception is immediately signaled to the downstream, skipping + * if {@code true}, an exception from the current {@code Flowable} is delayed until all buffered elements have been + * consumed by the downstream; if {@code false}, an exception is immediately signaled to the downstream, skipping * any buffered element * @param unbounded - * if true, the capacity value is interpreted as the internal "island" size of the unbounded buffer + * if {@code true}, the capacity value is interpreted as the internal "island" size of the unbounded buffer * @param onOverflow action to execute if an item needs to be buffered, but there are no available slots. Null is allowed. * @return the source {@code Publisher} modified to buffer items up to the given capacity * @see ReactiveX operators documentation: backpressure operators @@ -11997,8 +11995,8 @@ public final Flowable onBackpressureBuffer(int capacity, boolean delayError, } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer up to - * a given amount of items until they can be emitted. The resulting Publisher will signal + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer up to + * a given amount of items until they can be emitted. The resulting {@code Publisher} will signal * a {@code BufferOverflowException} via {@code onError} as soon as the buffer's capacity is exceeded, dropping all undelivered * items, canceling the source, and notifying the producer with {@code onOverflow}. *

@@ -12026,8 +12024,8 @@ public final Flowable onBackpressureBuffer(int capacity, @NonNull Action onOv } /** - * Instructs a Publisher that is emitting items faster than its Subscriber can consume them to buffer up to - * a given amount of items until they can be emitted. The resulting Publisher will behave as determined + * Instructs a {@link Publisher} that is emitting items faster than its {@link Subscriber} can consume them to buffer up to + * a given amount of items until they can be emitted. The resulting {@code Publisher} will behave as determined * by {@code overflowStrategy} if the buffer capacity is exceeded. * *