summaryrefslogtreecommitdiffstats
path: root/src/plugins/multimedia/ffmpeg/recordingengine
diff options
context:
space:
mode:
Diffstat (limited to 'src/plugins/multimedia/ffmpeg/recordingengine')
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder.cpp343
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder_p.h77
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils.cpp97
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils_p.h28
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions.cpp362
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions_p.h32
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread.cpp40
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread_p.h72
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer.cpp165
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer_p.h77
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer.cpp64
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer_p.h41
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine.cpp278
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine_p.h121
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils.cpp63
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils_p.h81
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder.cpp259
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder_p.h64
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils.cpp214
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils_p.h64
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder.cpp547
-rw-r--r--src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder_p.h104
22 files changed, 3193 insertions, 0 deletions
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder.cpp
new file mode 100644
index 000000000..d8eaae58b
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder.cpp
@@ -0,0 +1,343 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#include "qffmpegaudioencoder_p.h"
+#include "qffmpegrecordingengineutils_p.h"
+#include "qffmpegaudioencoderutils_p.h"
+#include "qffmpegaudioinput_p.h"
+#include "qffmpegencoderoptions_p.h"
+#include "qffmpegmuxer_p.h"
+#include "qffmpegrecordingengine_p.h"
+#include "qffmpegmediaformatinfo_p.h"
+#include <QtCore/qloggingcategory.h>
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+Q_STATIC_LOGGING_CATEGORY(qLcFFmpegAudioEncoder, "qt.multimedia.ffmpeg.audioencoder");
+
+AudioEncoder::AudioEncoder(RecordingEngine &recordingEngine, const QAudioFormat &sourceFormat,
+ const QMediaEncoderSettings &settings)
+ : EncoderThread(recordingEngine), m_format(sourceFormat), m_settings(settings)
+{
+ setObjectName(QLatin1String("AudioEncoder"));
+ qCDebug(qLcFFmpegAudioEncoder) << "AudioEncoder" << settings.audioCodec();
+
+ auto codecID = QFFmpegMediaFormatInfo::codecIdForAudioCodec(settings.audioCodec());
+ Q_ASSERT(avformat_query_codec(recordingEngine.avFormatContext()->oformat, codecID,
+ FF_COMPLIANCE_NORMAL));
+
+ const AVAudioFormat requestedAudioFormat(m_format);
+
+ m_avCodec = QFFmpeg::findAVEncoder(codecID, {}, requestedAudioFormat.sampleFormat);
+
+ if (!m_avCodec)
+ m_avCodec = QFFmpeg::findAVEncoder(codecID);
+
+ qCDebug(qLcFFmpegAudioEncoder) << "found audio codec" << m_avCodec->name;
+
+ Q_ASSERT(m_avCodec);
+
+ m_stream = avformat_new_stream(recordingEngine.avFormatContext(), nullptr);
+ m_stream->id = recordingEngine.avFormatContext()->nb_streams - 1;
+ m_stream->codecpar->codec_type = AVMEDIA_TYPE_AUDIO;
+ m_stream->codecpar->codec_id = codecID;
+#if QT_FFMPEG_OLD_CHANNEL_LAYOUT
+ m_stream->codecpar->channel_layout =
+ adjustChannelLayout(m_avCodec->channel_layouts, requestedAudioFormat.channelLayoutMask);
+ m_stream->codecpar->channels = qPopulationCount(m_stream->codecpar->channel_layout);
+#else
+ m_stream->codecpar->ch_layout =
+ adjustChannelLayout(m_avCodec->ch_layouts, requestedAudioFormat.channelLayout);
+#endif
+ const auto sampleRate =
+ adjustSampleRate(m_avCodec->supported_samplerates, requestedAudioFormat.sampleRate);
+
+ m_stream->codecpar->sample_rate = sampleRate;
+ m_stream->codecpar->frame_size = 1024;
+ m_stream->codecpar->format =
+ adjustSampleFormat(m_avCodec->sample_fmts, requestedAudioFormat.sampleFormat);
+
+ m_stream->time_base = AVRational{ 1, sampleRate };
+
+ qCDebug(qLcFFmpegAudioEncoder) << "set stream time_base" << m_stream->time_base.num << "/"
+ << m_stream->time_base.den;
+}
+
+void AudioEncoder::open()
+{
+ m_codecContext.reset(avcodec_alloc_context3(m_avCodec));
+
+ if (m_stream->time_base.num != 1 || m_stream->time_base.den != m_format.sampleRate()) {
+ qCDebug(qLcFFmpegAudioEncoder) << "Most likely, av_format_write_header changed time base from"
+ << 1 << "/" << m_format.sampleRate() << "to"
+ << m_stream->time_base;
+ }
+
+ m_codecContext->time_base = m_stream->time_base;
+
+ avcodec_parameters_to_context(m_codecContext.get(), m_stream->codecpar);
+
+ AVDictionaryHolder opts;
+ applyAudioEncoderOptions(m_settings, m_avCodec->name, m_codecContext.get(), opts);
+ applyExperimentalCodecOptions(m_avCodec, opts);
+
+ const int res = avcodec_open2(m_codecContext.get(), m_avCodec, opts);
+
+ qCDebug(qLcFFmpegAudioEncoder) << "audio codec opened" << res;
+ qCDebug(qLcFFmpegAudioEncoder) << "audio codec params: fmt=" << m_codecContext->sample_fmt
+ << "rate=" << m_codecContext->sample_rate;
+
+ updateResampler();
+}
+
+void AudioEncoder::addBuffer(const QAudioBuffer &buffer)
+{
+ if (!buffer.isValid()) {
+ setEndOfSourceStream();
+ return;
+ }
+
+ {
+ const std::chrono::microseconds bufferDuration(buffer.duration());
+ auto guard = lockLoopData();
+
+ resetEndOfSourceStream();
+
+ if (m_paused)
+ return;
+
+ // TODO: apply logic with canPushFrame
+
+ m_audioBufferQueue.push(buffer);
+ m_queueDuration += bufferDuration;
+ }
+
+ dataReady();
+}
+
+QAudioBuffer AudioEncoder::takeBuffer()
+{
+ auto locker = lockLoopData();
+ QAudioBuffer result = dequeueIfPossible(m_audioBufferQueue);
+ m_queueDuration -= std::chrono::microseconds(result.duration());
+ return result;
+}
+
+void AudioEncoder::init()
+{
+ open();
+
+ // TODO: try to address this dependency here.
+ if (auto input = qobject_cast<QFFmpegAudioInput *>(source()))
+ input->setFrameSize(m_codecContext->frame_size);
+
+ qCDebug(qLcFFmpegAudioEncoder) << "AudioEncoder::init started audio device thread.";
+}
+
+void AudioEncoder::cleanup()
+{
+ while (!m_audioBufferQueue.empty())
+ processOne();
+
+ if (m_avFrameSamplesOffset) {
+ // the size of the last frame can be less than m_codecContext->frame_size
+
+ retrievePackets();
+ sendPendingFrameToAVCodec();
+ }
+
+ while (avcodec_send_frame(m_codecContext.get(), nullptr) == AVERROR(EAGAIN))
+ retrievePackets();
+ retrievePackets();
+}
+
+bool AudioEncoder::hasData() const
+{
+ return !m_audioBufferQueue.empty();
+}
+
+void AudioEncoder::retrievePackets()
+{
+ while (1) {
+ AVPacketUPtr packet(av_packet_alloc());
+ int ret = avcodec_receive_packet(m_codecContext.get(), packet.get());
+ if (ret < 0) {
+ if (ret != AVERROR(EOF))
+ break;
+ if (ret != AVERROR(EAGAIN)) {
+ char errStr[1024];
+ av_strerror(ret, errStr, 1024);
+ qCDebug(qLcFFmpegAudioEncoder) << "receive packet" << ret << errStr;
+ }
+ break;
+ }
+
+ // qCDebug(qLcFFmpegEncoder) << "writing audio packet" << packet->size << packet->pts <<
+ // packet->dts;
+ packet->stream_index = m_stream->id;
+ m_recordingEngine.getMuxer()->addPacket(std::move(packet));
+ }
+}
+
+void AudioEncoder::processOne()
+{
+ QAudioBuffer buffer = takeBuffer();
+ Q_ASSERT(buffer.isValid());
+
+ // qCDebug(qLcFFmpegEncoder) << "new audio buffer" << buffer.byteCount() << buffer.format()
+ // << buffer.frameCount() << codec->frame_size;
+
+ if (buffer.format() != m_format) {
+ m_format = buffer.format();
+ updateResampler();
+ }
+
+ int samplesOffset = 0;
+ const int bufferSamplesCount = static_cast<int>(buffer.frameCount());
+
+ while (samplesOffset < bufferSamplesCount)
+ handleAudioData(buffer.constData<uint8_t>(), samplesOffset, bufferSamplesCount);
+
+ Q_ASSERT(samplesOffset == bufferSamplesCount);
+}
+
+bool AudioEncoder::checkIfCanPushFrame() const
+{
+ if (isRunning())
+ return m_audioBufferQueue.size() <= 1 || m_queueDuration < m_maxQueueDuration;
+ if (!isFinished())
+ return m_audioBufferQueue.empty();
+
+ return false;
+}
+
+void AudioEncoder::updateResampler()
+{
+ m_resampler.reset();
+
+ const AVAudioFormat requestedAudioFormat(m_format);
+ const AVAudioFormat codecAudioFormat(m_codecContext.get());
+
+ if (requestedAudioFormat != codecAudioFormat)
+ m_resampler = createResampleContext(requestedAudioFormat, codecAudioFormat);
+
+ qCDebug(qLcFFmpegAudioEncoder)
+ << "Resampler updated. Input format:" << m_format << "Resampler:" << m_resampler.get();
+}
+
+void AudioEncoder::ensurePendingFrame(int availableSamplesCount)
+{
+ Q_ASSERT(availableSamplesCount >= 0);
+
+ if (m_avFrame)
+ return;
+
+ m_avFrame = makeAVFrame();
+
+ m_avFrame->format = m_codecContext->sample_fmt;
+#if QT_FFMPEG_OLD_CHANNEL_LAYOUT
+ m_avFrame->channel_layout = m_codecContext->channel_layout;
+ m_avFrame->channels = m_codecContext->channels;
+#else
+ m_avFrame->ch_layout = m_codecContext->ch_layout;
+#endif
+ m_avFrame->sample_rate = m_codecContext->sample_rate;
+
+ const bool isFixedFrameSize = !(m_avCodec->capabilities & AV_CODEC_CAP_VARIABLE_FRAME_SIZE)
+ && m_codecContext->frame_size;
+ m_avFrame->nb_samples = isFixedFrameSize ? m_codecContext->frame_size : availableSamplesCount;
+ if (m_avFrame->nb_samples)
+ av_frame_get_buffer(m_avFrame.get(), 0);
+
+ const auto &timeBase = m_stream->time_base;
+ const auto pts = timeBase.den && timeBase.num
+ ? timeBase.den * m_samplesWritten / (m_codecContext->sample_rate * timeBase.num)
+ : m_samplesWritten;
+ setAVFrameTime(*m_avFrame, pts, timeBase);
+}
+
+void AudioEncoder::writeDataToPendingFrame(const uchar *data, int &samplesOffset, int samplesCount)
+{
+ Q_ASSERT(m_avFrame);
+ Q_ASSERT(m_avFrameSamplesOffset <= m_avFrame->nb_samples);
+
+ const int bytesPerSample = av_get_bytes_per_sample(m_codecContext->sample_fmt);
+ const bool isPlanar = av_sample_fmt_is_planar(m_codecContext->sample_fmt);
+
+#if QT_FFMPEG_OLD_CHANNEL_LAYOUT
+ const int channelsCount = m_codecContext->channels;
+#else
+ const int channelsCount = m_codecContext->ch_layout.nb_channels;
+#endif
+
+ const int audioDataOffset = isPlanar ? bytesPerSample * m_avFrameSamplesOffset
+ : bytesPerSample * m_avFrameSamplesOffset * channelsCount;
+
+ const int planesCount = isPlanar ? channelsCount : 1;
+ m_avFramePlanesData.resize(planesCount);
+ for (int plane = 0; plane < planesCount; ++plane)
+ m_avFramePlanesData[plane] = m_avFrame->extended_data[plane] + audioDataOffset;
+
+ const int samplesToRead =
+ std::min(m_avFrame->nb_samples - m_avFrameSamplesOffset, samplesCount - samplesOffset);
+
+ data += m_format.bytesForFrames(samplesOffset);
+
+ if (m_resampler) {
+ m_avFrameSamplesOffset += swr_convert(m_resampler.get(), m_avFramePlanesData.data(),
+ samplesToRead, &data, samplesToRead);
+ } else {
+ Q_ASSERT(planesCount == 1);
+ m_avFrameSamplesOffset += samplesToRead;
+ memcpy(m_avFramePlanesData[0], data, m_format.bytesForFrames(samplesToRead));
+ }
+
+ samplesOffset += samplesToRead;
+}
+
+void AudioEncoder::sendPendingFrameToAVCodec()
+{
+ Q_ASSERT(m_avFrame);
+ Q_ASSERT(m_avFrameSamplesOffset <= m_avFrame->nb_samples);
+
+ m_avFrame->nb_samples = m_avFrameSamplesOffset;
+
+ m_samplesWritten += m_avFrameSamplesOffset;
+
+ const qint64 time = m_format.durationForFrames(m_samplesWritten);
+ m_recordingEngine.newTimeStamp(time / 1000);
+
+ // qCDebug(qLcFFmpegEncoder) << "sending audio frame" << buffer.byteCount() << frame->pts <<
+ // ((double)buffer.frameCount()/frame->sample_rate);
+
+ int ret = avcodec_send_frame(m_codecContext.get(), m_avFrame.get());
+ if (ret < 0) {
+ char errStr[AV_ERROR_MAX_STRING_SIZE];
+ av_strerror(ret, errStr, AV_ERROR_MAX_STRING_SIZE);
+ qCDebug(qLcFFmpegAudioEncoder) << "error sending frame" << ret << errStr;
+ }
+
+ m_avFrame = nullptr;
+ m_avFrameSamplesOffset = 0;
+ std::fill(m_avFramePlanesData.begin(), m_avFramePlanesData.end(), nullptr);
+}
+
+void AudioEncoder::handleAudioData(const uchar *data, int &samplesOffset, int samplesCount)
+{
+ ensurePendingFrame(samplesCount - samplesOffset);
+
+ writeDataToPendingFrame(data, samplesOffset, samplesCount);
+
+ // The frame is not ready yet
+ if (m_avFrameSamplesOffset < m_avFrame->nb_samples)
+ return;
+
+ retrievePackets();
+
+ sendPendingFrameToAVCodec();
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder_p.h
new file mode 100644
index 000000000..4408ff54f
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoder_p.h
@@ -0,0 +1,77 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGAUDIOENCODER_P_H
+#define QFFMPEGAUDIOENCODER_P_H
+
+#include "qffmpeg_p.h"
+#include "qffmpegencoderthread_p.h"
+#include "private/qplatformmediarecorder_p.h"
+#include <qaudiobuffer.h>
+#include <queue>
+#include <chrono>
+
+QT_BEGIN_NAMESPACE
+
+class QMediaEncoderSettings;
+
+namespace QFFmpeg {
+
+class AudioEncoder : public EncoderThread
+{
+public:
+ AudioEncoder(RecordingEngine &recordingEngine, const QAudioFormat &sourceFormat,
+ const QMediaEncoderSettings &settings);
+
+ void addBuffer(const QAudioBuffer &buffer);
+
+protected:
+ bool checkIfCanPushFrame() const override;
+
+private:
+ void open();
+
+ QAudioBuffer takeBuffer();
+ void retrievePackets();
+ void updateResampler();
+
+ void init() override;
+ void cleanup() override;
+ bool hasData() const override;
+ void processOne() override;
+
+ void handleAudioData(const uchar *data, int &samplesOffset, int samplesCount);
+
+ void ensurePendingFrame(int availableSamplesCount);
+
+ void writeDataToPendingFrame(const uchar *data, int &samplesOffset, int samplesCount);
+
+ void sendPendingFrameToAVCodec();
+
+private:
+ std::queue<QAudioBuffer> m_audioBufferQueue;
+
+ // Arbitrarily chosen to limit audio queue duration
+ const std::chrono::microseconds m_maxQueueDuration = std::chrono::seconds(5);
+
+ std::chrono::microseconds m_queueDuration{ 0 };
+
+ AVStream *m_stream = nullptr;
+ AVCodecContextUPtr m_codecContext;
+ QAudioFormat m_format;
+
+ SwrContextUPtr m_resampler;
+ qint64 m_samplesWritten = 0;
+ const AVCodec *m_avCodec = nullptr;
+ QMediaEncoderSettings m_settings;
+
+ AVFrameUPtr m_avFrame;
+ int m_avFrameSamplesOffset = 0;
+ std::vector<uint8_t *> m_avFramePlanesData;
+};
+
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils.cpp
new file mode 100644
index 000000000..4d4dc69d2
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils.cpp
@@ -0,0 +1,97 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#include "qffmpegaudioencoderutils_p.h"
+#include "qalgorithms.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+AVSampleFormat adjustSampleFormat(const AVSampleFormat *supportedFormats, AVSampleFormat requested)
+{
+ auto calcScore = [requested](AVSampleFormat format) {
+ return format == requested ? BestAVScore
+ : format == av_get_planar_sample_fmt(requested) ? BestAVScore - 1
+ : 0;
+ };
+
+ const auto result = findBestAVValue(supportedFormats, calcScore).first;
+ return result == AV_SAMPLE_FMT_NONE ? requested : result;
+}
+
+int adjustSampleRate(const int *supportedRates, int requested)
+{
+ auto calcScore = [requested](int rate) {
+ return requested == rate ? BestAVScore
+ : requested <= rate ? rate - requested
+ : requested - rate - 1000000;
+ };
+
+ const auto result = findBestAVValue(supportedRates, calcScore).first;
+ return result == 0 ? requested : result;
+}
+
+static AVScore calculateScoreByChannelsCount(int supportedChannelsNumber,
+ int requestedChannelsNumber)
+{
+ if (supportedChannelsNumber >= requestedChannelsNumber)
+ return requestedChannelsNumber - supportedChannelsNumber;
+
+ return supportedChannelsNumber - requestedChannelsNumber - 10000;
+}
+
+static AVScore calculateScoreByChannelsMask(int supportedChannelsNumber, uint64_t supportedMask,
+ int requestedChannelsNumber, uint64_t requestedMask)
+{
+ if ((supportedMask & requestedMask) == requestedMask)
+ return BestAVScore - qPopulationCount(supportedMask & ~requestedMask);
+
+ return calculateScoreByChannelsCount(supportedChannelsNumber, requestedChannelsNumber);
+}
+
+#if QT_FFMPEG_OLD_CHANNEL_LAYOUT
+
+uint64_t adjustChannelLayout(const uint64_t *supportedMasks, uint64_t requested)
+{
+ auto calcScore = [requested](uint64_t mask) {
+ return calculateScoreByChannelsMask(qPopulationCount(mask), mask,
+ qPopulationCount(requested), requested);
+ };
+
+ const auto result = findBestAVValue(supportedMasks, calcScore).first;
+ return result == 0 ? requested : result;
+}
+
+#else
+
+AVChannelLayout adjustChannelLayout(const AVChannelLayout *supportedLayouts,
+ const AVChannelLayout &requested)
+{
+ auto calcScore = [&requested](const AVChannelLayout &layout) {
+ if (layout == requested)
+ return BestAVScore;
+
+ // The only realistic case for now:
+ // layout.order == requested.order == AV_CHANNEL_ORDER_NATIVE
+ // Let's consider other orders to make safe code
+
+ if (layout.order == AV_CHANNEL_ORDER_CUSTOM || requested.order == AV_CHANNEL_ORDER_CUSTOM)
+ return calculateScoreByChannelsCount(layout.nb_channels, requested.nb_channels) - 1000;
+
+ const auto offset = layout.order == requested.order ? 1 : 100;
+
+ return calculateScoreByChannelsMask(layout.nb_channels, layout.u.mask,
+ requested.nb_channels, requested.u.mask)
+ - offset;
+ };
+
+ const auto result = findBestAVValue(supportedLayouts, calcScore);
+ return result.second == NotSuitableAVScore ? requested : result.first;
+}
+
+#endif
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils_p.h
new file mode 100644
index 000000000..8a7c184ec
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegaudioencoderutils_p.h
@@ -0,0 +1,28 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#ifndef QFFMPEGAUDIOENCODERUTILS_P_H
+#define QFFMPEGAUDIOENCODERUTILS_P_H
+
+#include "qffmpeg_p.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+AVSampleFormat adjustSampleFormat(const AVSampleFormat *supportedFormats, AVSampleFormat requested);
+
+int adjustSampleRate(const int *supportedRates, int requested);
+
+#if QT_FFMPEG_OLD_CHANNEL_LAYOUT
+uint64_t adjustChannelLayout(const uint64_t *supportedLayouts, uint64_t requested);
+#else
+AVChannelLayout adjustChannelLayout(const AVChannelLayout *supportedLayouts,
+ const AVChannelLayout &requested);
+#endif
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif // QFFMPEGAUDIOENCODERUTILS_P_H
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions.cpp
new file mode 100644
index 000000000..bd6a8e09b
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions.cpp
@@ -0,0 +1,362 @@
+// Copyright (C) 2022 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#include "qffmpegencoderoptions_p.h"
+
+#if QT_CONFIG(vaapi)
+#include <va/va.h>
+#endif
+
+QT_BEGIN_NAMESPACE
+
+// unfortunately there is no common way to specify options for the encoders. The code here tries to map our settings sensibly
+// to options available in different encoders
+
+// For constant quality options, we're trying to map things to approx those bit rates for 1080p@30fps (in Mbps):
+// VeryLow Low Normal High VeryHigh
+// H264: 0.8M 1.5M 3.5M 6M 10M
+// H265: 0.5M 1.0M 2.5M 4M 7M
+
+[[maybe_unused]]
+static int bitrateForSettings(const QMediaEncoderSettings &settings, bool hdr = false)
+{
+ // calculate an acceptable bitrate depending on video codec, resolution, framerate and requested quality
+ // The calculations are rather heuristic here, trying to take into account how well codecs compress using
+ // the tables above.
+
+ // The table here is for 30FPS
+ const double bitsPerPixel[int(QMediaFormat::VideoCodec::LastVideoCodec)+1][QMediaRecorder::VeryHighQuality+1] = {
+ { 1.2, 2.25, 5, 9, 15 }, // MPEG1,
+ { 0.8, 1.5, 3.5, 6, 10 }, // MPEG2
+ { 0.4, 0.75, 1.75, 3, 5 }, // MPEG4
+ { 0.4, 0.75, 1.75, 3, 5 }, // H264
+ { 0.3, 0.5, 0.2, 2, 3 }, // H265
+ { 0.4, 0.75, 1.75, 3, 5 }, // VP8
+ { 0.3, 0.5, 0.2, 2, 3 }, // VP9
+ { 0.2, 0.4, 0.9, 1.5, 2.5 }, // AV1
+ { 0.4, 0.75, 1.75, 3, 5 }, // Theora
+ { 0.8, 1.5, 3.5, 6, 10 }, // WMV
+ { 16, 24, 32, 40, 48 }, // MotionJPEG
+ };
+
+ QSize s = settings.videoResolution();
+ double bitrate = bitsPerPixel[int(settings.videoCodec())][settings.quality()]*s.width()*s.height();
+
+ if (settings.videoCodec() != QMediaFormat::VideoCodec::MotionJPEG) {
+ // We assume that doubling the framerate requires 1.5 times the amount of data (not twice, as intraframe
+ // differences will be smaller). 4 times the frame rate uses thus 2.25 times the data, etc.
+ float rateMultiplier = log2(settings.videoFrameRate()/30.);
+ bitrate *= pow(1.5, rateMultiplier);
+ } else {
+ // MotionJPEG doesn't optimize between frames, so we have a linear dependency on framerate
+ bitrate *= settings.videoFrameRate()/30.;
+ }
+
+ // HDR requires 10bits per pixel instead of 8, so apply a factor of 1.25.
+ if (hdr)
+ bitrate *= 1.25;
+ return bitrate;
+}
+
+static void apply_openh264(const QMediaEncoderSettings &settings, AVCodecContext *codec,
+ AVDictionary **opts)
+{
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding
+ || settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ av_dict_set(opts, "rc_mode", "bitrate", 0);
+ } else {
+ av_dict_set(opts, "rc_mode", "quality", 0);
+ static const int q[] = { 51, 48, 38, 25, 5 };
+ codec->qmax = codec->qmin = q[settings.quality()];
+ }
+}
+
+static void apply_x264(const QMediaEncoderSettings &settings, AVCodecContext *codec, AVDictionary **opts)
+{
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding || settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ } else {
+ const char *scales[] = {
+ "29", "26", "23", "21", "19"
+ };
+ av_dict_set(opts, "crf", scales[settings.quality()], 0);
+ }
+}
+
+static void apply_x265(const QMediaEncoderSettings &settings, AVCodecContext *codec, AVDictionary **opts)
+{
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding || settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ } else {
+ const char *scales[QMediaRecorder::VeryHighQuality+1] = {
+ "40", "34", "28", "26", "24",
+ };
+ av_dict_set(opts, "crf", scales[settings.quality()], 0);
+ }
+}
+
+static void apply_libvpx(const QMediaEncoderSettings &settings, AVCodecContext *codec, AVDictionary **opts)
+{
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding || settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ } else {
+ const char *scales[QMediaRecorder::VeryHighQuality+1] = {
+ "38", "34", "31", "28", "25",
+ };
+ av_dict_set(opts, "crf", scales[settings.quality()], 0);
+ av_dict_set(opts, "b", 0, 0);
+ }
+ av_dict_set(opts, "row-mt", "1", 0); // better multithreading
+}
+
+#ifdef Q_OS_DARWIN
+static void apply_videotoolbox(const QMediaEncoderSettings &settings, AVCodecContext *codec, AVDictionary **opts)
+{
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding || settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ } else {
+ // only use quality on macOS/ARM, as FFmpeg doesn't support it on the other platforms and would throw
+ // an error when initializing the codec
+#if defined(Q_OS_MACOS) && defined(Q_PROCESSOR_ARM_64)
+ // Videotoolbox describes quality as a number from 0 to 1, with low == 0.25, normal 0.5, high 0.75 and lossless = 1
+ // ffmpeg uses a different scale going from 0 to 11800.
+ // Values here are adjusted to agree approximately with the target bit rates listed above
+ const int scales[] = {
+ 3000, 4800, 5900, 6900, 7700,
+ };
+ codec->global_quality = scales[settings.quality()];
+ codec->flags |= AV_CODEC_FLAG_QSCALE;
+#else
+ codec->bit_rate = bitrateForSettings(settings);
+#endif
+ }
+
+ // Videotooldox hw acceleration fails of some hardwares,
+ // allow_sw makes sw encoding available if hw encoding failed.
+ // Under the hood, ffmpeg sets
+ // kVTVideoEncoderSpecification_EnableHardwareAcceleratedVideoEncoder instead of
+ // kVTVideoEncoderSpecification_RequireHardwareAcceleratedVideoEncoder
+ av_dict_set(opts, "allow_sw", "1", 0);
+}
+#endif
+
+#if QT_CONFIG(vaapi)
+static void apply_vaapi(const QMediaEncoderSettings &settings, AVCodecContext *codec, AVDictionary **/*opts*/)
+{
+ // See also vaapi_encode_init_rate_control() in libavcodec
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ codec->rc_max_rate = settings.videoBitRate();
+ } else if (settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ } else {
+ const int *quality = nullptr;
+ // unfortunately, all VA codecs use different quality scales :/
+ switch (settings.videoCodec()) {
+ case QMediaFormat::VideoCodec::MPEG2: {
+ static const int q[] = { 20, 15, 10, 8, 6 };
+ quality = q;
+ break;
+ }
+ case QMediaFormat::VideoCodec::MPEG4:
+ case QMediaFormat::VideoCodec::H264: {
+ static const int q[] = { 29, 26, 23, 21, 19 };
+ quality = q;
+ break;
+ }
+ case QMediaFormat::VideoCodec::H265: {
+ static const int q[] = { 40, 34, 28, 26, 24 };
+ quality = q;
+ break;
+ }
+ case QMediaFormat::VideoCodec::VP8: {
+ static const int q[] = { 56, 48, 40, 34, 28 };
+ quality = q;
+ break;
+ }
+ case QMediaFormat::VideoCodec::VP9: {
+ static const int q[] = { 124, 112, 100, 88, 76 };
+ quality = q;
+ break;
+ }
+ case QMediaFormat::VideoCodec::MotionJPEG: {
+ static const int q[] = { 40, 60, 80, 90, 95 };
+ quality = q;
+ break;
+ }
+ case QMediaFormat::VideoCodec::AV1:
+ case QMediaFormat::VideoCodec::Theora:
+ case QMediaFormat::VideoCodec::WMV:
+ default:
+ break;
+ }
+
+ if (quality)
+ codec->global_quality = quality[settings.quality()];
+ }
+}
+#endif
+
+static void apply_nvenc(const QMediaEncoderSettings &settings, AVCodecContext *codec,
+ AVDictionary **opts)
+{
+ switch (settings.encodingMode()) {
+ case QMediaRecorder::EncodingMode::AverageBitRateEncoding:
+ av_dict_set(opts, "vbr", "1", 0);
+ codec->bit_rate = settings.videoBitRate();
+ break;
+ case QMediaRecorder::EncodingMode::ConstantBitRateEncoding:
+ av_dict_set(opts, "cbr", "1", 0);
+ codec->bit_rate = settings.videoBitRate();
+ codec->rc_max_rate = codec->rc_min_rate = codec->bit_rate;
+ break;
+ case QMediaRecorder::EncodingMode::ConstantQualityEncoding: {
+ static const char *q[] = { "51", "48", "35", "15", "1" };
+ av_dict_set(opts, "cq", q[settings.quality()], 0);
+ } break;
+ default:
+ break;
+ }
+}
+
+#ifdef Q_OS_WINDOWS
+static void apply_mf(const QMediaEncoderSettings &settings, AVCodecContext *codec, AVDictionary **opts)
+{
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding || settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding) {
+ codec->bit_rate = settings.videoBitRate();
+ av_dict_set(opts, "rate_control", "cbr", 0);
+ } else {
+ av_dict_set(opts, "rate_control", "quality", 0);
+ const char *scales[] = {
+ "25", "50", "75", "90", "100"
+ };
+ av_dict_set(opts, "quality", scales[settings.quality()], 0);
+ }
+}
+#endif
+
+#ifdef Q_OS_ANDROID
+static void apply_mediacodec(const QMediaEncoderSettings &settings, AVCodecContext *codec,
+ AVDictionary **opts)
+{
+ codec->bit_rate = settings.videoBitRate();
+
+ const int quality[] = { 25, 50, 75, 90, 100 };
+ codec->global_quality = quality[settings.quality()];
+
+ switch (settings.encodingMode()) {
+ case QMediaRecorder::EncodingMode::AverageBitRateEncoding:
+ av_dict_set(opts, "bitrate_mode", "vbr", 1);
+ break;
+ case QMediaRecorder::EncodingMode::ConstantBitRateEncoding:
+ av_dict_set(opts, "bitrate_mode", "cbr", 1);
+ break;
+ case QMediaRecorder::EncodingMode::ConstantQualityEncoding:
+ // av_dict_set(opts, "bitrate_mode", "cq", 1);
+ av_dict_set(opts, "bitrate_mode", "cbr", 1);
+ break;
+ default:
+ break;
+ }
+
+ switch (settings.videoCodec()) {
+ case QMediaFormat::VideoCodec::H264: {
+ const char *levels[] = { "2.2", "3.2", "4.2", "5.2", "6.2" };
+ av_dict_set(opts, "level", levels[settings.quality()], 1);
+ codec->profile = FF_PROFILE_H264_HIGH;
+ break;
+ }
+ case QMediaFormat::VideoCodec::H265: {
+ const char *levels[] = { "h2.1", "h3.1", "h4.1", "h5.1", "h6.1" };
+ av_dict_set(opts, "level", levels[settings.quality()], 1);
+ codec->profile = FF_PROFILE_HEVC_MAIN;
+ break;
+ }
+ default:
+ break;
+ }
+}
+#endif
+
+namespace QFFmpeg {
+
+using ApplyOptions = void (*)(const QMediaEncoderSettings &settings, AVCodecContext *codec, AVDictionary **opts);
+
+const struct {
+ const char *name;
+ ApplyOptions apply;
+} videoCodecOptionTable[] = { { "libx264", apply_x264 },
+ { "libx265xx", apply_x265 },
+ { "libvpx", apply_libvpx },
+ { "libvpx_vp9", apply_libvpx },
+ { "libopenh264", apply_openh264 },
+ { "h264_nvenc", apply_nvenc },
+ { "hevc_nvenc", apply_nvenc },
+ { "av1_nvenc", apply_nvenc },
+#ifdef Q_OS_DARWIN
+ { "h264_videotoolbox", apply_videotoolbox },
+ { "hevc_videotoolbox", apply_videotoolbox },
+ { "prores_videotoolbox", apply_videotoolbox },
+ { "vp9_videotoolbox", apply_videotoolbox },
+#endif
+#if QT_CONFIG(vaapi)
+ { "mpeg2_vaapi", apply_vaapi },
+ { "mjpeg_vaapi", apply_vaapi },
+ { "h264_vaapi", apply_vaapi },
+ { "hevc_vaapi", apply_vaapi },
+ { "vp8_vaapi", apply_vaapi },
+ { "vp9_vaapi", apply_vaapi },
+#endif
+#ifdef Q_OS_WINDOWS
+ { "hevc_mf", apply_mf },
+ { "h264_mf", apply_mf },
+#endif
+#ifdef Q_OS_ANDROID
+ { "hevc_mediacodec", apply_mediacodec },
+ { "h264_mediacodec", apply_mediacodec },
+#endif
+ { nullptr, nullptr } };
+
+const struct {
+ const char *name;
+ ApplyOptions apply;
+} audioCodecOptionTable[] = {
+ { nullptr, nullptr }
+};
+
+void applyVideoEncoderOptions(const QMediaEncoderSettings &settings, const QByteArray &codecName, AVCodecContext *codec, AVDictionary **opts)
+{
+ av_dict_set(opts, "threads", "auto", 0); // we always want automatic threading
+
+ auto *table = videoCodecOptionTable;
+ while (table->name) {
+ if (codecName == table->name) {
+ table->apply(settings, codec, opts);
+ return;
+ }
+
+ ++table;
+ }
+}
+
+void applyAudioEncoderOptions(const QMediaEncoderSettings &settings, const QByteArray &codecName, AVCodecContext *codec, AVDictionary **opts)
+{
+ codec->thread_count = -1; // we always want automatic threading
+ if (settings.encodingMode() == QMediaRecorder::ConstantBitRateEncoding || settings.encodingMode() == QMediaRecorder::AverageBitRateEncoding)
+ codec->bit_rate = settings.audioBitRate();
+
+ auto *table = audioCodecOptionTable;
+ while (table->name) {
+ if (codecName == table->name) {
+ table->apply(settings, codec, opts);
+ return;
+ }
+
+ ++table;
+ }
+
+}
+
+}
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions_p.h
new file mode 100644
index 000000000..005ad7652
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderoptions_p.h
@@ -0,0 +1,32 @@
+// Copyright (C) 2022 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGENCODEROPTIONS_P_H
+#define QFFMPEGENCODEROPTIONS_P_H
+
+//
+// W A R N I N G
+// -------------
+//
+// This file is not part of the Qt API. It exists purely as an
+// implementation detail. This header file may change from version to
+// version without notice, or even be removed.
+//
+// We mean it.
+//
+
+#include "qffmpeghwaccel_p.h"
+#include "qvideoframeformat.h"
+#include "private/qplatformmediarecorder_p.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+void applyVideoEncoderOptions(const QMediaEncoderSettings &settings, const QByteArray &codecName, AVCodecContext *codec, AVDictionary **opts);
+void applyAudioEncoderOptions(const QMediaEncoderSettings &settings, const QByteArray &codecName, AVCodecContext *codec, AVDictionary **opts);
+
+}
+
+QT_END_NAMESPACE
+
+#endif
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread.cpp
new file mode 100644
index 000000000..61fe954c8
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread.cpp
@@ -0,0 +1,40 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#include "qffmpegencoderthread_p.h"
+#include "qmetaobject.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+EncoderThread::EncoderThread(RecordingEngine &recordingEngine) : m_recordingEngine(recordingEngine)
+{
+}
+
+void EncoderThread::setPaused(bool paused)
+{
+ auto guard = lockLoopData();
+ m_paused = paused;
+}
+
+void EncoderThread::setAutoStop(bool autoStop)
+{
+ auto guard = lockLoopData();
+ m_autoStop = autoStop;
+}
+
+void EncoderThread::setEndOfSourceStream()
+{
+ {
+ auto guard = lockLoopData();
+ m_endOfSourceStream = true;
+ }
+
+ emit endOfSourceStream();
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#include "moc_qffmpegencoderthread_p.cpp"
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread_p.h
new file mode 100644
index 000000000..f1f6b610a
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencoderthread_p.h
@@ -0,0 +1,72 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGENCODERTHREAD_P_H
+#define QFFMPEGENCODERTHREAD_P_H
+
+#include "qffmpegthread_p.h"
+#include "qpointer.h"
+
+#include "private/qmediainputencoderinterface_p.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+class RecordingEngine;
+
+class EncoderThread : public ConsumerThread, public QMediaInputEncoderInterface
+{
+ Q_OBJECT
+public:
+ EncoderThread(RecordingEngine &recordingEngine);
+
+ void setPaused(bool paused);
+
+ void setAutoStop(bool autoStop);
+
+ void setSource(QObject *source) { m_source = source; }
+
+ QObject *source() const { return m_source; }
+
+ bool canPushFrame() const override { return m_canPushFrame.load(std::memory_order_relaxed); }
+
+ void setEndOfSourceStream();
+
+ bool isEndOfSourceStream() const { return m_endOfSourceStream; }
+
+protected:
+ void updateCanPushFrame();
+
+ virtual bool checkIfCanPushFrame() const = 0;
+
+ void resetEndOfSourceStream() { m_endOfSourceStream = false; }
+
+ auto lockLoopData()
+ {
+ return QScopeGuard([this, locker = ConsumerThread::lockLoopData()]() mutable {
+ const bool autoStopActivated = m_endOfSourceStream && m_autoStop;
+ const bool canPush = !autoStopActivated && !m_paused && checkIfCanPushFrame();
+ locker.unlock();
+ if (m_canPushFrame.exchange(canPush, std::memory_order_relaxed) != canPush)
+ emit canPushFrameChanged();
+ });
+ }
+
+Q_SIGNALS:
+ void canPushFrameChanged();
+ void endOfSourceStream();
+
+protected:
+ bool m_paused = false;
+ bool m_endOfSourceStream = false;
+ bool m_autoStop = false;
+ std::atomic_bool m_canPushFrame = false;
+ RecordingEngine &m_recordingEngine;
+ QPointer<QObject> m_source;
+};
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer.cpp
new file mode 100644
index 000000000..4f8c21bd5
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer.cpp
@@ -0,0 +1,165 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#include "qffmpegencodinginitializer_p.h"
+#include "qffmpegrecordingengineutils_p.h"
+#include "qffmpegrecordingengine_p.h"
+#include "qffmpegaudioinput_p.h"
+#include "qvideoframe.h"
+
+#include "private/qplatformvideoframeinput_p.h"
+#include "private/qplatformaudiobufferinput_p.h"
+#include "private/qplatformaudiobufferinput_p.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+EncodingInitializer::EncodingInitializer(RecordingEngine &engine) : m_recordingEngine(engine) { }
+
+EncodingInitializer::~EncodingInitializer()
+{
+ for (QObject *source : m_pendingSources)
+ setEncoderInterface(source, nullptr);
+}
+
+void EncodingInitializer::start(const std::vector<QPlatformAudioBufferInputBase *> &audioSources,
+ const std::vector<QPlatformVideoSource *> &videoSources)
+{
+ for (auto source : audioSources) {
+ if (auto audioInput = qobject_cast<QFFmpegAudioInput *>(source))
+ m_recordingEngine.addAudioInput(audioInput);
+ else if (auto audioBufferInput = qobject_cast<QPlatformAudioBufferInput *>(source))
+ addAudioBufferInput(audioBufferInput);
+ else
+ Q_ASSERT(!"Undefined source type");
+ }
+
+ for (auto source : videoSources)
+ addVideoSource(source);
+
+ tryStartRecordingEngine();
+}
+
+void EncodingInitializer::addAudioBufferInput(QPlatformAudioBufferInput *input)
+{
+ Q_ASSERT(input);
+
+ if (input->audioFormat().isValid())
+ m_recordingEngine.addAudioBufferInput(input, {});
+ else
+ addPendingAudioBufferInput(input);
+}
+
+void EncodingInitializer::addPendingAudioBufferInput(QPlatformAudioBufferInput *input)
+{
+ addPendingSource(input);
+
+ connect(input, &QPlatformAudioBufferInput::destroyed, this, [this, input]() {
+ erasePendingSource(input, QStringLiteral("Audio source deleted"), true);
+ });
+
+ connect(input, &QPlatformAudioBufferInput::newAudioBuffer, this,
+ [this, input](const QAudioBuffer &buffer) {
+ if (buffer.isValid())
+ erasePendingSource(
+ input, [&]() { m_recordingEngine.addAudioBufferInput(input, buffer); });
+ else
+ erasePendingSource(input,
+ QStringLiteral("Audio source has sent the end frame"));
+ });
+}
+
+void EncodingInitializer::addVideoSource(QPlatformVideoSource *source)
+{
+ Q_ASSERT(source);
+ Q_ASSERT(source->isActive());
+
+ if (source->frameFormat().isValid())
+ m_recordingEngine.addVideoSource(source, {});
+ else if (source->hasError())
+ emitStreamInitializationError(QStringLiteral("Video source error: ")
+ + source->errorString());
+ else
+ addPendingVideoSource(source);
+}
+
+void EncodingInitializer::addPendingVideoSource(QPlatformVideoSource *source)
+{
+ addPendingSource(source);
+
+ connect(source, &QPlatformVideoSource::errorChanged, this, [this, source]() {
+ if (source->hasError())
+ erasePendingSource(source,
+ QStringLiteral("Videio source error: ") + source->errorString());
+ });
+
+ connect(source, &QPlatformVideoSource::destroyed, this, [this, source]() {
+ erasePendingSource(source, QStringLiteral("Source deleted"), true);
+ });
+
+ connect(source, &QPlatformVideoSource::activeChanged, this, [this, source]() {
+ if (!source->isActive())
+ erasePendingSource(source, QStringLiteral("Video source deactivated"));
+ });
+
+ connect(source, &QPlatformVideoSource::newVideoFrame, this,
+ [this, source](const QVideoFrame &frame) {
+ if (frame.isValid())
+ erasePendingSource(source,
+ [&]() { m_recordingEngine.addVideoSource(source, frame); });
+ else
+ erasePendingSource(source,
+ QStringLiteral("Video source has sent the end frame"));
+ });
+}
+
+void EncodingInitializer::tryStartRecordingEngine()
+{
+ if (m_pendingSources.empty())
+ m_recordingEngine.start();
+}
+
+void EncodingInitializer::emitStreamInitializationError(QString error)
+{
+ emit m_recordingEngine.streamInitializationError(
+ QMediaRecorder::ResourceError,
+ QStringLiteral("Video steam initialization error. ") + error);
+}
+
+void EncodingInitializer::addPendingSource(QObject *source)
+{
+ Q_ASSERT(m_pendingSources.count(source) == 0);
+
+ setEncoderInterface(source, this);
+ m_pendingSources.emplace(source);
+}
+
+template <typename F>
+void EncodingInitializer::erasePendingSource(QObject *source, F &&functionOrError, bool destroyed)
+{
+ const auto erasedCount = m_pendingSources.erase(source);
+ if (erasedCount == 0)
+ return; // got a queued event, just ignore it.
+
+ if (!destroyed) {
+ setEncoderInterface(source, nullptr);
+ disconnect(source, nullptr, this, nullptr);
+ }
+
+ if constexpr (std::is_invocable_v<F>)
+ functionOrError();
+ else
+ emitStreamInitializationError(functionOrError);
+
+ tryStartRecordingEngine();
+}
+
+bool EncodingInitializer::canPushFrame() const
+{
+ return true;
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer_p.h
new file mode 100644
index 000000000..e3bcb3428
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegencodinginitializer_p.h
@@ -0,0 +1,77 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#ifndef QENCODINGINITIALIZER_P_H
+#define QENCODINGINITIALIZER_P_H
+
+#include "qobject.h"
+#include "private/qmediainputencoderinterface_p.h"
+#include <unordered_set>
+#include <vector>
+
+//
+// W A R N I N G
+// -------------
+//
+// This file is not part of the Qt API. It exists purely as an
+// implementation detail. This header file may change from version to
+// version without notice, or even be removed.
+//
+// We mean it.
+//
+
+QT_BEGIN_NAMESPACE
+
+class QFFmpegAudioInput;
+class QPlatformVideoSource;
+class QPlatformAudioBufferInput;
+class QPlatformAudioBufferInputBase;
+class QMediaInputEncoderInterface;
+
+namespace QFFmpeg {
+
+class RecordingEngine;
+
+// Initializes RecordingEngine with audio and video sources, potentially lazily
+// upon first frame arrival if video frame format is not pre-determined.
+class EncodingInitializer : public QObject, private QMediaInputEncoderInterface
+{
+public:
+ EncodingInitializer(RecordingEngine &engine);
+
+ ~EncodingInitializer() override;
+
+ void start(const std::vector<QPlatformAudioBufferInputBase *> &audioSources,
+ const std::vector<QPlatformVideoSource *> &videoSources);
+
+private:
+ void addAudioBufferInput(QPlatformAudioBufferInput *input);
+
+ void addPendingAudioBufferInput(QPlatformAudioBufferInput *input);
+
+ void addVideoSource(QPlatformVideoSource *source);
+
+ void addPendingVideoSource(QPlatformVideoSource *source);
+
+ void addPendingSource(QObject *source);
+
+ void tryStartRecordingEngine();
+
+private:
+ void emitStreamInitializationError(QString error);
+
+ template <typename F>
+ void erasePendingSource(QObject *source, F &&functionOrError, bool destroyed = false);
+
+ bool canPushFrame() const override;
+
+private:
+ RecordingEngine &m_recordingEngine;
+ std::unordered_set<QObject *> m_pendingSources;
+};
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif // QENCODINGINITIALIZER_P_H
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer.cpp
new file mode 100644
index 000000000..dbb96d00c
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer.cpp
@@ -0,0 +1,64 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#include "qffmpegmuxer_p.h"
+#include "qffmpegrecordingengine_p.h"
+#include "qffmpegrecordingengineutils_p.h"
+#include <QtCore/qloggingcategory.h>
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+Q_STATIC_LOGGING_CATEGORY(qLcFFmpegMuxer, "qt.multimedia.ffmpeg.muxer");
+
+Muxer::Muxer(RecordingEngine *encoder) : m_encoder(encoder)
+{
+ setObjectName(QLatin1String("Muxer"));
+}
+
+void Muxer::addPacket(AVPacketUPtr packet)
+{
+ {
+ QMutexLocker locker = lockLoopData();
+ m_packetQueue.push(std::move(packet));
+ }
+
+ // qCDebug(qLcFFmpegEncoder) << "Muxer::addPacket" << packet->pts << packet->stream_index;
+ dataReady();
+}
+
+AVPacketUPtr Muxer::takePacket()
+{
+ QMutexLocker locker = lockLoopData();
+ return dequeueIfPossible(m_packetQueue);
+}
+
+void Muxer::init()
+{
+ qCDebug(qLcFFmpegMuxer) << "Muxer::init started thread.";
+}
+
+void Muxer::cleanup()
+{
+ while (!m_packetQueue.empty())
+ processOne();
+}
+
+bool QFFmpeg::Muxer::hasData() const
+{
+ return !m_packetQueue.empty();
+}
+
+void Muxer::processOne()
+{
+ auto packet = takePacket();
+ // qCDebug(qLcFFmpegEncoder) << "writing packet to file" << packet->pts << packet->duration <<
+ // packet->stream_index;
+
+ // the function takes ownership for the packet
+ av_interleaved_write_frame(m_encoder->avFormatContext(), packet.release());
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer_p.h
new file mode 100644
index 000000000..4f8f4d27a
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegmuxer_p.h
@@ -0,0 +1,41 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGMUXER_P_H
+#define QFFMPEGMUXER_P_H
+
+#include "qffmpegthread_p.h"
+#include "qffmpeg_p.h"
+#include <queue>
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+class RecordingEngine;
+
+class Muxer : public ConsumerThread
+{
+public:
+ Muxer(RecordingEngine *encoder);
+
+ void addPacket(AVPacketUPtr packet);
+
+private:
+ AVPacketUPtr takePacket();
+
+ void init() override;
+ void cleanup() override;
+ bool hasData() const override;
+ void processOne() override;
+
+private:
+ std::queue<AVPacketUPtr> m_packetQueue;
+
+ RecordingEngine *m_encoder;
+};
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine.cpp
new file mode 100644
index 000000000..469cd1c48
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine.cpp
@@ -0,0 +1,278 @@
+// Copyright (C) 2021 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#include "qffmpegrecordingengine_p.h"
+#include "qffmpegencodinginitializer_p.h"
+#include "qffmpegaudioencoder_p.h"
+#include "qffmpegaudioinput_p.h"
+#include "qffmpegrecordingengineutils_p.h"
+
+#include "private/qmultimediautils_p.h"
+#include "private/qplatformaudiobufferinput_p.h"
+#include "private/qplatformvideosource_p.h"
+#include "private/qplatformvideoframeinput_p.h"
+
+#include "qdebug.h"
+#include "qffmpegvideoencoder_p.h"
+#include "qffmpegmediametadata_p.h"
+#include "qffmpegmuxer_p.h"
+#include "qloggingcategory.h"
+
+QT_BEGIN_NAMESPACE
+
+Q_STATIC_LOGGING_CATEGORY(qLcFFmpegEncoder, "qt.multimedia.ffmpeg.encoder");
+
+namespace QFFmpeg
+{
+
+RecordingEngine::RecordingEngine(const QMediaEncoderSettings &settings,
+ std::unique_ptr<EncodingFormatContext> context)
+ : m_settings(settings), m_formatContext(std::move(context)), m_muxer(new Muxer(this))
+{
+ Q_ASSERT(m_formatContext);
+ Q_ASSERT(m_formatContext->isAVIOOpen());
+}
+
+RecordingEngine::~RecordingEngine()
+{
+}
+
+void RecordingEngine::addAudioInput(QFFmpegAudioInput *input)
+{
+ Q_ASSERT(input);
+
+ if (input->device.isNull()) {
+ emit streamInitializationError(QMediaRecorder::ResourceError,
+ QLatin1StringView("Audio device is null"));
+ return;
+ }
+
+ const QAudioFormat format = input->device.preferredFormat();
+
+ if (!format.isValid()) {
+ emit streamInitializationError(
+ QMediaRecorder::FormatError,
+ QLatin1StringView("Audio device has invalid preferred format"));
+ return;
+ }
+
+ AudioEncoder *audioEncoder = createAudioEncoder(format);
+ connectEncoderToSource(audioEncoder, input);
+
+ input->setRunning(true);
+}
+
+void RecordingEngine::addAudioBufferInput(QPlatformAudioBufferInput *input,
+ const QAudioBuffer &firstBuffer)
+{
+ Q_ASSERT(input);
+ const QAudioFormat format = firstBuffer.isValid() ? firstBuffer.format() : input->audioFormat();
+
+ AudioEncoder *audioEncoder = createAudioEncoder(format);
+
+ // set the buffer before connecting to avoid potential races
+ if (firstBuffer.isValid())
+ audioEncoder->addBuffer(firstBuffer);
+
+ connectEncoderToSource(audioEncoder, input);
+}
+
+AudioEncoder *RecordingEngine::createAudioEncoder(const QAudioFormat &format)
+{
+ Q_ASSERT(format.isValid());
+
+ auto audioEncoder = new AudioEncoder(*this, format, m_settings);
+ m_audioEncoders.push_back(audioEncoder);
+ connect(audioEncoder, &EncoderThread::endOfSourceStream, this,
+ &RecordingEngine::handleSourceEndOfStream);
+ if (m_autoStop)
+ audioEncoder->setAutoStop(true);
+
+ return audioEncoder;
+}
+
+void RecordingEngine::addVideoSource(QPlatformVideoSource *source, const QVideoFrame &firstFrame)
+{
+ QVideoFrameFormat frameFormat =
+ firstFrame.isValid() ? firstFrame.surfaceFormat() : source->frameFormat();
+
+ Q_ASSERT(frameFormat.isValid());
+
+ if (firstFrame.isValid() && frameFormat.streamFrameRate() <= 0.f) {
+ const qint64 startTime = firstFrame.startTime();
+ const qint64 endTime = firstFrame.endTime();
+ if (startTime != -1 && endTime > startTime)
+ frameFormat.setStreamFrameRate(static_cast<qreal>(VideoFrameTimeBase)
+ / (endTime - startTime));
+ }
+
+ std::optional<AVPixelFormat> hwPixelFormat = source->ffmpegHWPixelFormat()
+ ? AVPixelFormat(*source->ffmpegHWPixelFormat())
+ : std::optional<AVPixelFormat>{};
+
+ qCDebug(qLcFFmpegEncoder) << "adding video source" << source->metaObject()->className() << ":"
+ << "pixelFormat=" << frameFormat.pixelFormat()
+ << "frameSize=" << frameFormat.frameSize()
+ << "frameRate=" << frameFormat.streamFrameRate()
+ << "ffmpegHWPixelFormat=" << (hwPixelFormat ? *hwPixelFormat : AV_PIX_FMT_NONE);
+
+ auto veUPtr = std::make_unique<VideoEncoder>(*this, m_settings, frameFormat, hwPixelFormat);
+ if (!veUPtr->isValid()) {
+ emit streamInitializationError(QMediaRecorder::FormatError,
+ QLatin1StringView("Cannot initialize encoder"));
+ return;
+ }
+
+ auto videoEncoder = veUPtr.release();
+ m_videoEncoders.append(videoEncoder);
+ if (m_autoStop)
+ videoEncoder->setAutoStop(true);
+
+ connect(videoEncoder, &EncoderThread::endOfSourceStream, this,
+ &RecordingEngine::handleSourceEndOfStream);
+
+ // set the frame before connecting to avoid potential races
+ if (firstFrame.isValid())
+ videoEncoder->addFrame(firstFrame);
+
+ connectEncoderToSource(videoEncoder, source);
+}
+
+void RecordingEngine::start()
+{
+ Q_ASSERT(m_initializer);
+ m_initializer.reset();
+
+ if (m_audioEncoders.empty() && m_videoEncoders.empty()) {
+ emit sessionError(QMediaRecorder::ResourceError,
+ QLatin1StringView("No valid stream found for encoding"));
+ return;
+ }
+
+ qCDebug(qLcFFmpegEncoder) << "RecordingEngine::start!";
+
+ avFormatContext()->metadata = QFFmpegMetaData::toAVMetaData(m_metaData);
+
+ Q_ASSERT(!m_isHeaderWritten);
+
+ int res = avformat_write_header(avFormatContext(), nullptr);
+ if (res < 0) {
+ qWarning() << "could not write header, error:" << res << err2str(res);
+ emit sessionError(QMediaRecorder::ResourceError,
+ QLatin1StringView("Cannot start writing the stream"));
+ return;
+ }
+
+ m_isHeaderWritten = true;
+
+ qCDebug(qLcFFmpegEncoder) << "stream header is successfully written";
+
+ m_muxer->start();
+
+ forEachEncoder([](QThread *thread) { thread->start(); });
+}
+
+void RecordingEngine::initialize(const std::vector<QPlatformAudioBufferInputBase *> &audioSources,
+ const std::vector<QPlatformVideoSource *> &videoSources)
+{
+ qCDebug(qLcFFmpegEncoder) << ">>>>>>>>>>>>>>> initialize";
+
+ m_initializer = std::make_unique<EncodingInitializer>(*this);
+ m_initializer->start(audioSources, videoSources);
+}
+
+RecordingEngine::EncodingFinalizer::EncodingFinalizer(RecordingEngine &recordingEngine)
+ : m_recordingEngine(recordingEngine)
+{
+ connect(this, &QThread::finished, this, &QObject::deleteLater);
+}
+
+void RecordingEngine::EncodingFinalizer::run()
+{
+ m_recordingEngine.forEachEncoder(&EncoderThread::stopAndDelete);
+ m_recordingEngine.m_muxer->stopAndDelete();
+
+ if (m_recordingEngine.m_isHeaderWritten) {
+ const int res = av_write_trailer(m_recordingEngine.avFormatContext());
+ if (res < 0) {
+ const auto errorDescription = err2str(res);
+ qCWarning(qLcFFmpegEncoder) << "could not write trailer" << res << errorDescription;
+ emit m_recordingEngine.sessionError(QMediaRecorder::FormatError,
+ QLatin1String("Cannot write trailer: ")
+ + errorDescription);
+ }
+ }
+ // else ffmpeg might crash
+
+ // close AVIO before emitting finalizationDone.
+ m_recordingEngine.m_formatContext->closeAVIO();
+
+ qCDebug(qLcFFmpegEncoder) << " done finalizing.";
+ emit m_recordingEngine.finalizationDone();
+ auto recordingEnginePtr = &m_recordingEngine;
+ delete recordingEnginePtr;
+}
+
+void RecordingEngine::finalize()
+{
+ qCDebug(qLcFFmpegEncoder) << ">>>>>>>>>>>>>>> finalize";
+
+ m_initializer.reset();
+
+ forEachEncoder(&disconnectEncoderFromSource);
+
+ auto *finalizer = new EncodingFinalizer(*this);
+ finalizer->start();
+}
+
+void RecordingEngine::setPaused(bool paused)
+{
+ forEachEncoder(&EncoderThread::setPaused, paused);
+}
+
+void RecordingEngine::setAutoStop(bool autoStop)
+{
+ m_autoStop = autoStop;
+ forEachEncoder(&EncoderThread::setAutoStop, autoStop);
+ handleSourceEndOfStream();
+}
+
+void RecordingEngine::setMetaData(const QMediaMetaData &metaData)
+{
+ m_metaData = metaData;
+}
+
+void RecordingEngine::newTimeStamp(qint64 time)
+{
+ QMutexLocker locker(&m_timeMutex);
+ if (time > m_timeRecorded) {
+ m_timeRecorded = time;
+ emit durationChanged(time);
+ }
+}
+
+bool RecordingEngine::isEndOfSourceStreams() const
+{
+ auto isAtEnd = [](EncoderThread *encoder) { return encoder->isEndOfSourceStream(); };
+ return std::all_of(m_videoEncoders.cbegin(), m_videoEncoders.cend(), isAtEnd)
+ && std::all_of(m_audioEncoders.cbegin(), m_audioEncoders.cend(), isAtEnd);
+}
+
+void RecordingEngine::handleSourceEndOfStream()
+{
+ if (m_autoStop && isEndOfSourceStreams())
+ emit autoStopped();
+}
+
+template <typename F, typename... Args>
+void RecordingEngine::forEachEncoder(F &&f, Args &&...args)
+{
+ for (AudioEncoder *audioEncoder : m_audioEncoders)
+ std::invoke(f, audioEncoder, args...);
+ for (VideoEncoder *videoEncoder : m_videoEncoders)
+ std::invoke(f, videoEncoder, args...);
+}
+}
+
+QT_END_NAMESPACE
+
+#include "moc_qffmpegrecordingengine_p.cpp"
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine_p.h
new file mode 100644
index 000000000..ce3aaa6bb
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengine_p.h
@@ -0,0 +1,121 @@
+// Copyright (C) 2021 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGENCODER_P_H
+#define QFFMPEGENCODER_P_H
+
+//
+// W A R N I N G
+// -------------
+//
+// This file is not part of the Qt API. It exists purely as an
+// implementation detail. This header file may change from version to
+// version without notice, or even be removed.
+//
+// We mean it.
+//
+
+#include "qffmpegthread_p.h"
+#include "qffmpegencodingformatcontext_p.h"
+
+#include <private/qplatformmediarecorder_p.h>
+#include <qmediarecorder.h>
+
+QT_BEGIN_NAMESPACE
+
+class QFFmpegAudioInput;
+class QPlatformAudioBufferInput;
+class QPlatformAudioBufferInputBase;
+class QVideoFrame;
+class QAudioBuffer;
+class QPlatformVideoSource;
+
+namespace QFFmpeg
+{
+
+class RecordingEngine;
+class Muxer;
+class AudioEncoder;
+class VideoEncoder;
+class VideoFrameEncoder;
+class EncodingInitializer;
+
+class RecordingEngine : public QObject
+{
+ Q_OBJECT
+public:
+ RecordingEngine(const QMediaEncoderSettings &settings, std::unique_ptr<EncodingFormatContext> context);
+ ~RecordingEngine();
+
+ void initialize(const std::vector<QPlatformAudioBufferInputBase *> &audioSources,
+ const std::vector<QPlatformVideoSource *> &videoSources);
+ void finalize();
+
+ void setPaused(bool p);
+
+ void setAutoStop(bool autoStop);
+
+ bool autoStop() const { return m_autoStop; }
+
+ void setMetaData(const QMediaMetaData &metaData);
+ AVFormatContext *avFormatContext() { return m_formatContext->avFormatContext(); }
+ Muxer *getMuxer() { return m_muxer; }
+
+ bool isEndOfSourceStreams() const;
+
+public Q_SLOTS:
+ void newTimeStamp(qint64 time);
+
+Q_SIGNALS:
+ void durationChanged(qint64 duration);
+ void sessionError(QMediaRecorder::Error code, const QString &description);
+ void streamInitializationError(QMediaRecorder::Error code, const QString &description);
+ void finalizationDone();
+ void autoStopped();
+
+private:
+ class EncodingFinalizer : public QThread
+ {
+ public:
+ EncodingFinalizer(RecordingEngine &recordingEngine);
+
+ void run() override;
+
+ private:
+ RecordingEngine &m_recordingEngine;
+ };
+
+ friend class EncodingInitializer;
+ void addAudioInput(QFFmpegAudioInput *input);
+ void addAudioBufferInput(QPlatformAudioBufferInput *input, const QAudioBuffer &firstBuffer);
+ AudioEncoder *createAudioEncoder(const QAudioFormat &format);
+
+ void addVideoSource(QPlatformVideoSource *source, const QVideoFrame &firstFrame);
+ void handleSourceEndOfStream();
+
+ void start();
+
+ template <typename F, typename... Args>
+ void forEachEncoder(F &&f, Args &&...args);
+
+private:
+ QMediaEncoderSettings m_settings;
+ QMediaMetaData m_metaData;
+ std::unique_ptr<EncodingFormatContext> m_formatContext;
+ Muxer *m_muxer = nullptr;
+
+ QList<AudioEncoder *> m_audioEncoders;
+ QList<VideoEncoder *> m_videoEncoders;
+ std::unique_ptr<EncodingInitializer> m_initializer;
+
+ QMutex m_timeMutex;
+ qint64 m_timeRecorded = 0;
+
+ bool m_isHeaderWritten = false;
+ bool m_autoStop = false;
+};
+
+}
+
+QT_END_NAMESPACE
+
+#endif
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils.cpp
new file mode 100644
index 000000000..6c2ba8b15
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils.cpp
@@ -0,0 +1,63 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#include "recordingengine/qffmpegrecordingengineutils_p.h"
+#include "recordingengine/qffmpegencoderthread_p.h"
+#include "private/qplatformaudiobufferinput_p.h"
+#include "private/qplatformvideoframeinput_p.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+template <typename F>
+void doWithMediaFrameInput(QObject *source, F &&f)
+{
+ if (auto videoFrameInput = qobject_cast<QPlatformVideoFrameInput *>(source))
+ f(videoFrameInput);
+ else if (auto audioBufferInput = qobject_cast<QPlatformAudioBufferInput *>(source))
+ f(audioBufferInput);
+}
+
+void setEncoderInterface(QObject *source, QMediaInputEncoderInterface *interface)
+{
+ doWithMediaFrameInput(source, [&](auto source) {
+ using Source = std::remove_pointer_t<decltype(source)>;
+
+ source->setEncoderInterface(interface);
+ if (interface)
+ // Postpone emit 'encoderUpdated' as the encoding pipeline may be not
+ // completely ready at the moment. The case is calling QMediaRecorder::stop
+ // upon handling 'readyToSendFrame'
+ QMetaObject::invokeMethod(source, &Source::encoderUpdated, Qt::QueuedConnection);
+ else
+ emit source->encoderUpdated();
+ });
+}
+
+void setEncoderUpdateConnection(QObject *source, EncoderThread *encoder)
+{
+ doWithMediaFrameInput(source, [&](auto source) {
+ using Source = std::remove_pointer_t<decltype(source)>;
+ QObject::connect(encoder, &EncoderThread::canPushFrameChanged, source,
+ &Source::encoderUpdated);
+ });
+}
+
+void disconnectEncoderFromSource(EncoderThread *encoder)
+{
+ QObject *source = encoder->source();
+ if (!source)
+ return;
+
+ // We should address the dependency AudioEncoder from QFFmpegAudioInput to
+ // set null source here.
+ // encoder->setSource(nullptr);
+
+ QObject::disconnect(source, nullptr, encoder, nullptr);
+ setEncoderInterface(source, nullptr);
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils_p.h
new file mode 100644
index 000000000..a60f81696
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegrecordingengineutils_p.h
@@ -0,0 +1,81 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#ifndef QFFMPEGRECORDINGENGINEUTILS_P_H
+#define QFFMPEGRECORDINGENGINEUTILS_P_H
+
+//
+// W A R N I N G
+// -------------
+//
+// This file is not part of the Qt API. It exists purely as an
+// implementation detail. This header file may change from version to
+// version without notice, or even be removed.
+//
+// We mean it.
+//
+
+#include "qobject.h"
+#include <queue>
+
+QT_BEGIN_NAMESPACE
+
+class QMediaInputEncoderInterface;
+class QPlatformVideoSource;
+
+namespace QFFmpeg {
+
+constexpr qint64 VideoFrameTimeBase = 1000000; // us in sec
+
+class EncoderThread;
+
+template <typename T>
+T dequeueIfPossible(std::queue<T> &queue)
+{
+ if (queue.empty())
+ return T{};
+
+ auto result = std::move(queue.front());
+ queue.pop();
+ return result;
+}
+
+void setEncoderInterface(QObject *source, QMediaInputEncoderInterface *interface);
+
+void setEncoderUpdateConnection(QObject *source, EncoderThread *encoder);
+
+template <typename Encoder, typename Source>
+void connectEncoderToSource(Encoder *encoder, Source *source)
+{
+ Q_ASSERT(!encoder->source());
+ encoder->setSource(source);
+
+ if constexpr (std::is_same_v<Source, QPlatformVideoSource>) {
+ QObject::connect(source, &Source::newVideoFrame, encoder, &Encoder::addFrame,
+ Qt::DirectConnection);
+
+ QObject::connect(source, &Source::activeChanged, encoder, [=]() {
+ if (!source->isActive())
+ encoder->setEndOfSourceStream();
+ });
+ } else {
+ QObject::connect(source, &Source::newAudioBuffer, encoder, &Encoder::addBuffer,
+ Qt::DirectConnection);
+ }
+
+ // TODO:
+ // QObject::connect(source, &Source::disconnectedFromSession, encoder, [=]() {
+ // encoder->setSourceEndOfStream();
+ // });
+
+ setEncoderUpdateConnection(source, encoder);
+ setEncoderInterface(source, encoder);
+}
+
+void disconnectEncoderFromSource(EncoderThread *encoder);
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif // QFFMPEGRECORDINGENGINEUTILS_P_H
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder.cpp
new file mode 100644
index 000000000..27706580b
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder.cpp
@@ -0,0 +1,259 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#include "qffmpegvideoencoder_p.h"
+#include "qffmpegmuxer_p.h"
+#include "qffmpegvideobuffer_p.h"
+#include "qffmpegrecordingengine_p.h"
+#include "qffmpegvideoframeencoder_p.h"
+#include "qffmpegrecordingengineutils_p.h"
+#include "private/qvideoframe_p.h"
+#include <QtCore/qloggingcategory.h>
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+Q_STATIC_LOGGING_CATEGORY(qLcFFmpegVideoEncoder, "qt.multimedia.ffmpeg.videoencoder");
+
+VideoEncoder::VideoEncoder(RecordingEngine &recordingEngine, const QMediaEncoderSettings &settings,
+ const QVideoFrameFormat &format, std::optional<AVPixelFormat> hwFormat)
+ : EncoderThread(recordingEngine)
+{
+ setObjectName(QLatin1String("VideoEncoder"));
+
+ const AVPixelFormat swFormat = QFFmpegVideoBuffer::toAVPixelFormat(format.pixelFormat());
+ qreal frameRate = format.streamFrameRate();
+ if (frameRate <= 0.) {
+ qWarning() << "Invalid frameRate" << frameRate << "; Using the default instead";
+
+ // set some default frame rate since ffmpeg has UB if it's 0.
+ frameRate = 30.;
+ }
+
+ VideoFrameEncoder::SourceParams sourceParams;
+ sourceParams.size = format.frameSize();
+ sourceParams.format = hwFormat && *hwFormat != AV_PIX_FMT_NONE ? *hwFormat : swFormat;
+ sourceParams.swFormat = swFormat;
+ sourceParams.rotation = format.rotation();
+ sourceParams.xMirrored = format.isMirrored();
+ sourceParams.yMirrored = format.scanLineDirection() == QVideoFrameFormat::BottomToTop;
+ sourceParams.frameRate = frameRate;
+ sourceParams.colorTransfer = QFFmpeg::toAvColorTransfer(format.colorTransfer());
+ sourceParams.colorSpace = QFFmpeg::toAvColorSpace(format.colorSpace());
+ sourceParams.colorRange = QFFmpeg::toAvColorRange(format.colorRange());
+
+ m_frameEncoder =
+ VideoFrameEncoder::create(settings, sourceParams, recordingEngine.avFormatContext());
+}
+
+VideoEncoder::~VideoEncoder() = default;
+
+bool VideoEncoder::isValid() const
+{
+ return m_frameEncoder != nullptr;
+}
+
+void VideoEncoder::addFrame(const QVideoFrame &frame)
+{
+ if (!frame.isValid()) {
+ setEndOfSourceStream();
+ return;
+ }
+
+ {
+ auto guard = lockLoopData();
+
+ resetEndOfSourceStream();
+
+ if (m_paused) {
+ m_shouldAdjustTimeBaseForNextFrame = true;
+ return;
+ }
+
+ // Drop frames if encoder can not keep up with the video source data rate;
+ // canPushFrame might be used instead
+ const bool queueFull = m_videoFrameQueue.size() >= m_maxQueueSize;
+
+ if (queueFull) {
+ qCDebug(qLcFFmpegVideoEncoder) << "RecordingEngine frame queue full. Frame lost.";
+ return;
+ }
+
+ m_videoFrameQueue.push({ frame, m_shouldAdjustTimeBaseForNextFrame });
+ m_shouldAdjustTimeBaseForNextFrame = false;
+ }
+
+ dataReady();
+}
+
+VideoEncoder::FrameInfo VideoEncoder::takeFrame()
+{
+ auto guard = lockLoopData();
+ return dequeueIfPossible(m_videoFrameQueue);
+}
+
+void VideoEncoder::retrievePackets()
+{
+ if (!m_frameEncoder)
+ return;
+ while (auto packet = m_frameEncoder->retrievePacket())
+ m_recordingEngine.getMuxer()->addPacket(std::move(packet));
+}
+
+void VideoEncoder::init()
+{
+ Q_ASSERT(isValid());
+
+ qCDebug(qLcFFmpegVideoEncoder) << "VideoEncoder::init started video device thread.";
+ bool ok = m_frameEncoder->open();
+ if (!ok)
+ emit m_recordingEngine.sessionError(QMediaRecorder::ResourceError,
+ "Could not initialize encoder");
+}
+
+void VideoEncoder::cleanup()
+{
+ while (!m_videoFrameQueue.empty())
+ processOne();
+ if (m_frameEncoder) {
+ while (m_frameEncoder->sendFrame(nullptr) == AVERROR(EAGAIN))
+ retrievePackets();
+ retrievePackets();
+ }
+}
+
+bool VideoEncoder::hasData() const
+{
+ return !m_videoFrameQueue.empty();
+}
+
+struct QVideoFrameHolder
+{
+ QVideoFrame f;
+ QImage i;
+};
+
+static void freeQVideoFrame(void *opaque, uint8_t *)
+{
+ delete reinterpret_cast<QVideoFrameHolder *>(opaque);
+}
+
+void VideoEncoder::processOne()
+{
+ retrievePackets();
+
+ FrameInfo frameInfo = takeFrame();
+ QVideoFrame &frame = frameInfo.frame;
+ Q_ASSERT(frame.isValid());
+
+ if (!isValid())
+ return;
+
+ // qCDebug(qLcFFmpegEncoder) << "new video buffer" << frame.startTime();
+
+ AVFrameUPtr avFrame;
+
+ auto *videoBuffer = dynamic_cast<QFFmpegVideoBuffer *>(QVideoFramePrivate::hwBuffer(frame));
+ if (videoBuffer) {
+ // ffmpeg video buffer, let's use the native AVFrame stored in there
+ auto *hwFrame = videoBuffer->getHWFrame();
+ if (hwFrame && hwFrame->format == m_frameEncoder->sourceFormat())
+ avFrame.reset(av_frame_clone(hwFrame));
+ }
+
+ if (!avFrame) {
+ frame.map(QtVideo::MapMode::ReadOnly);
+ auto size = frame.size();
+ avFrame = makeAVFrame();
+ avFrame->format = m_frameEncoder->sourceFormat();
+ avFrame->width = size.width();
+ avFrame->height = size.height();
+
+ for (int i = 0; i < 4; ++i) {
+ avFrame->data[i] = const_cast<uint8_t *>(frame.bits(i));
+ avFrame->linesize[i] = frame.bytesPerLine(i);
+ }
+
+ // TODO: investigate if we need to set color params to AVFrame.
+ // Setting only codec carameters might be sufficient.
+ // What happens if frame color params are set and not equal codec prms?
+ //
+ // QVideoFrameFormat format = frame.surfaceFormat();
+ // avFrame->color_trc = QFFmpeg::toAvColorTransfer(format.colorTransfer());
+ // avFrame->colorspace = QFFmpeg::toAvColorSpace(format.colorSpace());
+ // avFrame->color_range = QFFmpeg::toAvColorRange(format.colorRange());
+
+ QImage img;
+ if (frame.pixelFormat() == QVideoFrameFormat::Format_Jpeg) {
+ // the QImage is cached inside the video frame, so we can take the pointer to the image
+ // data here
+ img = frame.toImage();
+ avFrame->data[0] = (uint8_t *)img.bits();
+ avFrame->linesize[0] = img.bytesPerLine();
+ }
+
+ Q_ASSERT(avFrame->data[0]);
+ // ensure the video frame and it's data is alive as long as it's being used in the encoder
+ avFrame->opaque_ref = av_buffer_create(nullptr, 0, freeQVideoFrame,
+ new QVideoFrameHolder{ frame, img }, 0);
+ }
+
+ const auto [startTime, endTime] = frameTimeStamps(frame);
+
+ if (frameInfo.shouldAdjustTimeBase) {
+ m_baseTime += startTime - m_lastFrameTime;
+ qCDebug(qLcFFmpegVideoEncoder)
+ << ">>>> adjusting base time to" << m_baseTime << startTime << m_lastFrameTime;
+ }
+
+ const qint64 time = startTime - m_baseTime;
+ m_lastFrameTime = endTime;
+
+ setAVFrameTime(*avFrame, m_frameEncoder->getPts(time), m_frameEncoder->getTimeBase());
+
+ m_recordingEngine.newTimeStamp(time / 1000);
+
+ qCDebug(qLcFFmpegVideoEncoder)
+ << ">>> sending frame" << avFrame->pts << time << m_lastFrameTime;
+ int ret = m_frameEncoder->sendFrame(std::move(avFrame));
+ if (ret < 0) {
+ qCDebug(qLcFFmpegVideoEncoder) << "error sending frame" << ret << err2str(ret);
+ emit m_recordingEngine.sessionError(QMediaRecorder::ResourceError, err2str(ret));
+ }
+}
+
+bool VideoEncoder::checkIfCanPushFrame() const
+{
+ if (isRunning())
+ return m_videoFrameQueue.size() < m_maxQueueSize;
+ if (!isFinished())
+ return m_videoFrameQueue.empty();
+
+ return false;
+}
+
+std::pair<qint64, qint64> VideoEncoder::frameTimeStamps(const QVideoFrame &frame) const
+{
+ qint64 startTime = frame.startTime();
+ qint64 endTime = frame.endTime();
+
+ if (startTime == -1) {
+ startTime = m_lastFrameTime;
+ endTime = -1;
+ }
+
+ if (endTime == -1) {
+ qreal frameRate = frame.streamFrameRate();
+ if (frameRate <= 0.)
+ frameRate = m_frameEncoder->settings().videoFrameRate();
+
+ Q_ASSERT(frameRate > 0.f);
+ endTime = startTime + static_cast<qint64>(std::round(VideoFrameTimeBase / frameRate));
+ }
+
+ return { startTime, endTime };
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder_p.h
new file mode 100644
index 000000000..ff6a74fc8
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoder_p.h
@@ -0,0 +1,64 @@
+// Copyright (C) 2024 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGVIDEOENCODER_P_H
+#define QFFMPEGVIDEOENCODER_P_H
+
+#include "qffmpegencoderthread_p.h"
+#include "qffmpeg_p.h"
+#include <qvideoframe.h>
+#include <queue>
+
+QT_BEGIN_NAMESPACE
+
+class QVideoFrameFormat;
+class QMediaEncoderSettings;
+
+namespace QFFmpeg {
+class VideoFrameEncoder;
+
+class VideoEncoder : public EncoderThread
+{
+public:
+ VideoEncoder(RecordingEngine &recordingEngine, const QMediaEncoderSettings &settings,
+ const QVideoFrameFormat &format, std::optional<AVPixelFormat> hwFormat);
+ ~VideoEncoder() override;
+
+ bool isValid() const;
+
+ void addFrame(const QVideoFrame &frame);
+
+protected:
+ bool checkIfCanPushFrame() const override;
+
+private:
+ struct FrameInfo
+ {
+ QVideoFrame frame;
+ bool shouldAdjustTimeBase = false;
+ };
+
+ FrameInfo takeFrame();
+ void retrievePackets();
+
+ void init() override;
+ void cleanup() override;
+ bool hasData() const override;
+ void processOne() override;
+
+ std::pair<qint64, qint64> frameTimeStamps(const QVideoFrame &frame) const;
+
+private:
+ std::queue<FrameInfo> m_videoFrameQueue;
+ const size_t m_maxQueueSize = 10; // Arbitrarily chosen to limit memory usage (332 MB @ 4K)
+
+ std::unique_ptr<VideoFrameEncoder> m_frameEncoder;
+ qint64 m_baseTime = 0;
+ bool m_shouldAdjustTimeBaseForNextFrame = true;
+ qint64 m_lastFrameTime = 0;
+};
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils.cpp
new file mode 100644
index 000000000..69073688b
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils.cpp
@@ -0,0 +1,214 @@
+// Copyright (C) 2022 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#include "qffmpegvideoencoderutils_p.h"
+#include "private/qmultimediautils_p.h"
+
+extern "C" {
+#include <libavutil/pixdesc.h>
+}
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+static AVScore calculateTargetSwFormatScore(const AVPixFmtDescriptor *sourceSwFormatDesc,
+ AVPixelFormat fmt)
+{
+ // determine the format used by the encoder.
+ // We prefer YUV422 based formats such as NV12 or P010. Selection trues to find the best
+ // matching format for the encoder depending on the bit depth of the source format
+
+ const auto *desc = av_pix_fmt_desc_get(fmt);
+ if (!desc)
+ return NotSuitableAVScore;
+
+ const int sourceDepth = sourceSwFormatDesc ? sourceSwFormatDesc->comp[0].depth : 0;
+
+ if (desc->flags & AV_PIX_FMT_FLAG_HWACCEL)
+ // we really don't want HW accelerated formats here
+ return NotSuitableAVScore;
+
+ auto score = DefaultAVScore;
+
+ if (desc == sourceSwFormatDesc)
+ // prefer exact matches
+ score += 10;
+ if (desc->comp[0].depth == sourceDepth)
+ score += 100;
+ else if (desc->comp[0].depth < sourceDepth)
+ score -= 100 + (sourceDepth - desc->comp[0].depth);
+ if (desc->log2_chroma_h == 1)
+ score += 1;
+ if (desc->log2_chroma_w == 1)
+ score += 1;
+ if (desc->flags & AV_PIX_FMT_FLAG_BE)
+ score -= 10;
+ if (desc->flags & AV_PIX_FMT_FLAG_PAL)
+ // we don't want paletted formats
+ score -= 10000;
+ if (desc->flags & AV_PIX_FMT_FLAG_RGB)
+ // we don't want RGB formats
+ score -= 1000;
+
+ // qCDebug(qLcVideoFrameEncoder)
+ // << "checking format" << fmt << Qt::hex << desc->flags << desc->comp[0].depth
+ // << desc->log2_chroma_h << desc->log2_chroma_w << "score:" << score;
+
+ return score;
+}
+
+static auto targetSwFormatScoreCalculator(AVPixelFormat sourceFormat)
+{
+ const auto sourceSwFormatDesc = av_pix_fmt_desc_get(sourceFormat);
+ return [=](AVPixelFormat fmt) { return calculateTargetSwFormatScore(sourceSwFormatDesc, fmt); };
+}
+
+static bool isHwFormatAcceptedByCodec(AVPixelFormat pixFormat)
+{
+ switch (pixFormat) {
+ case AV_PIX_FMT_MEDIACODEC:
+ // Mediacodec doesn't accept AV_PIX_FMT_MEDIACODEC (QTBUG-116836)
+ return false;
+ default:
+ return true;
+ }
+}
+
+AVPixelFormat findTargetSWFormat(AVPixelFormat sourceSWFormat, const AVCodec *codec,
+ const HWAccel &accel)
+{
+ auto scoreCalculator = targetSwFormatScoreCalculator(sourceSWFormat);
+
+ const auto constraints = accel.constraints();
+ if (constraints && constraints->valid_sw_formats)
+ return findBestAVValue(constraints->valid_sw_formats, scoreCalculator).first;
+
+ // Some codecs, e.g. mediacodec, don't expose constraints, let's find the format in
+ // codec->pix_fmts
+ if (codec->pix_fmts)
+ return findBestAVValue(codec->pix_fmts, scoreCalculator).first;
+
+ return AV_PIX_FMT_NONE;
+}
+
+AVPixelFormat findTargetFormat(AVPixelFormat sourceFormat, AVPixelFormat sourceSWFormat,
+ const AVCodec *codec, const HWAccel *accel)
+{
+ Q_UNUSED(sourceFormat);
+
+ if (accel) {
+ const auto hwFormat = accel->hwFormat();
+
+ // TODO: handle codec->capabilities & AV_CODEC_CAP_HARDWARE here
+ if (!isHwFormatAcceptedByCodec(hwFormat))
+ return findTargetSWFormat(sourceSWFormat, codec, *accel);
+
+ const auto constraints = accel->constraints();
+ if (constraints && hasAVFormat(constraints->valid_hw_formats, hwFormat))
+ return hwFormat;
+
+ // Some codecs, don't expose constraints,
+ // let's find the format in codec->pix_fmts and hw_config
+ if (isAVFormatSupported(codec, hwFormat))
+ return hwFormat;
+ }
+
+ if (!codec->pix_fmts) {
+ qWarning() << "Codec pix formats are undefined, it's likely to behave incorrectly";
+
+ return sourceSWFormat;
+ }
+
+ auto swScoreCalculator = targetSwFormatScoreCalculator(sourceSWFormat);
+ return findBestAVValue(codec->pix_fmts, swScoreCalculator).first;
+}
+
+std::pair<const AVCodec *, std::unique_ptr<HWAccel>> findHwEncoder(AVCodecID codecID,
+ const QSize &resolution)
+{
+ auto matchesSizeConstraints = [&resolution](const HWAccel &accel) {
+ const auto constraints = accel.constraints();
+ if (!constraints)
+ return true;
+
+ return resolution.width() >= constraints->min_width
+ && resolution.height() >= constraints->min_height
+ && resolution.width() <= constraints->max_width
+ && resolution.height() <= constraints->max_height;
+ };
+
+ // 1st - attempt to find hw accelerated encoder
+ auto result = HWAccel::findEncoderWithHwAccel(codecID, matchesSizeConstraints);
+ Q_ASSERT(!!result.first == !!result.second);
+
+ return result;
+}
+
+const AVCodec *findSwEncoder(AVCodecID codecID, AVPixelFormat sourceSWFormat)
+{
+ auto formatScoreCalculator = targetSwFormatScoreCalculator(sourceSWFormat);
+
+ return findAVEncoder(codecID, [&formatScoreCalculator](const AVCodec *codec) {
+ if (!codec->pix_fmts)
+ // codecs without pix_fmts are suspicious
+ return MinAVScore;
+
+ return findBestAVValue(codec->pix_fmts, formatScoreCalculator).second;
+ });
+}
+
+AVRational adjustFrameRate(const AVRational *supportedRates, qreal requestedRate)
+{
+ auto calcScore = [requestedRate](const AVRational &rate) {
+ // relative comparison
+ return qMin(requestedRate * rate.den, qreal(rate.num))
+ / qMax(requestedRate * rate.den, qreal(rate.num));
+ };
+
+ const auto result = findBestAVValue(supportedRates, calcScore).first;
+ if (result.num && result.den)
+ return result;
+
+ const auto [num, den] = qRealToFraction(requestedRate);
+ return { num, den };
+}
+
+AVRational adjustFrameTimeBase(const AVRational *supportedRates, AVRational frameRate)
+{
+ // TODO: user-specified frame rate might be required.
+ if (supportedRates) {
+ auto hasFrameRate = [&]() {
+ for (auto rate = supportedRates; rate->num && rate->den; ++rate)
+ if (rate->den == frameRate.den && rate->num == frameRate.num)
+ return true;
+
+ return false;
+ };
+
+ Q_ASSERT(hasFrameRate());
+
+ return { frameRate.den, frameRate.num };
+ }
+
+ constexpr int TimeScaleFactor = 1000; // Allows not to follow fixed rate
+ return { frameRate.den, frameRate.num * TimeScaleFactor };
+}
+
+QSize adjustVideoResolution(const AVCodec *codec, QSize requestedResolution)
+{
+#ifdef Q_OS_WINDOWS
+ // TODO: investigate, there might be more encoders not supporting odd resolution
+ if (strcmp(codec->name, "h264_mf") == 0) {
+ auto makeEven = [](int size) { return size & ~1; };
+ return QSize(makeEven(requestedResolution.width()), makeEven(requestedResolution.height()));
+ }
+#else
+ Q_UNUSED(codec);
+#endif
+ return requestedResolution;
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils_p.h
new file mode 100644
index 000000000..3a16a7de3
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoencoderutils_p.h
@@ -0,0 +1,64 @@
+// Copyright (C) 2022 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGVIDEOENCODERUTILS_P_H
+#define QFFMPEGVIDEOENCODERUTILS_P_H
+
+//
+// W A R N I N G
+// -------------
+//
+// This file is not part of the Qt API. It exists purely as an
+// implementation detail. This header file may change from version to
+// version without notice, or even be removed.
+//
+// We mean it.
+//
+
+#include "qffmpeg_p.h"
+#include "qffmpeghwaccel_p.h"
+
+QT_BEGIN_NAMESPACE
+
+namespace QFFmpeg {
+
+AVPixelFormat findTargetSWFormat(AVPixelFormat sourceSWFormat, const AVCodec *codec,
+ const HWAccel &accel);
+
+AVPixelFormat findTargetFormat(AVPixelFormat sourceFormat, AVPixelFormat sourceSWFormat,
+ const AVCodec *codec, const HWAccel *accel);
+
+std::pair<const AVCodec *, std::unique_ptr<HWAccel>> findHwEncoder(AVCodecID codecID,
+ const QSize &sourceSize);
+
+const AVCodec *findSwEncoder(AVCodecID codecID, AVPixelFormat sourceSWFormat);
+
+/**
+ * @brief adjustFrameRate get a rational frame rate be requested qreal rate.
+ * If the codec supports fixed frame rate (non-null supportedRates),
+ * the function selects the most suitable one,
+ * otherwise just makes AVRational from qreal.
+ */
+AVRational adjustFrameRate(const AVRational *supportedRates, qreal requestedRate);
+
+/**
+ * @brief adjustFrameTimeBase gets adjusted timebase by a list of supported frame rates
+ * and an already adjusted frame rate.
+ *
+ * Timebase is the fundamental unit of time (in seconds) in terms
+ * of which frame timestamps are represented.
+ * For fixed-fps content (non-null supportedRates),
+ * timebase should be 1/framerate.
+ *
+ * For more information, see AVStream::time_base and AVCodecContext::time_base.
+ *
+ * The adjusted time base is supposed to be set to stream and codec context.
+ */
+AVRational adjustFrameTimeBase(const AVRational *supportedRates, AVRational frameRate);
+
+QSize adjustVideoResolution(const AVCodec *codec, QSize requestedResolution);
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
+
+#endif // QFFMPEGVIDEOENCODERUTILS_P_H
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder.cpp b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder.cpp
new file mode 100644
index 000000000..ce2a1af28
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder.cpp
@@ -0,0 +1,547 @@
+// Copyright (C) 2022 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+
+#include "qffmpegvideoframeencoder_p.h"
+#include "qffmpegmediaformatinfo_p.h"
+#include "qffmpegencoderoptions_p.h"
+#include "qffmpegvideoencoderutils_p.h"
+#include <qloggingcategory.h>
+#include <QtMultimedia/private/qmaybe_p.h>
+
+extern "C" {
+#include "libavutil/display.h"
+}
+
+QT_BEGIN_NAMESPACE
+
+Q_STATIC_LOGGING_CATEGORY(qLcVideoFrameEncoder, "qt.multimedia.ffmpeg.videoencoder");
+
+namespace QFFmpeg {
+
+std::unique_ptr<VideoFrameEncoder>
+VideoFrameEncoder::create(const QMediaEncoderSettings &encoderSettings,
+ const SourceParams &sourceParams, AVFormatContext *formatContext)
+{
+ Q_ASSERT(isSwPixelFormat(sourceParams.swFormat));
+ Q_ASSERT(isHwPixelFormat(sourceParams.format) || sourceParams.swFormat == sourceParams.format);
+
+ std::unique_ptr<VideoFrameEncoder> result(new VideoFrameEncoder);
+
+ result->m_settings = encoderSettings;
+ result->m_sourceSize = sourceParams.size;
+ result->m_sourceFormat = sourceParams.format;
+
+ // Temporary: check isSwPixelFormat because of android issue (QTBUG-116836)
+ result->m_sourceSWFormat =
+ isSwPixelFormat(sourceParams.format) ? sourceParams.format : sourceParams.swFormat;
+
+ if (!result->m_settings.videoResolution().isValid())
+ result->m_settings.setVideoResolution(sourceParams.size);
+
+ if (result->m_settings.videoFrameRate() <= 0.)
+ result->m_settings.setVideoFrameRate(sourceParams.frameRate);
+
+ if (!result->initCodec() || !result->initTargetFormats()
+ || !result->initCodecContext(sourceParams, formatContext)) {
+ return nullptr;
+ }
+
+ // TODO: make VideoFrameEncoder::private and do openning here
+ // if (!open()) {
+ // m_error = QMediaRecorder::FormatError;
+ // m_errorStr = QLatin1StringView("Cannot open codec");
+ // return;
+ // }
+
+ result->updateConversions();
+
+ return result;
+}
+
+bool VideoFrameEncoder::initCodec()
+{
+ const auto qVideoCodec = m_settings.videoCodec();
+ const auto codecID = QFFmpegMediaFormatInfo::codecIdForVideoCodec(qVideoCodec);
+ const auto resolution = m_settings.videoResolution();
+
+ std::tie(m_codec, m_accel) = findHwEncoder(codecID, resolution);
+
+ if (!m_codec)
+ m_codec = findSwEncoder(codecID, m_sourceSWFormat);
+
+ if (!m_codec) {
+ qWarning() << "Could not find encoder for codecId" << codecID;
+ return false;
+ }
+
+ qCDebug(qLcVideoFrameEncoder) << "found encoder" << m_codec->name << "for id" << m_codec->id;
+
+#ifdef Q_OS_WINDOWS
+ // TODO: investigate, there might be more encoders not supporting odd resolution
+ if (strcmp(m_codec->name, "h264_mf") == 0) {
+ auto makeEven = [](int size) { return size & ~1; };
+ const QSize fixedResolution(makeEven(resolution.width()), makeEven(resolution.height()));
+ if (fixedResolution != resolution) {
+ qCDebug(qLcVideoFrameEncoder) << "Fix odd video resolution for codec" << m_codec->name
+ << ":" << resolution << "->" << fixedResolution;
+ m_settings.setVideoResolution(fixedResolution);
+ }
+ }
+#endif
+
+ auto fixedResolution = adjustVideoResolution(m_codec, m_settings.videoResolution());
+ if (resolution != fixedResolution) {
+ qCDebug(qLcVideoFrameEncoder) << "Fix odd video resolution for codec" << m_codec->name
+ << ":" << resolution << "->" << fixedResolution;
+
+ m_settings.setVideoResolution(fixedResolution);
+ }
+
+ if (m_codec->supported_framerates && qLcVideoFrameEncoder().isEnabled(QtDebugMsg))
+ for (auto rate = m_codec->supported_framerates; rate->num && rate->den; ++rate)
+ qCDebug(qLcVideoFrameEncoder) << "supported frame rate:" << *rate;
+
+ m_codecFrameRate = adjustFrameRate(m_codec->supported_framerates, m_settings.videoFrameRate());
+ qCDebug(qLcVideoFrameEncoder) << "Adjusted frame rate:" << m_codecFrameRate;
+
+ return true;
+}
+
+bool VideoFrameEncoder::initTargetFormats()
+{
+ m_targetFormat = findTargetFormat(m_sourceFormat, m_sourceSWFormat, m_codec, m_accel.get());
+
+ if (m_targetFormat == AV_PIX_FMT_NONE) {
+ qWarning() << "Could not find target format for codecId" << m_codec->id;
+ return false;
+ }
+
+ if (isHwPixelFormat(m_targetFormat)) {
+ Q_ASSERT(m_accel);
+
+ m_targetSWFormat = findTargetSWFormat(m_sourceSWFormat, m_codec, *m_accel);
+
+ if (m_targetSWFormat == AV_PIX_FMT_NONE) {
+ qWarning() << "Cannot find software target format. sourceSWFormat:" << m_sourceSWFormat
+ << "targetFormat:" << m_targetFormat;
+ return false;
+ }
+
+ m_accel->createFramesContext(m_targetSWFormat, m_settings.videoResolution());
+ if (!m_accel->hwFramesContextAsBuffer())
+ return false;
+ } else {
+ m_targetSWFormat = m_targetFormat;
+ }
+
+ return true;
+}
+
+VideoFrameEncoder::~VideoFrameEncoder() = default;
+
+bool VideoFrameEncoder::initCodecContext(const SourceParams &sourceParams,
+ AVFormatContext *formatContext)
+{
+ m_stream = avformat_new_stream(formatContext, nullptr);
+ m_stream->id = formatContext->nb_streams - 1;
+ //qCDebug(qLcVideoFrameEncoder) << "Video stream: index" << d->stream->id;
+ m_stream->codecpar->codec_type = AVMEDIA_TYPE_VIDEO;
+ m_stream->codecpar->codec_id = m_codec->id;
+
+ // Apples HEVC decoders don't like the hev1 tag ffmpeg uses by default, use hvc1 as the more commonly accepted tag
+ if (m_codec->id == AV_CODEC_ID_HEVC)
+ m_stream->codecpar->codec_tag = MKTAG('h', 'v', 'c', '1');
+
+ const auto resolution = m_settings.videoResolution();
+
+ // ### Fix hardcoded values
+ m_stream->codecpar->format = m_targetFormat;
+ m_stream->codecpar->width = resolution.width();
+ m_stream->codecpar->height = resolution.height();
+ m_stream->codecpar->sample_aspect_ratio = AVRational{ 1, 1 };
+ m_stream->codecpar->color_trc = sourceParams.colorTransfer;
+ m_stream->codecpar->color_space = sourceParams.colorSpace;
+ m_stream->codecpar->color_range = sourceParams.colorRange;
+
+ if (sourceParams.rotation != QtVideo::Rotation::None || sourceParams.xMirrored
+ || sourceParams.yMirrored) {
+ constexpr auto displayMatrixSize = sizeof(int32_t) * 9;
+ AVPacketSideData sideData = { reinterpret_cast<uint8_t *>(av_malloc(displayMatrixSize)),
+ displayMatrixSize, AV_PKT_DATA_DISPLAYMATRIX };
+ int32_t *matrix = reinterpret_cast<int32_t *>(sideData.data);
+ av_display_rotation_set(matrix, static_cast<double>(sourceParams.rotation));
+ av_display_matrix_flip(matrix, sourceParams.xMirrored, sourceParams.yMirrored);
+
+ addStreamSideData(m_stream, sideData);
+ }
+
+ Q_ASSERT(m_codec);
+
+ m_stream->time_base = adjustFrameTimeBase(m_codec->supported_framerates, m_codecFrameRate);
+ m_codecContext.reset(avcodec_alloc_context3(m_codec));
+ if (!m_codecContext) {
+ qWarning() << "Could not allocate codec context";
+ return false;
+ }
+
+ avcodec_parameters_to_context(m_codecContext.get(), m_stream->codecpar);
+ m_codecContext->time_base = m_stream->time_base;
+ qCDebug(qLcVideoFrameEncoder) << "codecContext time base" << m_codecContext->time_base.num
+ << m_codecContext->time_base.den;
+
+ m_codecContext->framerate = m_codecFrameRate;
+ m_codecContext->pix_fmt = m_targetFormat;
+ m_codecContext->width = resolution.width();
+ m_codecContext->height = resolution.height();
+
+ if (m_accel) {
+ auto deviceContext = m_accel->hwDeviceContextAsBuffer();
+ Q_ASSERT(deviceContext);
+ m_codecContext->hw_device_ctx = av_buffer_ref(deviceContext);
+
+ if (auto framesContext = m_accel->hwFramesContextAsBuffer())
+ m_codecContext->hw_frames_ctx = av_buffer_ref(framesContext);
+ }
+
+ return true;
+}
+
+bool VideoFrameEncoder::open()
+{
+ if (!m_codecContext)
+ return false;
+
+ AVDictionaryHolder opts;
+ applyVideoEncoderOptions(m_settings, m_codec->name, m_codecContext.get(), opts);
+ applyExperimentalCodecOptions(m_codec, opts);
+
+ int res = avcodec_open2(m_codecContext.get(), m_codec, opts);
+ if (res < 0) {
+ m_codecContext.reset();
+ qWarning() << "Couldn't open codec for writing" << err2str(res);
+ return false;
+ }
+ qCDebug(qLcVideoFrameEncoder) << "video codec opened" << res << "time base"
+ << m_codecContext->time_base;
+ return true;
+}
+
+qint64 VideoFrameEncoder::getPts(qint64 us) const
+{
+ qint64 div = 1'000'000 * m_stream->time_base.num;
+ return div != 0 ? (us * m_stream->time_base.den + div / 2) / div : 0;
+}
+
+const AVRational &VideoFrameEncoder::getTimeBase() const
+{
+ return m_stream->time_base;
+}
+
+namespace {
+struct FrameConverter
+{
+ FrameConverter(AVFrameUPtr inputFrame) : m_inputFrame{ std::move(inputFrame) } { }
+
+ int downloadFromHw()
+ {
+ AVFrameUPtr cpuFrame = makeAVFrame();
+
+ int err = av_hwframe_transfer_data(cpuFrame.get(), currentFrame(), 0);
+ if (err < 0) {
+ qCDebug(qLcVideoFrameEncoder)
+ << "Error transferring frame data to surface." << err2str(err);
+ return err;
+ }
+
+ setFrame(std::move(cpuFrame));
+ return 0;
+ }
+
+ void convert(SwsContext *converter, AVPixelFormat format, const QSize &size)
+ {
+ AVFrameUPtr scaledFrame = makeAVFrame();
+
+ scaledFrame->format = format;
+ scaledFrame->width = size.width();
+ scaledFrame->height = size.height();
+
+ av_frame_get_buffer(scaledFrame.get(), 0);
+ const auto scaledHeight =
+ sws_scale(converter, currentFrame()->data, currentFrame()->linesize, 0, currentFrame()->height,
+ scaledFrame->data, scaledFrame->linesize);
+
+ if (scaledHeight != scaledFrame->height)
+ qCWarning(qLcVideoFrameEncoder)
+ << "Scaled height" << scaledHeight << "!=" << scaledFrame->height;
+
+ setFrame(std::move(scaledFrame));
+ }
+
+ int uploadToHw(HWAccel *accel)
+ {
+ auto *hwFramesContext = accel->hwFramesContextAsBuffer();
+ Q_ASSERT(hwFramesContext);
+ AVFrameUPtr hwFrame = makeAVFrame();
+ if (!hwFrame)
+ return AVERROR(ENOMEM);
+
+ int err = av_hwframe_get_buffer(hwFramesContext, hwFrame.get(), 0);
+ if (err < 0) {
+ qCDebug(qLcVideoFrameEncoder) << "Error getting HW buffer" << err2str(err);
+ return err;
+ } else {
+ qCDebug(qLcVideoFrameEncoder) << "got HW buffer";
+ }
+ if (!hwFrame->hw_frames_ctx) {
+ qCDebug(qLcVideoFrameEncoder) << "no hw frames context";
+ return AVERROR(ENOMEM);
+ }
+ err = av_hwframe_transfer_data(hwFrame.get(), currentFrame(), 0);
+ if (err < 0) {
+ qCDebug(qLcVideoFrameEncoder)
+ << "Error transferring frame data to surface." << err2str(err);
+ return err;
+ }
+
+ setFrame(std::move(hwFrame));
+
+ return 0;
+ }
+
+ QMaybe<AVFrameUPtr, int> takeResultFrame()
+ {
+ // Ensure that object is reset to empty state
+ AVFrameUPtr converted = std::move(m_convertedFrame);
+ AVFrameUPtr input = std::move(m_inputFrame);
+
+ if (!converted)
+ return input;
+
+ // Copy metadata except size and format from input frame
+ const int status = av_frame_copy_props(converted.get(), input.get());
+ if (status != 0)
+ return status;
+
+ return converted;
+ }
+
+private:
+ void setFrame(AVFrameUPtr frame) { m_convertedFrame = std::move(frame); }
+
+ AVFrame *currentFrame() const
+ {
+ if (m_convertedFrame)
+ return m_convertedFrame.get();
+ return m_inputFrame.get();
+ }
+
+ AVFrameUPtr m_inputFrame;
+ AVFrameUPtr m_convertedFrame;
+};
+}
+
+int VideoFrameEncoder::sendFrame(AVFrameUPtr inputFrame)
+{
+ if (!m_codecContext) {
+ qWarning() << "codec context is not initialized!";
+ return AVERROR(EINVAL);
+ }
+
+ if (!inputFrame)
+ return avcodec_send_frame(m_codecContext.get(), nullptr); // Flush
+
+ if (!updateSourceFormatAndSize(inputFrame.get()))
+ return AVERROR(EINVAL);
+
+ FrameConverter converter{ std::move(inputFrame) };
+
+ if (m_downloadFromHW) {
+ const int status = converter.downloadFromHw();
+ if (status != 0)
+ return status;
+ }
+
+ if (m_converter)
+ converter.convert(m_converter.get(), m_targetSWFormat, m_settings.videoResolution());
+
+ if (m_uploadToHW) {
+ const int status = converter.uploadToHw(m_accel.get());
+ if (status != 0)
+ return status;
+ }
+
+ const QMaybe<AVFrameUPtr, int> resultFrame = converter.takeResultFrame();
+ if (!resultFrame)
+ return resultFrame.error();
+
+ AVRational timeBase{};
+ int64_t pts{};
+ getAVFrameTime(*resultFrame.value(), pts, timeBase);
+ qCDebug(qLcVideoFrameEncoder) << "sending frame" << pts << "*" << timeBase;
+
+ return avcodec_send_frame(m_codecContext.get(), resultFrame.value().get());
+}
+
+qint64 VideoFrameEncoder::estimateDuration(const AVPacket &packet, bool isFirstPacket)
+{
+ qint64 duration = 0; // In stream units, multiply by time_base to get seconds
+
+ if (isFirstPacket) {
+ // First packet - Estimate duration from frame rate. Duration must
+ // be set for single-frame videos, otherwise they won't open in
+ // media player.
+ const AVRational frameDuration = av_inv_q(m_codecContext->framerate);
+ duration = av_rescale_q(1, frameDuration, m_stream->time_base);
+ } else {
+ // Duration is calculated from actual packet times. TODO: Handle discontinuities
+ duration = packet.pts - m_lastPacketTime;
+ }
+
+ return duration;
+}
+
+AVPacketUPtr VideoFrameEncoder::retrievePacket()
+{
+ if (!m_codecContext)
+ return nullptr;
+
+ auto getPacket = [&]() {
+ AVPacketUPtr packet(av_packet_alloc());
+ const int ret = avcodec_receive_packet(m_codecContext.get(), packet.get());
+ if (ret < 0) {
+ if (ret != AVERROR(EOF) && ret != AVERROR(EAGAIN) && ret != AVERROR_EOF)
+ qCDebug(qLcVideoFrameEncoder) << "Error receiving packet" << ret << err2str(ret);
+ return AVPacketUPtr{};
+ }
+ auto ts = timeStampMs(packet->pts, m_stream->time_base);
+
+ qCDebug(qLcVideoFrameEncoder)
+ << "got a packet" << packet->pts << packet->dts << (ts ? *ts : 0);
+
+ packet->stream_index = m_stream->id;
+
+ if (packet->duration == 0) {
+ const bool firstFrame = m_lastPacketTime == AV_NOPTS_VALUE;
+ packet->duration = estimateDuration(*packet, firstFrame);
+ }
+
+ m_lastPacketTime = packet->pts;
+
+ return packet;
+ };
+
+ auto fixPacketDts = [&](AVPacket &packet) {
+ // Workaround for some ffmpeg codecs bugs (e.g. nvenc)
+ // Ideally, packet->pts < packet->dts is not expected
+
+ if (packet.dts == AV_NOPTS_VALUE)
+ return true;
+
+ packet.dts -= m_packetDtsOffset;
+
+ if (packet.pts != AV_NOPTS_VALUE && packet.pts < packet.dts) {
+ m_packetDtsOffset += packet.dts - packet.pts;
+ packet.dts = packet.pts;
+
+ if (m_prevPacketDts != AV_NOPTS_VALUE && packet.dts < m_prevPacketDts) {
+ qCWarning(qLcVideoFrameEncoder)
+ << "Skip packet; failed to fix dts:" << packet.dts << m_prevPacketDts;
+ return false;
+ }
+ }
+
+ m_prevPacketDts = packet.dts;
+
+ return true;
+ };
+
+ while (auto packet = getPacket()) {
+ if (fixPacketDts(*packet))
+ return packet;
+ }
+
+ return nullptr;
+}
+
+bool VideoFrameEncoder::updateSourceFormatAndSize(const AVFrame *frame)
+{
+ Q_ASSERT(frame);
+
+ const QSize frameSize(frame->width, frame->height);
+ const AVPixelFormat frameFormat = static_cast<AVPixelFormat>(frame->format);
+
+ if (frameSize == m_sourceSize && frameFormat == m_sourceFormat)
+ return true;
+
+ auto applySourceFormatAndSize = [&](AVPixelFormat swFormat) {
+ m_sourceSize = frameSize;
+ m_sourceFormat = frameFormat;
+ m_sourceSWFormat = swFormat;
+ updateConversions();
+ return true;
+ };
+
+ if (frameFormat == m_sourceFormat)
+ return applySourceFormatAndSize(m_sourceSWFormat);
+
+ if (frameFormat == AV_PIX_FMT_NONE) {
+ qWarning() << "Got a frame with invalid pixel format";
+ return false;
+ }
+
+ if (isSwPixelFormat(frameFormat))
+ return applySourceFormatAndSize(frameFormat);
+
+ auto framesCtx = reinterpret_cast<const AVHWFramesContext *>(frame->hw_frames_ctx->data);
+ if (!framesCtx || framesCtx->sw_format == AV_PIX_FMT_NONE) {
+ qWarning() << "Cannot update conversions as hw frame has invalid framesCtx" << framesCtx;
+ return false;
+ }
+
+ return applySourceFormatAndSize(framesCtx->sw_format);
+}
+
+void VideoFrameEncoder::updateConversions()
+{
+ const bool needToScale = m_sourceSize != m_settings.videoResolution();
+ const bool zeroCopy = m_sourceFormat == m_targetFormat && !needToScale;
+
+ m_converter.reset();
+
+ if (zeroCopy) {
+ m_downloadFromHW = false;
+ m_uploadToHW = false;
+
+ qCDebug(qLcVideoFrameEncoder) << "zero copy encoding, format" << m_targetFormat;
+ // no need to initialize any converters
+ return;
+ }
+
+ m_downloadFromHW = m_sourceFormat != m_sourceSWFormat;
+ m_uploadToHW = m_targetFormat != m_targetSWFormat;
+
+ if (m_sourceSWFormat != m_targetSWFormat || needToScale) {
+ const auto targetSize = m_settings.videoResolution();
+ qCDebug(qLcVideoFrameEncoder)
+ << "video source and encoder use different formats:" << m_sourceSWFormat
+ << m_targetSWFormat << "or sizes:" << m_sourceSize << targetSize;
+
+ m_converter.reset(sws_getContext(m_sourceSize.width(), m_sourceSize.height(),
+ m_sourceSWFormat, targetSize.width(), targetSize.height(),
+ m_targetSWFormat, SWS_FAST_BILINEAR, nullptr, nullptr,
+ nullptr));
+ }
+
+ qCDebug(qLcVideoFrameEncoder) << "VideoFrameEncoder conversions initialized:"
+ << "sourceFormat:" << m_sourceFormat
+ << (isHwPixelFormat(m_sourceFormat) ? "(hw)" : "(sw)")
+ << "targetFormat:" << m_targetFormat
+ << (isHwPixelFormat(m_targetFormat) ? "(hw)" : "(sw)")
+ << "sourceSWFormat:" << m_sourceSWFormat
+ << "targetSWFormat:" << m_targetSWFormat
+ << "converter:" << m_converter.get();
+}
+
+} // namespace QFFmpeg
+
+QT_END_NAMESPACE
diff --git a/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder_p.h b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder_p.h
new file mode 100644
index 000000000..731789926
--- /dev/null
+++ b/src/plugins/multimedia/ffmpeg/recordingengine/qffmpegvideoframeencoder_p.h
@@ -0,0 +1,104 @@
+// Copyright (C) 2022 The Qt Company Ltd.
+// SPDX-License-Identifier: LicenseRef-Qt-Commercial OR LGPL-3.0-only OR GPL-2.0-only OR GPL-3.0-only
+#ifndef QFFMPEGVIDEOFRAMEENCODER_P_H
+#define QFFMPEGVIDEOFRAMEENCODER_P_H
+
+//
+// W A R N I N G
+// -------------
+//
+// This file is not part of the Qt API. It exists purely as an
+// implementation detail. This header file may change from version to
+// version without notice, or even be removed.
+//
+// We mean it.
+//
+
+#include "qffmpeghwaccel_p.h"
+#include "private/qplatformmediarecorder_p.h"
+
+QT_BEGIN_NAMESPACE
+
+class QMediaEncoderSettings;
+
+namespace QFFmpeg {
+
+class VideoFrameEncoder
+{
+public:
+ struct SourceParams
+ {
+ QSize size;
+ AVPixelFormat format = AV_PIX_FMT_NONE;
+ AVPixelFormat swFormat = AV_PIX_FMT_NONE;
+ QtVideo::Rotation rotation = QtVideo::Rotation::None;
+ bool xMirrored = false;
+ bool yMirrored = false;
+ qreal frameRate = 0.;
+ AVColorTransferCharacteristic colorTransfer = AVCOL_TRC_UNSPECIFIED;
+ AVColorSpace colorSpace = AVCOL_SPC_UNSPECIFIED;
+ AVColorRange colorRange = AVCOL_RANGE_UNSPECIFIED;
+ };
+ static std::unique_ptr<VideoFrameEncoder> create(const QMediaEncoderSettings &encoderSettings,
+ const SourceParams &sourceParams,
+ AVFormatContext *formatContext);
+
+ ~VideoFrameEncoder();
+
+ bool open();
+
+ AVPixelFormat sourceFormat() const { return m_sourceFormat; }
+ AVPixelFormat targetFormat() const { return m_targetFormat; }
+
+ qint64 getPts(qint64 ms) const;
+
+ const AVRational &getTimeBase() const;
+
+ int sendFrame(AVFrameUPtr inputFrame);
+ AVPacketUPtr retrievePacket();
+
+ const QMediaEncoderSettings &settings() { return m_settings; }
+
+private:
+ VideoFrameEncoder() = default;
+
+ bool updateSourceFormatAndSize(const AVFrame *frame);
+
+ void updateConversions();
+
+ bool initCodec();
+
+ bool initTargetFormats();
+
+ bool initCodecContext(const SourceParams &sourceParams, AVFormatContext *formatContext);
+
+ qint64 estimateDuration(const AVPacket &packet, bool isFirstPacket);
+
+private:
+ QMediaEncoderSettings m_settings;
+ QSize m_sourceSize;
+
+ std::unique_ptr<HWAccel> m_accel;
+ const AVCodec *m_codec = nullptr;
+ AVStream *m_stream = nullptr;
+ qint64 m_lastPacketTime = AV_NOPTS_VALUE;
+ AVCodecContextUPtr m_codecContext;
+ std::unique_ptr<SwsContext, decltype(&sws_freeContext)> m_converter = { nullptr,
+ &sws_freeContext };
+ AVPixelFormat m_sourceFormat = AV_PIX_FMT_NONE;
+ AVPixelFormat m_sourceSWFormat = AV_PIX_FMT_NONE;
+ AVPixelFormat m_targetFormat = AV_PIX_FMT_NONE;
+ AVPixelFormat m_targetSWFormat = AV_PIX_FMT_NONE;
+ bool m_downloadFromHW = false;
+ bool m_uploadToHW = false;
+
+ AVRational m_codecFrameRate = { 0, 1 };
+
+ int64_t m_prevPacketDts = AV_NOPTS_VALUE;
+ int64_t m_packetDtsOffset = 0;
+};
+}
+
+QT_END_NAMESPACE
+
+#endif