diff --git a/checks.gradle b/checks.gradle index 7a689d62..f3b2b89d 100644 --- a/checks.gradle +++ b/checks.gradle @@ -15,6 +15,9 @@ jacocoTestCoverageVerification { '*QueueService.migrateQueues()', '*.ShutdownHandler.*', '*FfmpegExecutor.runFfmpeg$lambda$?(java.lang.Process)', + '*FfmpegExecutorKt.getProgressRegex()', + '*FilterSettings.*', + 'se.svt.oss.encore.service.EncoreService.handleProgress.1.1.emit(int, kotlin.coroutines.Continuation)', ] limit { counter = 'LINE' diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/config/EncodingProperties.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/config/EncodingProperties.kt index 3dd7fb0e..d87674c9 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/config/EncodingProperties.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/config/EncodingProperties.kt @@ -5,8 +5,13 @@ package se.svt.oss.encore.config import org.springframework.boot.context.properties.NestedConfigurationProperty +import se.svt.oss.encore.model.AudioEncodingMode import se.svt.oss.encore.model.profile.ChannelLayout +data class SegmentedEncodingProperties( + val audioEncodingMode: AudioEncodingMode = AudioEncodingMode.ENCODE_WITH_VIDEO, +) + data class EncodingProperties( @NestedConfigurationProperty val audioMixPresets: Map = mapOf("default" to AudioMixPreset()), @@ -15,4 +20,6 @@ data class EncodingProperties( val flipWidthHeightIfPortrait: Boolean = true, val exitOnError: Boolean = true, val globalParams: LinkedHashMap = linkedMapOf(), + @NestedConfigurationProperty + val segmentedEncoding: SegmentedEncodingProperties = SegmentedEncodingProperties(), ) diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/AudioEncodingMode.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/AudioEncodingMode.kt new file mode 100644 index 00000000..0316735b --- /dev/null +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/AudioEncodingMode.kt @@ -0,0 +1,28 @@ +// SPDX-FileCopyrightText: 2025 Eyevinn Technology AB +// +// SPDX-License-Identifier: EUPL-1.2 + +package se.svt.oss.encore.model + +/** + * Defines how audio should be encoded when using segmented encoding. + */ +enum class AudioEncodingMode { + /** + * Encode audio and video together in the same segments. + * Creates N tasks of type AUDIOVIDEOSEGMENT. + */ + ENCODE_WITH_VIDEO, + + /** + * Encode audio separately from video as a single full-length file (not segmented). + * Creates 1 AUDIOFULL task + N VIDEOSEGMENT tasks. + */ + ENCODE_SEPARATELY_FULL, + + /** + * Encode audio separately from video, with both audio and video segmented. + * Creates N AUDIOSEGMENT tasks + N VIDEOSEGMENT tasks (2N total tasks). + */ + ENCODE_SEPARATELY_SEGMENTED, +} diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/EncoreJob.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/EncoreJob.kt index 4784433b..c2ace3fd 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/EncoreJob.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/EncoreJob.kt @@ -21,6 +21,55 @@ import se.svt.oss.mediaanalyzer.file.MediaFile import java.time.OffsetDateTime import java.util.UUID +data class SegmentedEncodingInfo( + @field:Schema( + description = "Length of each segment in seconds. Should be a multiple of target GOP.", + example = "19.2", + readOnly = true, + nullable = false, + ) + val segmentLength: Double, + @field:Schema( + description = "Number of video segments", + nullable = false, + readOnly = true, + ) + val numSegments: Int, + @field:Schema( + description = "Number of encoding tasks used for this job. This will be equal to numSegments plus numAudioSegments", + nullable = false, + readOnly = true, + ) + val numTasks: Int, + @field:Schema( + description = "The audio encoding mode used for this job.", + example = "ENCODE_WITH_VIDEO", + nullable = false, + readOnly = true, + ) + val audioEncodingMode: AudioEncodingMode, + @field:Schema( + description = "Audio segment padding in seconds (added at start/end of segments to avoid artifacts). Only relevant in ENCODE_SEPARATELY_SEGMENTED mode.", + example = "0.04267", + nullable = false, + readOnly = true, + ) + val audioSegmentPadding: Double = 0.0, + @field:Schema( + description = "Length of each audio segment in seconds. Only relevant in ENCODE_SEPARATELY_SEGMENTED mode.", + example = "256.0", + nullable = false, + readOnly = true, + ) + val audioSegmentLength: Double = 0.0, + @field:Schema( + description = "Number of audio segments", + nullable = false, + readOnly = true, + ) + val numAudioSegments: Int, +) + @Validated @RedisHash("encore-jobs", timeToLive = (60 * 60 * 24 * 7).toLong()) // 1 week ttl @Tag(name = "encorejob") @@ -107,6 +156,29 @@ data class EncoreJob( @field:Positive val segmentLength: Double? = null, + @field:Schema( + description = "Defines how audio should be encoded when using segmented encoding. ENCODE_WITH_VIDEO: audio and video together in segments; ENCODE_SEPARATELY_FULL: audio separately as full file; ENCODE_SEPARATELY_SEGMENTED: audio separately in segments.", + example = "ENCODE_WITH_VIDEO", + defaultValue = "ENCODE_WITH_VIDEO", + nullable = true, + ) + val audioEncodingMode: AudioEncodingMode? = null, + + @field:Schema( + description = "Length of audio segments in seconds when using ENCODE_SEPARATELY_SEGMENTED mode. If not specified, a value close to 256s will be calculated that is a multiple of the audio frame size.", + example = "256.0", + nullable = true, + ) + @field:Positive + val audioSegmentLength: Double? = null, + + @field:Schema( + description = "Properties for segmented encoding, or null if not used", + nullable = true, + readOnly = true, + ) + var segmentedEncodingInfo: SegmentedEncodingInfo? = null, + @field:Schema( description = "The exception message, if the EncoreJob failed", example = "input/output error", diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncode.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncode.kt index bbb5bf3c..9defa6dd 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncode.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncode.kt @@ -17,7 +17,7 @@ import se.svt.oss.encore.model.output.AudioStreamEncode import se.svt.oss.encore.model.output.Output data class AudioEncode( - val codec: String = "libfdk_aac", + override val codec: String = "libfdk_aac", val bitrate: String? = null, val samplerate: Int = 48000, val channelLayout: ChannelLayout = ChannelLayout.CH_LAYOUT_STEREO, diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncoder.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncoder.kt index 6d9bb18c..f4a4e6f2 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncoder.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/AudioEncoder.kt @@ -12,7 +12,8 @@ private val log = KotlinLogging.logger { } abstract class AudioEncoder : OutputProducer { abstract val optional: Boolean - abstract val enabled: Boolean + abstract val codec: String + abstract override val enabled: Boolean fun logOrThrow(message: String): Output? { if (optional || !enabled) { diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/OutputProducer.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/OutputProducer.kt index 3c91f345..5df27233 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/OutputProducer.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/OutputProducer.kt @@ -21,5 +21,6 @@ import se.svt.oss.encore.model.output.Output JsonSubTypes.Type(value = ThumbnailMapEncode::class, name = "ThumbnailMapEncode"), ) interface OutputProducer { + val enabled: Boolean fun getOutput(job: EncoreJob, encodingProperties: EncodingProperties): Output? } diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/SimpleAudioEncode.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/SimpleAudioEncode.kt index f1296df6..2a67c655 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/SimpleAudioEncode.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/SimpleAudioEncode.kt @@ -13,7 +13,7 @@ import se.svt.oss.encore.model.output.AudioStreamEncode import se.svt.oss.encore.model.output.Output data class SimpleAudioEncode( - val codec: String = "libfdk_aac", + override val codec: String = "libfdk_aac", val bitrate: String? = null, val samplerate: Int? = null, val suffix: String = "_$codec", diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailEncode.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailEncode.kt index ddc64c74..0d0267b2 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailEncode.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailEncode.kt @@ -25,7 +25,7 @@ data class ThumbnailEncode( val suffixZeroPad: Int = 2, val inputLabel: String = DEFAULT_VIDEO_LABEL, val optional: Boolean = false, - val enabled: Boolean = true, + override val enabled: Boolean = true, val intervalSeconds: Double? = null, val decodeOutput: Int? = null, ) : OutputProducer { diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailMapEncode.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailMapEncode.kt index 1c840180..e4a83d70 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailMapEncode.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/ThumbnailMapEncode.kt @@ -26,7 +26,7 @@ data class ThumbnailMapEncode( val rows: Int = 20, val quality: Int = 5, val optional: Boolean = true, - val enabled: Boolean = true, + override val enabled: Boolean = true, val suffix: String = "_${cols}x${rows}_${tileWidth}x${tileHeight}_thumbnail_map", val format: String = "jpg", val inputLabel: String = DEFAULT_VIDEO_LABEL, diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/VideoEncode.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/VideoEncode.kt index f31693a0..ede181d3 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/VideoEncode.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/profile/VideoEncode.kt @@ -34,7 +34,7 @@ interface VideoEncode : OutputProducer { val codec: String val inputLabel: String val optional: Boolean - val enabled: Boolean + override val enabled: Boolean val cropTo: FractionString? val padTo: FractionString? diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/model/queue/QueueItem.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/model/queue/QueueItem.kt index 26ddb4bd..087c7d62 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/model/queue/QueueItem.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/model/queue/QueueItem.kt @@ -14,5 +14,18 @@ data class QueueItem( val id: String, val priority: Int = 0, val created: LocalDateTime = LocalDateTime.now(), - val segment: Int? = null, + val task: Task? = null, +) + +enum class TaskType { + AUDIOVIDEOSEGMENT, + VIDEOSEGMENT, + AUDIOFULL, + AUDIOSEGMENT, +} + +data class Task( + val type: TaskType, + val taskNo: Int, + val segment: Int, ) diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/process/SegmentUtil.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/process/SegmentUtil.kt index c0c8ec05..51b02228 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/process/SegmentUtil.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/process/SegmentUtil.kt @@ -5,30 +5,30 @@ package se.svt.oss.encore.process import se.svt.oss.encore.model.EncoreJob -import se.svt.oss.mediaanalyzer.file.MediaContainer -import kotlin.math.ceil - -fun EncoreJob.segmentLengthOrThrow() = segmentLength ?: throw RuntimeException("No segmentLength in job!") - -fun EncoreJob.numSegments(): Int { - val segLen = segmentLengthOrThrow() - val readDuration = duration - return if (readDuration != null) { - ceil(readDuration / segLen).toInt() - } else { - val segments = - inputs.map { ceil(((it.analyzed as MediaContainer).duration - (it.seekTo ?: 0.0)) / segLen).toInt() }.toSet() - if (segments.size > 1) { - throw RuntimeException("Inputs differ in length") - } - segments.first() + +fun EncoreJob.segmentLengthOrThrow() = segmentedEncodingInfoOrThrow().segmentLength + +fun EncoreJob.segmentedEncodingInfoOrThrow() = segmentedEncodingInfo ?: throw RuntimeException("No segmentedEncodingInfo in job!") + +fun EncoreJob.segmentDuration(segmentNumber: Int): Double { + val numSegments = segmentedEncodingInfoOrThrow().numSegments + return when { + duration == null -> segmentLengthOrThrow() + segmentNumber < numSegments - 1 -> segmentLengthOrThrow() + segmentNumber == numSegments - 1 -> + // This correctly handles the case where the duration is an exact multiple of the segment length + duration!! - segmentLengthOrThrow() * (numSegments - 1) + else -> throw IllegalArgumentException("segmentNumber $segmentNumber is out of range for job with $numSegments segments") } } -fun EncoreJob.segmentDuration(segmentNumber: Int): Double = when { - duration == null -> segmentLengthOrThrow() - segmentNumber < numSegments() - 1 -> segmentLengthOrThrow() - else -> duration!! % segmentLengthOrThrow() +fun EncoreJob.baseName(segmentNumber: Int) = "${baseName}_%05d".format(segmentNumber) + +fun EncoreJob.segmentSuffixFromFilename(file: String): String { + val regex = Regex("${baseName}_\\d{5}(.*)") + val match = regex.find(file) ?: throw RuntimeException("Could not find segment suffix for file $file") + return match.groupValues[1] } -fun EncoreJob.baseName(segmentNumber: Int) = "${baseName}_%05d".format(segmentNumber) +fun EncoreJob.targetFilenameFromSegmentFilename(segmentFile: String) = + segmentFile.replace(Regex("^${baseName}_\\d{5}"), baseName) diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/service/EncoreService.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/service/EncoreService.kt index 5e5f0111..ad9c45c5 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/service/EncoreService.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/service/EncoreService.kt @@ -36,15 +36,18 @@ import se.svt.oss.encore.model.RedisEvent import se.svt.oss.encore.model.SegmentProgressEvent import se.svt.oss.encore.model.Status import se.svt.oss.encore.model.queue.QueueItem +import se.svt.oss.encore.model.queue.Task +import se.svt.oss.encore.model.queue.TaskType import se.svt.oss.encore.process.baseName -import se.svt.oss.encore.process.numSegments import se.svt.oss.encore.process.segmentDuration import se.svt.oss.encore.process.segmentLengthOrThrow +import se.svt.oss.encore.process.segmentedEncodingInfoOrThrow import se.svt.oss.encore.repository.EncoreJobRepository import se.svt.oss.encore.service.callback.CallbackService import se.svt.oss.encore.service.localencode.LocalEncodeService import se.svt.oss.encore.service.mediaanalyzer.MediaAnalyzerService import se.svt.oss.encore.service.queue.QueueService +import se.svt.oss.encore.service.segmentedencode.SegmentedEncodeService import se.svt.oss.mediaanalyzer.file.MediaContainer import se.svt.oss.mediaanalyzer.file.MediaFile import java.io.File @@ -67,6 +70,7 @@ class EncoreService( private val localEncodeService: LocalEncodeService, private val encoreProperties: EncoreProperties, private val queueService: QueueService, + private val segmentedEncodeService: SegmentedEncodeService, ) { private val cancelTopicName = "cancel" @@ -79,9 +83,19 @@ class EncoreService( ?: throw IllegalStateException("Shared work dir has not been configured") fun encode(queueItem: QueueItem, job: EncoreJob) { + try { + initJob(job) + } catch (e: Exception) { + log.error(e) { "Job initialization failed: ${e.message}" } + job.status = Status.FAILED + job.message = e.message + repository.save(job) + callbackService.sendProgressCallback(job) + return + } when { - queueItem.segment != null -> encodeSegment(job, queueItem.segment) - job.segmentLength != null -> encodeSegmented(job) + queueItem.task != null -> encodeSegment(job, queueItem.task) + job.segmentedEncodingInfo != null -> encodeSegmented(job) else -> encode(job) } } @@ -91,22 +105,23 @@ class EncoreService( val cancelListener = CancellationListener(objectMapper, encoreJob.id, coroutineJob) var progressListener: SegmentProgressListener? = null try { - initJob(encoreJob) - val numSegments = encoreJob.numSegments() - log.debug { "Encoding using $numSegments segments" } + startJob(encoreJob) + val tasks = segmentedEncodeService.createTasks(encoreJob) + val numTasks = tasks.size + redisMessageListerenerContainer.addMessageListener(cancelListener, ChannelTopic.of(cancelTopicName)) val progressChannel = Channel() progressListener = - SegmentProgressListener(objectMapper, encoreJob.id, coroutineJob, numSegments, progressChannel) + SegmentProgressListener(objectMapper, encoreJob.id, coroutineJob, numTasks, progressChannel) redisMessageListerenerContainer.addMessageListener(progressListener, ChannelTopic.of("segment-progress")) val timedOutput = measureTimedValue { sharedWorkDir(encoreJob).mkdirs() - repeat(numSegments) { + tasks.forEach { queueService.enqueue( QueueItem( id = encoreJob.id.toString(), priority = encoreJob.priority, - segment = it, + task = it, ), ) } @@ -117,7 +132,7 @@ class EncoreService( progressChannel.trySendBlocking(0) while (!progressListener.completed()) { ShutdownHandler.checkShutdown() - log.info { "Awaiting completion ${progressListener.completionCount()}/$numSegments..." } + log.info { "Awaiting completion ${progressListener.completionCount()}/$numTasks..." } delay(1000) } } @@ -126,26 +141,7 @@ class EncoreService( throw RuntimeException("Some segments failed") } log.info { "All segments completed" } - val outWorkDir = sharedWorkDir(encoreJob) - val suffixes = mutableSetOf() - repeat(numSegments) { segmentNum -> - val segmentBaseName = encoreJob.baseName(segmentNum) - outWorkDir.list()?.filter { it.startsWith(segmentBaseName) } - ?.forEach { - val suffix = it.replaceFirst(segmentBaseName, "") - suffixes.add(suffix) - outWorkDir.resolve("$suffix.txt").appendText("file $it\n") - } - } - val outputFolder = File(encoreJob.outputFolder) - outputFolder.mkdirs() - val outputFiles = suffixes.map { - val targetName = encoreJob.baseName + it - log.info { "Joining segments for $targetName" } - val targetFile = outputFolder.resolve(targetName) - ffmpegExecutor.joinSegments(encoreJob, outWorkDir.resolve("$it.txt"), targetFile) - } - outputFiles + segmentedEncodeService.joinSegments(encoreJob, sharedWorkDir(encoreJob)) } updateSuccessfulJob(encoreJob, timedOutput) } catch (e: CancellationException) { @@ -165,25 +161,71 @@ class EncoreService( } } - private fun encodeSegment(encoreJob: EncoreJob, segmentNumber: Int) { + private fun encodeSegment(encoreJob: EncoreJob, task: Task) { + val taskNo = task.taskNo try { - log.info { "Start encoding ${encoreJob.baseName} segment $segmentNumber/${encoreJob.numSegments()} " } - val outputFolder = sharedWorkDir(encoreJob).absolutePath - val job = encoreJob.copy( - baseName = encoreJob.baseName(segmentNumber), - duration = encoreJob.segmentDuration(segmentNumber), - inputs = encoreJob.inputs.map { - it.withSeekTo((it.seekTo ?: 0.0) + encoreJob.segmentLengthOrThrow() * segmentNumber) - }, - ) - ffmpegExecutor.run(job, outputFolder, null) - redisTemplate.convertAndSend("segment-progress", SegmentProgressEvent(encoreJob.id, segmentNumber, true)) - log.info { "Completed ${encoreJob.baseName} segment $segmentNumber/${encoreJob.numSegments()} " } + log.info { "Start encoding ${encoreJob.baseName} task $taskNo/${encoreJob.segmentedEncodingInfo?.numTasks} (${task.type})" } + val encodingMode = when (task.type) { + TaskType.AUDIOFULL -> EncodingMode.AUDIO_ONLY + TaskType.AUDIOSEGMENT -> EncodingMode.AUDIO_ONLY + TaskType.VIDEOSEGMENT -> EncodingMode.VIDEO_ONLY + TaskType.AUDIOVIDEOSEGMENT -> EncodingMode.AUDIO_AND_VIDEO + } + val (job, outputFolder) = when (task.type) { + TaskType.AUDIOFULL -> { + // Full audio, no segmentation + Pair(encoreJob, sharedWorkDir(encoreJob).resolve("audio").absolutePath) + } + TaskType.AUDIOSEGMENT -> { + // Audio segment with timing and padding + val segmentNumber = task.segment + val segmentedInfo = encoreJob.segmentedEncodingInfoOrThrow() + val numSegments = segmentedInfo.numAudioSegments + val padding = segmentedInfo.audioSegmentPadding + val audioSegmentLength = segmentedInfo.audioSegmentLength + + // Add padding at start (except first segment) and end (except last segment) + val startPadding = if (segmentNumber == 0) 0.0 else padding + val endPadding = if (segmentNumber == numSegments - 1) 0.0 else padding + + // Calculate audio segment duration (use remainder for last segment if duration is set) + val baseDuration = when { + encoreJob.duration == null -> audioSegmentLength + segmentNumber < numSegments - 1 -> audioSegmentLength + else -> encoreJob.duration!! - audioSegmentLength * (numSegments - 1) + } + + val job = encoreJob.copy( + baseName = encoreJob.baseName(segmentNumber), + duration = baseDuration + startPadding + endPadding, + inputs = encoreJob.inputs.map { + val baseSeekTo = (it.seekTo ?: 0.0) + audioSegmentLength * segmentNumber + it.withSeekTo(baseSeekTo - startPadding) + }, + ) + Pair(job, sharedWorkDir(encoreJob).resolve("audio").absolutePath) + } + TaskType.VIDEOSEGMENT, TaskType.AUDIOVIDEOSEGMENT -> { + // Video or audio+video segment with timing + val segmentNumber = task.segment + val job = encoreJob.copy( + baseName = encoreJob.baseName(segmentNumber), + duration = encoreJob.segmentDuration(segmentNumber), + inputs = encoreJob.inputs.map { + it.withSeekTo((it.seekTo ?: 0.0) + encoreJob.segmentLengthOrThrow() * segmentNumber) + }, + ) + Pair(job, sharedWorkDir(encoreJob).absolutePath) + } + } + ffmpegExecutor.run(job, outputFolder, null, encodingMode) + redisTemplate.convertAndSend("segment-progress", SegmentProgressEvent(encoreJob.id, taskNo, true)) + log.info { "Completed ${encoreJob.baseName} task $taskNo/${encoreJob.segmentedEncodingInfo?.numTasks} " } } catch (e: ApplicationShutdownException) { throw e } catch (e: Exception) { - log.error(e) { "Error encoding segment $segmentNumber: ${e.message}" } - redisTemplate.convertAndSend("segment-progress", SegmentProgressEvent(encoreJob.id, segmentNumber, false)) + log.error(e) { "Error encoding task $taskNo: ${e.message}" } + redisTemplate.convertAndSend("segment-progress", SegmentProgressEvent(encoreJob.id, taskNo, false)) } } @@ -197,7 +239,7 @@ class EncoreService( outputFolder = localEncodeService.outputFolder(encoreJob) val timedOutput = measureTimedValue { - initJob(encoreJob) + startJob(encoreJob) val outputFiles = runBlocking(coroutineJob + MDCContext()) { val progressChannel = Channel() @@ -273,6 +315,11 @@ class EncoreService( encoreJob.inputs.forEach { input -> mediaAnalyzerService.analyzeInput(input) } + + encoreJob.segmentedEncodingInfo = segmentedEncodeService.segmentedEncodingInfo(encoreJob) + } + + private fun startJob(encoreJob: EncoreJob) { log.info { "Start encoding" } encoreJob.status = Status.IN_PROGRESS repository.save(encoreJob) diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/service/FfmpegExecutor.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/service/FfmpegExecutor.kt index 737f40e6..c8fcec65 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/service/FfmpegExecutor.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/service/FfmpegExecutor.kt @@ -12,6 +12,7 @@ import se.svt.oss.encore.config.EncoreProperties import se.svt.oss.encore.model.EncoreJob import se.svt.oss.encore.model.input.maxDuration import se.svt.oss.encore.model.mediafile.toParams +import se.svt.oss.encore.model.output.Output import se.svt.oss.encore.process.CommandBuilder import se.svt.oss.encore.process.createTempDir import se.svt.oss.encore.service.profile.ProfileService @@ -24,6 +25,27 @@ import kotlin.math.round private val log = KotlinLogging.logger { } +enum class EncodingMode { + AUDIO_AND_VIDEO, + VIDEO_ONLY, + AUDIO_ONLY, +} + +val progressRegex = + Regex(".*time=(?\\d{2}):(?\\d{2}):(?\\d{2}\\.\\d+) .* speed= *(?[0-9.e-]+x) *") + +fun getProgress(duration: Double?, line: String): Int? = if (duration != null && duration > 0) { + progressRegex.matchEntire(line)?.let { + val hours = it.groups["hours"]!!.value.toInt() + val minutes = it.groups["minutes"]!!.value.toInt() + val seconds = it.groups["seconds"]!!.value.toDouble() + val time = hours * 3600 + minutes * 60 + seconds + min(100, round(100 * time / duration).toInt()) + } +} else { + null +} + @Service class FfmpegExecutor( private val mediaAnalyzer: MediaAnalyzer, @@ -35,13 +57,11 @@ class FfmpegExecutor( fun getLoglevel(line: String) = logLevelRegex.matchEntire(line)?.groups?.get("level")?.value - val progressRegex = - Regex(".*time=(?\\d{2}):(?\\d{2}):(?\\d{2}\\.\\d+) .* speed= *(?[0-9.e-]+x) .*") - fun run( encoreJob: EncoreJob, outputFolder: String, progressChannel: SendChannel?, + encodingMode: EncodingMode = EncodingMode.AUDIO_AND_VIDEO, ): List { ShutdownHandler.checkShutdown() val profile = profileService.getProfile(encoreJob) @@ -50,10 +70,11 @@ class FfmpegExecutor( encoreJob, encoreProperties.encoding, ) - } + }.mapNotNull { adaptOutputToEncodingMode(it, encodingMode) } check(outputs.isNotEmpty()) { "No outputs to encode! Check your profile and inputs!" } + check(outputs.distinctBy { it.id }.size == outputs.size) { "Profile ${encoreJob.profile} contains duplicate suffixes: ${outputs.map { it.id }}!" } @@ -79,6 +100,23 @@ class FfmpegExecutor( } } + private fun adaptOutputToEncodingMode(output: Output, encodingMode: EncodingMode): Output? = when (encodingMode) { + EncodingMode.AUDIO_AND_VIDEO -> output + EncodingMode.VIDEO_ONLY -> + if (output.video == null) { + null + } else { + output.copy(audioStreams = emptyList()) + } + + EncodingMode.AUDIO_ONLY -> + if (output.audioStreams.isEmpty()) { + null + } else { + output.copy(video = null) + } + } + private fun runFfmpeg( command: List, workDir: File, @@ -165,34 +203,41 @@ class FfmpegExecutor( private fun totalProgress(subtaskProgress: Int, subtaskIndex: Int, subtaskCount: Int) = (subtaskIndex * 100 + subtaskProgress) / subtaskCount - private fun getProgress(duration: Double?, line: String): Int? = if (duration != null && duration > 0) { - progressRegex.matchEntire(line)?.let { - val hours = it.groups["hours"]!!.value.toInt() - val minutes = it.groups["minutes"]!!.value.toInt() - val seconds = it.groups["seconds"]!!.value.toDouble() - val time = hours * 3600 + minutes * 60 + seconds - min(100, round(100 * time / duration).toInt()) + fun joinSegments( + encoreJob: EncoreJob, + segmentList: File, + targetFile: File, + audioFile: File?, + audioSegmentList: File?, + ): MediaFile { + require(audioFile == null || audioSegmentList == null) { + "Cannot provide both audioFile and audioSegmentList" } - } else { - null - } - fun joinSegments(encoreJob: EncoreJob, segmentList: File, targetFile: File): MediaFile { val joinParams = profileService.getProfile(encoreJob).joinSegmentParams.toParams() + val inputArgs = mutableListOf() + val mapArgs = mutableListOf("-map", "0") + + // Add video segments input with concat demuxer + inputArgs.addAll(listOf("-f", "concat", "-safe", "0", "-i", "$segmentList")) + + // Add audio input (either segment list with concat or direct file) + if (audioSegmentList != null) { + inputArgs.addAll(listOf("-f", "concat", "-safe", "0", "-i", "$audioSegmentList")) + mapArgs.addAll(listOf("-map", "1")) + } else if (audioFile != null) { + inputArgs.addAll(listOf("-i", audioFile.absolutePath)) + mapArgs.addAll(listOf("-map", "1")) + } + val command = listOf( "ffmpeg", "-hide_banner", "-loglevel", "+level", "-y", - "-f", - "concat", - "-safe", - "0", - "-i", - "$segmentList", - "-map", - "0", + *inputArgs.toTypedArray(), + *mapArgs.toTypedArray(), "-ignore_unknown", "-c", "copy", diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/service/mediaanalyzer/MediaAnalyzerService.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/service/mediaanalyzer/MediaAnalyzerService.kt index c52aef9f..a6f85831 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/service/mediaanalyzer/MediaAnalyzerService.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/service/mediaanalyzer/MediaAnalyzerService.kt @@ -22,6 +22,7 @@ import se.svt.oss.mediaanalyzer.ffprobe.SideData import se.svt.oss.mediaanalyzer.ffprobe.UnknownSideData import se.svt.oss.mediaanalyzer.ffprobe.UnknownStream import se.svt.oss.mediaanalyzer.file.AudioFile +import se.svt.oss.mediaanalyzer.file.MediaFile import se.svt.oss.mediaanalyzer.file.VideoFile import se.svt.oss.mediaanalyzer.mediainfo.AudioTrack import se.svt.oss.mediaanalyzer.mediainfo.GeneralTrack @@ -74,4 +75,6 @@ class MediaAnalyzerService(private val mediaAnalyzer: MediaAnalyzer) { } } } + + fun analyze(absolutePath: String): MediaFile = mediaAnalyzer.analyze(absolutePath) } diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/service/queue/QueueService.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/service/queue/QueueService.kt index ebbc3613..79854a91 100644 --- a/encore-common/src/main/kotlin/se/svt/oss/encore/service/queue/QueueService.kt +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/service/queue/QueueService.kt @@ -54,7 +54,7 @@ class QueueService( log.info { "Job was cancelled" } return true } - if (queueItem.segment != null && job.status == Status.FAILED) { + if (queueItem.task != null && job.status == Status.FAILED) { log.info { "Main job has failed" } return true } @@ -91,13 +91,13 @@ class QueueService( try { log.info { "Adding job to queue (repost on interrupt)" } enqueue(queueItem) - if (queueItem.segment == null) { + if (queueItem.task == null) { job.status = Status.QUEUED repository.save(job) } log.info { "Added job to queue (repost on interrupt)" } } catch (e: Exception) { - if (queueItem.segment == null) { + if (queueItem.task == null) { val message = "Failed to add interrupted job to queue" log.error(e) { message } job.message = message diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/service/segmentedencode/SegmentedEncodeService.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/service/segmentedencode/SegmentedEncodeService.kt new file mode 100644 index 00000000..08797799 --- /dev/null +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/service/segmentedencode/SegmentedEncodeService.kt @@ -0,0 +1,395 @@ +package se.svt.oss.encore.service.segmentedencode + +import io.github.oshai.kotlinlogging.KotlinLogging +import org.springframework.stereotype.Service +import se.svt.oss.encore.config.EncoreProperties +import se.svt.oss.encore.model.AudioEncodingMode +import se.svt.oss.encore.model.EncoreJob +import se.svt.oss.encore.model.SegmentedEncodingInfo +import se.svt.oss.encore.model.profile.AudioEncode +import se.svt.oss.encore.model.profile.AudioEncoder +import se.svt.oss.encore.model.profile.OutputProducer +import se.svt.oss.encore.model.profile.Profile +import se.svt.oss.encore.model.profile.SimpleAudioEncode +import se.svt.oss.encore.model.profile.VideoEncode +import se.svt.oss.encore.model.queue.Task +import se.svt.oss.encore.model.queue.TaskType +import se.svt.oss.encore.process.segmentedEncodingInfoOrThrow +import se.svt.oss.encore.process.targetFilenameFromSegmentFilename +import se.svt.oss.encore.service.FfmpegExecutor +import se.svt.oss.encore.service.mediaanalyzer.MediaAnalyzerService +import se.svt.oss.encore.service.profile.ProfileService +import se.svt.oss.encore.util.allAudioEncodes +import se.svt.oss.encore.util.hasAudioEncodes +import se.svt.oss.encore.util.hasVideoEncodes +import se.svt.oss.mediaanalyzer.file.MediaContainer +import se.svt.oss.mediaanalyzer.file.MediaFile +import java.io.File +import kotlin.math.ceil + +private val log = KotlinLogging.logger {} + +fun OutputProducer?.audioSamplerates(): List = when { + this == null -> emptyList() + !this.enabled -> emptyList() + this is AudioEncode -> listOf(this.samplerate) + this is SimpleAudioEncode -> this.samplerate?.let { listOf(it) } ?: emptyList() + this is VideoEncode -> this.audioEncodes.flatMap { it.audioSamplerates() } + this.audioEncode.audioSamplerates() + else -> emptyList() +} + +data class AudioEncodingConfig( + val audioEncodingMode: AudioEncodingMode, + val audioSegmentPadding: Double, + val audioSegmentLength: Double, + val numSegments: Int, +) + +@Service +class SegmentedEncodeService( + private val ffmpegExecutor: FfmpegExecutor, + private val mediaAnalyzerService: MediaAnalyzerService, + private val profileService: ProfileService, + private val encoreProperties: EncoreProperties, +) { + + fun segmentedEncodingInfo(encoreJob: EncoreJob): SegmentedEncodingInfo? { + val profile = profileService.getProfile(encoreJob) + val hasVideo = profile.hasVideoEncodes() + if (encoreJob.segmentLength == null && hasVideo) { + return null + } + if (encoreJob.segmentLength == null && + encoreJob.audioEncodingMode != AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED + ) { + return null + } + val segmentLength = encoreJob.segmentLength ?: 0.0 + val audioEncodingConfig = audioEncodingConfig(encoreJob, profile) + val numVideoSegments = if (hasVideo) numSegments(encoreJob, segmentLength) else 0 + val numTasks = numVideoSegments + audioEncodingConfig.numSegments + return SegmentedEncodingInfo( + segmentLength = segmentLength, + audioEncodingMode = audioEncodingConfig.audioEncodingMode, + numTasks = numTasks, + numSegments = numVideoSegments, + numAudioSegments = audioEncodingConfig.numSegments, + audioSegmentPadding = audioEncodingConfig.audioSegmentPadding, + audioSegmentLength = audioEncodingConfig.audioSegmentLength, + ) + } + + fun audioEncodingConfig(encoreJob: EncoreJob, profile: Profile): AudioEncodingConfig { + if (!profile.hasAudioEncodes()) { + log.info { "No audio encodes found in profile ${profile.name}, skipping audio segmented encoding configuration." } + return AudioEncodingConfig(AudioEncodingMode.ENCODE_WITH_VIDEO, 0.0, 0.0, 0) + } + + val audioSampleRates = profile.encodes + .filter { it.enabled } + .flatMap { it.audioSamplerates() }.distinct() + + val audioEncodingMode = audioEncodingMode(encoreJob, profile, audioSampleRates) + + if (audioEncodingMode != AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) { + val numAudioSegments = + if (audioEncodingMode == AudioEncodingMode.ENCODE_SEPARATELY_FULL) { + 1 + } else { + 0 + } + return AudioEncodingConfig(audioEncodingMode, 0.0, 0.0, numAudioSegments) + } + + // Each audio segment is padded with two audio frames at start and end + // This padding is then removed when joining segments + // This avoids priming samples causing artifacts at segment boundaries + // Calculate audio segment padding: 2 * (frame_size / sample_rate) + val audioFrameSize = 1024.0 // Standard AAC frame size in samples + val maxSampleRate = audioSampleRates.maxOrNull() ?: 48000 + val audioSegmentPadding = 2.0 * audioFrameSize / maxSampleRate + + // Calculate audio segment length for ENCODE_SEPARATELY_SEGMENTED mode + val audioSegmentLength = + encoreJob.audioSegmentLength ?: run { + // Calculate a value close to 256s that is a multiple of the audio frame size + // 256 is selected because it is an integer number of audio frames for both + // 44.1kHz and 48kHz sample rates. + val frameDuration = audioFrameSize / maxSampleRate + val targetDuration = 256.0 + val numFrames = kotlin.math.round(targetDuration / frameDuration).toLong() + numFrames * frameDuration + } + + return AudioEncodingConfig( + audioEncodingMode, + audioSegmentPadding, + audioSegmentLength, + numSegments(encoreJob, audioSegmentLength), + ) + } + + fun audioEncodingMode(encoreJob: EncoreJob, profile: Profile, audioSampleRates: List): AudioEncodingMode { + // Get the requested audio encoding mode from job or fall back to default + val requestedMode = encoreJob.audioEncodingMode + ?: encoreProperties.encoding.segmentedEncoding.audioEncodingMode + + val hasNonAacAudioEncode = profile.allAudioEncodes() + .filterNot { it.isAacEncoder() } + .isNotEmpty() + + return when { + requestedMode == AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED && audioSampleRates.size > 1 -> { + log.warn { "Multiple audio sample rates detected (${audioSampleRates.joinToString()}), downgrading from ENCODE_SEPARATELY_SEGMENTED to ENCODE_SEPARATELY_FULL" } + AudioEncodingMode.ENCODE_SEPARATELY_FULL + } + requestedMode == AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED && hasNonAacAudioEncode -> { + log.warn { "Non-AAC audio encode detected, downgrading from ENCODE_SEPARATELY_SEGMENTED to ENCODE_SEPARATELY_FULL" } + AudioEncodingMode.ENCODE_SEPARATELY_FULL + } + else -> requestedMode + } + } + + private fun AudioEncoder.isAacEncoder(): Boolean = this.codec == "aac" || this.codec == "libfdk_aac" + + private fun numSegments(encoreJob: EncoreJob, segmentLength: Double): Int { + val readDuration = encoreJob.duration + return if (readDuration != null) { + ceil(readDuration / segmentLength).toInt() + } else { + val segments = + encoreJob.inputs.map { ceil(((it.analyzed as MediaContainer).duration - (it.seekTo ?: 0.0)) / segmentLength).toInt() } + .toSet() + if (segments.size > 1) { + throw RuntimeException("Inputs differ in length") + } + segments.first() + } + } + + /** + * Create tasks for segmented encoding based on the audio encoding mode. + * - ENCODE_WITH_VIDEO: N tasks of type AUDIOVIDEOSEGMENT + * - ENCODE_SEPARATELY_FULL: 1 AUDIOFULL task + N VIDEOSEGMENT tasks + * - ENCODE_SEPARATELY_SEGMENTED: M AUDIOSEGMENT tasks + N VIDEOSEGMENT tasks + */ + fun createTasks(encoreJob: EncoreJob): List { + val segmentedEncodingInfo = encoreJob.segmentedEncodingInfoOrThrow() + val audioEncodingMode = segmentedEncodingInfo.audioEncodingMode + val numSegments = segmentedEncodingInfo.numSegments + + log.debug { "Encoding using $numSegments video segments, ${segmentedEncodingInfo.numAudioSegments} audio segments,`0 with audio mode: $audioEncodingMode" } + + val tasks = mutableListOf() + var taskNo = 0 + + when (audioEncodingMode) { + AudioEncodingMode.ENCODE_WITH_VIDEO -> { + // Audio and video together in each segment + repeat(numSegments) { segmentIndex -> + tasks.add( + Task( + type = TaskType.AUDIOVIDEOSEGMENT, + taskNo = taskNo++, + segment = segmentIndex, + ), + ) + } + } + AudioEncodingMode.ENCODE_SEPARATELY_FULL -> { + // One full audio task + tasks.add( + Task( + type = TaskType.AUDIOFULL, + taskNo = taskNo++, + segment = 0, + ), + ) + // N video segment tasks + repeat(numSegments) { segmentIndex -> + tasks.add( + Task( + type = TaskType.VIDEOSEGMENT, + taskNo = taskNo++, + segment = segmentIndex, + ), + ) + } + } + AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED -> { + val numAudioSegments = segmentedEncodingInfo.numAudioSegments + // N audio segment tasks + repeat(numAudioSegments) { segmentIndex -> + tasks.add( + Task( + type = TaskType.AUDIOSEGMENT, + taskNo = taskNo++, + segment = segmentIndex, + ), + ) + } + // N video segment tasks + repeat(numSegments) { segmentIndex -> + tasks.add( + Task( + type = TaskType.VIDEOSEGMENT, + taskNo = taskNo++, + segment = segmentIndex, + ), + ) + } + } + } + return tasks + } + + data class JoinSegmentOperation( + val target: File, + val segmentFiles: List, + val audioFile: File? = null, + val audioSegmentFiles: List? = null, + ) + + fun prepareJoinSegment(encoreJob: EncoreJob, sharedWorkDir: File): Map { + val segmentedEncodingInfo = encoreJob.segmentedEncodingInfoOrThrow() + val audioEncodingMode = segmentedEncodingInfo.audioEncodingMode + + // Groups video segment files by suffix/target file + val videoSegmentFileMap = sharedWorkDir.listFiles() + .filter { it.isFile } + .sorted() + .groupBy { encoreJob.targetFilenameFromSegmentFilename(it.name) } + + val audioFilesMap = if (audioEncodingMode == AudioEncodingMode.ENCODE_SEPARATELY_FULL) { + // Full audio files are in the audio subfolder + sharedWorkDir.resolve("audio").listFiles().filter { it.isFile }.associateBy { it.name } + } else { + emptyMap() + } + + val audioSegmentFileMap = if (audioEncodingMode == AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) { + // Audio segments are in the audio subfolder + val audioDir = sharedWorkDir.resolve("audio") + audioDir.listFiles() + .filter { it.isFile } + .sorted() + .groupBy { encoreJob.targetFilenameFromSegmentFilename(it.name) } + } else { + emptyMap() + } + + val outputFolder = File(encoreJob.outputFolder) + outputFolder.mkdirs() + val joinSegmentOperations = LinkedHashMap() + + // Handle video segments and determine corresponding audio + videoSegmentFileMap.forEach { (targetName, files) -> + val targetFile = outputFolder.resolve(targetName) + val audioFile = audioFilesMap[targetName] + val audioSegmentFiles = audioSegmentFileMap[targetName] + + joinSegmentOperations[targetFile.name] = JoinSegmentOperation( + target = targetFile, + segmentFiles = files.map { sharedWorkDir.resolve(it) }, + audioFile = audioFile, + audioSegmentFiles = audioSegmentFiles, + ) + } + + // Handle audio-only outputs + audioFilesMap.forEach { (targetName, audioFile) -> + if (!joinSegmentOperations.containsKey(targetName)) { + val targetFile = outputFolder.resolve(targetName) + joinSegmentOperations[targetName] = JoinSegmentOperation( + target = targetFile, + segmentFiles = emptyList(), + audioFile = audioFile, + audioSegmentFiles = null, + ) + } + } + audioSegmentFileMap.forEach { (targetName, audioSegmentFiles) -> + if (!joinSegmentOperations.containsKey(targetName)) { + val targetFile = outputFolder.resolve(targetName) + joinSegmentOperations[targetName] = JoinSegmentOperation( + target = targetFile, + segmentFiles = emptyList(), + audioFile = null, + audioSegmentFiles = audioSegmentFiles, + ) + } + } + + return joinSegmentOperations + } + + fun joinSegments(encoreJob: EncoreJob, sharedWorkDir: File): List { + val joinSegmentOperations = prepareJoinSegment(encoreJob, sharedWorkDir) + + return joinSegmentOperations.values.map { joinSegmentOperation -> + joinSegments(encoreJob, sharedWorkDir, joinSegmentOperation) + } + } + + fun joinSegments(encoreJob: EncoreJob, sharedWorkDir: File, joinSegmentOperation: JoinSegmentOperation): MediaFile { + log.info { "Joining segments for ${joinSegmentOperation.target.name}" } + + // Create audio segment list file if audio segments are present + val audioSegmentListFile = if (joinSegmentOperation.audioSegmentFiles != null) { + log.info { "Preparing to join ${joinSegmentOperation.audioSegmentFiles.size} audio segments with video" } + val audioSegmentListFile = sharedWorkDir.resolve("audio/${joinSegmentOperation.target.nameWithoutExtension}_audio_filelist.txt") + val segmentedInfo = encoreJob.segmentedEncodingInfoOrThrow() + val padding = segmentedInfo.audioSegmentPadding + val audioSegmentLength = segmentedInfo.audioSegmentLength + val numSegments = joinSegmentOperation.audioSegmentFiles.size + + joinSegmentOperation.audioSegmentFiles.forEachIndexed { index, file -> + audioSegmentListFile.appendText("file ${file.absolutePath}\n") + + // Add inpoint/outpoint to trim padding + val isFirst = index == 0 + val isLast = index == numSegments - 1 + // Trim start padding for all segments except first + val inPoint = if (isFirst) 0.0 else padding + + audioSegmentListFile.appendText("inpoint $inPoint\n") + + if (!isLast) { + // Trim end padding for all segments except last + val outpoint = if (isFirst) audioSegmentLength else audioSegmentLength + padding + audioSegmentListFile.appendText("outpoint $outpoint\n") + } + } + audioSegmentListFile + } else { + null + } + + // Join video segments with audio (or just copy audio if no video) + return if (joinSegmentOperation.segmentFiles.isNotEmpty()) { + val segmentListFile = sharedWorkDir.resolve("${joinSegmentOperation.target.nameWithoutExtension}_filelist.txt") + joinSegmentOperation.segmentFiles.forEach { file -> + segmentListFile.appendText("file ${file.absolutePath}\n") + } + ffmpegExecutor.joinSegments( + encoreJob, + segmentListFile, + joinSegmentOperation.target, + joinSegmentOperation.audioFile, + audioSegmentListFile, + ) + } else { + // Audio-only output: either segments or full file + if (audioSegmentListFile != null) { + log.info { "Joining audio-only segments for ${joinSegmentOperation.target.name}" } + ffmpegExecutor.joinSegments(encoreJob, audioSegmentListFile, joinSegmentOperation.target, null, null) + } else { + log.info { "Moving audio file ${joinSegmentOperation.audioFile} to output folder" } + val target = joinSegmentOperation.target + joinSegmentOperation.audioFile!!.copyTo(target, overwrite = true) + mediaAnalyzerService.analyze(target.absolutePath) + } + } + } +} diff --git a/encore-common/src/main/kotlin/se/svt/oss/encore/util/ProfileUtil.kt b/encore-common/src/main/kotlin/se/svt/oss/encore/util/ProfileUtil.kt new file mode 100644 index 00000000..5ec16fe4 --- /dev/null +++ b/encore-common/src/main/kotlin/se/svt/oss/encore/util/ProfileUtil.kt @@ -0,0 +1,51 @@ +package se.svt.oss.encore.util + +import se.svt.oss.encore.model.profile.AudioEncode +import se.svt.oss.encore.model.profile.AudioEncoder +import se.svt.oss.encore.model.profile.Profile +import se.svt.oss.encore.model.profile.SimpleAudioEncode +import se.svt.oss.encore.model.profile.VideoEncode + +fun Profile.hasAudioEncodes(): Boolean = this.encodes + .filter { it.enabled } + .any { encode -> + when (encode) { + is VideoEncode -> + encode.audioEncode?.enabled == true || encode.audioEncodes.any { it.enabled } + is AudioEncode, + is SimpleAudioEncode, + -> true + else -> false + } + } + +fun Profile.allAudioEncodes(): List = + this.encodes + .filter { it.enabled } + .flatMap { encode -> + when (encode) { + is VideoEncode -> encode.allAudioEncodes() + is AudioEncoder -> listOf(encode) + else -> emptyList() + } + } + +fun VideoEncode.allAudioEncodes(): List { + val audioEncodes = mutableListOf() + this.audioEncode?.let { + if (it.enabled) { + audioEncodes.add(it) + } + } + audioEncodes.addAll(this.audioEncodes.filter { it.enabled }) + return audioEncodes +} + +fun Profile.hasVideoEncodes(): Boolean = this.encodes + .filter { it.enabled } + .any { encode -> + when (encode) { + is VideoEncode -> true + else -> false + } + } diff --git a/encore-common/src/test/kotlin/se/svt/oss/encore/EncoreIntegrationTest.kt b/encore-common/src/test/kotlin/se/svt/oss/encore/EncoreIntegrationTest.kt index 423f63f5..ecc30826 100644 --- a/encore-common/src/test/kotlin/se/svt/oss/encore/EncoreIntegrationTest.kt +++ b/encore-common/src/test/kotlin/se/svt/oss/encore/EncoreIntegrationTest.kt @@ -28,7 +28,7 @@ class EncoreIntegrationTest(wireMockRuntimeInfo: WireMockRuntimeInfo) : EncoreIn @Test fun jobIsSuccessfulSurround(@TempDir outputDir: File) { - successfulTest( + val createdJob = successfulTest( job(outputDir = outputDir, file = testFileSurround), defaultExpectedOutputFiles(outputDir, testFileSurround) + listOf( @@ -39,6 +39,93 @@ class EncoreIntegrationTest(wireMockRuntimeInfo: WireMockRuntimeInfo) : EncoreIn ) } + @Test + fun jobIsSuccessfulSurroundSegmentedEncode(@TempDir outputDir: File) { + val job = job(outputDir = outputDir, file = testFileSurround).copy( + profile = "separate-video-audio", + segmentLength = 3.84, + priority = 100, + ) + val expectedFiles = listOf( + "x264_3100.mp4", + "STEREO.mp4", + ) + .map { expectedFile(outputDir, testFileSurround, it) } + + listOf( + expectedFile(outputDir, testFileSurround, "STEREO_DE.mp4"), + expectedFile(outputDir, testFileSurround, "SURROUND.mp4"), + expectedFile(outputDir, testFileSurround, "SURROUND_DE.mp4"), + ) + + val createdJob = successfulTest( + job, + expectedFiles, + ) + assertThat(createdJob.segmentedEncodingInfo) + .hasAudioEncodingMode(se.svt.oss.encore.model.AudioEncodingMode.ENCODE_WITH_VIDEO) + .hasNumSegments(3) + .hasNumAudioSegments(0) + .hasNumTasks(3) + } + + @Test + fun jobIsSuccessfulSurroundSegmentedEncodeSeparateAudio(@TempDir outputDir: File) { + val job = job(outputDir = outputDir, file = testFileSurround).copy( + profile = "separate-video-audio", + segmentLength = 3.84, + audioEncodingMode = se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_FULL, + priority = 100, + ) + val expectedFiles = listOf( + "x264_3100.mp4", + "STEREO.mp4", + ) + .map { expectedFile(outputDir, testFileSurround, it) } + + listOf( + expectedFile(outputDir, testFileSurround, "STEREO_DE.mp4"), + expectedFile(outputDir, testFileSurround, "SURROUND.mp4"), + expectedFile(outputDir, testFileSurround, "SURROUND_DE.mp4"), + ) + + val createdJob = successfulTest( + job, + expectedFiles, + ) + assertThat(createdJob.segmentedEncodingInfo) + .hasAudioEncodingMode(se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_FULL) + .hasNumSegments(3) + .hasNumTasks(4) + } + + @Test + fun jobIsSuccessfulSegmentedEncodeSeparatelySegmentedAudio(@TempDir outputDir: File) { + val job = job(outputDir = outputDir, file = testFileSurround).copy( + profile = "separate-video-audio", + segmentLength = 3.84, + audioEncodingMode = se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, + audioSegmentLength = 8.0, + priority = 100, + profileParams = mapOf("enableSurround" to "false"), // segmented audio encode not supported for surround + ) + val expectedFiles = listOf( + "x264_3100.mp4", + "STEREO.mp4", + ) + .map { expectedFile(outputDir, testFileSurround, it) } + + listOf( + expectedFile(outputDir, testFileSurround, "STEREO_DE.mp4"), + ) + + val createdJob = successfulTest( + job, + expectedFiles, + ) + assertThat(createdJob.segmentedEncodingInfo) + .hasAudioEncodingMode(se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + .hasNumSegments(3) + .hasNumTasks(5) // 2 audio segments + 3 video segments + } + @Test fun multipleAudioStreamsOutputSegmentedEncode(@TempDir outputDir: File) { val baseName = "multiple_audio" @@ -50,11 +137,52 @@ class EncoreIntegrationTest(wireMockRuntimeInfo: WireMockRuntimeInfo) : EncoreIn ) val expectedOutPut = listOf(outputDir.resolve("$baseName.mp4").absolutePath) val createdJob = successfulTest(job, expectedOutPut) + assertThat(createdJob.segmentedEncodingInfo) + .hasAudioEncodingMode(se.svt.oss.encore.model.AudioEncodingMode.ENCODE_WITH_VIDEO) + .hasNumSegments(3) + .hasNumTasks(3) + assertThat(createdJob.output) + .hasSize(1) + assertThat(createdJob.output[0]) + .isInstanceOf(VideoFile::class.java) + val audioStreams = (createdJob.output[0] as VideoFile).audioStreams + assertThat(audioStreams).hasSize(2) + assertThat(audioStreams[0]) + .hasFormat("AC-3") + .hasCodec("ac3") + .hasDurationCloseTo(10.0, 0.1) + .hasChannels(6) + .hasSamplingRate(48000) + assertThat(audioStreams[1]) + .hasFormat("AAC") + .hasCodec("aac") + .hasDurationCloseTo(10.0, 0.1) + .hasChannels(2) + .hasSamplingRate(48000) + } + + @Test + fun multipleAudioStreamsOutputSegmentedEncodeSeparateAudio(@TempDir outputDir: File) { + val baseName = "multiple_audio" + val job = job(outputDir).copy( + baseName = baseName, + profile = "audio-streams", + segmentLength = 3.84, + priority = 100, + audioEncodingMode = se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_FULL, + ) + val expectedOutPut = listOf(outputDir.resolve("$baseName.mp4").absolutePath) + val createdJob = successfulTest(job, expectedOutPut) assertThat(createdJob.output) .hasSize(1) assertThat(createdJob.output[0]) .isInstanceOf(VideoFile::class.java) + assertThat(createdJob.segmentedEncodingInfo) + .hasAudioEncodingMode(se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_FULL) + .hasNumSegments(3) + .hasNumTasks(4) + val audioStreams = (createdJob.output[0] as VideoFile).audioStreams assertThat(audioStreams).hasSize(2) assertThat(audioStreams[0]) @@ -222,4 +350,43 @@ class EncoreIntegrationTest(wireMockRuntimeInfo: WireMockRuntimeInfo) : EncoreIn assertThat(createdJob.message) .contains("Coding might not be compatible on all devices") } + + @Test + fun jobIsSuccessfulAudioOnlySegmentedEncode(@TempDir outputDir: File) { + val job = job(outputDir = outputDir, file = testFileSurround).copy( + profile = "audio-only", + segmentLength = 3.84, + audioEncodingMode = se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, + audioSegmentLength = 235 * 1024 / 48000.0, // 235 audio frames ~= 5.013333s + priority = 100, + ) + val expectedFiles = listOf( + "STEREO.mp4", + "STEREO_DE.mp4", + ).map { expectedFile(outputDir, testFileSurround, it) } + + val createdJob = successfulTest( + job, + expectedFiles, + ) + assertThat(createdJob.segmentedEncodingInfo) + .hasAudioEncodingMode(se.svt.oss.encore.model.AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + .hasNumSegments(0) // No video segments + .hasNumTasks(2) // Only 2 audio segments + } + + @Test + fun jobIsFailedOnUnknownProfile(@TempDir outputDir: File) { + val createdJob = createAndAwaitJob( + job = job(outputDir).copy(profile = "unknown_profile"), + pollInterval = Durations.ONE_SECOND, + timeout = Durations.ONE_MINUTE, + ) { it.status.isCompleted } + + assertThat(createdJob) + .hasStatus(Status.FAILED) + + assertThat(createdJob.message) + .contains("Could not find location for profile unknown_profile!") + } } diff --git a/encore-common/src/test/kotlin/se/svt/oss/encore/process/SegmentUtilTest.kt b/encore-common/src/test/kotlin/se/svt/oss/encore/process/SegmentUtilTest.kt index 62b3f4d5..5fe212f6 100644 --- a/encore-common/src/test/kotlin/se/svt/oss/encore/process/SegmentUtilTest.kt +++ b/encore-common/src/test/kotlin/se/svt/oss/encore/process/SegmentUtilTest.kt @@ -5,78 +5,134 @@ package se.svt.oss.encore.process import org.assertj.core.data.Offset +import org.junit.jupiter.api.Nested import org.junit.jupiter.api.Test import se.svt.oss.encore.Assertions.assertThat import se.svt.oss.encore.Assertions.assertThatThrownBy import se.svt.oss.encore.defaultEncoreJob -import se.svt.oss.encore.defaultVideoFile import se.svt.oss.encore.longVideoFile +import se.svt.oss.encore.model.AudioEncodingMode +import se.svt.oss.encore.model.SegmentedEncodingInfo import se.svt.oss.encore.model.input.AudioVideoInput +import kotlin.math.ceil class SegmentUtilTest { private val job = defaultEncoreJob().copy( baseName = "segment_test", - segmentLength = 19.2, duration = null, inputs = listOf(AudioVideoInput(uri = "test", analyzed = longVideoFile)), + segmentedEncodingInfo = SegmentedEncodingInfo( + segmentLength = 19.2, + numSegments = ceil(longVideoFile.duration / 19.2).toInt(), + numTasks = ceil(longVideoFile.duration / 19.2).toInt(), + audioEncodingMode = AudioEncodingMode.ENCODE_WITH_VIDEO, + audioSegmentPadding = 0.0, + audioSegmentLength = 0.0, + numAudioSegments = 0, + ), ) - @Test - fun baseName() { - assertThat(job.baseName(2)).isEqualTo("segment_test_00002") - } + @Nested + inner class SegmentDurationTest { - @Test - fun missingSegmentLength() { - val encoreJob = job.copy(segmentLength = null) - val message = "No segmentLength in job!" - assertThatThrownBy { - encoreJob.segmentLengthOrThrow() - }.hasMessage(message) - assertThatThrownBy { - encoreJob.numSegments() - }.hasMessage(message) - assertThatThrownBy { - encoreJob.segmentDuration(1) - }.hasMessage(message) - } + @Test + fun `returns full segment length when duration not set`() { + assertThat(job.segmentDuration(0)).isEqualTo(19.2) + assertThat(job.segmentDuration(5)).isEqualTo(19.2) + assertThat(job.segmentDuration(140)).isEqualTo(19.2) + } - @Test - fun hasSegmentLength() { - assertThat(job.segmentLengthOrThrow()).isEqualTo(19.2) - } + @Test + fun `returns full segment length for non-last segments when duration set`() { + val jobWithDuration = job.copy( + duration = 125.0, + segmentedEncodingInfo = job.segmentedEncodingInfo!!.copy( + numSegments = 7, + ), + ) + assertThat(jobWithDuration.segmentDuration(0)).isEqualTo(19.2) + assertThat(jobWithDuration.segmentDuration(3)).isEqualTo(19.2) + assertThat(jobWithDuration.segmentDuration(5)).isEqualTo(19.2) + } - @Test - fun numSegmentsDurationSet() { - val encoreJob = job.copy(duration = 125.0) - assertThat(encoreJob.numSegments()).isEqualTo(7) - } + @Test + fun `returns remainder for last segment when duration set`() { + val jobWithDuration = job.copy( + duration = 125.0, + segmentedEncodingInfo = job.segmentedEncodingInfo!!.copy( + numSegments = 7, + ), + ) + // 125.0 % 19.2 = 9.8 + assertThat(jobWithDuration.segmentDuration(6)).isCloseTo(9.8, Offset.offset(0.001)) + } - @Test - fun numSegmentsDurationNotSet() { - assertThat(job.numSegments()).isEqualTo(141) - } + @Test + fun `handles exact multiple of segment length`() { + val jobWithDuration = job.copy( + duration = 96.0, // Exactly 5 segments of 19.2 + segmentedEncodingInfo = job.segmentedEncodingInfo!!.copy( + numSegments = 5, + ), + ) + // When duration is an exact multiple of segment length, + // the last segment should still have the full segment length + assertThat(jobWithDuration.segmentDuration(4)).isCloseTo(19.2, Offset.offset(0.001)) + } - @Test - fun numSegmentsInputsDiffer() { - val encoreJob = job.copy(inputs = job.inputs + AudioVideoInput(uri = "test", analyzed = defaultVideoFile)) - assertThatThrownBy { encoreJob.numSegments() } - .hasMessage("Inputs differ in length") - } + @Test + fun `throws exception for invalid segment number`() { + val jobWithDuration = job.copy( + duration = 96.0, // Exactly 5 segments of 19.2 + segmentedEncodingInfo = job.segmentedEncodingInfo!!.copy( + numSegments = 5, + ), + ) - @Test - fun segmentDurationDurationNotSet() { - assertThat(job.segmentDuration(140)).isEqualTo(19.2) + assertThatThrownBy { jobWithDuration.segmentDuration(5) } + .hasMessage("segmentNumber 5 is out of range for job with 5 segments") + } + + @Test + fun `throws exception when segmentedEncodingInfo is missing`() { + val jobWithoutInfo = job.copy(segmentedEncodingInfo = null) + assertThatThrownBy { + jobWithoutInfo.segmentDuration(0) + }.hasMessage("No segmentedEncodingInfo in job!") + } } - @Test - fun segmentDurationDurationSetFirst() { - assertThat(job.copy(duration = 125.0).segmentDuration(0)).isEqualTo(19.2) + @Nested + inner class BaseNameTest { + + @Test + fun `generates correct base name with segment number`() { + assertThat(job.baseName(2)).isEqualTo("segment_test_00002") + } } - @Test - fun segmentDurationDurationSetLast() { - assertThat(job.copy(duration = 125.0).segmentDuration(6)).isCloseTo(9.8, Offset.offset(0.001)) + @Nested + inner class SegmentSuffixFromFilenameTest { + + @Test + fun `extracts suffix from segment filename`() { + val encoreJob = job.copy(baseName = "test_video") + assertThat(encoreJob.segmentSuffixFromFilename("test_video_00003_720p.mp4")).isEqualTo("_720p.mp4") + } + + @Test + fun `extracts suffix when no additional suffix present`() { + val encoreJob = job.copy(baseName = "test_video") + assertThat(encoreJob.segmentSuffixFromFilename("test_video_00000.mp4")).isEqualTo(".mp4") + } + + @Test + fun `throws exception for invalid segment filename`() { + val encoreJob = job.copy(baseName = "test_video") + assertThatThrownBy { + encoreJob.segmentSuffixFromFilename("wrong_name.mp4") + }.hasMessageContaining("Could not find segment suffix for file wrong_name.mp4") + } } } diff --git a/encore-common/src/test/kotlin/se/svt/oss/encore/service/FfmpegExecutorTest.kt b/encore-common/src/test/kotlin/se/svt/oss/encore/service/FfmpegExecutorTest.kt new file mode 100644 index 00000000..5afe224f --- /dev/null +++ b/encore-common/src/test/kotlin/se/svt/oss/encore/service/FfmpegExecutorTest.kt @@ -0,0 +1,38 @@ +package se.svt.oss.encore.service + +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Assertions.assertNotNull +import org.junit.jupiter.api.Assertions.assertNull +import org.junit.jupiter.api.Nested +import org.junit.jupiter.api.Test + +class FfmpegExecutorTest { + @Nested + inner class TestGetProgress { + @Test + fun `valid time and duration, returns progress`() { + val logLine = "frame= 240 fps= 24 q=28.0 size= 1024kB time=00:00:10.00 bitrate= 838.9kbits/s speed=1.00x" + val duration = 20.0 // seconds + val progress = getProgress(duration, logLine) + assertNotNull(progress) + assertEquals(50, progress) + } + + @Test + fun `invalid logline, returns null`() { + val logLine = "RANDOM LOG LINE" + val duration = 20.0 // seconds + val progress = getProgress(duration, logLine) + assertNull(progress) + } + + @Test + fun `null duration, returns null`() { + val logLine = + "frame= 240 fps= 24 q=28.0 size= 1024kB time=00:00:10.00 bitrate= 838.9kbits/s speed=1.00x" + val duration: Double? = null + val progress = getProgress(duration, logLine) + assertNull(progress) + } + } +} diff --git a/encore-common/src/test/kotlin/se/svt/oss/encore/service/mediaanalyzer/MediaAnalyzerServiceTest.kt b/encore-common/src/test/kotlin/se/svt/oss/encore/service/mediaanalyzer/MediaAnalyzerServiceTest.kt new file mode 100644 index 00000000..2e705f9c --- /dev/null +++ b/encore-common/src/test/kotlin/se/svt/oss/encore/service/mediaanalyzer/MediaAnalyzerServiceTest.kt @@ -0,0 +1,25 @@ +package se.svt.oss.encore.service.mediaanalyzer + +import io.mockk.every +import io.mockk.mockk +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test +import se.svt.oss.mediaanalyzer.MediaAnalyzer +import se.svt.oss.mediaanalyzer.file.MediaFile + +class MediaAnalyzerServiceTest { + private val mediaAnalyzer = mockk() + + private val mediaAnalyzerService = MediaAnalyzerService(mediaAnalyzer) + + @Test + fun testAnalyze() { + val mediaFile = mockk() + val slot = io.mockk.slot() + every { mediaAnalyzer.analyze(capture(slot)) } returns mediaFile + + val actual = mediaAnalyzerService.analyze("testInput") + assertEquals(mediaFile, actual) + assertEquals("testInput", slot.captured) + } +} diff --git a/encore-common/src/test/kotlin/se/svt/oss/encore/service/segmentedencode/SegmentedEncodeServiceTest.kt b/encore-common/src/test/kotlin/se/svt/oss/encore/service/segmentedencode/SegmentedEncodeServiceTest.kt new file mode 100644 index 00000000..7a78261a --- /dev/null +++ b/encore-common/src/test/kotlin/se/svt/oss/encore/service/segmentedencode/SegmentedEncodeServiceTest.kt @@ -0,0 +1,769 @@ +// SPDX-FileCopyrightText: 2025 Eyevinn Technology AB +// +// SPDX-License-Identifier: EUPL-1.2 + +package se.svt.oss.encore.service.segmentedencode + +import io.mockk.every +import io.mockk.mockk +import io.mockk.slot +import io.mockk.verify +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Assertions.assertNull +import org.junit.jupiter.api.Nested +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.io.TempDir +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.CsvSource +import se.svt.oss.encore.config.EncoreProperties +import se.svt.oss.encore.defaultEncoreJob +import se.svt.oss.encore.model.AudioEncodingMode +import se.svt.oss.encore.model.SegmentedEncodingInfo +import se.svt.oss.encore.model.profile.Profile +import se.svt.oss.encore.model.profile.SimpleAudioEncode +import se.svt.oss.encore.model.profile.X264Encode +import se.svt.oss.encore.model.queue.TaskType +import se.svt.oss.encore.service.FfmpegExecutor +import se.svt.oss.encore.service.mediaanalyzer.MediaAnalyzerService +import se.svt.oss.encore.service.profile.ProfileService +import se.svt.oss.mediaanalyzer.file.MediaFile +import java.io.File + +class SegmentedEncodeServiceTest { + + private val ffmpegExecutor: FfmpegExecutor = mockk() + private val mediaAnalyzerService: MediaAnalyzerService = mockk() + private val profileService: ProfileService = mockk() + private val encoreProperties: EncoreProperties = mockk() + private val service = SegmentedEncodeService(ffmpegExecutor, mediaAnalyzerService, profileService, encoreProperties) + + private fun createJobWithSegmentedEncoding( + numSegments: Int, + audioEncodingMode: AudioEncodingMode, + outputFolder: String = "/output/path", + baseName: String = "test", + audioSegmentPadding: Double = 0.0, + audioSegmentLength: Double = 0.0, + numAudioSegments: Int = when (audioEncodingMode) { + AudioEncodingMode.ENCODE_WITH_VIDEO -> 0 + AudioEncodingMode.ENCODE_SEPARATELY_FULL -> 1 + AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED -> numSegments + }, + ) = defaultEncoreJob().copy( + outputFolder = outputFolder, + baseName = baseName, + segmentedEncodingInfo = SegmentedEncodingInfo( + segmentLength = 10.0, + numSegments = numSegments, + numTasks = numSegments + numAudioSegments, + audioEncodingMode = audioEncodingMode, + audioSegmentPadding = audioSegmentPadding, + audioSegmentLength = audioSegmentLength, + numAudioSegments = numAudioSegments, + ), + ) + + private fun createSegmentFiles(workDir: File, baseName: String, suffixes: List, segmentCount: Int) { + suffixes.forEach { suffix -> + repeat(segmentCount) { i -> + File(workDir, "${baseName}_${"%05d".format(i)}$suffix").writeText("segment$i") + } + } + } + + private fun setupDirectories(tempDir: File): Pair { + val outputFolder = File(tempDir, "output").apply { mkdirs() } + val workDir = File(tempDir, "work").apply { mkdirs() } + return outputFolder to workDir + } + + private fun assertOperationMatches( + operation: SegmentedEncodeService.JoinSegmentOperation, + expectedTarget: File, + expectedSegments: List, + expectedAudio: File? = null, + expectedAudioSegments: List? = null, + ) { + assertEquals(expectedTarget, operation.target) + assertEquals(expectedSegments.size, operation.segmentFiles.size) + expectedSegments.forEachIndexed { index, expectedFile -> + assertEquals(expectedFile, operation.segmentFiles[index]) + } + if (expectedAudio != null) { + assertEquals(expectedAudio, operation.audioFile) + } else { + assertNull(operation.audioFile) + } + if (expectedAudioSegments != null) { + assertEquals(expectedAudioSegments.size, operation.audioSegmentFiles?.size) + expectedAudioSegments.forEachIndexed { index, expectedFile -> + assertEquals(expectedFile, operation.audioSegmentFiles?.get(index)) + } + } else { + assertNull(operation.audioSegmentFiles) + } + } + + private fun expectedSegmentFiles(workDir: File, baseName: String, suffix: String, count: Int): List = + (0 until count).map { File(workDir, "${baseName}_${"%05d".format(it)}$suffix") } + + private fun createSegmentFilesList(workDir: File, baseName: String, suffix: String, count: Int): List = + expectedSegmentFiles(workDir, baseName, suffix, count).onEach { it.writeText("segment") } + + private fun expectedSegmentListContent(segments: List): String = + segments.joinToString("\n", postfix = "\n") { "file ${it.absolutePath}" } + + private fun assertSegmentListFileMatches(actual: File, expectedFile: File, expectedSegments: List) { + assertEquals(expectedFile, actual) + assertEquals(true, actual.exists()) + assertEquals(expectedSegmentListContent(expectedSegments), actual.readText()) + } + + @Nested + inner class CreateTasksTest { + + @ParameterizedTest(name = "numSegments={0}, audioMode={1} creates {2} tasks") + @CsvSource( + "3, ENCODE_WITH_VIDEO, 3", + "10, ENCODE_WITH_VIDEO, 10", + "3, ENCODE_SEPARATELY_FULL, 4", + "1, ENCODE_SEPARATELY_FULL, 2", + "3, ENCODE_SEPARATELY_SEGMENTED, 6", + "5, ENCODE_SEPARATELY_SEGMENTED, 10", + ) + fun `creates correct number of tasks`(numSegments: Int, audioEncodingMode: AudioEncodingMode, expectedTasks: Int) { + val encoreJob = createJobWithSegmentedEncoding(numSegments, audioEncodingMode) + + val tasks = service.createTasks(encoreJob) + + assertEquals(expectedTasks, tasks.size) + } + + @Test + fun `creates audio-video segment tasks when using ENCODE_WITH_VIDEO mode`() { + val encoreJob = createJobWithSegmentedEncoding(numSegments = 3, audioEncodingMode = AudioEncodingMode.ENCODE_WITH_VIDEO) + + val tasks = service.createTasks(encoreJob) + + tasks.forEachIndexed { index, task -> + assertEquals(TaskType.AUDIOVIDEOSEGMENT, task.type) + assertEquals(index, task.taskNo) + assertEquals(index, task.segment) + } + } + + @Test + fun `creates separate full audio task and video segments when using ENCODE_SEPARATELY_FULL mode`() { + val encoreJob = createJobWithSegmentedEncoding(numSegments = 3, audioEncodingMode = AudioEncodingMode.ENCODE_SEPARATELY_FULL) + + val tasks = service.createTasks(encoreJob) + + assertEquals(TaskType.AUDIOFULL, tasks[0].type) + assertEquals(0, tasks[0].taskNo) + + tasks.drop(1).forEachIndexed { index, task -> + assertEquals(TaskType.VIDEOSEGMENT, task.type) + assertEquals(index + 1, task.taskNo) + assertEquals(index, task.segment) + } + } + + @Test + fun `creates audio segments and video segments when using ENCODE_SEPARATELY_SEGMENTED mode`() { + val encoreJob = createJobWithSegmentedEncoding(numSegments = 3, audioEncodingMode = AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val tasks = service.createTasks(encoreJob) + + assertEquals(6, tasks.size) + + // First 3 tasks should be audio segments + tasks.take(3).forEachIndexed { index, task -> + assertEquals(TaskType.AUDIOSEGMENT, task.type) + assertEquals(index, task.taskNo) + assertEquals(index, task.segment) + } + + // Next 3 tasks should be video segments + tasks.drop(3).forEachIndexed { index, task -> + assertEquals(TaskType.VIDEOSEGMENT, task.type) + assertEquals(index + 3, task.taskNo) + assertEquals(index, task.segment) + } + } + } + + @Nested + inner class PrepareJoinSegmentTest { + + @TempDir + lateinit var tempDir: File + + @Test + fun `prepares join operations for ENCODE_WITH_VIDEO mode`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + createSegmentFiles(workDir, "test", listOf("_720p.mp4", "_1080p.mp4"), 3) + + val encoreJob = createJobWithSegmentedEncoding( + numSegments = 3, + audioEncodingMode = AudioEncodingMode.ENCODE_WITH_VIDEO, + outputFolder = outputFolder.absolutePath, + ) + + val operations = service.prepareJoinSegment(encoreJob, workDir) + + assertEquals(2, operations.size) + assertOperationMatches( + operations["test_720p.mp4"]!!, + expectedTarget = File(outputFolder, "test_720p.mp4"), + expectedSegments = expectedSegmentFiles(workDir, "test", "_720p.mp4", 3), + ) + assertOperationMatches( + operations["test_1080p.mp4"]!!, + expectedTarget = File(outputFolder, "test_1080p.mp4"), + expectedSegments = expectedSegmentFiles(workDir, "test", "_1080p.mp4", 3), + ) + } + + @Test + fun `prepares join operations for ENCODE_SEPARATELY_FULL mode`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + val audioDir = File(workDir, "audio").apply { mkdirs() } + + createSegmentFiles(workDir, "test", listOf("_720p.mp4"), 2) + File(audioDir, "test_720p.mp4").writeText("audio") + File(audioDir, "test_audio.mp4").writeText("audio_only") + + val encoreJob = createJobWithSegmentedEncoding( + numSegments = 2, + audioEncodingMode = AudioEncodingMode.ENCODE_SEPARATELY_FULL, + outputFolder = outputFolder.absolutePath, + ) + + val operations = service.prepareJoinSegment(encoreJob, workDir) + + assertEquals(2, operations.size) + assertOperationMatches( + operations["test_720p.mp4"]!!, + expectedTarget = File(outputFolder, "test_720p.mp4"), + expectedSegments = expectedSegmentFiles(workDir, "test", "_720p.mp4", 2), + expectedAudio = File(audioDir, "test_720p.mp4"), + ) + assertOperationMatches( + operations["test_audio.mp4"]!!, + expectedTarget = File(outputFolder, "test_audio.mp4"), + expectedSegments = emptyList(), + expectedAudio = File(audioDir, "test_audio.mp4"), + ) + } + + @Test + fun `prepares join operations for ENCODE_SEPARATELY_SEGMENTED mode`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + val audioDir = File(workDir, "audio").apply { mkdirs() } + + // Create video segments + createSegmentFiles(workDir, "test", listOf("_720p.mp4"), 2) + // Create audio segments + createSegmentFiles(audioDir, "test", listOf("_720p.mp4"), 2) + // Create audio-only segments + createSegmentFiles(audioDir, "test", listOf("_audio.mp4"), 2) + + val encoreJob = createJobWithSegmentedEncoding( + numSegments = 2, + audioEncodingMode = AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, + outputFolder = outputFolder.absolutePath, + ) + + val operations = service.prepareJoinSegment(encoreJob, workDir) + + // Should have 2 operations: + // 1. Video join operation for test_720p.mp4 (with audioSegmentFiles) + // 2. Audio-only join operation for test_audio.mp4 + assertEquals(2, operations.size) + + // Video operation should have audio segment files + assertOperationMatches( + operations["test_720p.mp4"]!!, + expectedTarget = File(outputFolder, "test_720p.mp4"), + expectedSegments = expectedSegmentFiles(workDir, "test", "_720p.mp4", 2), + expectedAudioSegments = expectedSegmentFiles(audioDir, "test", "_720p.mp4", 2), + ) + + // Audio-only output + assertOperationMatches( + operations["test_audio.mp4"]!!, + expectedTarget = File(outputFolder, "test_audio.mp4"), + expectedSegments = emptyList(), + expectedAudioSegments = expectedSegmentFiles(audioDir, "test", "_audio.mp4", 2), + ) + } + + @Test + fun `groups segment files by suffix correctly`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + createSegmentFiles(workDir, "test", listOf(".mp4", "_high.mp4", "_low.mp4"), 2) + + val encoreJob = createJobWithSegmentedEncoding( + numSegments = 2, + audioEncodingMode = AudioEncodingMode.ENCODE_WITH_VIDEO, + outputFolder = outputFolder.absolutePath, + ) + + val operations = service.prepareJoinSegment(encoreJob, workDir) + + assertEquals(3, operations.size) + assertOperationMatches( + operations["test.mp4"]!!, + expectedTarget = File(outputFolder, "test.mp4"), + expectedSegments = expectedSegmentFiles(workDir, "test", ".mp4", 2), + ) + assertOperationMatches( + operations["test_high.mp4"]!!, + expectedTarget = File(outputFolder, "test_high.mp4"), + expectedSegments = expectedSegmentFiles(workDir, "test", "_high.mp4", 2), + ) + assertOperationMatches( + operations["test_low.mp4"]!!, + expectedTarget = File(outputFolder, "test_low.mp4"), + expectedSegments = expectedSegmentFiles(workDir, "test", "_low.mp4", 2), + ) + } + } + + @Nested + inner class JoinSegmentsTest { + + @TempDir + lateinit var tempDir: File + + @Test + fun `joins segments with audio file`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + val audioDir = File(workDir, "audio").apply { mkdirs() } + + val segments = createSegmentFilesList(workDir, "test", "_720p.mp4", 2) + val audioFile = File(audioDir, "test_720p.mp4").apply { writeText("audio") } + val targetFile = File(outputFolder, "test_720p.mp4") + + val segmentListFileSlot = slot() + val mockMediaFile = mockk() + every { ffmpegExecutor.joinSegments(any(), capture(segmentListFileSlot), any(), any(), any()) } returns mockMediaFile + + val operation = SegmentedEncodeService.JoinSegmentOperation(targetFile, segments, audioFile) + val encoreJob = defaultEncoreJob().copy(outputFolder = outputFolder.absolutePath, baseName = "test") + + val result = service.joinSegments(encoreJob, workDir, operation) + + assertEquals(mockMediaFile, result) + verify { ffmpegExecutor.joinSegments(encoreJob, any(), targetFile, audioFile, null) } + assertSegmentListFileMatches(segmentListFileSlot.captured, File(workDir, "test_720p_filelist.txt"), segments) + } + + @Test + fun `joins segments without audio file`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + + val segments = createSegmentFilesList(workDir, "test", "_720p.mp4", 2) + val targetFile = File(outputFolder, "test_720p.mp4") + + val segmentListFileSlot = slot() + val mockMediaFile = mockk() + every { ffmpegExecutor.joinSegments(any(), capture(segmentListFileSlot), any(), null, any()) } returns mockMediaFile + + val operation = SegmentedEncodeService.JoinSegmentOperation(targetFile, segments, null) + val encoreJob = defaultEncoreJob().copy(outputFolder = outputFolder.absolutePath, baseName = "test") + + val result = service.joinSegments(encoreJob, workDir, operation) + + assertEquals(mockMediaFile, result) + verify { ffmpegExecutor.joinSegments(encoreJob, any(), targetFile, null, null) } + assertSegmentListFileMatches(segmentListFileSlot.captured, File(workDir, "test_720p_filelist.txt"), segments) + } + + @Test + fun `copies audio file when no video segments exist`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + val audioDir = File(workDir, "audio").apply { mkdirs() } + + val audioFile = File(audioDir, "test_audio.mp4").apply { writeText("audio_only") } + val targetFile = File(outputFolder, "test_audio.mp4") + + val mockMediaFile = mockk() + every { mediaAnalyzerService.analyze(targetFile.absolutePath) } returns mockMediaFile + + val operation = SegmentedEncodeService.JoinSegmentOperation(targetFile, emptyList(), audioFile) + val encoreJob = defaultEncoreJob().copy(outputFolder = outputFolder.absolutePath, baseName = "test") + + val result = service.joinSegments(encoreJob, workDir, operation) + + assertEquals(mockMediaFile, result) + assertEquals("audio_only", targetFile.readText()) + verify { mediaAnalyzerService.analyze(targetFile.absolutePath) } + } + + @Test + fun `joins audio segments with padding`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + val audioDir = File(workDir, "audio").apply { mkdirs() } + + val audioSegments = createSegmentFilesList(audioDir, "test", "_audio.mp4", 3) + val targetFile = File(outputFolder, "test_audio.mp4") + + val mockMediaFile = mockk() + val capturedSegmentList = slot() + every { ffmpegExecutor.joinSegments(any(), capture(capturedSegmentList), any(), null, null) } returns mockMediaFile + + val operation = SegmentedEncodeService.JoinSegmentOperation(targetFile, emptyList(), null, audioSegments) + val encoreJob = defaultEncoreJob().copy( + outputFolder = outputFolder.absolutePath, + baseName = "test", + segmentedEncodingInfo = SegmentedEncodingInfo( + segmentLength = 10.0, + numSegments = 3, + numTasks = 3, + audioEncodingMode = AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, + audioSegmentPadding = 2 * 8.0 / 375.0, // 2 audio frames for 48khz + audioSegmentLength = 8.0, + numAudioSegments = 4, + ), + ) + + val result = service.joinSegments(encoreJob, workDir, operation) + + assertEquals(mockMediaFile, result) + verify { ffmpegExecutor.joinSegments(encoreJob, any(), targetFile, null, null) } + + // Verify the audio segment list file was created with proper padding + val audioSegmentListFile = capturedSegmentList.captured + assertEquals(true, audioSegmentListFile.exists()) + + val expectedContent = """ + file ${audioSegments[0].absolutePath} + inpoint 0.0 + outpoint 8.0 + file ${audioSegments[1].absolutePath} + inpoint 0.042666666666666665 + outpoint 8.042666666666667 + file ${audioSegments[2].absolutePath} + inpoint 0.042666666666666665 + + """.trimIndent() + + assertEquals(expectedContent, audioSegmentListFile.readText()) + } + + @Test + fun `joins video segments with audio segment list`() { + val (outputFolder, workDir) = setupDirectories(tempDir) + val audioDir = File(workDir, "audio").apply { mkdirs() } + + val videoSegments = createSegmentFilesList(workDir, "test", "_720p.mp4", 2) + val audioSegments = createSegmentFilesList(audioDir, "test", "_720p.mp4", 2) + val targetFile = File(outputFolder, "test_720p.mp4") + + val mockMediaFile = mockk() + val capturedAudioSegmentList = slot() + every { + ffmpegExecutor.joinSegments(any(), any(), any(), null, capture(capturedAudioSegmentList)) + } returns mockMediaFile + + val operation = SegmentedEncodeService.JoinSegmentOperation(targetFile, videoSegments, null, audioSegments) + val encoreJob = defaultEncoreJob().copy( + outputFolder = outputFolder.absolutePath, + baseName = "test", + segmentedEncodingInfo = SegmentedEncodingInfo( + segmentLength = 10.0, + numSegments = 2, + numTasks = 4, + audioEncodingMode = AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, + audioSegmentPadding = 0.2, + audioSegmentLength = 8.0, + numAudioSegments = 2, + ), + ) + + val result = service.joinSegments(encoreJob, workDir, operation) + + assertEquals(mockMediaFile, result) + verify { ffmpegExecutor.joinSegments(encoreJob, any(), targetFile, null, any()) } + + // Verify the audio segment list file was created + val audioSegmentListFile = capturedAudioSegmentList.captured + assertEquals(true, audioSegmentListFile.exists()) + } + } + + @Nested + inner class AudioEncodingModeTest { + + private fun setupEncoreProperties(audioEncodingMode: AudioEncodingMode) { + val segmentedEncodingProps = mockk { + every { this@mockk.audioEncodingMode } returns audioEncodingMode + } + val encodingProps = mockk { + every { segmentedEncoding } returns segmentedEncodingProps + } + every { encoreProperties.encoding } returns encodingProps + } + + @Test + fun `returns nulls for ENCODE_WITH_VIDEO mode`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 48000), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_WITH_VIDEO) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_WITH_VIDEO, config.audioEncodingMode) + assertEquals(0.0, config.audioSegmentPadding, 0.0001) + assertEquals(0.0, config.audioSegmentLength, 0.0001) + } + + @Test + fun `returns nulls for ENCODE_SEPARATELY_FULL mode`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 48000), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_FULL) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_SEPARATELY_FULL, config.audioEncodingMode) + assertEquals(0.0, config.audioSegmentPadding, 0.0001) + assertEquals(0.0, config.audioSegmentLength, 0.0001) + } + + @Test + fun `downgrades to ENCODE_WITH_VIDEO when no audio encodes`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + X264Encode(width = 1920, height = 1080, twoPass = false, suffix = "_test"), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_WITH_VIDEO, config.audioEncodingMode) + assertEquals(0.0, config.audioSegmentPadding, 0.0001) + assertEquals(0.0, config.audioSegmentLength, 0.0001) + } + + @Test + fun `downgrades to ENCODE_SEPARATELY_FULL when multiple sample rates with ENCODE_SEPARATELY_SEGMENTED`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 48000), + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 44100), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_SEPARATELY_FULL, config.audioEncodingMode) + assertEquals(0.0, config.audioSegmentPadding, 0.0001) + assertEquals(0.0, config.audioSegmentLength, 0.0001) + } + + @Test + fun `calculates audioSegmentPadding for ENCODE_SEPARATELY_SEGMENTED with single sample rate`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 48000), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, config.audioEncodingMode) + // audioSegmentPadding = 2 * 1024 / 48000 = 0.04266... + assertEquals(2.0 * 1024.0 / 48000.0, config.audioSegmentPadding, 0.0001) + } + + @Test + fun `calculates audioSegmentLength close to 256s for ENCODE_SEPARATELY_SEGMENTED`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 48000), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, config.audioEncodingMode) + // Should be close to 256s and a multiple of frame duration (1024 / 48000 = 0.021333...) + val frameDuration = 1024.0 / 48000.0 + val expectedLength = kotlin.math.round(256.0 / frameDuration) * frameDuration + assertEquals(expectedLength, config.audioSegmentLength, 0.0001) + // Verify it's actually close to 256s + assertEquals(256.0, config.audioSegmentLength, 0.1) + } + + @Test + fun `uses custom audioSegmentLength from job when specified`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 48000), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val customLength = 300.0 + val job = defaultEncoreJob().copy(audioSegmentLength = customLength) + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, config.audioEncodingMode) + assertEquals(customLength, config.audioSegmentLength) + } + + @Test + fun `uses job audioEncodingMode when specified`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 48000), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_WITH_VIDEO) + + val job = defaultEncoreJob().copy(audioEncodingMode = AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + val config = service.audioEncodingConfig(job, profile) + + assertEquals(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED, config.audioEncodingMode) + } + + @Test + fun `downgrades when multiple sample rates within same output`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.GenericVideoEncode( + width = null, + height = 720, + twoPass = false, + params = linkedMapOf(), + audioEncode = null, + audioEncodes = listOf( + se.svt.oss.encore.model.profile.SimpleAudioEncode(samplerate = 48000), + se.svt.oss.encore.model.profile.SimpleAudioEncode(samplerate = 44100), + ), + suffix = "_test", + format = "mp4", + codec = "libx264", + ), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + // Should downgrade to ENCODE_SEPARATELY_FULL because multiple sample rates are present + assertEquals(AudioEncodingMode.ENCODE_SEPARATELY_FULL, config.audioEncodingMode) + assertEquals(0.0, config.audioSegmentPadding, 0.0001) + assertEquals(0.0, config.audioSegmentLength, 0.0001) + } + + @Test + fun `uses default sample rate of 48000 when no sample rates found`() { + val profile = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.GenericVideoEncode( + width = null, + height = 720, + twoPass = false, + params = linkedMapOf(), + audioEncode = SimpleAudioEncode(), + suffix = "_test", + format = "mp4", + codec = "libx264", + ), + ), + ) + every { profileService.getProfile(any()) } returns profile + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val job = defaultEncoreJob() + val config = service.audioEncodingConfig(job, profile) + + // Should use default 48000 + assertEquals(2.0 * 1024.0 / 48000.0, config.audioSegmentPadding, 0.0001) + } + + @Test + fun `calculates different padding for different sample rates`() { + // Test with 44100 Hz + val profile44100 = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 44100), + ), + ) + every { profileService.getProfile(any()) } returns profile44100 + setupEncoreProperties(AudioEncodingMode.ENCODE_SEPARATELY_SEGMENTED) + + val job1 = defaultEncoreJob() + val config1 = service.audioEncodingConfig(job1, profile44100) + assertEquals(2.0 * 1024.0 / 44100.0, config1.audioSegmentPadding, 0.0001) + + // Test with 96000 Hz + val profile96000 = Profile( + name = "test", + description = "test", + encodes = listOf( + se.svt.oss.encore.model.profile.AudioEncode(samplerate = 96000), + ), + ) + every { profileService.getProfile(any()) } returns profile96000 + + val job2 = defaultEncoreJob() + val config2 = service.audioEncodingConfig(job2, profile96000) + assertEquals(2.0 * 1024.0 / 96000.0, config2.audioSegmentPadding, 0.0001) + + // Higher sample rate should have smaller padding + assert(config2.audioSegmentPadding < config1.audioSegmentPadding) + } + } +} diff --git a/encore-common/src/test/resources/profile/audio-only.yml b/encore-common/src/test/resources/profile/audio-only.yml new file mode 100644 index 00000000..71109e4b --- /dev/null +++ b/encore-common/src/test/resources/profile/audio-only.yml @@ -0,0 +1,14 @@ +name: audio-only +description: Audio-only profile for testing segmented audio encoding +encodes: + - type: AudioEncode + bitrate: 128k + suffix: _STEREO + + - type: AudioEncode + bitrate: 128k + suffix: _STEREO_DE + dialogueEnhancement: + enabled: true + audioMixPreset: de + optional: true diff --git a/encore-common/src/test/resources/profile/profiles.yml b/encore-common/src/test/resources/profile/profiles.yml index 1e6174cf..9867513c 100644 --- a/encore-common/src/test/resources/profile/profiles.yml +++ b/encore-common/src/test/resources/profile/profiles.yml @@ -4,6 +4,8 @@ dpb_size_failed: dpb_size_failed.yml program-x265: program-x265.yml archive: archive.yml audio-streams: audio-streams.yml +audio-only: audio-only.yml test-invalid: test_profile_invalid.yml test-invalid-location: test_profile_invalid_location.yml +separate-video-audio: separate-video-audio.yml none: diff --git a/encore-common/src/test/resources/profile/separate-video-audio.yml b/encore-common/src/test/resources/profile/separate-video-audio.yml new file mode 100644 index 00000000..0d5fd1b8 --- /dev/null +++ b/encore-common/src/test/resources/profile/separate-video-audio.yml @@ -0,0 +1,53 @@ +name: separate-video-audio +description: Testing profile with one video/audio output and multiple audio only outputs +scaling: bicubic +encodes: + - type: X264Encode + suffix: _x264_3100 + twoPass: true + height: 1080 + params: + b:v: 3100k + maxrate: 4700k + bufsize: 6200k + r: 25 + fps_mode: cfr + pix_fmt: yuv420p + force_key_frames: expr:not(mod(n,96)) + profile:v: high + level: 4.1 + audioEncode: + type: AudioEncode + bitrate: 128k + suffix: STEREO + + - type: AudioEncode + bitrate: 128k + suffix: _STEREO + + - type: AudioEncode + bitrate: 128k + suffix: _STEREO_DE + dialogueEnhancement: + enabled: true + audioMixPreset: de + optional: true + + - type: AudioEncode + codec: ac3 + bitrate: 448k + suffix: _SURROUND + optional: true + channelLayout: '5.1' + enabled: #{profileParams['enableSurround'] ?: 'true'} + + - type: AudioEncode + codec: ac3 + bitrate: 448k + suffix: _SURROUND_DE + dialogueEnhancement: + enabled: true + audioMixPreset: de + optional: true + channelLayout: '5.1' + enabled: #{profileParams['enableSurround'] ?: 'true'}