Replace RxJava in Downloader (#9256)

* Rename removeFromQueueByPredicate to removeFromQueueIf

Follow-up to PR comment in #9511

* Make Download hashCode stable

Mutating pages would previously change the Download hashCode, which
breaks HashMap lookups.

* Convert Donwloader subscription to coroutine

Replace downloadsRelay with activeDownloadsFlow. Instead of managing
a PublishRelay independent from the queue, derive a Flow of active
downloads directly from the queue StateFlow. (This will allow
updating the queue without pausing the downloader, to be done in a
follow-up PR.)

When a download completes successfully, the downloads is removed from
queueState. This updates activeDownloadsFlow and causes the
downloaderJob start the download job for the next active download.

When a download fails, the download is left in the queue, so
queueState is not modified. To make activeDownloadsFlow update
without a change to queueState, use transformLatest and use the
Download statusFlows to suspend until a download reaches the ERROR
state.

To avoid stopping and starting downloads every time
activeDownloadsFlow emits a new value, maintain a map of current
download Jobs and only start/stop jobs in the difference between
downloadJobs and activeDownloads. To make sure all child download
jobs are cancelled when the top-level downloader job is cancelled,
use supervisorScope.

* Remove obsolete main thread references in Downloader

Thread safety of the queue state used to be guaranteed by running all
queue mutation on the main thread, but this has not been true for
some time. Since the queue state is now backed by a StateFlow,
queueState can be safely updated by any thread.
This commit is contained in:
Two-Ai 2023-05-24 18:02:27 -04:00 committed by GitHub
parent d8998aacb4
commit 3ae1e37c40
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 90 additions and 82 deletions

View File

@ -2,7 +2,6 @@ package eu.kanade.tachiyomi.data.download
import android.content.Context import android.content.Context
import com.hippo.unifile.UniFile import com.hippo.unifile.UniFile
import com.jakewharton.rxrelay.PublishRelay
import eu.kanade.domain.chapter.model.toSChapter import eu.kanade.domain.chapter.model.toSChapter
import eu.kanade.domain.manga.model.getComicInfo import eu.kanade.domain.manga.model.getComicInfo
import eu.kanade.tachiyomi.R import eu.kanade.tachiyomi.R
@ -17,26 +16,31 @@ import eu.kanade.tachiyomi.util.storage.DiskUtil
import eu.kanade.tachiyomi.util.storage.DiskUtil.NOMEDIA_FILE import eu.kanade.tachiyomi.util.storage.DiskUtil.NOMEDIA_FILE
import eu.kanade.tachiyomi.util.storage.saveTo import eu.kanade.tachiyomi.util.storage.saveTo
import kotlinx.coroutines.CancellationException import kotlinx.coroutines.CancellationException
import kotlinx.coroutines.CoroutineScope
import kotlinx.coroutines.Dispatchers import kotlinx.coroutines.Dispatchers
import kotlinx.coroutines.Job
import kotlinx.coroutines.SupervisorJob
import kotlinx.coroutines.async import kotlinx.coroutines.async
import kotlinx.coroutines.delay import kotlinx.coroutines.delay
import kotlinx.coroutines.flow.MutableStateFlow import kotlinx.coroutines.flow.MutableStateFlow
import kotlinx.coroutines.flow.asFlow import kotlinx.coroutines.flow.asFlow
import kotlinx.coroutines.flow.asStateFlow import kotlinx.coroutines.flow.asStateFlow
import kotlinx.coroutines.flow.collectLatest
import kotlinx.coroutines.flow.combine
import kotlinx.coroutines.flow.distinctUntilChanged
import kotlinx.coroutines.flow.filter
import kotlinx.coroutines.flow.first import kotlinx.coroutines.flow.first
import kotlinx.coroutines.flow.flatMapMerge import kotlinx.coroutines.flow.flatMapMerge
import kotlinx.coroutines.flow.flow import kotlinx.coroutines.flow.flow
import kotlinx.coroutines.flow.flowOn import kotlinx.coroutines.flow.flowOn
import kotlinx.coroutines.flow.retryWhen import kotlinx.coroutines.flow.retryWhen
import kotlinx.coroutines.flow.transformLatest
import kotlinx.coroutines.flow.update import kotlinx.coroutines.flow.update
import kotlinx.coroutines.runBlocking import kotlinx.coroutines.launch
import kotlinx.coroutines.supervisorScope
import logcat.LogPriority import logcat.LogPriority
import nl.adaptivity.xmlutil.serialization.XML import nl.adaptivity.xmlutil.serialization.XML
import okhttp3.Response import okhttp3.Response
import rx.Observable
import rx.Subscription
import rx.android.schedulers.AndroidSchedulers
import rx.schedulers.Schedulers
import tachiyomi.core.metadata.comicinfo.COMIC_INFO_FILE import tachiyomi.core.metadata.comicinfo.COMIC_INFO_FILE
import tachiyomi.core.metadata.comicinfo.ComicInfo import tachiyomi.core.metadata.comicinfo.ComicInfo
import tachiyomi.core.util.lang.awaitSingle import tachiyomi.core.util.lang.awaitSingle
@ -61,11 +65,7 @@ import java.util.zip.ZipOutputStream
/** /**
* This class is the one in charge of downloading chapters. * This class is the one in charge of downloading chapters.
* *
* Its queue contains the list of chapters to download. In order to download them, the downloader * Its queue contains the list of chapters to download.
* subscription must be running and the list of chapters must be sent to them by [downloadsRelay].
*
* The queue manipulation must be done in one thread (currently the main thread) to avoid unexpected
* behavior, but it's safe to read it from multiple threads.
*/ */
class Downloader( class Downloader(
private val context: Context, private val context: Context,
@ -93,21 +93,14 @@ class Downloader(
*/ */
private val notifier by lazy { DownloadNotifier(context) } private val notifier by lazy { DownloadNotifier(context) }
/** private val scope = CoroutineScope(SupervisorJob() + Dispatchers.IO)
* Downloader subscription. private var downloaderJob: Job? = null
*/
private var subscription: Subscription? = null
/**
* Relay to send a list of downloads to the downloader.
*/
private val downloadsRelay = PublishRelay.create<List<Download>>()
/** /**
* Whether the downloader is running. * Whether the downloader is running.
*/ */
val isRunning: Boolean val isRunning: Boolean
get() = subscription != null get() = downloaderJob?.isActive ?: false
/** /**
* Whether the downloader is paused * Whether the downloader is paused
@ -129,18 +122,17 @@ class Downloader(
* @return true if the downloader is started, false otherwise. * @return true if the downloader is started, false otherwise.
*/ */
fun start(): Boolean { fun start(): Boolean {
if (subscription != null || queueState.value.isEmpty()) { if (isRunning || queueState.value.isEmpty()) {
return false return false
} }
initializeSubscription()
val pending = queueState.value.filter { it.status != Download.State.DOWNLOADED } val pending = queueState.value.filter { it.status != Download.State.DOWNLOADED }
pending.forEach { if (it.status != Download.State.QUEUE) it.status = Download.State.QUEUE } pending.forEach { if (it.status != Download.State.QUEUE) it.status = Download.State.QUEUE }
isPaused = false isPaused = false
downloadsRelay.call(pending) launchDownloaderJob()
return pending.isNotEmpty() return pending.isNotEmpty()
} }
@ -148,7 +140,7 @@ class Downloader(
* Stops the downloader. * Stops the downloader.
*/ */
fun stop(reason: String? = null) { fun stop(reason: String? = null) {
destroySubscription() cancelDownloaderJob()
queueState.value queueState.value
.filter { it.status == Download.State.DOWNLOADING } .filter { it.status == Download.State.DOWNLOADING }
.forEach { it.status = Download.State.ERROR } .forEach { it.status = Download.State.ERROR }
@ -176,7 +168,7 @@ class Downloader(
* Pauses the downloader * Pauses the downloader
*/ */
fun pause() { fun pause() {
destroySubscription() cancelDownloaderJob()
queueState.value queueState.value
.filter { it.status == Download.State.DOWNLOADING } .filter { it.status == Download.State.DOWNLOADING }
.forEach { it.status = Download.State.QUEUE } .forEach { it.status = Download.State.QUEUE }
@ -187,7 +179,7 @@ class Downloader(
* Removes everything from the queue. * Removes everything from the queue.
*/ */
fun clearQueue() { fun clearQueue() {
destroySubscription() cancelDownloaderJob()
_clearQueue() _clearQueue()
notifier.dismissProgress() notifier.dismissProgress()
@ -196,49 +188,74 @@ class Downloader(
/** /**
* Prepares the subscriptions to start downloading. * Prepares the subscriptions to start downloading.
*/ */
private fun initializeSubscription() { private fun launchDownloaderJob() {
if (subscription != null) return if (isRunning) return
subscription = downloadsRelay.concatMapIterable { it } downloaderJob = scope.launch {
// Concurrently download from 5 different sources val activeDownloadsFlow = queueState.transformLatest { queue ->
.groupBy { it.source } while (true) {
.flatMap( val activeDownloads = queue.asSequence()
{ bySource -> .filter { it.status.value <= Download.State.DOWNLOADING.value } // Ignore completed downloads, leave them in the queue
bySource.concatMap { download -> .groupBy { it.source }
Observable.fromCallable { .toList().take(5) // Concurrently download from 5 different sources
runBlocking { downloadChapter(download) } .map { (_, downloads) -> downloads.first() }
download emit(activeDownloads)
}.subscribeOn(Schedulers.io())
if (activeDownloads.isEmpty()) break
// Suspend until a download enters the ERROR state
val activeDownloadsErroredFlow =
combine(activeDownloads.map(Download::statusFlow)) { states ->
states.contains(Download.State.ERROR)
}.filter { it }
activeDownloadsErroredFlow.first()
}
}.distinctUntilChanged()
// Use supervisorScope to cancel child jobs when the downloader job is cancelled
supervisorScope {
val downloadJobs = mutableMapOf<Download, Job>()
activeDownloadsFlow.collectLatest { activeDownloads ->
val downloadJobsToStop = downloadJobs.filter { it.key !in activeDownloads }
downloadJobsToStop.forEach { (download, job) ->
job.cancel()
downloadJobs.remove(download)
} }
},
5, val downloadsToStart = activeDownloads.filter { it !in downloadJobs }
) downloadsToStart.forEach { download ->
.onBackpressureLatest() downloadJobs[download] = launchDownloadJob(download)
.observeOn(AndroidSchedulers.mainThread())
.subscribe(
{
// Remove successful download from queue
if (it.status == Download.State.DOWNLOADED) {
removeFromQueue(it)
} }
if (areAllDownloadsFinished()) { }
stop() }
} }
}, }
{ error ->
logcat(LogPriority.ERROR, error) private fun CoroutineScope.launchDownloadJob(download: Download) = launchIO {
notifier.onError(error.message) try {
stop() downloadChapter(download)
},
) // Remove successful download from queue
if (download.status == Download.State.DOWNLOADED) {
removeFromQueue(download)
}
if (areAllDownloadsFinished()) {
stop()
}
} catch (e: Throwable) {
if (e is CancellationException) throw e
logcat(LogPriority.ERROR, e)
notifier.onError(e.message)
stop()
}
} }
/** /**
* Destroys the downloader subscriptions. * Destroys the downloader subscriptions.
*/ */
private fun destroySubscription() { private fun cancelDownloaderJob() {
subscription?.unsubscribe() downloaderJob?.cancel()
subscription = null downloaderJob = null
} }
/** /**
@ -255,17 +272,13 @@ class Downloader(
val source = sourceManager.get(manga.source) as? HttpSource ?: return@launchIO val source = sourceManager.get(manga.source) as? HttpSource ?: return@launchIO
val wasEmpty = queueState.value.isEmpty() val wasEmpty = queueState.value.isEmpty()
// Called in background thread, the operation can be slow with SAF. val chaptersWithoutDir = chapters
val chaptersWithoutDir = async { // Filter out those already downloaded.
chapters .filter { provider.findChapterDir(it.name, it.scanlator, manga.title, source) == null }
// Filter out those already downloaded. // Add chapters to queue from the start.
.filter { provider.findChapterDir(it.name, it.scanlator, manga.title, source) == null } .sortedByDescending { it.sourceOrder }
// Add chapters to queue from the start.
.sortedByDescending { it.sourceOrder }
}
// Runs in main thread (synchronization needed). val chaptersToQueue = chaptersWithoutDir
val chaptersToQueue = chaptersWithoutDir.await()
// Filter out those already enqueued. // Filter out those already enqueued.
.filter { chapter -> queueState.value.none { it.chapter.id == chapter.id } } .filter { chapter -> queueState.value.none { it.chapter.id == chapter.id } }
// Create a download for each one. // Create a download for each one.
@ -274,11 +287,6 @@ class Downloader(
if (chaptersToQueue.isNotEmpty()) { if (chaptersToQueue.isNotEmpty()) {
addAllToQueue(chaptersToQueue) addAllToQueue(chaptersToQueue)
if (isRunning) {
// Send the list of downloads to the downloader.
downloadsRelay.call(chaptersToQueue)
}
// Start downloader if needed // Start downloader if needed
if (autoStart && wasEmpty) { if (autoStart && wasEmpty) {
val queuedDownloads = queueState.value.count { it.source !is UnmeteredSource } val queuedDownloads = queueState.value.count { it.source !is UnmeteredSource }
@ -656,7 +664,7 @@ class Downloader(
} }
} }
private inline fun removeFromQueueByPredicate(predicate: (Download) -> Boolean) { private inline fun removeFromQueueIf(predicate: (Download) -> Boolean) {
_queueState.update { queue -> _queueState.update { queue ->
val downloads = queue.filter { predicate(it) } val downloads = queue.filter { predicate(it) }
store.removeAll(downloads) store.removeAll(downloads)
@ -671,11 +679,11 @@ class Downloader(
fun removeFromQueue(chapters: List<Chapter>) { fun removeFromQueue(chapters: List<Chapter>) {
val chapterIds = chapters.map { it.id } val chapterIds = chapters.map { it.id }
removeFromQueueByPredicate { it.chapter.id in chapterIds } removeFromQueueIf { it.chapter.id in chapterIds }
} }
fun removeFromQueue(manga: Manga) { fun removeFromQueue(manga: Manga) {
removeFromQueueByPredicate { it.manga.id == manga.id } removeFromQueueIf { it.manga.id == manga.id }
} }
private fun _clearQueue() { private fun _clearQueue() {

View File

@ -22,8 +22,8 @@ data class Download(
val source: HttpSource, val source: HttpSource,
val manga: Manga, val manga: Manga,
val chapter: Chapter, val chapter: Chapter,
var pages: List<Page>? = null,
) { ) {
var pages: List<Page>? = null
val totalProgress: Int val totalProgress: Int
get() = pages?.sumOf(Page::progress) ?: 0 get() = pages?.sumOf(Page::progress) ?: 0