Blame SOURCES/0044-Move-the-wayland-socket-polling-to-a-separate-event-.patch

9fb289
From 4644d51f4b52e83fc1b4d02b380d80d9d57e76fa Mon Sep 17 00:00:00 2001
9fb289
From: Adrien Faveraux <af@brain-networks.fr>
9fb289
Date: Fri, 26 Nov 2021 09:18:58 +0100
9fb289
Subject: [PATCH 44/52] Move the wayland socket polling to a separate event
9fb289
 thread
9fb289
9fb289
New event threads is introduced which calls poll() on the wayland fd,
9fb289
instead of relying on the event dispatcher by using the QSocketNotifier.
9fb289
This allows to call in the proper order the wl_display_prepare_read(),
9fb289
poll() and wl_display_read_events() functions.
9fb289
9fb289
One thread is responsible for the default queue; when needed, it emit
9fb289
a signal so that the main thread can dispatch the queue. Another thread
9fb289
is responsible for the dedicated queue for frame callbacks; this thread
9fb289
will dispatch events on the thread itself.
9fb289
9fb289
QWaylandWindow is updated to, instead of each window's dedicated event
9fb289
queue, use this queue for frame callbacks.
9fb289
9fb289
Co-authored-by: Ratchanan Srirattanamet <ratchanan@ubports.com>
9fb289
Task-number: QTBUG-66075
9fb289
Change-Id: Ibb33ad7f4193b866d1b8d7a0405a94d59dcad5eb
9fb289
Reviewed-by: Qt CI Bot <qt_ci_bot@qt-project.org>
9fb289
Reviewed-by: Eskil Abrahamsen Blomfeldt <eskil.abrahamsen-blomfeldt@qt.io>
9fb289
(cherry picked from commit 92a7904d9651348b0c307e84251c8440c6f75b22)
9fb289
---
9fb289
 src/client/qwaylanddisplay.cpp     | 302 +++++++++++++++++++++--------
9fb289
 src/client/qwaylanddisplay_p.h     |  21 +-
9fb289
 src/client/qwaylandintegration.cpp |   4 +-
9fb289
 src/client/qwaylandwindow.cpp      |  34 +++-
9fb289
 src/client/qwaylandwindow_p.h      |   2 +-
9fb289
 5 files changed, 255 insertions(+), 108 deletions(-)
9fb289
9fb289
diff --git a/src/client/qwaylanddisplay.cpp b/src/client/qwaylanddisplay.cpp
9fb289
index 0f75cb7e..a7ce280a 100644
9fb289
--- a/src/client/qwaylanddisplay.cpp
9fb289
+++ b/src/client/qwaylanddisplay.cpp
9fb289
@@ -85,10 +85,203 @@
9fb289
 
9fb289
 #include <errno.h>
9fb289
 
9fb289
+#include <tuple> // for std::tie
9fb289
+
9fb289
+static void checkWaylandError(struct wl_display *display)
9fb289
+{
9fb289
+    int ecode = wl_display_get_error(display);
9fb289
+    if ((ecode == EPIPE || ecode == ECONNRESET)) {
9fb289
+        // special case this to provide a nicer error
9fb289
+        qWarning("The Wayland connection broke. Did the Wayland compositor die?");
9fb289
+    } else {
9fb289
+        qWarning("The Wayland connection experienced a fatal error: %s", strerror(ecode));
9fb289
+    }
9fb289
+    _exit(1);
9fb289
+}
9fb289
+
9fb289
 QT_BEGIN_NAMESPACE
9fb289
 
9fb289
 namespace QtWaylandClient {
9fb289
 
9fb289
+class EventThread : public QThread
9fb289
+{
9fb289
+    Q_OBJECT
9fb289
+public:
9fb289
+    enum OperatingMode {
9fb289
+        EmitToDispatch, // Emit the signal, allow dispatching in a differnt thread.
9fb289
+        SelfDispatch, // Dispatch the events inside this thread.
9fb289
+    };
9fb289
+
9fb289
+    EventThread(struct wl_display * wl, struct wl_event_queue * ev_queue,
9fb289
+                OperatingMode mode)
9fb289
+        : m_fd(wl_display_get_fd(wl))
9fb289
+        , m_pipefd{ -1, -1 }
9fb289
+        , m_wldisplay(wl)
9fb289
+        , m_wlevqueue(ev_queue)
9fb289
+        , m_mode(mode)
9fb289
+        , m_reading(true)
9fb289
+        , m_quitting(false)
9fb289
+    {
9fb289
+        setObjectName(QStringLiteral("WaylandEventThread"));
9fb289
+    }
9fb289
+
9fb289
+    void readAndDispatchEvents()
9fb289
+    {
9fb289
+        /*
9fb289
+         * Dispatch pending events and flush the requests at least once. If the event thread
9fb289
+         * is not reading, try to call _prepare_read() to allow the event thread to poll().
9fb289
+         * If that fails, re-try dispatch & flush again until _prepare_read() is successful.
9fb289
+         *
9fb289
+         * This allow any call to readAndDispatchEvents() to start event thread's polling,
9fb289
+         * not only the one issued from event thread's waitForReading(), which means functions
9fb289
+         * called from dispatch_pending() can safely spin an event loop.
9fb289
+         */
9fb289
+        for (;;) {
9fb289
+            if (dispatchQueuePending() < 0) {
9fb289
+                checkWaylandError(m_wldisplay);
9fb289
+                return;
9fb289
+            }
9fb289
+
9fb289
+            wl_display_flush(m_wldisplay);
9fb289
+
9fb289
+            // We have to check if event thread is reading every time we dispatch
9fb289
+            // something, as that may recursively call this function.
9fb289
+            if (m_reading.loadAcquire())
9fb289
+                break;
9fb289
+
9fb289
+            if (prepareReadQueue() == 0) {
9fb289
+                QMutexLocker l(&m_mutex);
9fb289
+                m_reading.storeRelease(true);
9fb289
+                m_cond.wakeOne();
9fb289
+                break;
9fb289
+            }
9fb289
+        }
9fb289
+    }
9fb289
+
9fb289
+    void stop()
9fb289
+    {
9fb289
+        // We have to both write to the pipe and set the flag, as the thread may be
9fb289
+        // either in the poll() or waiting for _prepare_read().
9fb289
+        if (m_pipefd[1] != -1 && write(m_pipefd[1], "\0", 1) == -1)
9fb289
+            qWarning("Failed to write to the pipe: %s.", strerror(errno));
9fb289
+
9fb289
+        {
9fb289
+            QMutexLocker l(&m_mutex);
9fb289
+            m_quitting = true;
9fb289
+            m_cond.wakeOne();
9fb289
+        }
9fb289
+
9fb289
+        wait();
9fb289
+    }
9fb289
+
9fb289
+Q_SIGNALS:
9fb289
+    void needReadAndDispatch();
9fb289
+
9fb289
+protected:
9fb289
+    void run() override
9fb289
+    {
9fb289
+        // we use this pipe to make the loop exit otherwise if we simply used a flag on the loop condition, if stop() gets
9fb289
+        // called while poll() is blocking the thread will never quit since there are no wayland messages coming anymore.
9fb289
+        struct Pipe
9fb289
+        {
9fb289
+            Pipe(int *fds)
9fb289
+                : fds(fds)
9fb289
+            {
9fb289
+                if (qt_safe_pipe(fds) != 0)
9fb289
+                    qWarning("Pipe creation failed. Quitting may hang.");
9fb289
+            }
9fb289
+            ~Pipe()
9fb289
+            {
9fb289
+                if (fds[0] != -1) {
9fb289
+                    close(fds[0]);
9fb289
+                    close(fds[1]);
9fb289
+                }
9fb289
+            }
9fb289
+
9fb289
+            int *fds;
9fb289
+        } pipe(m_pipefd);
9fb289
+
9fb289
+        // Make the main thread call wl_prepare_read(), dispatch the pending messages and flush the
9fb289
+        // outbound ones. Wait until it's done before proceeding, unless we're told to quit.
9fb289
+        while (waitForReading()) {
9fb289
+            pollfd fds[2] = { { m_fd, POLLIN, 0 }, { m_pipefd[0], POLLIN, 0 } };
9fb289
+            poll(fds, 2, -1);
9fb289
+
9fb289
+            if (fds[1].revents & POLLIN) {
9fb289
+                // we don't really care to read the byte that was written here since we're closing down
9fb289
+                wl_display_cancel_read(m_wldisplay);
9fb289
+                break;
9fb289
+            }
9fb289
+
9fb289
+            if (fds[0].revents & POLLIN)
9fb289
+                wl_display_read_events(m_wldisplay);
9fb289
+                // The polll was succesfull and the event thread did the wl_display_read_events(). On the next iteration of the loop
9fb289
+                // the event sent to the main thread will cause it to dispatch the messages just read, unless the loop exits in which
9fb289
+                // case we don't care anymore about them.
9fb289
+            else
9fb289
+                wl_display_cancel_read(m_wldisplay);
9fb289
+        }
9fb289
+    }
9fb289
+
9fb289
+private:
9fb289
+    bool waitForReading()
9fb289
+    {
9fb289
+        Q_ASSERT(QThread::currentThread() == this);
9fb289
+
9fb289
+        m_reading.storeRelease(false);
9fb289
+
9fb289
+        if (m_mode == SelfDispatch) {
9fb289
+            readAndDispatchEvents();
9fb289
+        } else {
9fb289
+            Q_EMIT needReadAndDispatch();
9fb289
+
9fb289
+            QMutexLocker lock(&m_mutex);
9fb289
+            // m_reading might be set from our emit or some other invocation of
9fb289
+            // readAndDispatchEvents().
9fb289
+            while (!m_reading.loadRelaxed() && !m_quitting)
9fb289
+                m_cond.wait(&m_mutex);
9fb289
+        }
9fb289
+
9fb289
+        return !m_quitting;
9fb289
+    }
9fb289
+
9fb289
+    int dispatchQueuePending()
9fb289
+    {
9fb289
+        if (m_wlevqueue)
9fb289
+            return wl_display_dispatch_queue_pending(m_wldisplay, m_wlevqueue);
9fb289
+        else
9fb289
+            return wl_display_dispatch_pending(m_wldisplay);
9fb289
+    }
9fb289
+
9fb289
+    int prepareReadQueue()
9fb289
+    {
9fb289
+        if (m_wlevqueue)
9fb289
+            return wl_display_prepare_read_queue(m_wldisplay, m_wlevqueue);
9fb289
+        else
9fb289
+            return wl_display_prepare_read(m_wldisplay);
9fb289
+    }
9fb289
+
9fb289
+    int m_fd;
9fb289
+    int m_pipefd[2];
9fb289
+    wl_display *m_wldisplay;
9fb289
+    wl_event_queue *m_wlevqueue;
9fb289
+    OperatingMode m_mode;
9fb289
+
9fb289
+    /* Concurrency note when operating in EmitToDispatch mode:
9fb289
+     * m_reading is set to false inside event thread's waitForReading(), and is
9fb289
+     * set to true inside main thread's readAndDispatchEvents().
9fb289
+     * The lock is not taken when setting m_reading to false, as the main thread
9fb289
+     * is not actively waiting for it to turn false. However, the lock is taken
9fb289
+     * inside readAndDispatchEvents() before setting m_reading to true,
9fb289
+     * as the event thread is actively waiting for it under the wait condition.
9fb289
+     */
9fb289
+
9fb289
+    QAtomicInteger<bool> m_reading;
9fb289
+    bool m_quitting;
9fb289
+    QMutex m_mutex;
9fb289
+    QWaitCondition m_cond;
9fb289
+};
9fb289
+
9fb289
 Q_LOGGING_CATEGORY(lcQpaWayland, "qt.qpa.wayland"); // for general (uncategorized) Wayland platform logging
9fb289
 
9fb289
 struct wl_surface *QWaylandDisplay::createSurface(void *handle)
9fb289
@@ -162,6 +355,12 @@ QWaylandDisplay::QWaylandDisplay(QWaylandIntegration *waylandIntegration)
9fb289
 
9fb289
 QWaylandDisplay::~QWaylandDisplay(void)
9fb289
 {
9fb289
+    if (m_eventThread)
9fb289
+        m_eventThread->stop();
9fb289
+
9fb289
+    if (m_frameEventQueueThread)
9fb289
+        m_frameEventQueueThread->stop();
9fb289
+
9fb289
     if (mSyncCallback)
9fb289
         wl_callback_destroy(mSyncCallback);
9fb289
 
9fb289
@@ -208,98 +407,37 @@ void QWaylandDisplay::ensureScreen()
9fb289
 
9fb289
 void QWaylandDisplay::checkError() const
9fb289
 {
9fb289
-    int ecode = wl_display_get_error(mDisplay);
9fb289
-    if ((ecode == EPIPE || ecode == ECONNRESET)) {
9fb289
-        // special case this to provide a nicer error
9fb289
-        qWarning("The Wayland connection broke. Did the Wayland compositor die?");
9fb289
-    } else {
9fb289
-        qWarning("The Wayland connection experienced a fatal error: %s", strerror(ecode));
9fb289
-    }
9fb289
-    _exit(1);
9fb289
+    checkWaylandError(mDisplay);
9fb289
 }
9fb289
 
9fb289
+// Called in main thread, either from queued signal or directly.
9fb289
 void QWaylandDisplay::flushRequests()
9fb289
 {
9fb289
-    if (wl_display_prepare_read(mDisplay) == 0) {
9fb289
-        wl_display_read_events(mDisplay);
9fb289
-    }
9fb289
-
9fb289
-    if (wl_display_dispatch_pending(mDisplay) < 0)
9fb289
-        checkError();
9fb289
-
9fb289
-    {
9fb289
-        QReadLocker locker(&m_frameQueueLock);
9fb289
-        for (const FrameQueue &q : mExternalQueues) {
9fb289
-            QMutexLocker locker(q.mutex);
9fb289
-            while (wl_display_prepare_read_queue(mDisplay, q.queue) != 0)
9fb289
-                wl_display_dispatch_queue_pending(mDisplay, q.queue);
9fb289
-            wl_display_read_events(mDisplay);
9fb289
-            wl_display_dispatch_queue_pending(mDisplay, q.queue);
9fb289
-        }
9fb289
-    }
9fb289
-
9fb289
-    wl_display_flush(mDisplay);
9fb289
-}
9fb289
-
9fb289
-void QWaylandDisplay::blockingReadEvents()
9fb289
-{
9fb289
-    if (wl_display_dispatch(mDisplay) < 0)
9fb289
-        checkError();
9fb289
-}
9fb289
-
9fb289
-void QWaylandDisplay::destroyFrameQueue(const QWaylandDisplay::FrameQueue &q)
9fb289
-{
9fb289
-    QWriteLocker locker(&m_frameQueueLock);
9fb289
-    auto it = std::find_if(mExternalQueues.begin(),
9fb289
-                           mExternalQueues.end(),
9fb289
-                           [&q] (const QWaylandDisplay::FrameQueue &other){ return other.queue == q.queue; });
9fb289
-    Q_ASSERT(it != mExternalQueues.end());
9fb289
-    mExternalQueues.erase(it);
9fb289
-    if (q.queue != nullptr)
9fb289
-        wl_event_queue_destroy(q.queue);
9fb289
-    delete q.mutex;
9fb289
+    m_eventThread->readAndDispatchEvents();
9fb289
 }
9fb289
 
9fb289
-QWaylandDisplay::FrameQueue QWaylandDisplay::createFrameQueue()
9fb289
+// We have to wait until we have an eventDispatcher before creating the eventThread,
9fb289
+// otherwise forceRoundTrip() may block inside _events_read() because eventThread is
9fb289
+// polling.
9fb289
+void QWaylandDisplay::initEventThread()
9fb289
 {
9fb289
-    QWriteLocker locker(&m_frameQueueLock);
9fb289
-    FrameQueue q{createEventQueue()};
9fb289
-    mExternalQueues.append(q);
9fb289
-    return q;
9fb289
-}
9fb289
+    m_eventThread.reset(
9fb289
+            new EventThread(mDisplay, /* default queue */ nullptr, EventThread::EmitToDispatch));
9fb289
+    connect(m_eventThread.get(), &EventThread::needReadAndDispatch, this,
9fb289
+            &QWaylandDisplay::flushRequests, Qt::QueuedConnection);
9fb289
+    m_eventThread->start();
9fb289
 
9fb289
-wl_event_queue *QWaylandDisplay::createEventQueue()
9fb289
-{
9fb289
-    return wl_display_create_queue(mDisplay);
9fb289
+    // wl_display_disconnect() free this.
9fb289
+    m_frameEventQueue = wl_display_create_queue(mDisplay);
9fb289
+    m_frameEventQueueThread.reset(
9fb289
+            new EventThread(mDisplay, m_frameEventQueue, EventThread::SelfDispatch));
9fb289
+    m_frameEventQueueThread->start();
9fb289
 }
9fb289
 
9fb289
-void QWaylandDisplay::dispatchQueueWhile(wl_event_queue *queue, std::function<bool ()> condition, int timeout)
9fb289
+void QWaylandDisplay::blockingReadEvents()
9fb289
 {
9fb289
-    if (!condition())
9fb289
-        return;
9fb289
-
9fb289
-    QElapsedTimer timer;
9fb289
-    timer.start();
9fb289
-    struct pollfd pFd = qt_make_pollfd(wl_display_get_fd(mDisplay), POLLIN);
9fb289
-    while (timeout == -1 || timer.elapsed() < timeout) {
9fb289
-        while (wl_display_prepare_read_queue(mDisplay, queue) != 0)
9fb289
-            wl_display_dispatch_queue_pending(mDisplay, queue);
9fb289
-
9fb289
-        wl_display_flush(mDisplay);
9fb289
-
9fb289
-        const int remaining = qMax(timeout - timer.elapsed(), 0ll);
9fb289
-        const int pollTimeout = timeout == -1 ? -1 : remaining;
9fb289
-        if (qt_poll_msecs(&pFd, 1, pollTimeout) > 0)
9fb289
-            wl_display_read_events(mDisplay);
9fb289
-        else
9fb289
-            wl_display_cancel_read(mDisplay);
9fb289
-
9fb289
-        if (wl_display_dispatch_queue_pending(mDisplay, queue) < 0)
9fb289
-            checkError();
9fb289
-
9fb289
-        if (!condition())
9fb289
-            break;
9fb289
-    }
9fb289
+    if (wl_display_dispatch(mDisplay) < 0)
9fb289
+        checkWaylandError(mDisplay);
9fb289
 }
9fb289
 
9fb289
 QWaylandScreen *QWaylandDisplay::screenForOutput(struct wl_output *output) const
9fb289
@@ -674,4 +812,6 @@ QWaylandCursorTheme *QWaylandDisplay::loadCursorTheme(const QString &name, int p
9fb289
 
9fb289
 } // namespace QtWaylandClient
9fb289
 
9fb289
+#include "qwaylanddisplay.moc"
9fb289
+
9fb289
 QT_END_NAMESPACE
9fb289
diff --git a/src/client/qwaylanddisplay_p.h b/src/client/qwaylanddisplay_p.h
9fb289
index d9c8849f..42bc661d 100644
9fb289
--- a/src/client/qwaylanddisplay_p.h
9fb289
+++ b/src/client/qwaylanddisplay_p.h
9fb289
@@ -109,6 +109,7 @@ class QWaylandSurface;
9fb289
 class QWaylandShellIntegration;
9fb289
 class QWaylandCursor;
9fb289
 class QWaylandCursorTheme;
9fb289
+class EventThread;
9fb289
 
9fb289
 typedef void (*RegistryListener)(void *data,
9fb289
                                  struct wl_registry *registry,
9fb289
@@ -120,12 +121,6 @@ class Q_WAYLAND_CLIENT_EXPORT QWaylandDisplay : public QObject, public QtWayland
9fb289
     Q_OBJECT
9fb289
 
9fb289
 public:
9fb289
-    struct FrameQueue {
9fb289
-        FrameQueue(wl_event_queue *q = nullptr) : queue(q), mutex(new QMutex) {}
9fb289
-        wl_event_queue *queue;
9fb289
-        QMutex *mutex;
9fb289
-    };
9fb289
-
9fb289
     QWaylandDisplay(QWaylandIntegration *waylandIntegration);
9fb289
     ~QWaylandDisplay(void) override;
9fb289
 
9fb289
@@ -212,12 +207,11 @@ public:
9fb289
     void handleKeyboardFocusChanged(QWaylandInputDevice *inputDevice);
9fb289
     void handleWindowDestroyed(QWaylandWindow *window);
9fb289
 
9fb289
-    wl_event_queue *createEventQueue();
9fb289
-    FrameQueue createFrameQueue();
9fb289
-    void destroyFrameQueue(const FrameQueue &q);
9fb289
-    void dispatchQueueWhile(wl_event_queue *queue, std::function<bool()> condition, int timeout = -1);
9fb289
+    wl_event_queue *frameEventQueue() { return m_frameEventQueue; };
9fb289
 
9fb289
     bool isKeyboardAvailable() const;
9fb289
+
9fb289
+    void initEventThread();
9fb289
 public slots:
9fb289
     void blockingReadEvents();
9fb289
     void flushRequests();
9fb289
@@ -240,6 +234,9 @@ private:
9fb289
     };
9fb289
 
9fb289
     struct wl_display *mDisplay = nullptr;
9fb289
+    QScopedPointer<EventThread> m_eventThread;
9fb289
+    wl_event_queue *m_frameEventQueue = nullptr;
9fb289
+    QScopedPointer<EventThread> m_frameEventQueueThread;
9fb289
     QtWayland::wl_compositor mCompositor;
9fb289
     QScopedPointer<QWaylandShm> mShm;
9fb289
     QList<QWaylandScreen *> mWaitingScreens;
9fb289
@@ -276,11 +273,9 @@ private:
9fb289
     QWaylandInputDevice *mLastInputDevice = nullptr;
9fb289
     QPointer<QWaylandWindow> mLastInputWindow;
9fb289
     QPointer<QWaylandWindow> mLastKeyboardFocus;
9fb289
-    QVector<QWaylandWindow *> mActiveWindows;
9fb289
-    QVector<FrameQueue> mExternalQueues;
9fb289
+    QList<QWaylandWindow *> mActiveWindows;
9fb289
     struct wl_callback *mSyncCallback = nullptr;
9fb289
     static const wl_callback_listener syncCallbackListener;
9fb289
-    QReadWriteLock m_frameQueueLock;
9fb289
 
9fb289
     bool mClientSideInputContextRequested = !QPlatformInputContextFactory::requested().isNull();
9fb289
 
9fb289
diff --git a/src/client/qwaylandintegration.cpp b/src/client/qwaylandintegration.cpp
9fb289
index 3a6fa651..3b876047 100644
9fb289
--- a/src/client/qwaylandintegration.cpp
9fb289
+++ b/src/client/qwaylandintegration.cpp
9fb289
@@ -192,9 +192,7 @@ QAbstractEventDispatcher *QWaylandIntegration::createEventDispatcher() const
9fb289
 
9fb289
 void QWaylandIntegration::initialize()
9fb289
 {
9fb289
-    int fd = wl_display_get_fd(mDisplay->wl_display());
9fb289
-    QSocketNotifier *sn = new QSocketNotifier(fd, QSocketNotifier::Read, mDisplay.data());
9fb289
-    QObject::connect(sn, SIGNAL(activated(QSocketDescriptor)), mDisplay.data(), SLOT(flushRequests()));
9fb289
+    mDisplay->initEventThread();
9fb289
 
9fb289
     // Call after eventDispatcher is fully connected, for QWaylandDisplay::forceRoundTrip()
9fb289
     mDisplay->initialize();
9fb289
diff --git a/src/client/qwaylandwindow.cpp b/src/client/qwaylandwindow.cpp
9fb289
index 1597f67e..7de19a74 100644
9fb289
--- a/src/client/qwaylandwindow.cpp
9fb289
+++ b/src/client/qwaylandwindow.cpp
9fb289
@@ -76,7 +76,6 @@ QWaylandWindow *QWaylandWindow::mMouseGrab = nullptr;
9fb289
 QWaylandWindow::QWaylandWindow(QWindow *window, QWaylandDisplay *display)
9fb289
     : QPlatformWindow(window)
9fb289
     , mDisplay(display)
9fb289
-    , mFrameQueue(mDisplay->createFrameQueue())
9fb289
     , mResizeAfterSwap(qEnvironmentVariableIsSet("QT_WAYLAND_RESIZE_AFTER_SWAP"))
9fb289
 {
9fb289
     {
9fb289
@@ -95,8 +94,6 @@ QWaylandWindow::QWaylandWindow(QWindow *window, QWaylandDisplay *display)
9fb289
 
9fb289
 QWaylandWindow::~QWaylandWindow()
9fb289
 {
9fb289
-    mDisplay->destroyFrameQueue(mFrameQueue);
9fb289
-
9fb289
     delete mWindowDecoration;
9fb289
 
9fb289
     if (mSurface)
9fb289
@@ -635,6 +632,8 @@ const wl_callback_listener QWaylandWindow::callbackListener = {
9fb289
 
9fb289
 void QWaylandWindow::handleFrameCallback()
9fb289
 {
9fb289
+    QMutexLocker locker(&mFrameSyncMutex);
9fb289
+
9fb289
     mWaitingForFrameCallback = false;
9fb289
     mFrameCallbackElapsedTimer.invalidate();
9fb289
 
9fb289
@@ -656,12 +655,16 @@ void QWaylandWindow::handleFrameCallback()
9fb289
         mWaitingForUpdateDelivery = true;
9fb289
         QMetaObject::invokeMethod(this, doHandleExpose, Qt::QueuedConnection);
9fb289
     }
9fb289
+
9fb289
+    mFrameSyncWait.notify_all();
9fb289
 }
9fb289
 
9fb289
 bool QWaylandWindow::waitForFrameSync(int timeout)
9fb289
 {
9fb289
-    QMutexLocker locker(mFrameQueue.mutex);
9fb289
-    mDisplay->dispatchQueueWhile(mFrameQueue.queue, [&]() { return mWaitingForFrameCallback; }, timeout);
9fb289
+    QMutexLocker locker(&mFrameSyncMutex);
9fb289
+
9fb289
+    QDeadlineTimer deadline(timeout);
9fb289
+    while (mWaitingForFrameCallback && mFrameSyncWait.wait(&mFrameSyncMutex, deadline)) { }
9fb289
 
9fb289
     if (mWaitingForFrameCallback) {
9fb289
         qCDebug(lcWaylandBackingstore) << "Didn't receive frame callback in time, window should now be inexposed";
9fb289
@@ -1157,8 +1160,11 @@ void QWaylandWindow::requestUpdate()
9fb289
     Q_ASSERT(hasPendingUpdateRequest()); // should be set by QPA
9fb289
 
9fb289
     // If we have a frame callback all is good and will be taken care of there
9fb289
-    if (mWaitingForFrameCallback)
9fb289
-        return;
9fb289
+    {
9fb289
+        QMutexLocker locker(&mFrameSyncMutex);
9fb289
+        if (mWaitingForFrameCallback)
9fb289
+            return;
9fb289
+    }
9fb289
 
9fb289
     // If we've already called deliverUpdateRequest(), but haven't seen any attach+commit/swap yet
9fb289
     // This is a somewhat redundant behavior and might indicate a bug in the calling code, so log
9fb289
@@ -1171,7 +1177,12 @@ void QWaylandWindow::requestUpdate()
9fb289
     // so use invokeMethod to delay the delivery a bit.
9fb289
     QMetaObject::invokeMethod(this, [this] {
9fb289
         // Things might have changed in the meantime
9fb289
-        if (hasPendingUpdateRequest() && !mWaitingForFrameCallback)
9fb289
+        {
9fb289
+            QMutexLocker locker(&mFrameSyncMutex);
9fb289
+            if (mWaitingForFrameCallback)
9fb289
+                return;
9fb289
+        }
9fb289
+        if (hasPendingUpdateRequest())
9fb289
             deliverUpdateRequest();
9fb289
     }, Qt::QueuedConnection);
9fb289
 }
9fb289
@@ -1191,9 +1202,10 @@ void QWaylandWindow::handleUpdate()
9fb289
     if (!mSurface)
9fb289
         return;
9fb289
 
9fb289
-    QMutexLocker locker(mFrameQueue.mutex);
9fb289
+    QMutexLocker locker(&mFrameSyncMutex);
9fb289
+
9fb289
     struct ::wl_surface *wrappedSurface = reinterpret_cast<struct ::wl_surface *>(wl_proxy_create_wrapper(mSurface->object()));
9fb289
-    wl_proxy_set_queue(reinterpret_cast<wl_proxy *>(wrappedSurface), mFrameQueue.queue);
9fb289
+    wl_proxy_set_queue(reinterpret_cast<wl_proxy *>(wrappedSurface), mDisplay->frameEventQueue());
9fb289
     mFrameCallback = wl_surface_frame(wrappedSurface);
9fb289
     wl_proxy_wrapper_destroy(wrappedSurface);
9fb289
     wl_callback_add_listener(mFrameCallback, &QWaylandWindow::callbackListener, this);
9fb289
@@ -1203,6 +1215,8 @@ void QWaylandWindow::handleUpdate()
9fb289
     // Start a timer for handling the case when the compositor stops sending frame callbacks.
9fb289
     if (mFrameCallbackTimeout > 0) {
9fb289
         QMetaObject::invokeMethod(this, [this] {
9fb289
+            QMutexLocker locker(&mFrameSyncMutex);
9fb289
+
9fb289
             if (mWaitingForFrameCallback) {
9fb289
                 if (mFrameCallbackCheckIntervalTimerId < 0)
9fb289
                     mFrameCallbackCheckIntervalTimerId = startTimer(mFrameCallbackTimeout);
9fb289
diff --git a/src/client/qwaylandwindow_p.h b/src/client/qwaylandwindow_p.h
9fb289
index e0687962..d45980a8 100644
9fb289
--- a/src/client/qwaylandwindow_p.h
9fb289
+++ b/src/client/qwaylandwindow_p.h
9fb289
@@ -232,7 +232,7 @@ protected:
9fb289
     int mFrameCallbackCheckIntervalTimerId = -1;
9fb289
     QElapsedTimer mFrameCallbackElapsedTimer;
9fb289
     struct ::wl_callback *mFrameCallback = nullptr;
9fb289
-    QWaylandDisplay::FrameQueue mFrameQueue;
9fb289
+    QMutex mFrameSyncMutex;
9fb289
     QWaitCondition mFrameSyncWait;
9fb289
 
9fb289
     // True when we have called deliverRequestUpdate, but the client has not yet attached a new buffer
9fb289
-- 
9fb289
2.35.1
9fb289