Create a VsyncSchedule per display
In order to determine the vsync offsets between displays, keep track of
a VsyncSchedule for each display.
Store the VsyncSchedules in a SmallMap. Update getVsyncSchedule with a
parameter to choose the display. The default parameter uses the leader's
display, which is what current external callers want.
Update VsyncDispatches when the leader changes, so that they are always
listening to the leader.
Enable and disable vsync callbacks per display. Earlier attempts to turn
them on and off together could leave a secondary display on a bad
schedule. Move state and logic for enabling/disabling the callbacks into
VsyncSchedule. Add a method for resyncing all displays at once.
Use std::shared_ptrs for VsyncDispatches. This prevents lifetime issues
if a VsyncSchedule gets removed while its VsyncDispatch is still in use.
Same for VsyncTracker, which is referenced by VsyncDispatch.
When the leader VsyncSchedule changes, call cancel on
VsyncCallbackRegistrations and replace them with new ones using the new
VsyncDispatches. If a callback was scheduled, schedule a new one.
Update VsyncSchedule's members' traces so that there is a separate track
for each display.
Move SF's record of the last HWC Vsync states into VsyncSchedule, so it
sits with other related logic. Remove the pending HWC Vsync state, which
did not affect behavior.
For refresh rate changes, modulate vsync config based on the leader
display. When switching leaders, force a period transition to ensure
that a potential refresh rate change is completed.
Bug: 255601557
Bug: 256196556
Bug: 241285473
Bug: 241286146
Test: libsurfaceflinger_unittest
Test: manual (look at perfetto traces)
Change-Id: If60218e85292c786b9fa70ecb33ee374d3a385e0
diff --git a/services/surfaceflinger/CompositionEngine/tests/MockHWComposer.h b/services/surfaceflinger/CompositionEngine/tests/MockHWComposer.h
index 6199a5a..933f616 100644
--- a/services/surfaceflinger/CompositionEngine/tests/MockHWComposer.h
+++ b/services/surfaceflinger/CompositionEngine/tests/MockHWComposer.h
@@ -93,7 +93,7 @@
MOCK_METHOD2(onHotplug,
std::optional<DisplayIdentificationInfo>(hal::HWDisplayId, hal::Connection));
MOCK_CONST_METHOD0(updatesDeviceProductInfoOnHotplugReconnect, bool());
- MOCK_METHOD2(onVsync, bool(hal::HWDisplayId, int64_t));
+ MOCK_METHOD(std::optional<PhysicalDisplayId>, onVsync, (hal::HWDisplayId, int64_t));
MOCK_METHOD2(setVsyncEnabled, void(PhysicalDisplayId, hal::Vsync));
MOCK_CONST_METHOD1(isConnected, bool(PhysicalDisplayId));
MOCK_CONST_METHOD1(getModes, std::vector<HWComposer::HWCDisplayMode>(PhysicalDisplayId));
diff --git a/services/surfaceflinger/DisplayHardware/HWComposer.cpp b/services/surfaceflinger/DisplayHardware/HWComposer.cpp
index 7dde6b4..8e74716 100644
--- a/services/surfaceflinger/DisplayHardware/HWComposer.cpp
+++ b/services/surfaceflinger/DisplayHardware/HWComposer.cpp
@@ -30,6 +30,7 @@
#include <compositionengine/Output.h>
#include <compositionengine/OutputLayer.h>
#include <compositionengine/impl/OutputLayerCompositionState.h>
+#include <ftl/concat.h>
#include <log/log.h>
#include <ui/DebugUtils.h>
#include <ui/GraphicBuffer.h>
@@ -148,16 +149,17 @@
return mUpdateDeviceProductInfoOnHotplugReconnect;
}
-bool HWComposer::onVsync(hal::HWDisplayId hwcDisplayId, nsecs_t timestamp) {
- const auto displayId = toPhysicalDisplayId(hwcDisplayId);
- if (!displayId) {
+std::optional<PhysicalDisplayId> HWComposer::onVsync(hal::HWDisplayId hwcDisplayId,
+ nsecs_t timestamp) {
+ const auto displayIdOpt = toPhysicalDisplayId(hwcDisplayId);
+ if (!displayIdOpt) {
LOG_HWC_DISPLAY_ERROR(hwcDisplayId, "Invalid HWC display");
- return false;
+ return {};
}
- RETURN_IF_INVALID_DISPLAY(*displayId, false);
+ RETURN_IF_INVALID_DISPLAY(*displayIdOpt, {});
- auto& displayData = mDisplayData[*displayId];
+ auto& displayData = mDisplayData[*displayIdOpt];
{
// There have been reports of HWCs that signal several vsync events
@@ -166,18 +168,18 @@
// out here so they don't cause havoc downstream.
if (timestamp == displayData.lastPresentTimestamp) {
ALOGW("Ignoring duplicate VSYNC event from HWC for display %s (t=%" PRId64 ")",
- to_string(*displayId).c_str(), timestamp);
- return false;
+ to_string(*displayIdOpt).c_str(), timestamp);
+ return {};
}
displayData.lastPresentTimestamp = timestamp;
}
- const auto tag = "HW_VSYNC_" + to_string(*displayId);
+ const ftl::Concat tag("HW_VSYNC_", displayIdOpt->value);
ATRACE_INT(tag.c_str(), displayData.vsyncTraceToggle);
displayData.vsyncTraceToggle = !displayData.vsyncTraceToggle;
- return true;
+ return displayIdOpt;
}
size_t HWComposer::getMaxVirtualDisplayCount() const {
diff --git a/services/surfaceflinger/DisplayHardware/HWComposer.h b/services/surfaceflinger/DisplayHardware/HWComposer.h
index f6155d2..acebfb2 100644
--- a/services/surfaceflinger/DisplayHardware/HWComposer.h
+++ b/services/surfaceflinger/DisplayHardware/HWComposer.h
@@ -221,7 +221,10 @@
// TODO(b/157555476): Remove when the framework has proper support for headless mode
virtual bool updatesDeviceProductInfoOnHotplugReconnect() const = 0;
- virtual bool onVsync(hal::HWDisplayId, nsecs_t timestamp) = 0;
+ // Called when a vsync happens. If the vsync is valid, returns the
+ // corresponding PhysicalDisplayId. Otherwise returns nullopt.
+ virtual std::optional<PhysicalDisplayId> onVsync(hal::HWDisplayId, nsecs_t timestamp) = 0;
+
virtual void setVsyncEnabled(PhysicalDisplayId, hal::Vsync enabled) = 0;
virtual bool isConnected(PhysicalDisplayId) const = 0;
@@ -402,7 +405,7 @@
bool updatesDeviceProductInfoOnHotplugReconnect() const override;
- bool onVsync(hal::HWDisplayId, nsecs_t timestamp) override;
+ std::optional<PhysicalDisplayId> onVsync(hal::HWDisplayId, nsecs_t timestamp) override;
void setVsyncEnabled(PhysicalDisplayId, hal::Vsync enabled) override;
bool isConnected(PhysicalDisplayId) const override;
diff --git a/services/surfaceflinger/Scheduler/EventThread.cpp b/services/surfaceflinger/Scheduler/EventThread.cpp
index a902a8e..76e9416 100644
--- a/services/surfaceflinger/Scheduler/EventThread.cpp
+++ b/services/surfaceflinger/Scheduler/EventThread.cpp
@@ -238,29 +238,19 @@
namespace impl {
-EventThread::EventThread(const char* name, scheduler::VsyncSchedule& vsyncSchedule,
+EventThread::EventThread(const char* name, std::shared_ptr<scheduler::VsyncSchedule> vsyncSchedule,
+ IEventThreadCallback& eventThreadCallback,
android::frametimeline::TokenManager* tokenManager,
- ThrottleVsyncCallback throttleVsyncCallback,
- GetVsyncPeriodFunction getVsyncPeriodFunction,
std::chrono::nanoseconds workDuration,
std::chrono::nanoseconds readyDuration)
: mThreadName(name),
mVsyncTracer(base::StringPrintf("VSYNC-%s", name), 0),
mWorkDuration(base::StringPrintf("VsyncWorkDuration-%s", name), workDuration),
mReadyDuration(readyDuration),
- mVsyncSchedule(vsyncSchedule),
- mVsyncRegistration(
- vsyncSchedule.getDispatch(),
- [this](nsecs_t vsyncTime, nsecs_t wakeupTime, nsecs_t readyTime) {
- onVsync(vsyncTime, wakeupTime, readyTime);
- },
- name),
+ mVsyncSchedule(std::move(vsyncSchedule)),
+ mVsyncRegistration(mVsyncSchedule->getDispatch(), createDispatchCallback(), name),
mTokenManager(tokenManager),
- mThrottleVsyncCallback(std::move(throttleVsyncCallback)),
- mGetVsyncPeriodFunction(std::move(getVsyncPeriodFunction)) {
- LOG_ALWAYS_FATAL_IF(getVsyncPeriodFunction == nullptr,
- "getVsyncPeriodFunction must not be null");
-
+ mEventThreadCallback(eventThreadCallback) {
mThread = std::thread([this]() NO_THREAD_SAFETY_ANALYSIS {
std::unique_lock<std::mutex> lock(mMutex);
threadMain(lock);
@@ -371,16 +361,16 @@
}
VsyncEventData vsyncEventData;
- nsecs_t frameInterval = mGetVsyncPeriodFunction(connection->mOwnerUid);
- vsyncEventData.frameInterval = frameInterval;
+ const Fps frameInterval = mEventThreadCallback.getLeaderRenderFrameRate(connection->mOwnerUid);
+ vsyncEventData.frameInterval = frameInterval.getPeriodNsecs();
const auto [presentTime, deadline] = [&]() -> std::pair<nsecs_t, nsecs_t> {
std::lock_guard<std::mutex> lock(mMutex);
- const auto vsyncTime = mVsyncSchedule.getTracker().nextAnticipatedVSyncTimeFrom(
+ const auto vsyncTime = mVsyncSchedule->getTracker().nextAnticipatedVSyncTimeFrom(
systemTime() + mWorkDuration.get().count() + mReadyDuration.count());
return {vsyncTime, vsyncTime - mReadyDuration.count()};
}();
- generateFrameTimeline(vsyncEventData, frameInterval, systemTime(SYSTEM_TIME_MONOTONIC),
- presentTime, deadline);
+ generateFrameTimeline(vsyncEventData, frameInterval.getPeriodNsecs(),
+ systemTime(SYSTEM_TIME_MONOTONIC), presentTime, deadline);
return vsyncEventData;
}
@@ -541,9 +531,11 @@
bool EventThread::shouldConsumeEvent(const DisplayEventReceiver::Event& event,
const sp<EventThreadConnection>& connection) const {
const auto throttleVsync = [&] {
- return mThrottleVsyncCallback &&
- mThrottleVsyncCallback(event.vsync.vsyncData.preferredExpectedPresentationTime(),
- connection->mOwnerUid);
+ const auto& vsyncData = event.vsync.vsyncData;
+ const auto expectedPresentTime =
+ TimePoint::fromNs(vsyncData.preferredExpectedPresentationTime());
+ return !mEventThreadCallback.isVsyncTargetForUid(expectedPresentTime,
+ connection->mOwnerUid);
};
switch (event.header.type) {
@@ -631,9 +623,11 @@
for (const auto& consumer : consumers) {
DisplayEventReceiver::Event copy = event;
if (event.header.type == DisplayEventReceiver::DISPLAY_EVENT_VSYNC) {
- const int64_t frameInterval = mGetVsyncPeriodFunction(consumer->mOwnerUid);
- copy.vsync.vsyncData.frameInterval = frameInterval;
- generateFrameTimeline(copy.vsync.vsyncData, frameInterval, copy.header.timestamp,
+ const Fps frameInterval =
+ mEventThreadCallback.getLeaderRenderFrameRate(consumer->mOwnerUid);
+ copy.vsync.vsyncData.frameInterval = frameInterval.getPeriodNsecs();
+ generateFrameTimeline(copy.vsync.vsyncData, frameInterval.getPeriodNsecs(),
+ copy.header.timestamp,
event.vsync.vsyncData.preferredExpectedPresentationTime(),
event.vsync.vsyncData.preferredDeadlineTimestamp());
}
@@ -699,6 +693,26 @@
}
}
+void EventThread::onNewVsyncSchedule(std::shared_ptr<scheduler::VsyncSchedule> schedule) {
+ std::lock_guard<std::mutex> lock(mMutex);
+ const bool reschedule = mVsyncRegistration.cancel() == scheduler::CancelResult::Cancelled;
+ mVsyncSchedule = std::move(schedule);
+ mVsyncRegistration =
+ scheduler::VSyncCallbackRegistration(mVsyncSchedule->getDispatch(),
+ createDispatchCallback(), mThreadName);
+ if (reschedule) {
+ mVsyncRegistration.schedule({.workDuration = mWorkDuration.get().count(),
+ .readyDuration = mReadyDuration.count(),
+ .earliestVsync = mLastVsyncCallbackTime.ns()});
+ }
+}
+
+scheduler::VSyncDispatch::Callback EventThread::createDispatchCallback() {
+ return [this](nsecs_t vsyncTime, nsecs_t wakeupTime, nsecs_t readyTime) {
+ onVsync(vsyncTime, wakeupTime, readyTime);
+ };
+}
+
} // namespace impl
} // namespace android
diff --git a/services/surfaceflinger/Scheduler/EventThread.h b/services/surfaceflinger/Scheduler/EventThread.h
index ab9085e..b86553b 100644
--- a/services/surfaceflinger/Scheduler/EventThread.h
+++ b/services/surfaceflinger/Scheduler/EventThread.h
@@ -23,6 +23,7 @@
#include <sys/types.h>
#include <utils/Errors.h>
+#include <scheduler/Fps.h>
#include <scheduler/FrameRateMode.h>
#include <condition_variable>
#include <cstdint>
@@ -67,6 +68,15 @@
// Subsequent values are periods.
};
+class IEventThreadCallback {
+public:
+ virtual ~IEventThreadCallback() = default;
+
+ virtual bool isVsyncTargetForUid(TimePoint expectedVsyncTime, uid_t uid) const = 0;
+
+ virtual Fps getLeaderRenderFrameRate(uid_t uid) const = 0;
+};
+
class EventThreadConnection : public gui::BnDisplayEventConnection {
public:
EventThreadConnection(EventThread*, uid_t callingUid, ResyncCallback,
@@ -133,18 +143,17 @@
// Retrieves the number of event connections tracked by this EventThread.
virtual size_t getEventThreadConnectionCount() = 0;
+
+ virtual void onNewVsyncSchedule(std::shared_ptr<scheduler::VsyncSchedule>) = 0;
};
namespace impl {
class EventThread : public android::EventThread {
public:
- using ThrottleVsyncCallback = std::function<bool(nsecs_t, uid_t)>;
- using GetVsyncPeriodFunction = std::function<nsecs_t(uid_t)>;
-
- EventThread(const char* name, scheduler::VsyncSchedule&, frametimeline::TokenManager*,
- ThrottleVsyncCallback, GetVsyncPeriodFunction,
- std::chrono::nanoseconds workDuration, std::chrono::nanoseconds readyDuration);
+ EventThread(const char* name, std::shared_ptr<scheduler::VsyncSchedule>, IEventThreadCallback&,
+ frametimeline::TokenManager*, std::chrono::nanoseconds workDuration,
+ std::chrono::nanoseconds readyDuration);
~EventThread();
sp<EventThreadConnection> createEventConnection(
@@ -176,6 +185,8 @@
size_t getEventThreadConnectionCount() override;
+ void onNewVsyncSchedule(std::shared_ptr<scheduler::VsyncSchedule>) override;
+
private:
friend EventThreadTest;
@@ -199,17 +210,19 @@
nsecs_t timestamp, nsecs_t preferredExpectedPresentationTime,
nsecs_t preferredDeadlineTimestamp) const;
+ scheduler::VSyncDispatch::Callback createDispatchCallback();
+
const char* const mThreadName;
TracedOrdinal<int> mVsyncTracer;
TracedOrdinal<std::chrono::nanoseconds> mWorkDuration GUARDED_BY(mMutex);
std::chrono::nanoseconds mReadyDuration GUARDED_BY(mMutex);
- scheduler::VsyncSchedule& mVsyncSchedule;
+ std::shared_ptr<scheduler::VsyncSchedule> mVsyncSchedule;
TimePoint mLastVsyncCallbackTime GUARDED_BY(mMutex) = TimePoint::now();
scheduler::VSyncCallbackRegistration mVsyncRegistration GUARDED_BY(mMutex);
frametimeline::TokenManager* const mTokenManager;
- const ThrottleVsyncCallback mThrottleVsyncCallback;
- const GetVsyncPeriodFunction mGetVsyncPeriodFunction;
+ // mEventThreadCallback will outlive the EventThread.
+ IEventThreadCallback& mEventThreadCallback;
std::thread mThread;
mutable std::mutex mMutex;
diff --git a/services/surfaceflinger/Scheduler/ISchedulerCallback.h b/services/surfaceflinger/Scheduler/ISchedulerCallback.h
new file mode 100644
index 0000000..92c2189
--- /dev/null
+++ b/services/surfaceflinger/Scheduler/ISchedulerCallback.h
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2023 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <vector>
+
+#include <ui/DisplayId.h>
+
+#include "Display/DisplayModeRequest.h"
+
+namespace android::scheduler {
+
+struct ISchedulerCallback {
+ virtual void setVsyncEnabled(PhysicalDisplayId, bool) = 0;
+ virtual void requestDisplayModes(std::vector<display::DisplayModeRequest>) = 0;
+ virtual void kernelTimerChanged(bool expired) = 0;
+ virtual void triggerOnFrameRateOverridesChanged() = 0;
+
+protected:
+ ~ISchedulerCallback() = default;
+};
+
+} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/MessageQueue.cpp b/services/surfaceflinger/Scheduler/MessageQueue.cpp
index dec8f59..925f739 100644
--- a/services/surfaceflinger/Scheduler/MessageQueue.cpp
+++ b/services/surfaceflinger/Scheduler/MessageQueue.cpp
@@ -75,19 +75,37 @@
mHandler->dispatchFrame(vsyncId, expectedVsyncTime);
}
-void MessageQueue::initVsync(scheduler::VSyncDispatch& dispatch,
+void MessageQueue::initVsync(std::shared_ptr<scheduler::VSyncDispatch> dispatch,
frametimeline::TokenManager& tokenManager,
std::chrono::nanoseconds workDuration) {
std::lock_guard lock(mVsync.mutex);
mVsync.workDuration = workDuration;
mVsync.tokenManager = &tokenManager;
+ updateVsyncRegistrationLocked(std::move(dispatch));
+}
+
+void MessageQueue::updateVsyncRegistration(std::shared_ptr<scheduler::VSyncDispatch> dispatch) {
+ std::lock_guard lock(mVsync.mutex);
+ updateVsyncRegistrationLocked(std::move(dispatch));
+}
+
+void MessageQueue::updateVsyncRegistrationLocked(
+ std::shared_ptr<scheduler::VSyncDispatch> dispatch) {
+ const bool reschedule = mVsync.registration &&
+ mVsync.registration->cancel() == scheduler::CancelResult::Cancelled;
mVsync.registration = std::make_unique<
- scheduler::VSyncCallbackRegistration>(dispatch,
+ scheduler::VSyncCallbackRegistration>(std::move(dispatch),
std::bind(&MessageQueue::vsyncCallback, this,
std::placeholders::_1,
std::placeholders::_2,
std::placeholders::_3),
"sf");
+ if (reschedule) {
+ mVsync.scheduledFrameTime =
+ mVsync.registration->schedule({.workDuration = mVsync.workDuration.get().count(),
+ .readyDuration = 0,
+ .earliestVsync = mVsync.lastCallbackTime.ns()});
+ }
}
void MessageQueue::destroyVsync() {
diff --git a/services/surfaceflinger/Scheduler/MessageQueue.h b/services/surfaceflinger/Scheduler/MessageQueue.h
index 0d59337..ecb237d 100644
--- a/services/surfaceflinger/Scheduler/MessageQueue.h
+++ b/services/surfaceflinger/Scheduler/MessageQueue.h
@@ -65,7 +65,7 @@
public:
virtual ~MessageQueue() = default;
- virtual void initVsync(scheduler::VSyncDispatch&, frametimeline::TokenManager&,
+ virtual void initVsync(std::shared_ptr<scheduler::VSyncDispatch>, frametimeline::TokenManager&,
std::chrono::nanoseconds workDuration) = 0;
virtual void destroyVsync() = 0;
virtual void setDuration(std::chrono::nanoseconds workDuration) = 0;
@@ -106,6 +106,8 @@
void vsyncCallback(nsecs_t vsyncTime, nsecs_t targetWakeupTime, nsecs_t readyTime);
+ void updateVsyncRegistration(std::shared_ptr<scheduler::VSyncDispatch>) EXCLUDES(mVsync.mutex);
+
private:
virtual void onFrameSignal(ICompositor&, VsyncId, TimePoint expectedVsyncTime) = 0;
@@ -127,10 +129,13 @@
Vsync mVsync;
+ void updateVsyncRegistrationLocked(std::shared_ptr<scheduler::VSyncDispatch>)
+ REQUIRES(mVsync.mutex);
+
public:
explicit MessageQueue(ICompositor&);
- void initVsync(scheduler::VSyncDispatch&, frametimeline::TokenManager&,
+ void initVsync(std::shared_ptr<scheduler::VSyncDispatch>, frametimeline::TokenManager&,
std::chrono::nanoseconds workDuration) override;
void destroyVsync() override;
void setDuration(std::chrono::nanoseconds workDuration) override;
diff --git a/services/surfaceflinger/Scheduler/OneShotTimer.h b/services/surfaceflinger/Scheduler/OneShotTimer.h
index f95646c..02e8719 100644
--- a/services/surfaceflinger/Scheduler/OneShotTimer.h
+++ b/services/surfaceflinger/Scheduler/OneShotTimer.h
@@ -40,7 +40,7 @@
OneShotTimer(std::string name, const Interval& interval, const ResetCallback& resetCallback,
const TimeoutCallback& timeoutCallback,
- std::unique_ptr<Clock> clock = std::make_unique<SteadyClock>());
+ std::unique_ptr<android::Clock> clock = std::make_unique<SteadyClock>());
~OneShotTimer();
Duration interval() const { return mInterval; }
@@ -82,7 +82,7 @@
std::thread mThread;
// Clock object for the timer. Mocked in unit tests.
- std::unique_ptr<Clock> mClock;
+ std::unique_ptr<android::Clock> mClock;
// Semaphore to keep mThread synchronized.
sem_t mSemaphore;
diff --git a/services/surfaceflinger/Scheduler/Scheduler.cpp b/services/surfaceflinger/Scheduler/Scheduler.cpp
index 33c98ff..e6f4665 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.cpp
+++ b/services/surfaceflinger/Scheduler/Scheduler.cpp
@@ -113,10 +113,18 @@
}
void Scheduler::registerDisplay(PhysicalDisplayId displayId, RefreshRateSelectorPtr selectorPtr) {
+ registerDisplayInternal(displayId, std::move(selectorPtr),
+ std::make_shared<VsyncSchedule>(displayId, mFeatures));
+}
+
+void Scheduler::registerDisplayInternal(PhysicalDisplayId displayId,
+ RefreshRateSelectorPtr selectorPtr,
+ std::shared_ptr<VsyncSchedule> vsyncSchedule) {
demoteLeaderDisplay();
std::scoped_lock lock(mDisplayLock);
mRefreshRateSelectors.emplace_or_replace(displayId, std::move(selectorPtr));
+ mVsyncSchedules.emplace_or_replace(displayId, std::move(vsyncSchedule));
promoteLeaderDisplay();
}
@@ -126,6 +134,7 @@
std::scoped_lock lock(mDisplayLock);
mRefreshRateSelectors.erase(displayId);
+ mVsyncSchedules.erase(displayId);
// Do not allow removing the final display. Code in the scheduler expects
// there to be at least one display. (This may be relaxed in the future with
@@ -153,52 +162,49 @@
compositor.sample();
}
-void Scheduler::createVsyncSchedule(FeatureFlags features) {
- mVsyncSchedule.emplace(features);
+std::optional<Fps> Scheduler::getFrameRateOverride(uid_t uid) const {
+ std::scoped_lock lock(mDisplayLock);
+ return getFrameRateOverrideLocked(uid);
}
-std::optional<Fps> Scheduler::getFrameRateOverride(uid_t uid) const {
+std::optional<Fps> Scheduler::getFrameRateOverrideLocked(uid_t uid) const {
const bool supportsFrameRateOverrideByContent =
- leaderSelectorPtr()->supportsAppFrameRateOverrideByContent();
+ leaderSelectorPtrLocked()->supportsAppFrameRateOverrideByContent();
return mFrameRateOverrideMappings
.getFrameRateOverrideForUid(uid, supportsFrameRateOverrideByContent);
}
-bool Scheduler::isVsyncValid(TimePoint expectedVsyncTimestamp, uid_t uid) const {
+bool Scheduler::isVsyncTargetForUid(TimePoint expectedVsyncTime, uid_t uid) const {
const auto frameRate = getFrameRateOverride(uid);
if (!frameRate.has_value()) {
return true;
}
- return mVsyncSchedule->getTracker().isVSyncInPhase(expectedVsyncTimestamp.ns(), *frameRate);
+ return isVsyncInPhase(expectedVsyncTime, *frameRate);
}
-bool Scheduler::isVsyncInPhase(TimePoint timePoint, const Fps frameRate) const {
- return mVsyncSchedule->getTracker().isVSyncInPhase(timePoint.ns(), frameRate);
+bool Scheduler::isVsyncInPhase(TimePoint expectedVsyncTime, Fps frameRate) const {
+ return getVsyncSchedule()->getTracker().isVSyncInPhase(expectedVsyncTime.ns(), frameRate);
}
-impl::EventThread::ThrottleVsyncCallback Scheduler::makeThrottleVsyncCallback() const {
- return [this](nsecs_t expectedVsyncTimestamp, uid_t uid) {
- return !isVsyncValid(TimePoint::fromNs(expectedVsyncTimestamp), uid);
- };
-}
+Fps Scheduler::getLeaderRenderFrameRate(uid_t uid) const {
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+ auto vsyncSchedule = getVsyncScheduleLocked();
-impl::EventThread::GetVsyncPeriodFunction Scheduler::makeGetVsyncPeriodFunction() const {
- return [this](uid_t uid) {
- const Fps refreshRate = leaderSelectorPtr()->getActiveMode().fps;
- const nsecs_t currentPeriod = mVsyncSchedule->period().ns() ?: refreshRate.getPeriodNsecs();
+ const Fps refreshRate = leaderSelectorPtrLocked()->getActiveMode().fps;
+ const nsecs_t currentPeriod = vsyncSchedule->period().ns() ?: refreshRate.getPeriodNsecs();
- const auto frameRate = getFrameRateOverride(uid);
- if (!frameRate.has_value()) {
- return currentPeriod;
- }
+ const auto frameRate = getFrameRateOverrideLocked(uid);
+ if (!frameRate.has_value()) {
+ return Fps::fromPeriodNsecs(currentPeriod);
+ }
- const auto divisor = RefreshRateSelector::getFrameRateDivisor(refreshRate, *frameRate);
- if (divisor <= 1) {
- return currentPeriod;
- }
- return currentPeriod * divisor;
- };
+ const auto divisor = RefreshRateSelector::getFrameRateDivisor(refreshRate, *frameRate);
+ if (divisor <= 1) {
+ return Fps::fromPeriodNsecs(currentPeriod);
+ }
+ return Fps::fromPeriodNsecs(currentPeriod * divisor);
}
ConnectionHandle Scheduler::createEventThread(Cycle cycle,
@@ -206,9 +212,7 @@
std::chrono::nanoseconds workDuration,
std::chrono::nanoseconds readyDuration) {
auto eventThread = std::make_unique<impl::EventThread>(cycle == Cycle::Render ? "app" : "appSf",
- *mVsyncSchedule, tokenManager,
- makeThrottleVsyncCallback(),
- makeGetVsyncPeriodFunction(),
+ getVsyncSchedule(), *this, tokenManager,
workDuration, readyDuration);
auto& handle = cycle == Cycle::Render ? mAppConnectionHandle : mSfConnectionHandle;
@@ -265,7 +269,6 @@
thread = mConnections[handle].thread.get();
}
thread->onScreenAcquired();
- mScreenAcquired = true;
}
void Scheduler::onScreenReleased(ConnectionHandle handle) {
@@ -276,7 +279,6 @@
thread = mConnections[handle].thread.get();
}
thread->onScreenReleased();
- mScreenAcquired = false;
}
void Scheduler::onFrameRateOverridesChanged(ConnectionHandle handle, PhysicalDisplayId displayId) {
@@ -387,48 +389,57 @@
setDuration(config.sfWorkDuration);
}
-void Scheduler::enableHardwareVsync() {
- std::lock_guard<std::mutex> lock(mHWVsyncLock);
- if (!mPrimaryHWVsyncEnabled && mHWVsyncAvailable) {
- mVsyncSchedule->getTracker().resetModel();
- mSchedulerCallback.setVsyncEnabled(true);
- mPrimaryHWVsyncEnabled = true;
+void Scheduler::enableHardwareVsync(PhysicalDisplayId id) {
+ auto schedule = getVsyncSchedule(id);
+ schedule->enableHardwareVsync(mSchedulerCallback);
+}
+
+void Scheduler::disableHardwareVsync(PhysicalDisplayId id, bool disallow) {
+ auto schedule = getVsyncSchedule(id);
+ schedule->disableHardwareVsync(mSchedulerCallback, disallow);
+}
+
+void Scheduler::resyncAllToHardwareVsync(bool allowToEnable) {
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+
+ for (const auto& [id, _] : mRefreshRateSelectors) {
+ resyncToHardwareVsyncLocked(id, allowToEnable);
}
}
-void Scheduler::disableHardwareVsync(bool makeUnavailable) {
- std::lock_guard<std::mutex> lock(mHWVsyncLock);
- if (mPrimaryHWVsyncEnabled) {
- mSchedulerCallback.setVsyncEnabled(false);
- mPrimaryHWVsyncEnabled = false;
+void Scheduler::resyncToHardwareVsyncLocked(PhysicalDisplayId id, bool allowToEnable,
+ std::optional<Fps> refreshRate) {
+ if (!refreshRate) {
+ auto selectorPtr = mRefreshRateSelectors.get(id);
+ LOG_ALWAYS_FATAL_IF(!selectorPtr);
+ refreshRate = selectorPtr->get()->getActiveMode().modePtr->getFps();
}
- if (makeUnavailable) {
- mHWVsyncAvailable = false;
+ auto schedule = getVsyncScheduleLocked(id);
+ if (allowToEnable) {
+ schedule->allowHardwareVsync();
+ } else if (!schedule->isHardwareVsyncAllowed()) {
+ // Hardware vsync is not currently allowed, so abort the resync
+ // attempt for now.
+ return;
}
+
+ setVsyncPeriod(schedule, refreshRate->getPeriodNsecs(), false /* force */);
}
-void Scheduler::resyncToHardwareVsync(bool makeAvailable, Fps refreshRate) {
- {
- std::lock_guard<std::mutex> lock(mHWVsyncLock);
- if (makeAvailable) {
- mHWVsyncAvailable = makeAvailable;
- } else if (!mHWVsyncAvailable) {
- // Hardware vsync is not currently available, so abort the resync
- // attempt for now
- return;
- }
- }
+void Scheduler::setRenderRate(PhysicalDisplayId id, Fps renderFrameRate) {
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
- setVsyncPeriod(refreshRate.getPeriodNsecs());
-}
-
-void Scheduler::setRenderRate(Fps renderFrameRate) {
- const auto mode = leaderSelectorPtr()->getActiveMode();
+ auto selectorPtr = mRefreshRateSelectors.get(id);
+ LOG_ALWAYS_FATAL_IF(!selectorPtr);
+ const auto mode = selectorPtr->get()->getActiveMode();
using fps_approx_ops::operator!=;
LOG_ALWAYS_FATAL_IF(renderFrameRate != mode.fps,
- "Mismatch in render frame rates. Selector: %s, Scheduler: %s",
- to_string(mode.fps).c_str(), to_string(renderFrameRate).c_str());
+ "Mismatch in render frame rates. Selector: %s, Scheduler: %s, Display: "
+ "%" PRIu64,
+ to_string(mode.fps).c_str(), to_string(renderFrameRate).c_str(), id.value);
ALOGV("%s %s (%s)", __func__, to_string(mode.fps).c_str(),
to_string(mode.modePtr->getFps()).c_str());
@@ -437,7 +448,7 @@
LOG_ALWAYS_FATAL_IF(divisor == 0, "%s <> %s -- not divisors", to_string(mode.fps).c_str(),
to_string(mode.fps).c_str());
- mVsyncSchedule->getTracker().setDivisor(static_cast<unsigned>(divisor));
+ getVsyncScheduleLocked(id)->getTracker().setDivisor(static_cast<unsigned>(divisor));
}
void Scheduler::resync() {
@@ -447,49 +458,43 @@
const nsecs_t last = mLastResyncTime.exchange(now);
if (now - last > kIgnoreDelay) {
- const auto refreshRate = leaderSelectorPtr()->getActiveMode().modePtr->getFps();
- resyncToHardwareVsync(false, refreshRate);
+ resyncAllToHardwareVsync(false /* allowToEnable */);
}
}
-void Scheduler::setVsyncPeriod(nsecs_t period) {
+void Scheduler::setVsyncPeriod(const std::shared_ptr<VsyncSchedule>& schedule, nsecs_t period,
+ bool force) {
+ ALOGD("Scheduler::setVsyncPeriod");
if (period <= 0) return;
- std::lock_guard<std::mutex> lock(mHWVsyncLock);
- mVsyncSchedule->getController().startPeriodTransition(period);
-
- if (!mPrimaryHWVsyncEnabled) {
- mVsyncSchedule->getTracker().resetModel();
- mSchedulerCallback.setVsyncEnabled(true);
- mPrimaryHWVsyncEnabled = true;
- }
+ // TODO (b/266712910):The old code held mHWVsyncLock before calling
+ // startPeriodTransition. Move these into a new method on VsyncSchedule that
+ // encapsulates this behavior there and allows holding the lock the whole
+ // time.
+ schedule->getController().startPeriodTransition(period, force);
+ schedule->enableHardwareVsync(mSchedulerCallback);
}
-void Scheduler::addResyncSample(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriod,
- bool* periodFlushed) {
- bool needsHwVsync = false;
- *periodFlushed = false;
- { // Scope for the lock
- std::lock_guard<std::mutex> lock(mHWVsyncLock);
- if (mPrimaryHWVsyncEnabled) {
- needsHwVsync =
- mVsyncSchedule->getController().addHwVsyncTimestamp(timestamp, hwcVsyncPeriod,
- periodFlushed);
- }
+bool Scheduler::addResyncSample(PhysicalDisplayId id, nsecs_t timestamp,
+ std::optional<nsecs_t> hwcVsyncPeriod) {
+ bool periodFlushed = false;
+ auto schedule = getVsyncSchedule(id);
+ if (schedule->getController().addHwVsyncTimestamp(timestamp, hwcVsyncPeriod, &periodFlushed)) {
+ schedule->enableHardwareVsync(mSchedulerCallback);
+ } else {
+ schedule->disableHardwareVsync(mSchedulerCallback, false /* disallow */);
}
- if (needsHwVsync) {
- enableHardwareVsync();
- } else {
- disableHardwareVsync(false);
- }
+ return periodFlushed;
}
-void Scheduler::addPresentFence(std::shared_ptr<FenceTime> fence) {
- if (mVsyncSchedule->getController().addPresentFence(std::move(fence))) {
- enableHardwareVsync();
+void Scheduler::addPresentFence(PhysicalDisplayId id, std::shared_ptr<FenceTime> fence) {
+ auto schedule = getVsyncSchedule(id);
+ const bool needMoreSignals = schedule->getController().addPresentFence(std::move(fence));
+ if (needMoreSignals) {
+ schedule->enableHardwareVsync(mSchedulerCallback);
} else {
- disableHardwareVsync(false);
+ schedule->disableHardwareVsync(mSchedulerCallback, false /* disallow */);
}
}
@@ -541,12 +546,22 @@
}
}
-void Scheduler::setDisplayPowerMode(hal::PowerMode powerMode) {
- {
+void Scheduler::setDisplayPowerMode(PhysicalDisplayId id, hal::PowerMode powerMode) {
+ const bool isLeader = [this, id]() REQUIRES(kMainThreadContext) {
+ ftl::FakeGuard guard(mDisplayLock);
+ return id == mLeaderDisplayId;
+ }();
+ if (isLeader) {
+ // TODO (b/255657128): This needs to be handled per display.
std::lock_guard<std::mutex> lock(mPolicyLock);
mPolicy.displayPowerMode = powerMode;
}
- mVsyncSchedule->getController().setDisplayPowerMode(powerMode);
+ {
+ std::scoped_lock lock(mDisplayLock);
+ auto vsyncSchedule = getVsyncScheduleLocked(id);
+ vsyncSchedule->getController().setDisplayPowerMode(powerMode);
+ }
+ if (!isLeader) return;
if (mDisplayPowerTimer) {
mDisplayPowerTimer->reset();
@@ -557,6 +572,24 @@
mLayerHistory.clear();
}
+std::shared_ptr<const VsyncSchedule> Scheduler::getVsyncSchedule(
+ std::optional<PhysicalDisplayId> idOpt) const {
+ std::scoped_lock lock(mDisplayLock);
+ return getVsyncScheduleLocked(idOpt);
+}
+
+std::shared_ptr<const VsyncSchedule> Scheduler::getVsyncScheduleLocked(
+ std::optional<PhysicalDisplayId> idOpt) const {
+ ftl::FakeGuard guard(kMainThreadContext);
+ if (!idOpt) {
+ LOG_ALWAYS_FATAL_IF(!mLeaderDisplayId, "Missing a leader!");
+ idOpt = mLeaderDisplayId;
+ }
+ auto scheduleOpt = mVsyncSchedules.get(*idOpt);
+ LOG_ALWAYS_FATAL_IF(!scheduleOpt);
+ return std::const_pointer_cast<const VsyncSchedule>(scheduleOpt->get());
+}
+
void Scheduler::kernelIdleTimerCallback(TimerState state) {
ATRACE_INT("ExpiredKernelIdleTimer", static_cast<int>(state));
@@ -571,12 +604,17 @@
// If we're not in performance mode then the kernel timer shouldn't do
// anything, as the refresh rate during DPU power collapse will be the
// same.
- resyncToHardwareVsync(true /* makeAvailable */, refreshRate);
+ resyncAllToHardwareVsync(true /* allowToEnable */);
} else if (state == TimerState::Expired && refreshRate <= FPS_THRESHOLD_FOR_KERNEL_TIMER) {
// Disable HW VSYNC if the timer expired, as we don't need it enabled if
// we're not pushing frames, and if we're in PERFORMANCE mode then we'll
// need to update the VsyncController model anyway.
- disableHardwareVsync(false /* makeUnavailable */);
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+ constexpr bool disallow = false;
+ for (auto& [_, schedule] : mVsyncSchedules) {
+ schedule->disableHardwareVsync(mSchedulerCallback, disallow);
+ }
}
mSchedulerCallback.kernelTimerChanged(state == TimerState::Expired);
@@ -630,19 +668,23 @@
mFrameRateOverrideMappings.dump(dumper);
dumper.eol();
-
- {
- utils::Dumper::Section section(dumper, "Hardware VSYNC"sv);
-
- std::lock_guard lock(mHWVsyncLock);
- dumper.dump("screenAcquired"sv, mScreenAcquired.load());
- dumper.dump("hwVsyncAvailable"sv, mHWVsyncAvailable);
- dumper.dump("hwVsyncEnabled"sv, mPrimaryHWVsyncEnabled);
- }
}
void Scheduler::dumpVsync(std::string& out) const {
- mVsyncSchedule->dump(out);
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+ if (mLeaderDisplayId) {
+ base::StringAppendF(&out, "VsyncSchedule for leader %s:\n",
+ to_string(*mLeaderDisplayId).c_str());
+ getVsyncScheduleLocked()->dump(out);
+ }
+ for (auto& [id, vsyncSchedule] : mVsyncSchedules) {
+ if (id == mLeaderDisplayId) {
+ continue;
+ }
+ base::StringAppendF(&out, "VsyncSchedule for follower %s:\n", to_string(id).c_str());
+ vsyncSchedule->dump(out);
+ }
}
bool Scheduler::updateFrameRateOverrides(GlobalSignals consideredSignals, Fps displayRefreshRate) {
@@ -662,6 +704,7 @@
mLeaderDisplayId = leaderIdOpt.value_or(mRefreshRateSelectors.begin()->first);
ALOGI("Display %s is the leader", to_string(*mLeaderDisplayId).c_str());
+ auto vsyncSchedule = getVsyncScheduleLocked(*mLeaderDisplayId);
if (const auto leaderPtr = leaderSelectorPtrLocked()) {
leaderPtr->setIdleTimerCallbacks(
{.platform = {.onReset = [this] { idleTimerCallback(TimerState::Reset); },
@@ -671,6 +714,17 @@
[this] { kernelIdleTimerCallback(TimerState::Expired); }}});
leaderPtr->startIdleTimer();
+
+ const Fps refreshRate = leaderPtr->getActiveMode().modePtr->getFps();
+ setVsyncPeriod(vsyncSchedule, refreshRate.getPeriodNsecs(), true /* force */);
+ }
+
+ updateVsyncRegistration(vsyncSchedule->getDispatch());
+ {
+ std::lock_guard<std::mutex> lock(mConnectionsLock);
+ for (auto& [_, connection] : mConnections) {
+ connection.thread->onNewVsyncSchedule(vsyncSchedule);
+ }
}
}
diff --git a/services/surfaceflinger/Scheduler/Scheduler.h b/services/surfaceflinger/Scheduler/Scheduler.h
index e822448..8dc2def 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.h
+++ b/services/surfaceflinger/Scheduler/Scheduler.h
@@ -43,6 +43,7 @@
#include "Display/DisplayModeRequest.h"
#include "EventThread.h"
#include "FrameRateOverrideMappings.h"
+#include "ISchedulerCallback.h"
#include "LayerHistory.h"
#include "MessageQueue.h"
#include "OneShotTimer.h"
@@ -92,17 +93,7 @@
using GlobalSignals = RefreshRateSelector::GlobalSignals;
-struct ISchedulerCallback {
- virtual void setVsyncEnabled(bool) = 0;
- virtual void requestDisplayModes(std::vector<display::DisplayModeRequest>) = 0;
- virtual void kernelTimerChanged(bool expired) = 0;
- virtual void triggerOnFrameRateOverridesChanged() = 0;
-
-protected:
- ~ISchedulerCallback() = default;
-};
-
-class Scheduler : android::impl::MessageQueue {
+class Scheduler : android::impl::MessageQueue, public IEventThreadCallback {
using Impl = android::impl::MessageQueue;
public:
@@ -123,8 +114,6 @@
void run();
- void createVsyncSchedule(FeatureFlags);
-
using Impl::initVsync;
using Impl::getScheduledFrameTime;
@@ -177,9 +166,21 @@
const VsyncModulator& vsyncModulator() const { return *mVsyncModulator; }
+ // In some cases, we should only modulate for the leader display. In those
+ // cases, the caller should pass in the relevant display, and the method
+ // will no-op if it's not the leader. Other cases are not specific to a
+ // display.
template <typename... Args,
typename Handler = std::optional<VsyncConfig> (VsyncModulator::*)(Args...)>
- void modulateVsync(Handler handler, Args... args) {
+ void modulateVsync(std::optional<PhysicalDisplayId> id, Handler handler, Args... args) {
+ if (id) {
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+ if (id != mLeaderDisplayId) {
+ return;
+ }
+ }
+
if (const auto config = (*mVsyncModulator.*handler)(args...)) {
setVsyncConfig(*config, getLeaderVsyncPeriod());
}
@@ -188,24 +189,32 @@
void setVsyncConfigSet(const VsyncConfigSet&, Period vsyncPeriod);
// Sets the render rate for the scheduler to run at.
- void setRenderRate(Fps);
+ void setRenderRate(PhysicalDisplayId, Fps);
- void enableHardwareVsync();
- void disableHardwareVsync(bool makeUnavailable);
+ void enableHardwareVsync(PhysicalDisplayId);
+ void disableHardwareVsync(PhysicalDisplayId, bool makeUnavailable);
// Resyncs the scheduler to hardware vsync.
- // If makeAvailable is true, then hardware vsync will be turned on.
+ // If allowToEnable is true, then hardware vsync will be turned on.
// Otherwise, if hardware vsync is not already enabled then this method will
// no-op.
- void resyncToHardwareVsync(bool makeAvailable, Fps refreshRate);
+ // If refreshRate is nullopt, use the existing refresh rate of the display.
+ void resyncToHardwareVsync(PhysicalDisplayId id, bool allowToEnable,
+ std::optional<Fps> refreshRate = std::nullopt)
+ EXCLUDES(mDisplayLock) {
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+ resyncToHardwareVsyncLocked(id, allowToEnable, refreshRate);
+ }
void resync() EXCLUDES(mDisplayLock);
void forceNextResync() { mLastResyncTime = 0; }
- // Passes a vsync sample to VsyncController. periodFlushed will be true if
- // VsyncController detected that the vsync period changed, and false otherwise.
- void addResyncSample(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriod,
- bool* periodFlushed);
- void addPresentFence(std::shared_ptr<FenceTime>);
+ // Passes a vsync sample to VsyncController. Returns true if
+ // VsyncController detected that the vsync period changed and false
+ // otherwise.
+ bool addResyncSample(PhysicalDisplayId, nsecs_t timestamp,
+ std::optional<nsecs_t> hwcVsyncPeriod);
+ void addPresentFence(PhysicalDisplayId, std::shared_ptr<FenceTime>) EXCLUDES(mDisplayLock);
// Layers are registered on creation, and unregistered when the weak reference expires.
void registerLayer(Layer*);
@@ -223,20 +232,22 @@
// Indicates that touch interaction is taking place.
void onTouchHint();
- void setDisplayPowerMode(hal::PowerMode powerMode);
+ void setDisplayPowerMode(PhysicalDisplayId, hal::PowerMode powerMode)
+ REQUIRES(kMainThreadContext);
- VsyncSchedule& getVsyncSchedule() { return *mVsyncSchedule; }
+ std::shared_ptr<const VsyncSchedule> getVsyncSchedule(
+ std::optional<PhysicalDisplayId> idOpt = std::nullopt) const EXCLUDES(mDisplayLock);
+ std::shared_ptr<VsyncSchedule> getVsyncSchedule(
+ std::optional<PhysicalDisplayId> idOpt = std::nullopt) EXCLUDES(mDisplayLock) {
+ return std::const_pointer_cast<VsyncSchedule>(
+ static_cast<const Scheduler*>(this)->getVsyncSchedule(idOpt));
+ }
- // Returns true if a given vsync timestamp is considered valid vsync
- // for a given uid
- bool isVsyncValid(TimePoint expectedVsyncTimestamp, uid_t uid) const;
-
- // Checks if a vsync timestamp is in phase for a frame rate
- bool isVsyncInPhase(TimePoint timePoint, const Fps frameRate) const;
+ bool isVsyncInPhase(TimePoint expectedVsyncTime, Fps frameRate) const;
void dump(utils::Dumper&) const;
void dump(ConnectionHandle, std::string&) const;
- void dumpVsync(std::string&) const;
+ void dumpVsync(std::string&) const EXCLUDES(mDisplayLock);
// Returns the preferred refresh rate and frame rate for the leader display.
FrameRateMode getPreferredDisplayMode();
@@ -274,6 +285,10 @@
return mLayerHistory.getLayerFramerate(now, id);
}
+ // IEventThreadCallback overrides:
+ bool isVsyncTargetForUid(TimePoint expectedVsyncTime, uid_t uid) const override;
+ Fps getLeaderRenderFrameRate(uid_t uid) const override;
+
private:
friend class TestableScheduler;
@@ -295,7 +310,12 @@
void touchTimerCallback(TimerState);
void displayPowerTimerCallback(TimerState);
- void setVsyncPeriod(nsecs_t period);
+ void resyncToHardwareVsyncLocked(PhysicalDisplayId, bool allowToEnable,
+ std::optional<Fps> refreshRate = std::nullopt)
+ REQUIRES(kMainThreadContext, mDisplayLock);
+ void resyncAllToHardwareVsync(bool allowToEnable) EXCLUDES(mDisplayLock);
+ void setVsyncPeriod(const std::shared_ptr<VsyncSchedule>&, nsecs_t period, bool force)
+ REQUIRES(mDisplayLock);
void setVsyncConfig(const VsyncConfig&, Period vsyncPeriod);
// Chooses a leader among the registered displays, unless `leaderIdOpt` is specified. The new
@@ -307,6 +327,12 @@
// caller on the main thread to avoid deadlock, since the timer thread locks it before exit.
void demoteLeaderDisplay() REQUIRES(kMainThreadContext) EXCLUDES(mDisplayLock, mPolicyLock);
+ void registerDisplayInternal(PhysicalDisplayId, RefreshRateSelectorPtr,
+ std::shared_ptr<VsyncSchedule>) REQUIRES(kMainThreadContext)
+ EXCLUDES(mDisplayLock);
+
+ std::optional<Fps> getFrameRateOverrideLocked(uid_t) const REQUIRES(mDisplayLock);
+
struct Policy;
// Sets the S state of the policy to the T value under mPolicyLock, and chooses a display mode
@@ -344,9 +370,6 @@
void dispatchCachedReportedMode() REQUIRES(mPolicyLock) EXCLUDES(mDisplayLock);
- android::impl::EventThread::ThrottleVsyncCallback makeThrottleVsyncCallback() const;
- android::impl::EventThread::GetVsyncPeriodFunction makeGetVsyncPeriodFunction() const;
-
// Stores EventThread associated with a given VSyncSource, and an initial EventThreadConnection.
struct Connection {
sp<EventThreadConnection> connection;
@@ -360,14 +383,9 @@
ConnectionHandle mAppConnectionHandle;
ConnectionHandle mSfConnectionHandle;
- mutable std::mutex mHWVsyncLock;
- bool mPrimaryHWVsyncEnabled GUARDED_BY(mHWVsyncLock) = false;
- bool mHWVsyncAvailable GUARDED_BY(mHWVsyncLock) = false;
-
std::atomic<nsecs_t> mLastResyncTime = 0;
const FeatureFlags mFeatures;
- std::optional<VsyncSchedule> mVsyncSchedule;
// Shifts the VSYNC phase during certain transactions and refresh rate changes.
const sp<VsyncModulator> mVsyncModulator;
@@ -392,6 +410,10 @@
display::PhysicalDisplayMap<PhysicalDisplayId, RefreshRateSelectorPtr> mRefreshRateSelectors
GUARDED_BY(mDisplayLock) GUARDED_BY(kMainThreadContext);
+ // TODO (b/266715559): Store in the same map as mRefreshRateSelectors.
+ display::PhysicalDisplayMap<PhysicalDisplayId, std::shared_ptr<VsyncSchedule>> mVsyncSchedules
+ GUARDED_BY(mDisplayLock) GUARDED_BY(kMainThreadContext);
+
ftl::Optional<PhysicalDisplayId> mLeaderDisplayId GUARDED_BY(mDisplayLock)
GUARDED_BY(kMainThreadContext);
@@ -411,6 +433,14 @@
.value_or(std::cref(noLeader));
}
+ std::shared_ptr<const VsyncSchedule> getVsyncScheduleLocked(
+ std::optional<PhysicalDisplayId> idOpt = std::nullopt) const REQUIRES(mDisplayLock);
+ std::shared_ptr<VsyncSchedule> getVsyncScheduleLocked(
+ std::optional<PhysicalDisplayId> idOpt = std::nullopt) REQUIRES(mDisplayLock) {
+ return std::const_pointer_cast<VsyncSchedule>(
+ static_cast<const Scheduler*>(this)->getVsyncScheduleLocked(idOpt));
+ }
+
struct Policy {
// Policy for choosing the display mode.
LayerHistory::Summary contentRequirements;
@@ -437,9 +467,6 @@
static constexpr std::chrono::nanoseconds MAX_VSYNC_APPLIED_TIME = 200ms;
FrameRateOverrideMappings mFrameRateOverrideMappings;
-
- // Keeps track of whether the screen is acquired for debug
- std::atomic<bool> mScreenAcquired = false;
};
} // namespace scheduler
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatch.h b/services/surfaceflinger/Scheduler/VSyncDispatch.h
index 9520131..77875e3 100644
--- a/services/surfaceflinger/Scheduler/VSyncDispatch.h
+++ b/services/surfaceflinger/Scheduler/VSyncDispatch.h
@@ -161,7 +161,8 @@
*/
class VSyncCallbackRegistration {
public:
- VSyncCallbackRegistration(VSyncDispatch&, VSyncDispatch::Callback, std::string callbackName);
+ VSyncCallbackRegistration(std::shared_ptr<VSyncDispatch>, VSyncDispatch::Callback,
+ std::string callbackName);
~VSyncCallbackRegistration();
VSyncCallbackRegistration(VSyncCallbackRegistration&&);
@@ -177,7 +178,7 @@
CancelResult cancel();
private:
- std::reference_wrapper<VSyncDispatch> mDispatch;
+ std::shared_ptr<VSyncDispatch> mDispatch;
VSyncDispatch::CallbackToken mToken;
bool mValidToken;
};
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp
index 73d52cf..26389eb 100644
--- a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp
+++ b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp
@@ -215,10 +215,10 @@
}
VSyncDispatchTimerQueue::VSyncDispatchTimerQueue(std::unique_ptr<TimeKeeper> tk,
- VSyncTracker& tracker, nsecs_t timerSlack,
- nsecs_t minVsyncDistance)
+ VsyncSchedule::TrackerPtr tracker,
+ nsecs_t timerSlack, nsecs_t minVsyncDistance)
: mTimeKeeper(std::move(tk)),
- mTracker(tracker),
+ mTracker(std::move(tracker)),
mTimerSlack(timerSlack),
mMinVsyncDistance(minVsyncDistance) {}
@@ -255,7 +255,7 @@
}
if (it != skipUpdateIt) {
- callback->update(mTracker, now);
+ callback->update(*mTracker, now);
}
auto const wakeupTime = *callback->wakeupTime();
if (!min || *min > wakeupTime) {
@@ -365,10 +365,10 @@
auto const rearmImminent = now > mIntendedWakeupTime;
if (CC_UNLIKELY(rearmImminent)) {
callback->addPendingWorkloadUpdate(scheduleTiming);
- return getExpectedCallbackTime(mTracker, now, scheduleTiming);
+ return getExpectedCallbackTime(*mTracker, now, scheduleTiming);
}
- const ScheduleResult result = callback->schedule(scheduleTiming, mTracker, now);
+ const ScheduleResult result = callback->schedule(scheduleTiming, *mTracker, now);
if (!result.has_value()) {
return {};
}
@@ -434,15 +434,15 @@
}
}
-VSyncCallbackRegistration::VSyncCallbackRegistration(VSyncDispatch& dispatch,
+VSyncCallbackRegistration::VSyncCallbackRegistration(std::shared_ptr<VSyncDispatch> dispatch,
VSyncDispatch::Callback callback,
std::string callbackName)
- : mDispatch(dispatch),
- mToken(dispatch.registerCallback(std::move(callback), std::move(callbackName))),
+ : mDispatch(std::move(dispatch)),
+ mToken(mDispatch->registerCallback(std::move(callback), std::move(callbackName))),
mValidToken(true) {}
VSyncCallbackRegistration::VSyncCallbackRegistration(VSyncCallbackRegistration&& other)
- : mDispatch(other.mDispatch),
+ : mDispatch(std::move(other.mDispatch)),
mToken(std::move(other.mToken)),
mValidToken(std::move(other.mValidToken)) {
other.mValidToken = false;
@@ -457,28 +457,28 @@
}
VSyncCallbackRegistration::~VSyncCallbackRegistration() {
- if (mValidToken) mDispatch.get().unregisterCallback(mToken);
+ if (mValidToken) mDispatch->unregisterCallback(mToken);
}
ScheduleResult VSyncCallbackRegistration::schedule(VSyncDispatch::ScheduleTiming scheduleTiming) {
if (!mValidToken) {
return std::nullopt;
}
- return mDispatch.get().schedule(mToken, scheduleTiming);
+ return mDispatch->schedule(mToken, scheduleTiming);
}
ScheduleResult VSyncCallbackRegistration::update(VSyncDispatch::ScheduleTiming scheduleTiming) {
if (!mValidToken) {
return std::nullopt;
}
- return mDispatch.get().update(mToken, scheduleTiming);
+ return mDispatch->update(mToken, scheduleTiming);
}
CancelResult VSyncCallbackRegistration::cancel() {
if (!mValidToken) {
return CancelResult::Error;
}
- return mDispatch.get().cancel(mToken);
+ return mDispatch->cancel(mToken);
}
} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h
index c3af136..6499d69 100644
--- a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h
+++ b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h
@@ -26,11 +26,11 @@
#include <android-base/thread_annotations.h>
#include "VSyncDispatch.h"
+#include "VsyncSchedule.h"
namespace android::scheduler {
class TimeKeeper;
-class VSyncTracker;
// VSyncDispatchTimerQueueEntry is a helper class representing internal state for each entry in
// VSyncDispatchTimerQueue hoisted to public for unit testing.
@@ -120,8 +120,8 @@
// should be grouped into one wakeup.
// \param[in] minVsyncDistance The minimum distance between two vsync estimates before the
// vsyncs are considered the same vsync event.
- VSyncDispatchTimerQueue(std::unique_ptr<TimeKeeper>, VSyncTracker&, nsecs_t timerSlack,
- nsecs_t minVsyncDistance);
+ VSyncDispatchTimerQueue(std::unique_ptr<TimeKeeper>, VsyncSchedule::TrackerPtr,
+ nsecs_t timerSlack, nsecs_t minVsyncDistance);
~VSyncDispatchTimerQueue();
CallbackToken registerCallback(Callback, std::string callbackName) final;
@@ -148,7 +148,7 @@
static constexpr nsecs_t kInvalidTime = std::numeric_limits<int64_t>::max();
std::unique_ptr<TimeKeeper> const mTimeKeeper;
- VSyncTracker& mTracker;
+ VsyncSchedule::TrackerPtr mTracker;
nsecs_t const mTimerSlack;
nsecs_t const mMinVsyncDistance;
diff --git a/services/surfaceflinger/Scheduler/VSyncPredictor.cpp b/services/surfaceflinger/Scheduler/VSyncPredictor.cpp
index 02e12fd..a3b8a56 100644
--- a/services/surfaceflinger/Scheduler/VSyncPredictor.cpp
+++ b/services/surfaceflinger/Scheduler/VSyncPredictor.cpp
@@ -31,6 +31,7 @@
#include <android-base/stringprintf.h>
#include <cutils/compiler.h>
#include <cutils/properties.h>
+#include <gui/TraceUtils.h>
#include <utils/Log.h>
#include <utils/Trace.h>
@@ -40,14 +41,16 @@
namespace android::scheduler {
using base::StringAppendF;
+using base::StringPrintf;
static auto constexpr kMaxPercent = 100u;
VSyncPredictor::~VSyncPredictor() = default;
-VSyncPredictor::VSyncPredictor(nsecs_t idealPeriod, size_t historySize,
+VSyncPredictor::VSyncPredictor(std::string name, nsecs_t idealPeriod, size_t historySize,
size_t minimumSamplesForPrediction, uint32_t outlierTolerancePercent)
- : mTraceOn(property_get_bool("debug.sf.vsp_trace", false)),
+ : mName(name),
+ mTraceOn(property_get_bool("debug.sf.vsp_trace", false)),
kHistorySize(historySize),
kMinimumSamplesForPrediction(minimumSamplesForPrediction),
kOutlierTolerancePercent(std::min(outlierTolerancePercent, kMaxPercent)),
@@ -57,12 +60,14 @@
inline void VSyncPredictor::traceInt64If(const char* name, int64_t value) const {
if (CC_UNLIKELY(mTraceOn)) {
- ATRACE_INT64(name, value);
+ traceInt64(name, value);
}
}
inline void VSyncPredictor::traceInt64(const char* name, int64_t value) const {
- ATRACE_INT64(name, value);
+ // TODO (b/266817103): Pass in PhysicalDisplayId and use ftl::Concat to
+ // avoid unnecessary allocations.
+ ATRACE_INT64(StringPrintf("%s %s", name, mName.c_str()).c_str(), value);
}
inline size_t VSyncPredictor::next(size_t i) const {
@@ -214,8 +219,8 @@
it->second = {anticipatedPeriod, intercept};
- ALOGV("model update ts: %" PRId64 " slope: %" PRId64 " intercept: %" PRId64, timestamp,
- anticipatedPeriod, intercept);
+ ALOGV("model update ts %s: %" PRId64 " slope: %" PRId64 " intercept: %" PRId64, mName.c_str(),
+ timestamp, anticipatedPeriod, intercept);
return true;
}
@@ -327,7 +332,7 @@
}
void VSyncPredictor::setDivisor(unsigned divisor) {
- ALOGV("%s: %d", __func__, divisor);
+ ALOGV("%s %s: %d", __func__, mName.c_str(), divisor);
std::lock_guard lock(mMutex);
mDivisor = divisor;
}
@@ -343,7 +348,7 @@
}
void VSyncPredictor::setPeriod(nsecs_t period) {
- ATRACE_CALL();
+ ATRACE_FORMAT("%s %s", __func__, mName.c_str());
traceInt64("VSP-setPeriod", period);
std::lock_guard lock(mMutex);
diff --git a/services/surfaceflinger/Scheduler/VSyncPredictor.h b/services/surfaceflinger/Scheduler/VSyncPredictor.h
index 305cdb0..1ded54f 100644
--- a/services/surfaceflinger/Scheduler/VSyncPredictor.h
+++ b/services/surfaceflinger/Scheduler/VSyncPredictor.h
@@ -29,14 +29,15 @@
class VSyncPredictor : public VSyncTracker {
public:
/*
+ * \param [in] name The name of the display this corresponds to.
* \param [in] idealPeriod The initial ideal period to use.
* \param [in] historySize The internal amount of entries to store in the model.
* \param [in] minimumSamplesForPrediction The minimum number of samples to collect before
* predicting. \param [in] outlierTolerancePercent a number 0 to 100 that will be used to filter
* samples that fall outlierTolerancePercent from an anticipated vsync event.
*/
- VSyncPredictor(nsecs_t idealPeriod, size_t historySize, size_t minimumSamplesForPrediction,
- uint32_t outlierTolerancePercent);
+ VSyncPredictor(std::string name, nsecs_t idealPeriod, size_t historySize,
+ size_t minimumSamplesForPrediction, uint32_t outlierTolerancePercent);
~VSyncPredictor();
bool addVsyncTimestamp(nsecs_t timestamp) final EXCLUDES(mMutex);
@@ -76,6 +77,8 @@
VSyncPredictor& operator=(VSyncPredictor const&) = delete;
void clearTimestamps() REQUIRES(mMutex);
+ const std::string mName;
+
inline void traceInt64If(const char* name, int64_t value) const;
inline void traceInt64(const char* name, int64_t value) const;
bool const mTraceOn;
diff --git a/services/surfaceflinger/Scheduler/VSyncReactor.cpp b/services/surfaceflinger/Scheduler/VSyncReactor.cpp
index b5f212e..a831f66 100644
--- a/services/surfaceflinger/Scheduler/VSyncReactor.cpp
+++ b/services/surfaceflinger/Scheduler/VSyncReactor.cpp
@@ -21,6 +21,7 @@
#include <assert.h>
#include <cutils/properties.h>
+#include <gui/TraceUtils.h>
#include <log/log.h>
#include <utils/Trace.h>
@@ -32,6 +33,7 @@
namespace android::scheduler {
using base::StringAppendF;
+using base::StringPrintf;
VsyncController::~VsyncController() = default;
@@ -39,12 +41,12 @@
return systemTime(SYSTEM_TIME_MONOTONIC);
}
-VSyncReactor::VSyncReactor(std::unique_ptr<Clock> clock, VSyncTracker& tracker,
+VSyncReactor::VSyncReactor(std::string name, std::unique_ptr<Clock> clock, VSyncTracker& tracker,
size_t pendingFenceLimit, bool supportKernelIdleTimer)
- : mClock(std::move(clock)),
+ : mName(name),
+ mClock(std::move(clock)),
mTracker(tracker),
mPendingLimit(pendingFenceLimit),
- // TODO(adyabr): change mSupportKernelIdleTimer when the active display changes
mSupportKernelIdleTimer(supportKernelIdleTimer) {}
VSyncReactor::~VSyncReactor() = default;
@@ -114,7 +116,7 @@
}
void VSyncReactor::startPeriodTransitionInternal(nsecs_t newPeriod) {
- ATRACE_CALL();
+ ATRACE_FORMAT("%s %s", __func__, mName.c_str());
mPeriodConfirmationInProgress = true;
mPeriodTransitioningTo = newPeriod;
mMoreSamplesNeeded = true;
@@ -122,18 +124,20 @@
}
void VSyncReactor::endPeriodTransition() {
- ATRACE_CALL();
+ ATRACE_FORMAT("%s %s", __func__, mName.c_str());
mPeriodTransitioningTo.reset();
mPeriodConfirmationInProgress = false;
mLastHwVsync.reset();
}
-void VSyncReactor::startPeriodTransition(nsecs_t period) {
- ATRACE_INT64("VSR-startPeriodTransition", period);
+void VSyncReactor::startPeriodTransition(nsecs_t period, bool force) {
+ // TODO (b/266817103): Pass in PhysicalDisplayId and use ftl::Concat to
+ // avoid unnecessary allocations.
+ ATRACE_INT64(StringPrintf("VSR-startPeriodTransition %s", mName.c_str()).c_str(), period);
std::lock_guard lock(mMutex);
mLastHwVsync.reset();
- if (!mSupportKernelIdleTimer && period == mTracker.currentPeriod()) {
+ if (!mSupportKernelIdleTimer && period == mTracker.currentPeriod() && !force) {
endPeriodTransition();
setIgnorePresentFencesInternal(false);
mMoreSamplesNeeded = false;
@@ -181,7 +185,7 @@
std::lock_guard lock(mMutex);
if (periodConfirmed(timestamp, hwcVsyncPeriod)) {
- ATRACE_NAME("VSR: period confirmed");
+ ATRACE_FORMAT("VSR %s: period confirmed", mName.c_str());
if (mPeriodTransitioningTo) {
mTracker.setPeriod(*mPeriodTransitioningTo);
*periodFlushed = true;
@@ -195,12 +199,12 @@
endPeriodTransition();
mMoreSamplesNeeded = mTracker.needsMoreSamples();
} else if (mPeriodConfirmationInProgress) {
- ATRACE_NAME("VSR: still confirming period");
+ ATRACE_FORMAT("VSR %s: still confirming period", mName.c_str());
mLastHwVsync = timestamp;
mMoreSamplesNeeded = true;
*periodFlushed = false;
} else {
- ATRACE_NAME("VSR: adding sample");
+ ATRACE_FORMAT("VSR %s: adding sample", mName.c_str());
*periodFlushed = false;
mTracker.addVsyncTimestamp(timestamp);
mMoreSamplesNeeded = mTracker.needsMoreSamples();
diff --git a/services/surfaceflinger/Scheduler/VSyncReactor.h b/services/surfaceflinger/Scheduler/VSyncReactor.h
index 4501487..fd9ca42 100644
--- a/services/surfaceflinger/Scheduler/VSyncReactor.h
+++ b/services/surfaceflinger/Scheduler/VSyncReactor.h
@@ -22,6 +22,7 @@
#include <vector>
#include <android-base/thread_annotations.h>
+#include <ui/DisplayId.h>
#include <ui/FenceTime.h>
#include <scheduler/TimeKeeper.h>
@@ -37,14 +38,14 @@
// TODO (b/145217110): consider renaming.
class VSyncReactor : public VsyncController {
public:
- VSyncReactor(std::unique_ptr<Clock> clock, VSyncTracker& tracker, size_t pendingFenceLimit,
- bool supportKernelIdleTimer);
+ VSyncReactor(std::string name, std::unique_ptr<Clock> clock, VSyncTracker& tracker,
+ size_t pendingFenceLimit, bool supportKernelIdleTimer);
~VSyncReactor();
bool addPresentFence(std::shared_ptr<FenceTime>) final;
void setIgnorePresentFences(bool ignore) final;
- void startPeriodTransition(nsecs_t period) final;
+ void startPeriodTransition(nsecs_t period, bool force) final;
bool addHwVsyncTimestamp(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriod,
bool* periodFlushed) final;
@@ -61,6 +62,7 @@
bool periodConfirmed(nsecs_t vsync_timestamp, std::optional<nsecs_t> hwcVsyncPeriod)
REQUIRES(mMutex);
+ const std::string mName;
std::unique_ptr<Clock> const mClock;
VSyncTracker& mTracker;
size_t const mPendingLimit;
diff --git a/services/surfaceflinger/Scheduler/VsyncController.h b/services/surfaceflinger/Scheduler/VsyncController.h
index 726a420..9177899 100644
--- a/services/surfaceflinger/Scheduler/VsyncController.h
+++ b/services/surfaceflinger/Scheduler/VsyncController.h
@@ -63,8 +63,9 @@
* itself. The controller will end the period transition internally.
*
* \param [in] period The period that the system is changing into.
+ * \param [in] force True to recalibrate even if period matches the existing period.
*/
- virtual void startPeriodTransition(nsecs_t period) = 0;
+ virtual void startPeriodTransition(nsecs_t period, bool force) = 0;
/*
* Tells the tracker to stop using present fences to get a vsync signal.
diff --git a/services/surfaceflinger/Scheduler/VsyncSchedule.cpp b/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
index 95bc31f..951c1ec 100644
--- a/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
+++ b/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
@@ -21,6 +21,9 @@
#include "VsyncSchedule.h"
+#include "ISchedulerCallback.h"
+#include "Scheduler.h"
+#include "Utils/Dumper.h"
#include "VSyncDispatchTimerQueue.h"
#include "VSyncPredictor.h"
#include "VSyncReactor.h"
@@ -39,8 +42,8 @@
}
public:
- explicit PredictedVsyncTracer(VsyncDispatch& dispatch)
- : mRegistration(dispatch, makeVsyncCallback(), __func__) {
+ explicit PredictedVsyncTracer(std::shared_ptr<VsyncDispatch> dispatch)
+ : mRegistration(std::move(dispatch), makeVsyncCallback(), __func__) {
schedule();
}
@@ -51,21 +54,23 @@
VSyncCallbackRegistration mRegistration;
};
-VsyncSchedule::VsyncSchedule(FeatureFlags features)
- : mTracker(createTracker()),
- mDispatch(createDispatch(*mTracker)),
- mController(createController(*mTracker, features)) {
+VsyncSchedule::VsyncSchedule(PhysicalDisplayId id, FeatureFlags features)
+ : mId(id),
+ mTracker(createTracker(id)),
+ mDispatch(createDispatch(mTracker)),
+ mController(createController(id, *mTracker, features)) {
if (features.test(Feature::kTracePredictedVsync)) {
- mTracer = std::make_unique<PredictedVsyncTracer>(*mDispatch);
+ mTracer = std::make_unique<PredictedVsyncTracer>(mDispatch);
}
}
-VsyncSchedule::VsyncSchedule(TrackerPtr tracker, DispatchPtr dispatch, ControllerPtr controller)
- : mTracker(std::move(tracker)),
+VsyncSchedule::VsyncSchedule(PhysicalDisplayId id, TrackerPtr tracker, DispatchPtr dispatch,
+ ControllerPtr controller)
+ : mId(id),
+ mTracker(std::move(tracker)),
mDispatch(std::move(dispatch)),
mController(std::move(controller)) {}
-VsyncSchedule::VsyncSchedule(VsyncSchedule&&) = default;
VsyncSchedule::~VsyncSchedule() = default;
Period VsyncSchedule::period() const {
@@ -77,6 +82,13 @@
}
void VsyncSchedule::dump(std::string& out) const {
+ utils::Dumper dumper(out);
+ {
+ std::lock_guard<std::mutex> lock(mHwVsyncLock);
+ dumper.dump("hwVsyncState", ftl::enum_string(mHwVsyncState));
+ dumper.dump("lastHwVsyncState", ftl::enum_string(mLastHwVsyncState));
+ }
+
out.append("VsyncController:\n");
mController->dump(out);
@@ -84,40 +96,72 @@
mDispatch->dump(out);
}
-VsyncSchedule::TrackerPtr VsyncSchedule::createTracker() {
+VsyncSchedule::TrackerPtr VsyncSchedule::createTracker(PhysicalDisplayId id) {
// TODO(b/144707443): Tune constants.
constexpr nsecs_t kInitialPeriod = (60_Hz).getPeriodNsecs();
constexpr size_t kHistorySize = 20;
constexpr size_t kMinSamplesForPrediction = 6;
constexpr uint32_t kDiscardOutlierPercent = 20;
- return std::make_unique<VSyncPredictor>(kInitialPeriod, kHistorySize, kMinSamplesForPrediction,
- kDiscardOutlierPercent);
+ return std::make_unique<VSyncPredictor>(to_string(id), kInitialPeriod, kHistorySize,
+ kMinSamplesForPrediction, kDiscardOutlierPercent);
}
-VsyncSchedule::DispatchPtr VsyncSchedule::createDispatch(VsyncTracker& tracker) {
+VsyncSchedule::DispatchPtr VsyncSchedule::createDispatch(TrackerPtr tracker) {
using namespace std::chrono_literals;
// TODO(b/144707443): Tune constants.
constexpr std::chrono::nanoseconds kGroupDispatchWithin = 500us;
constexpr std::chrono::nanoseconds kSnapToSameVsyncWithin = 3ms;
- return std::make_unique<VSyncDispatchTimerQueue>(std::make_unique<Timer>(), tracker,
+ return std::make_unique<VSyncDispatchTimerQueue>(std::make_unique<Timer>(), std::move(tracker),
kGroupDispatchWithin.count(),
kSnapToSameVsyncWithin.count());
}
-VsyncSchedule::ControllerPtr VsyncSchedule::createController(VsyncTracker& tracker,
+VsyncSchedule::ControllerPtr VsyncSchedule::createController(PhysicalDisplayId id,
+ VsyncTracker& tracker,
FeatureFlags features) {
// TODO(b/144707443): Tune constants.
constexpr size_t kMaxPendingFences = 20;
const bool hasKernelIdleTimer = features.test(Feature::kKernelIdleTimer);
- auto reactor = std::make_unique<VSyncReactor>(std::make_unique<SystemClock>(), tracker,
- kMaxPendingFences, hasKernelIdleTimer);
+ auto reactor = std::make_unique<VSyncReactor>(to_string(id), std::make_unique<SystemClock>(),
+ tracker, kMaxPendingFences, hasKernelIdleTimer);
reactor->setIgnorePresentFences(!features.test(Feature::kPresentFences));
return reactor;
}
+void VsyncSchedule::enableHardwareVsync(ISchedulerCallback& callback) {
+ std::lock_guard<std::mutex> lock(mHwVsyncLock);
+ if (mHwVsyncState == HwVsyncState::Disabled) {
+ getTracker().resetModel();
+ callback.setVsyncEnabled(mId, true);
+ mHwVsyncState = HwVsyncState::Enabled;
+ mLastHwVsyncState = HwVsyncState::Enabled;
+ }
+}
+
+void VsyncSchedule::disableHardwareVsync(ISchedulerCallback& callback, bool disallow) {
+ std::lock_guard<std::mutex> lock(mHwVsyncLock);
+ if (mHwVsyncState == HwVsyncState::Enabled) {
+ callback.setVsyncEnabled(mId, false);
+ mLastHwVsyncState = HwVsyncState::Disabled;
+ }
+ mHwVsyncState = disallow ? HwVsyncState::Disallowed : HwVsyncState::Disabled;
+}
+
+bool VsyncSchedule::isHardwareVsyncAllowed() const {
+ std::lock_guard<std::mutex> lock(mHwVsyncLock);
+ return mHwVsyncState != HwVsyncState::Disallowed;
+}
+
+void VsyncSchedule::allowHardwareVsync() {
+ std::lock_guard<std::mutex> lock(mHwVsyncLock);
+ if (mHwVsyncState == HwVsyncState::Disallowed) {
+ mHwVsyncState = HwVsyncState::Disabled;
+ }
+}
+
} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/VsyncSchedule.h b/services/surfaceflinger/Scheduler/VsyncSchedule.h
index 173b1d0..ffb7ad5 100644
--- a/services/surfaceflinger/Scheduler/VsyncSchedule.h
+++ b/services/surfaceflinger/Scheduler/VsyncSchedule.h
@@ -19,8 +19,10 @@
#include <memory>
#include <string>
+#include <ftl/enum.h>
#include <scheduler/Features.h>
#include <scheduler/Time.h>
+#include <ui/DisplayId.h>
namespace android {
class EventThreadTest;
@@ -32,6 +34,8 @@
namespace android::scheduler {
+struct ISchedulerCallback;
+
// TODO(b/185535769): Rename classes, and remove aliases.
class VSyncDispatch;
class VSyncTracker;
@@ -43,8 +47,7 @@
// Schedule that synchronizes to hardware VSYNC of a physical display.
class VsyncSchedule {
public:
- explicit VsyncSchedule(FeatureFlags);
- VsyncSchedule(VsyncSchedule&&);
+ VsyncSchedule(PhysicalDisplayId, FeatureFlags);
~VsyncSchedule();
Period period() const;
@@ -55,30 +58,71 @@
VsyncTracker& getTracker() { return *mTracker; }
VsyncController& getController() { return *mController; }
+ // TODO(b/185535769): Once these are hidden behind the API, they may no
+ // longer need to be shared_ptrs.
+ using DispatchPtr = std::shared_ptr<VsyncDispatch>;
+ using TrackerPtr = std::shared_ptr<VsyncTracker>;
+
// TODO(b/185535769): Remove once VsyncSchedule owns all registrations.
- VsyncDispatch& getDispatch() { return *mDispatch; }
+ DispatchPtr getDispatch() { return mDispatch; }
void dump(std::string&) const;
+ // Turn on hardware vsyncs, unless mHwVsyncState is Disallowed, in which
+ // case this call is ignored.
+ void enableHardwareVsync(ISchedulerCallback&) EXCLUDES(mHwVsyncLock);
+
+ // Disable hardware vsyncs. If `disallow` is true, future calls to
+ // enableHardwareVsync are ineffective until allowHardwareVsync is called.
+ void disableHardwareVsync(ISchedulerCallback&, bool disallow) EXCLUDES(mHwVsyncLock);
+
+ // Restore the ability to enable hardware vsync.
+ void allowHardwareVsync() EXCLUDES(mHwVsyncLock);
+
+ // If true, enableHardwareVsync can enable hardware vsync (if not already
+ // enabled). If false, enableHardwareVsync does nothing.
+ bool isHardwareVsyncAllowed() const EXCLUDES(mHwVsyncLock);
+
+protected:
+ using ControllerPtr = std::unique_ptr<VsyncController>;
+
+ // For tests.
+ VsyncSchedule(PhysicalDisplayId, TrackerPtr, DispatchPtr, ControllerPtr);
+
private:
friend class TestableScheduler;
friend class android::EventThreadTest;
friend class android::fuzz::SchedulerFuzzer;
- using TrackerPtr = std::unique_ptr<VsyncTracker>;
- using DispatchPtr = std::unique_ptr<VsyncDispatch>;
- using ControllerPtr = std::unique_ptr<VsyncController>;
+ static TrackerPtr createTracker(PhysicalDisplayId);
+ static DispatchPtr createDispatch(TrackerPtr);
+ static ControllerPtr createController(PhysicalDisplayId, VsyncTracker&, FeatureFlags);
- // For tests.
- VsyncSchedule(TrackerPtr, DispatchPtr, ControllerPtr);
+ mutable std::mutex mHwVsyncLock;
+ enum class HwVsyncState {
+ // Hardware vsyncs are currently enabled.
+ Enabled,
- static TrackerPtr createTracker();
- static DispatchPtr createDispatch(VsyncTracker&);
- static ControllerPtr createController(VsyncTracker&, FeatureFlags);
+ // Hardware vsyncs are currently disabled. They can be enabled by a call
+ // to `enableHardwareVsync`.
+ Disabled,
+
+ // Hardware vsyncs are not currently allowed (e.g. because the display
+ // is off).
+ Disallowed,
+
+ ftl_last = Disallowed,
+ };
+ HwVsyncState mHwVsyncState GUARDED_BY(mHwVsyncLock) = HwVsyncState::Disallowed;
+
+ // The last state, which may be the current state, or the state prior to setting to Disallowed.
+ HwVsyncState mLastHwVsyncState GUARDED_BY(mHwVsyncLock) = HwVsyncState::Disabled;
class PredictedVsyncTracer;
using TracerPtr = std::unique_ptr<PredictedVsyncTracer>;
+ const PhysicalDisplayId mId;
+
// Effectively const except in move constructor.
TrackerPtr mTracker;
DispatchPtr mDispatch;
diff --git a/services/surfaceflinger/SurfaceFlinger.cpp b/services/surfaceflinger/SurfaceFlinger.cpp
index 43483e4..6020aba 100644
--- a/services/surfaceflinger/SurfaceFlinger.cpp
+++ b/services/surfaceflinger/SurfaceFlinger.cpp
@@ -1124,21 +1124,33 @@
return NO_ERROR;
}
-status_t SurfaceFlinger::getDisplayStats(const sp<IBinder>&, DisplayStatInfo* outStats) {
+status_t SurfaceFlinger::getDisplayStats(const sp<IBinder>& displayToken,
+ DisplayStatInfo* outStats) {
if (!outStats) {
return BAD_VALUE;
}
- const auto& schedule = mScheduler->getVsyncSchedule();
- outStats->vsyncTime = schedule.vsyncDeadlineAfter(TimePoint::now()).ns();
- outStats->vsyncPeriod = schedule.period().ns();
+ std::optional<PhysicalDisplayId> displayIdOpt;
+ {
+ Mutex::Autolock lock(mStateLock);
+ displayIdOpt = getPhysicalDisplayIdLocked(displayToken);
+ }
+
+ if (!displayIdOpt) {
+ ALOGE("%s: Invalid physical display token %p", __func__, displayToken.get());
+ return NAME_NOT_FOUND;
+ }
+ const auto schedule = mScheduler->getVsyncSchedule(displayIdOpt);
+ outStats->vsyncTime = schedule->vsyncDeadlineAfter(TimePoint::now()).ns();
+ outStats->vsyncPeriod = schedule->period().ns();
return NO_ERROR;
}
void SurfaceFlinger::setDesiredActiveMode(display::DisplayModeRequest&& request, bool force) {
ATRACE_CALL();
- auto display = getDisplayDeviceLocked(request.mode.modePtr->getPhysicalDisplayId());
+ const auto displayId = request.mode.modePtr->getPhysicalDisplayId();
+ const auto display = getDisplayDeviceLocked(displayId);
if (!display) {
ALOGW("%s: display is no longer valid", __func__);
return;
@@ -1151,23 +1163,25 @@
force)) {
case DisplayDevice::DesiredActiveModeAction::InitiateDisplayModeSwitch:
// Set the render rate as setDesiredActiveMode updated it.
- mScheduler->setRenderRate(display->refreshRateSelector().getActiveMode().fps);
+ mScheduler->setRenderRate(displayId,
+ display->refreshRateSelector().getActiveMode().fps);
// Schedule a new frame to initiate the display mode switch.
scheduleComposite(FrameHint::kNone);
// Start receiving vsync samples now, so that we can detect a period
// switch.
- mScheduler->resyncToHardwareVsync(true, mode.modePtr->getFps());
+ mScheduler->resyncToHardwareVsync(displayId, true /* allowToEnable */,
+ mode.modePtr->getFps());
+
// As we called to set period, we will call to onRefreshRateChangeCompleted once
// VsyncController model is locked.
- mScheduler->modulateVsync(&VsyncModulator::onRefreshRateChangeInitiated);
-
+ mScheduler->modulateVsync(displayId, &VsyncModulator::onRefreshRateChangeInitiated);
updatePhaseConfiguration(mode.fps);
mScheduler->setModeChangePending(true);
break;
case DisplayDevice::DesiredActiveModeAction::InitiateRenderRateSwitch:
- mScheduler->setRenderRate(mode.fps);
+ mScheduler->setRenderRate(displayId, mode.fps);
updatePhaseConfiguration(mode.fps);
mRefreshRateStats->setRefreshRate(mode.fps);
if (display->getPhysicalId() == mActiveDisplayId && emitEvent) {
@@ -1283,11 +1297,14 @@
}
void SurfaceFlinger::desiredActiveModeChangeDone(const sp<DisplayDevice>& display) {
- const auto displayFps = display->getDesiredActiveMode()->modeOpt->modePtr->getFps();
- const auto renderFps = display->getDesiredActiveMode()->modeOpt->fps;
+ const auto desiredActiveMode = display->getDesiredActiveMode();
+ const auto& modeOpt = desiredActiveMode->modeOpt;
+ const auto displayId = modeOpt->modePtr->getPhysicalDisplayId();
+ const auto displayFps = modeOpt->modePtr->getFps();
+ const auto renderFps = modeOpt->fps;
clearDesiredActiveModeState(display);
- mScheduler->resyncToHardwareVsync(true, displayFps);
- mScheduler->setRenderRate(renderFps);
+ mScheduler->resyncToHardwareVsync(displayId, true /* allowToEnable */, displayFps);
+ mScheduler->setRenderRate(displayId, renderFps);
updatePhaseConfiguration(renderFps);
}
@@ -2023,21 +2040,11 @@
ATRACE_FORMAT("onComposerHalVsync%s", tracePeriod.c_str());
Mutex::Autolock lock(mStateLock);
-
- if (!getHwComposer().onVsync(hwcDisplayId, timestamp)) {
- return;
- }
-
- if (const auto displayId = getHwComposer().toPhysicalDisplayId(hwcDisplayId);
- displayId != mActiveDisplayId) {
- // For now, we don't do anything with non active display vsyncs.
- return;
- }
-
- bool periodFlushed = false;
- mScheduler->addResyncSample(timestamp, vsyncPeriod, &periodFlushed);
- if (periodFlushed) {
- mScheduler->modulateVsync(&VsyncModulator::onRefreshRateChangeCompleted);
+ if (const auto displayIdOpt = getHwComposer().onVsync(hwcDisplayId, timestamp)) {
+ if (mScheduler->addResyncSample(*displayIdOpt, timestamp, vsyncPeriod)) {
+ // period flushed
+ mScheduler->modulateVsync(displayIdOpt, &VsyncModulator::onRefreshRateChangeCompleted);
+ }
}
}
@@ -2078,16 +2085,15 @@
mScheduler->forceNextResync();
}
-void SurfaceFlinger::setVsyncEnabled(bool enabled) {
- ATRACE_CALL();
+void SurfaceFlinger::setVsyncEnabled(PhysicalDisplayId id, bool enabled) {
+ const char* const whence = __func__;
+ ATRACE_FORMAT("%s (%d) for %" PRIu64, whence, enabled, id.value);
// On main thread to avoid race conditions with display power state.
static_cast<void>(mScheduler->schedule([=]() FTL_FAKE_GUARD(mStateLock) {
- mHWCVsyncPendingState = enabled ? hal::Vsync::ENABLE : hal::Vsync::DISABLE;
-
- if (const auto display = getDefaultDisplayDeviceLocked();
- display && display->isPoweredOn()) {
- setHWCVsyncEnabled(display->getPhysicalId(), mHWCVsyncPendingState);
+ ATRACE_FORMAT("%s (%d) for %" PRIu64 " (main thread)", whence, enabled, id.value);
+ if (const auto display = getDisplayDeviceLocked(id); display && display->isPoweredOn()) {
+ setHWCVsyncEnabled(id, enabled);
}
}));
}
@@ -2114,13 +2120,13 @@
TimePoint SurfaceFlinger::calculateExpectedPresentTime(TimePoint frameTime) const {
const auto& schedule = mScheduler->getVsyncSchedule();
- const TimePoint vsyncDeadline = schedule.vsyncDeadlineAfter(frameTime);
+ const TimePoint vsyncDeadline = schedule->vsyncDeadlineAfter(frameTime);
if (mScheduler->vsyncModulator().getVsyncConfig().sfOffset > 0) {
return vsyncDeadline;
}
// Inflate the expected present time if we're targeting the next vsync.
- return vsyncDeadline + schedule.period();
+ return vsyncDeadline + schedule->period();
}
void SurfaceFlinger::configure() FTL_FAKE_GUARD(kMainThreadContext) {
@@ -2147,7 +2153,7 @@
ticks<std::milli, float>(mExpectedPresentTime - TimePoint::now()),
mExpectedPresentTime == expectedVsyncTime ? "" : " (adjusted)");
- const Period vsyncPeriod = mScheduler->getVsyncSchedule().period();
+ const Period vsyncPeriod = mScheduler->getVsyncSchedule()->period();
const FenceTimePtr& previousPresentFence = getPreviousPresentFence(frameTime, vsyncPeriod);
// When backpressure propagation is enabled, we want to give a small grace period of 1ms
@@ -2417,7 +2423,7 @@
refreshArgs.devOptFlashDirtyRegionsDelay = std::chrono::milliseconds(mDebugFlashDelay);
}
- const auto prevVsyncTime = mExpectedPresentTime - mScheduler->getVsyncSchedule().period();
+ const auto prevVsyncTime = mExpectedPresentTime - mScheduler->getVsyncSchedule()->period();
const auto hwcMinWorkDuration = mVsyncConfiguration->getCurrentConfigs().hwcMinWorkDuration;
refreshArgs.earliestPresentTime = prevVsyncTime - hwcMinWorkDuration;
@@ -2503,7 +2509,7 @@
// TODO(b/160583065): Enable skip validation when SF caches all client composition layers.
const bool hasGpuUseOrReuse =
mCompositionCoverage.any(CompositionCoverage::Gpu | CompositionCoverage::GpuReuse);
- mScheduler->modulateVsync(&VsyncModulator::onDisplayRefresh, hasGpuUseOrReuse);
+ mScheduler->modulateVsync({}, &VsyncModulator::onDisplayRefresh, hasGpuUseOrReuse);
mLayersWithQueuedFrames.clear();
if (mLayerTracingEnabled && mLayerTracing.flagIsSet(LayerTracing::TRACE_COMPOSITION)) {
@@ -2645,9 +2651,9 @@
? mPresentLatencyTracker.trackPendingFrame(compositeTime, presentFenceTime)
: Duration::zero();
- const auto& schedule = mScheduler->getVsyncSchedule();
- const TimePoint vsyncDeadline = schedule.vsyncDeadlineAfter(presentTime);
- const Period vsyncPeriod = schedule.period();
+ const auto schedule = mScheduler->getVsyncSchedule();
+ const TimePoint vsyncDeadline = schedule->vsyncDeadlineAfter(presentTime);
+ const Period vsyncPeriod = schedule->period();
const nsecs_t vsyncPhase = mVsyncConfiguration->getCurrentConfigs().late.sfOffset;
const CompositorTiming compositorTiming(vsyncDeadline.ns(), vsyncPeriod.ns(), vsyncPhase,
@@ -2722,15 +2728,19 @@
mTimeStats->incrementTotalFrames();
mTimeStats->setPresentFenceGlobal(presentFenceTime);
- const bool isInternalDisplay = display &&
- FTL_FAKE_GUARD(mStateLock, mPhysicalDisplays)
- .get(display->getPhysicalId())
- .transform(&PhysicalDisplay::isInternal)
- .value_or(false);
-
- if (isInternalDisplay && display && display->getPowerMode() == hal::PowerMode::ON &&
- presentFenceTime->isValid()) {
- mScheduler->addPresentFence(std::move(presentFenceTime));
+ {
+ ftl::FakeGuard guard(mStateLock);
+ for (const auto& [id, physicalDisplay] : mPhysicalDisplays) {
+ if (auto displayDevice = getDisplayDeviceLocked(id);
+ displayDevice && displayDevice->isPoweredOn() && physicalDisplay.isInternal()) {
+ auto presentFenceTimeI = display && display->getPhysicalId() == id
+ ? std::move(presentFenceTime)
+ : std::make_shared<FenceTime>(getHwComposer().getPresentFence(id));
+ if (presentFenceTimeI->isValid()) {
+ mScheduler->addPresentFence(id, std::move(presentFenceTimeI));
+ }
+ }
+ }
}
const bool isDisplayConnected =
@@ -2738,7 +2748,7 @@
if (!hasSyncFramework) {
if (isDisplayConnected && display->isPoweredOn()) {
- mScheduler->enableHardwareVsync();
+ mScheduler->enableHardwareVsync(display->getPhysicalId());
}
}
@@ -2848,7 +2858,7 @@
// so we can call commitTransactionsLocked unconditionally.
// We clear the flags with mStateLock held to guarantee that
// mCurrentState won't change until the transaction is committed.
- mScheduler->modulateVsync(&VsyncModulator::onTransactionCommit);
+ mScheduler->modulateVsync({}, &VsyncModulator::onTransactionCommit);
commitTransactionsLocked(clearTransactionFlags(eTransactionMask));
mDebugInTransaction = 0;
@@ -3687,10 +3697,9 @@
mScheduler = std::make_unique<Scheduler>(static_cast<ICompositor&>(*this),
static_cast<ISchedulerCallback&>(*this), features,
std::move(modulatorPtr));
- mScheduler->createVsyncSchedule(features);
mScheduler->registerDisplay(display->getPhysicalId(), display->holdRefreshRateSelector());
- setVsyncEnabled(false);
+ setVsyncEnabled(display->getPhysicalId(), false);
mScheduler->startTimers();
const auto configs = mVsyncConfiguration->getCurrentConfigs();
@@ -3706,7 +3715,7 @@
/* workDuration */ activeRefreshRate.getPeriod(),
/* readyDuration */ configs.late.sfWorkDuration);
- mScheduler->initVsync(mScheduler->getVsyncSchedule().getDispatch(),
+ mScheduler->initVsync(mScheduler->getVsyncSchedule()->getDispatch(),
*mFrameTimeline->getTokenManager(), configs.late.sfWorkDuration);
mRegionSamplingThread =
@@ -3917,7 +3926,7 @@
void SurfaceFlinger::setTransactionFlags(uint32_t mask, TransactionSchedule schedule,
const sp<IBinder>& applyToken, FrameHint frameHint) {
- mScheduler->modulateVsync(&VsyncModulator::setTransactionSchedule, schedule, applyToken);
+ mScheduler->modulateVsync({}, &VsyncModulator::setTransactionSchedule, schedule, applyToken);
if (const bool scheduled = mTransactionFlags.fetch_or(mask) & mask; !scheduled) {
scheduleCommit(frameHint);
@@ -3944,7 +3953,7 @@
return TransactionReadiness::NotReady;
}
- if (!mScheduler->isVsyncValid(mExpectedPresentTime, transaction.originUid)) {
+ if (!mScheduler->isVsyncTargetForUid(mExpectedPresentTime, transaction.originUid)) {
ATRACE_NAME("!isVsyncValid");
return TransactionReadiness::NotReady;
}
@@ -4088,7 +4097,7 @@
return false;
}
- const Duration earlyLatchVsyncThreshold = mScheduler->getVsyncSchedule().period() / 2;
+ const Duration earlyLatchVsyncThreshold = mScheduler->getVsyncSchedule()->period() / 2;
return predictedPresentTime >= expectedPresentTime &&
predictedPresentTime - expectedPresentTime >= earlyLatchVsyncThreshold;
@@ -4985,10 +4994,11 @@
ALOGW("Couldn't set SCHED_FIFO on display on: %s\n", strerror(errno));
}
getHwComposer().setPowerMode(displayId, mode);
- if (isActiveDisplay && mode != hal::PowerMode::DOZE_SUSPEND) {
- setHWCVsyncEnabled(displayId, mHWCVsyncPendingState);
- mScheduler->onScreenAcquired(mAppConnectionHandle);
- mScheduler->resyncToHardwareVsync(true, refreshRate);
+ if (mode != hal::PowerMode::DOZE_SUSPEND) {
+ if (isActiveDisplay) {
+ mScheduler->onScreenAcquired(mAppConnectionHandle);
+ }
+ mScheduler->resyncToHardwareVsync(displayId, true /* allowToEnable */, refreshRate);
}
mVisibleRegionsDirty = true;
@@ -5001,33 +5011,34 @@
if (SurfaceFlinger::setSchedAttr(false) != NO_ERROR) {
ALOGW("Couldn't set uclamp.min on display off: %s\n", strerror(errno));
}
- if (isActiveDisplay && *currentModeOpt != hal::PowerMode::DOZE_SUSPEND) {
- mScheduler->disableHardwareVsync(true);
- mScheduler->onScreenReleased(mAppConnectionHandle);
+ if (*currentModeOpt != hal::PowerMode::DOZE_SUSPEND) {
+ mScheduler->disableHardwareVsync(displayId, true);
+ if (isActiveDisplay) {
+ mScheduler->onScreenReleased(mAppConnectionHandle);
+ }
}
- // Make sure HWVsync is disabled before turning off the display
- setHWCVsyncEnabled(displayId, hal::Vsync::DISABLE);
-
getHwComposer().setPowerMode(displayId, mode);
mVisibleRegionsDirty = true;
// from this point on, SF will stop drawing on this display
} else if (mode == hal::PowerMode::DOZE || mode == hal::PowerMode::ON) {
// Update display while dozing
getHwComposer().setPowerMode(displayId, mode);
- if (isActiveDisplay && *currentModeOpt == hal::PowerMode::DOZE_SUSPEND) {
+ if (*currentModeOpt == hal::PowerMode::DOZE_SUSPEND) {
+ if (isActiveDisplay) {
+ mScheduler->onScreenAcquired(mAppConnectionHandle);
+ }
ALOGI("Force repainting for DOZE_SUSPEND -> DOZE or ON.");
mVisibleRegionsDirty = true;
scheduleRepaint();
- mScheduler->onScreenAcquired(mAppConnectionHandle);
- mScheduler->resyncToHardwareVsync(true, refreshRate);
+ mScheduler->resyncToHardwareVsync(displayId, true /* allowToEnable */, refreshRate);
}
} else if (mode == hal::PowerMode::DOZE_SUSPEND) {
// Leave display going to doze
if (isActiveDisplay) {
- mScheduler->disableHardwareVsync(true);
mScheduler->onScreenReleased(mAppConnectionHandle);
}
+ mScheduler->disableHardwareVsync(displayId, true);
getHwComposer().setPowerMode(displayId, mode);
} else {
ALOGE("Attempting to set unknown power mode: %d\n", mode);
@@ -5037,8 +5048,8 @@
if (isActiveDisplay) {
mTimeStats->setPowerMode(mode);
mRefreshRateStats->setPowerMode(mode);
- mScheduler->setDisplayPowerMode(mode);
}
+ mScheduler->setDisplayPowerMode(displayId, mode);
ALOGD("Finished setting power mode %d on display %s", mode, to_string(displayId).c_str());
}
@@ -5216,14 +5227,6 @@
mScheduler->dump(dumper);
- // TODO(b/241286146): Move to Scheduler.
- {
- utils::Dumper::Indent indent(dumper);
- dumper.dump("lastHwcVsyncState"sv, mLastHWCVsyncState);
- dumper.dump("pendingHwcVsyncState"sv, mHWCVsyncPendingState);
- }
- dumper.eol();
-
// TODO(b/241285876): Move to DisplayModeController.
dumper.dump("debugDisplayModeSetByBackdoor"sv, mDebugDisplayModeSetByBackdoor);
dumper.eol();
diff --git a/services/surfaceflinger/SurfaceFlinger.h b/services/surfaceflinger/SurfaceFlinger.h
index 5b9bfd8..1eb1fda 100644
--- a/services/surfaceflinger/SurfaceFlinger.h
+++ b/services/surfaceflinger/SurfaceFlinger.h
@@ -74,6 +74,7 @@
#include "FrontEnd/LayerSnapshot.h"
#include "FrontEnd/TransactionHandler.h"
#include "LayerVector.h"
+#include "Scheduler/ISchedulerCallback.h"
#include "Scheduler/RefreshRateSelector.h"
#include "Scheduler/RefreshRateStats.h"
#include "Scheduler/Scheduler.h"
@@ -618,7 +619,7 @@
// Toggles hardware VSYNC by calling into HWC.
// TODO(b/241286146): Rename for self-explanatory API.
- void setVsyncEnabled(bool) override;
+ void setVsyncEnabled(PhysicalDisplayId, bool) override;
void requestDisplayModes(std::vector<display::DisplayModeRequest>) override;
void kernelTimerChanged(bool expired) override;
void triggerOnFrameRateOverridesChanged() override;
@@ -953,9 +954,9 @@
*/
nsecs_t getVsyncPeriodFromHWC() const REQUIRES(mStateLock);
- void setHWCVsyncEnabled(PhysicalDisplayId id, hal::Vsync enabled) {
- mLastHWCVsyncState = enabled;
- getHwComposer().setVsyncEnabled(id, enabled);
+ void setHWCVsyncEnabled(PhysicalDisplayId id, bool enabled) {
+ hal::Vsync halState = enabled ? hal::Vsync::ENABLE : hal::Vsync::DISABLE;
+ getHwComposer().setVsyncEnabled(id, halState);
}
using FenceTimePtr = std::shared_ptr<FenceTime>;
@@ -1090,7 +1091,15 @@
pid_t mPid;
std::future<void> mRenderEnginePrimeCacheFuture;
- // access must be protected by mStateLock
+ // mStateLock has conventions related to the current thread, because only
+ // the main thread should modify variables protected by mStateLock.
+ // - read access from a non-main thread must lock mStateLock, since the main
+ // thread may modify these variables.
+ // - write access from a non-main thread is not permitted.
+ // - read access from the main thread can use an ftl::FakeGuard, since other
+ // threads must not modify these variables.
+ // - write access from the main thread must lock mStateLock, since another
+ // thread may be reading these variables.
mutable Mutex mStateLock;
State mCurrentState{LayerVector::StateSet::Current};
std::atomic<int32_t> mTransactionFlags = 0;
@@ -1281,9 +1290,6 @@
TimePoint mScheduledPresentTime GUARDED_BY(kMainThreadContext);
TimePoint mExpectedPresentTime GUARDED_BY(kMainThreadContext);
- hal::Vsync mHWCVsyncPendingState = hal::Vsync::DISABLE;
- hal::Vsync mLastHWCVsyncState = hal::Vsync::DISABLE;
-
// below flags are set by main thread only
bool mSetActiveModePending = false;
diff --git a/services/surfaceflinger/fuzzer/surfaceflinger_fuzzers_utils.h b/services/surfaceflinger/fuzzer/surfaceflinger_fuzzers_utils.h
index cdffbb4..5303db3 100644
--- a/services/surfaceflinger/fuzzer/surfaceflinger_fuzzers_utils.h
+++ b/services/surfaceflinger/fuzzer/surfaceflinger_fuzzers_utils.h
@@ -226,27 +226,27 @@
TestableScheduler(const std::shared_ptr<scheduler::RefreshRateSelector>& selectorPtr,
sp<VsyncModulator> modulatorPtr, ISchedulerCallback& callback)
: TestableScheduler(std::make_unique<android::mock::VsyncController>(),
- std::make_unique<android::mock::VSyncTracker>(), selectorPtr,
+ std::make_shared<android::mock::VSyncTracker>(), selectorPtr,
std::move(modulatorPtr), callback) {}
TestableScheduler(std::unique_ptr<VsyncController> controller,
- std::unique_ptr<VSyncTracker> tracker,
+ VsyncSchedule::TrackerPtr tracker,
std::shared_ptr<RefreshRateSelector> selectorPtr,
sp<VsyncModulator> modulatorPtr, ISchedulerCallback& callback)
: Scheduler(*this, callback, Feature::kContentDetection, std::move(modulatorPtr)) {
- mVsyncSchedule.emplace(VsyncSchedule(std::move(tracker), nullptr, std::move(controller)));
-
const auto displayId = selectorPtr->getActiveMode().modePtr->getPhysicalDisplayId();
registerDisplay(displayId, std::move(selectorPtr));
+ mVsyncSchedules.emplace_or_replace(displayId,
+ std::shared_ptr<VsyncSchedule>(
+ new VsyncSchedule(displayId, std::move(tracker),
+ nullptr,
+ std::move(controller))));
}
ConnectionHandle createConnection(std::unique_ptr<EventThread> eventThread) {
return Scheduler::createConnection(std::move(eventThread));
}
- auto &mutablePrimaryHWVsyncEnabled() { return mPrimaryHWVsyncEnabled; }
- auto &mutableHWVsyncAvailable() { return mHWVsyncAvailable; }
-
auto &mutableLayerHistory() { return mLayerHistory; }
auto refreshRateSelector() { return leaderSelectorPtr(); }
@@ -649,10 +649,10 @@
// The ISchedulerCallback argument can be nullptr for a no-op implementation.
void setupScheduler(std::unique_ptr<scheduler::VsyncController> vsyncController,
- std::unique_ptr<scheduler::VSyncTracker> vsyncTracker,
+ std::shared_ptr<scheduler::VSyncTracker> vsyncTracker,
std::unique_ptr<EventThread> appEventThread,
std::unique_ptr<EventThread> sfEventThread,
- scheduler::ISchedulerCallback *callback = nullptr,
+ scheduler::ISchedulerCallback* callback = nullptr,
bool hasMultipleModes = false) {
constexpr DisplayModeId kModeId60{0};
DisplayModes modes = makeModes(mock::createDisplayMode(kModeId60, 60_Hz));
@@ -791,7 +791,7 @@
}
private:
- void setVsyncEnabled(bool) override {}
+ void setVsyncEnabled(PhysicalDisplayId, bool) override {}
void requestDisplayModes(std::vector<display::DisplayModeRequest>) override {}
void kernelTimerChanged(bool) override {}
void triggerOnFrameRateOverridesChanged() override {}
diff --git a/services/surfaceflinger/fuzzer/surfaceflinger_scheduler_fuzzer.cpp b/services/surfaceflinger/fuzzer/surfaceflinger_scheduler_fuzzer.cpp
index 44805db..b7b42ab 100644
--- a/services/surfaceflinger/fuzzer/surfaceflinger_scheduler_fuzzer.cpp
+++ b/services/surfaceflinger/fuzzer/surfaceflinger_scheduler_fuzzer.cpp
@@ -47,19 +47,23 @@
PowerMode::DOZE_SUSPEND, PowerMode::ON_SUSPEND};
constexpr uint16_t kRandomStringLength = 256;
-constexpr std::chrono::duration kSyncPeriod(16ms);
-
template <typename T>
void dump(T* component, FuzzedDataProvider* fdp) {
std::string res = fdp->ConsumeRandomLengthString(kRandomStringLength);
component->dump(res);
}
-class SchedulerFuzzer {
+class SchedulerFuzzer : public IEventThreadCallback {
public:
SchedulerFuzzer(const uint8_t* data, size_t size) : mFdp(data, size){};
void process();
+ // IEventThreadCallback overrides.
+ bool isVsyncTargetForUid(TimePoint /* expectedVsyncTime */, uid_t) const override {
+ return true;
+ }
+ Fps getLeaderRenderFrameRate(uid_t) const override { return 60_Hz; }
+
private:
void fuzzRefreshRateSelection();
void fuzzRefreshRateSelector();
@@ -76,7 +80,7 @@
FuzzedDataProvider mFdp;
- std::optional<scheduler::VsyncSchedule> mVsyncSchedule;
+ std::shared_ptr<scheduler::VsyncSchedule> mVsyncSchedule;
};
PhysicalDisplayId SchedulerFuzzer::getPhysicalDisplayId() {
@@ -90,12 +94,12 @@
}
void SchedulerFuzzer::fuzzEventThread() {
- mVsyncSchedule.emplace(scheduler::VsyncSchedule(std::make_unique<mock::VSyncTracker>(),
- std::make_unique<mock::VSyncDispatch>(),
- nullptr));
- const auto getVsyncPeriod = [](uid_t /* uid */) { return kSyncPeriod.count(); };
+ mVsyncSchedule = std::shared_ptr<scheduler::VsyncSchedule>(
+ new scheduler::VsyncSchedule(getPhysicalDisplayId(),
+ std::make_shared<mock::VSyncTracker>(),
+ std::make_shared<mock::VSyncDispatch>(), nullptr));
std::unique_ptr<android::impl::EventThread> thread = std::make_unique<
- android::impl::EventThread>("fuzzer", *mVsyncSchedule, nullptr, nullptr, getVsyncPeriod,
+ android::impl::EventThread>("fuzzer", mVsyncSchedule, *this, nullptr /* TokenManager */,
(std::chrono::nanoseconds)mFdp.ConsumeIntegral<uint64_t>(),
(std::chrono::nanoseconds)mFdp.ConsumeIntegral<uint64_t>());
@@ -132,7 +136,7 @@
}
void SchedulerFuzzer::fuzzVSyncDispatchTimerQueue() {
- FuzzImplVSyncTracker stubTracker{mFdp.ConsumeIntegral<nsecs_t>()};
+ auto stubTracker = std::make_shared<FuzzImplVSyncTracker>(mFdp.ConsumeIntegral<nsecs_t>());
scheduler::VSyncDispatchTimerQueue
mDispatch{std::make_unique<scheduler::ControllableClock>(), stubTracker,
mFdp.ConsumeIntegral<nsecs_t>() /*dispatchGroupThreshold*/,
@@ -145,17 +149,17 @@
scheduler::VSyncDispatchTimerQueueEntry entry(
"fuzz", [](auto, auto, auto) {},
mFdp.ConsumeIntegral<nsecs_t>() /*vSyncMoveThreshold*/);
- entry.update(stubTracker, 0);
+ entry.update(*stubTracker, 0);
entry.schedule({.workDuration = mFdp.ConsumeIntegral<nsecs_t>(),
.readyDuration = mFdp.ConsumeIntegral<nsecs_t>(),
.earliestVsync = mFdp.ConsumeIntegral<nsecs_t>()},
- stubTracker, 0);
+ *stubTracker, 0);
entry.disarm();
entry.ensureNotRunning();
entry.schedule({.workDuration = mFdp.ConsumeIntegral<nsecs_t>(),
.readyDuration = mFdp.ConsumeIntegral<nsecs_t>(),
.earliestVsync = mFdp.ConsumeIntegral<nsecs_t>()},
- stubTracker, 0);
+ *stubTracker, 0);
auto const wakeup = entry.wakeupTime();
auto const ready = entry.readyTime();
entry.callback(entry.executing(), *wakeup, *ready);
@@ -169,8 +173,8 @@
uint16_t now = mFdp.ConsumeIntegral<uint16_t>();
uint16_t historySize = mFdp.ConsumeIntegralInRange<uint16_t>(1, UINT16_MAX);
uint16_t minimumSamplesForPrediction = mFdp.ConsumeIntegralInRange<uint16_t>(1, UINT16_MAX);
- scheduler::VSyncPredictor tracker{mFdp.ConsumeIntegral<uint16_t>() /*period*/, historySize,
- minimumSamplesForPrediction,
+ scheduler::VSyncPredictor tracker{"predictor", mFdp.ConsumeIntegral<uint16_t>() /*period*/,
+ historySize, minimumSamplesForPrediction,
mFdp.ConsumeIntegral<uint32_t>() /*outlierTolerancePercent*/};
uint16_t period = mFdp.ConsumeIntegral<uint16_t>();
tracker.setPeriod(period);
@@ -242,13 +246,15 @@
void SchedulerFuzzer::fuzzVSyncReactor() {
std::shared_ptr<FuzzImplVSyncTracker> vSyncTracker = std::make_shared<FuzzImplVSyncTracker>();
- scheduler::VSyncReactor reactor(std::make_unique<ClockWrapper>(
+ scheduler::VSyncReactor reactor("fuzzer_reactor",
+ std::make_unique<ClockWrapper>(
std::make_shared<FuzzImplClock>()),
*vSyncTracker, mFdp.ConsumeIntegral<uint8_t>() /*pendingLimit*/,
false);
- reactor.startPeriodTransition(mFdp.ConsumeIntegral<nsecs_t>());
- bool periodFlushed = mFdp.ConsumeBool();
+ reactor.startPeriodTransition(mFdp.ConsumeIntegral<nsecs_t>(), mFdp.ConsumeBool());
+ bool periodFlushed = false; // Value does not matter, since this is an out
+ // param from addHwVsyncTimestamp.
reactor.addHwVsyncTimestamp(0, std::nullopt, &periodFlushed);
reactor.addHwVsyncTimestamp(mFdp.ConsumeIntegral<nsecs_t>() /*newPeriod*/, std::nullopt,
&periodFlushed);
diff --git a/services/surfaceflinger/tests/unittests/CompositionTest.cpp b/services/surfaceflinger/tests/unittests/CompositionTest.cpp
index 0416e93..419c818 100644
--- a/services/surfaceflinger/tests/unittests/CompositionTest.cpp
+++ b/services/surfaceflinger/tests/unittests/CompositionTest.cpp
@@ -139,7 +139,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp b/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp
index e0b508a..214b028 100644
--- a/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp
+++ b/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp
@@ -74,8 +74,8 @@
mock::EventThread::kCallingUid,
ResyncCallback())));
- mFlinger.setupScheduler(std::unique_ptr<scheduler::VsyncController>(mVsyncController),
- std::unique_ptr<scheduler::VSyncTracker>(mVSyncTracker),
+ mFlinger.setupScheduler(std::make_unique<mock::VsyncController>(),
+ std::make_shared<mock::VSyncTracker>(),
std::unique_ptr<EventThread>(mEventThread),
std::unique_ptr<EventThread>(mSFEventThread),
TestableSurfaceFlinger::SchedulerCallbackImpl::kMock);
diff --git a/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h b/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h
index 223f4db..c9245d6 100644
--- a/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h
+++ b/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h
@@ -128,8 +128,6 @@
renderengine::mock::RenderEngine* mRenderEngine = new renderengine::mock::RenderEngine();
Hwc2::mock::Composer* mComposer = nullptr;
- mock::VsyncController* mVsyncController = new mock::VsyncController;
- mock::VSyncTracker* mVSyncTracker = new mock::VSyncTracker;
mock::EventThread* mEventThread = new mock::EventThread;
mock::EventThread* mSFEventThread = new mock::EventThread;
diff --git a/services/surfaceflinger/tests/unittests/EventThreadTest.cpp b/services/surfaceflinger/tests/unittests/EventThreadTest.cpp
index b3aba37..5cecb8e 100644
--- a/services/surfaceflinger/tests/unittests/EventThreadTest.cpp
+++ b/services/surfaceflinger/tests/unittests/EventThreadTest.cpp
@@ -52,11 +52,9 @@
constexpr PhysicalDisplayId DISPLAY_ID_64BIT =
PhysicalDisplayId::fromEdid(0xffu, 0xffffu, 0xffff'ffffu);
-constexpr std::chrono::duration VSYNC_PERIOD(16ms);
-
} // namespace
-class EventThreadTest : public testing::Test {
+class EventThreadTest : public testing::Test, public IEventThreadCallback {
protected:
static constexpr std::chrono::nanoseconds kWorkDuration = 0ms;
static constexpr std::chrono::nanoseconds kReadyDuration = 3ms;
@@ -97,7 +95,7 @@
void expectConfigChangedEventReceivedByConnection(PhysicalDisplayId expectedDisplayId,
int32_t expectedConfigId,
nsecs_t expectedVsyncPeriod);
- void expectThrottleVsyncReceived(nsecs_t expectedTimestamp, uid_t);
+ void expectThrottleVsyncReceived(TimePoint expectedTimestamp, uid_t);
void expectUidFrameRateMappingEventReceivedByConnection(PhysicalDisplayId expectedDisplayId,
std::vector<FrameRateOverride>);
@@ -106,6 +104,14 @@
mThread->onVsync(expectedPresentationTime, timestamp, deadlineTimestamp);
}
+ // IEventThreadCallback overrides.
+ bool isVsyncTargetForUid(TimePoint expectedVsyncTime, uid_t uid) const override {
+ mThrottleVsyncCallRecorder.getInvocable()(expectedVsyncTime, uid);
+ return uid != mThrottledConnectionUid;
+ }
+
+ Fps getLeaderRenderFrameRate(uid_t uid) const override { return 60_Hz; }
+
AsyncCallRecorderWithCannedReturn<
scheduler::ScheduleResult (*)(scheduler::VSyncDispatch::CallbackToken,
scheduler::VSyncDispatch::ScheduleTiming)>
@@ -121,11 +127,11 @@
AsyncCallRecorder<void (*)(scheduler::VSyncDispatch::CallbackToken)>
mVSyncCallbackUnregisterRecorder;
AsyncCallRecorder<void (*)()> mResyncCallRecorder;
- AsyncCallRecorder<void (*)(nsecs_t, uid_t)> mThrottleVsyncCallRecorder;
+ mutable AsyncCallRecorder<void (*)(TimePoint, uid_t)> mThrottleVsyncCallRecorder;
ConnectionEventRecorder mConnectionEventCallRecorder{0};
ConnectionEventRecorder mThrottledConnectionEventCallRecorder{0};
- std::optional<scheduler::VsyncSchedule> mVsyncSchedule;
+ std::shared_ptr<scheduler::VsyncSchedule> mVsyncSchedule;
std::unique_ptr<impl::EventThread> mThread;
sp<MockEventThreadConnection> mConnection;
sp<MockEventThreadConnection> mThrottledConnection;
@@ -140,12 +146,12 @@
::testing::UnitTest::GetInstance()->current_test_info();
ALOGD("**** Setting up for %s.%s\n", test_info->test_case_name(), test_info->name());
- mVsyncSchedule.emplace(scheduler::VsyncSchedule(std::make_unique<mock::VSyncTracker>(),
- std::make_unique<mock::VSyncDispatch>(),
- nullptr));
-
- mock::VSyncDispatch& mockDispatch =
- *static_cast<mock::VSyncDispatch*>(&mVsyncSchedule->getDispatch());
+ auto mockDispatchPtr = std::make_shared<mock::VSyncDispatch>();
+ mVsyncSchedule = std::shared_ptr<scheduler::VsyncSchedule>(
+ new scheduler::VsyncSchedule(INTERNAL_DISPLAY_ID,
+ std::make_shared<mock::VSyncTracker>(), mockDispatchPtr,
+ nullptr));
+ mock::VSyncDispatch& mockDispatch = *mockDispatchPtr;
EXPECT_CALL(mockDispatch, registerCallback(_, _))
.WillRepeatedly(Invoke(mVSyncCallbackRegisterRecorder.getInvocable()));
EXPECT_CALL(mockDispatch, schedule(_, _))
@@ -180,19 +186,10 @@
}
void EventThreadTest::createThread() {
- const auto throttleVsync = [&](nsecs_t expectedVsyncTimestamp, uid_t uid) {
- mThrottleVsyncCallRecorder.getInvocable()(expectedVsyncTimestamp, uid);
- return (uid == mThrottledConnectionUid);
- };
- const auto getVsyncPeriod = [](uid_t uid) {
- return VSYNC_PERIOD.count();
- };
-
mTokenManager = std::make_unique<frametimeline::impl::TokenManager>();
mThread =
- std::make_unique<impl::EventThread>(/*std::move(source), */ "EventThreadTest",
- *mVsyncSchedule, mTokenManager.get(), throttleVsync,
- getVsyncPeriod, kWorkDuration, kReadyDuration);
+ std::make_unique<impl::EventThread>("EventThreadTest", mVsyncSchedule, *this,
+ mTokenManager.get(), kWorkDuration, kReadyDuration);
// EventThread should register itself as VSyncSource callback.
EXPECT_TRUE(mVSyncCallbackRegisterRecorder.waitForCall().has_value());
@@ -225,7 +222,7 @@
EXPECT_EQ(expectedReadyDuration.count(), std::get<1>(args.value()).readyDuration);
}
-void EventThreadTest::expectThrottleVsyncReceived(nsecs_t expectedTimestamp, uid_t uid) {
+void EventThreadTest::expectThrottleVsyncReceived(TimePoint expectedTimestamp, uid_t uid) {
auto args = mThrottleVsyncCallRecorder.waitForCall();
ASSERT_TRUE(args.has_value());
EXPECT_EQ(expectedTimestamp, std::get<0>(args.value()));
@@ -376,7 +373,7 @@
// Use the received callback to signal a first vsync event.
// The throttler should receive the event, as well as the connection.
onVSyncEvent(123, 456, 789);
- expectThrottleVsyncReceived(456, mConnectionUid);
+ expectThrottleVsyncReceived(TimePoint::fromNs(456), mConnectionUid);
expectVsyncEventReceivedByConnection(123, 1u);
// EventThread is requesting one more callback due to VsyncRequest::SingleSuppressCallback
@@ -494,17 +491,17 @@
// Send a vsync event. EventThread should then make a call to the
// throttler, and the connection.
onVSyncEvent(123, 456, 789);
- expectThrottleVsyncReceived(456, mConnectionUid);
+ expectThrottleVsyncReceived(TimePoint::fromNs(456), mConnectionUid);
expectVsyncEventReceivedByConnection(123, 1u);
// A second event should go to the same places.
onVSyncEvent(456, 123, 0);
- expectThrottleVsyncReceived(123, mConnectionUid);
+ expectThrottleVsyncReceived(TimePoint::fromNs(123), mConnectionUid);
expectVsyncEventReceivedByConnection(456, 2u);
// A third event should go to the same places.
onVSyncEvent(789, 777, 111);
- expectThrottleVsyncReceived(777, mConnectionUid);
+ expectThrottleVsyncReceived(TimePoint::fromNs(777), mConnectionUid);
expectVsyncEventReceivedByConnection(789, 3u);
}
@@ -743,7 +740,7 @@
// Use the received callback to signal a first vsync event.
// The throttler should receive the event, but not the connection.
onVSyncEvent(123, 456, 789);
- expectThrottleVsyncReceived(456, mThrottledConnectionUid);
+ expectThrottleVsyncReceived(TimePoint::fromNs(456), mThrottledConnectionUid);
mThrottledConnectionEventCallRecorder.waitForUnexpectedCall();
expectVSyncCallbackScheduleReceived(true);
@@ -751,7 +748,7 @@
// The throttler should receive the event, but the connection should
// not as it was only interested in the first.
onVSyncEvent(456, 123, 0);
- expectThrottleVsyncReceived(123, mThrottledConnectionUid);
+ expectThrottleVsyncReceived(TimePoint::fromNs(123), mThrottledConnectionUid);
EXPECT_FALSE(mConnectionEventCallRecorder.waitForUnexpectedCall().has_value());
expectVSyncCallbackScheduleReceived(true);
diff --git a/services/surfaceflinger/tests/unittests/FpsReporterTest.cpp b/services/surfaceflinger/tests/unittests/FpsReporterTest.cpp
index 1cd9e49..248061c 100644
--- a/services/surfaceflinger/tests/unittests/FpsReporterTest.cpp
+++ b/services/surfaceflinger/tests/unittests/FpsReporterTest.cpp
@@ -137,7 +137,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/FrameRateSelectionPriorityTest.cpp b/services/surfaceflinger/tests/unittests/FrameRateSelectionPriorityTest.cpp
index ac63a0e..ff7c2c9 100644
--- a/services/surfaceflinger/tests/unittests/FrameRateSelectionPriorityTest.cpp
+++ b/services/surfaceflinger/tests/unittests/FrameRateSelectionPriorityTest.cpp
@@ -125,7 +125,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/GameModeTest.cpp b/services/surfaceflinger/tests/unittests/GameModeTest.cpp
index 29aa717..ddf871b 100644
--- a/services/surfaceflinger/tests/unittests/GameModeTest.cpp
+++ b/services/surfaceflinger/tests/unittests/GameModeTest.cpp
@@ -76,7 +76,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/HWComposerTest.cpp b/services/surfaceflinger/tests/unittests/HWComposerTest.cpp
index afbc57a..9534f3b 100644
--- a/services/surfaceflinger/tests/unittests/HWComposerTest.cpp
+++ b/services/surfaceflinger/tests/unittests/HWComposerTest.cpp
@@ -118,6 +118,34 @@
}
}
+TEST_F(HWComposerTest, onVsync) {
+ constexpr hal::HWDisplayId kHwcDisplayId = 1;
+ expectHotplugConnect(kHwcDisplayId);
+
+ const auto info = mHwc.onHotplug(kHwcDisplayId, hal::Connection::CONNECTED);
+ ASSERT_TRUE(info);
+
+ const auto physicalDisplayId = info->id;
+
+ // Deliberately chosen not to match DisplayData.lastPresentTimestamp's
+ // initial value.
+ constexpr nsecs_t kTimestamp = 1;
+ auto displayIdOpt = mHwc.onVsync(kHwcDisplayId, kTimestamp);
+ ASSERT_TRUE(displayIdOpt);
+ EXPECT_EQ(physicalDisplayId, displayIdOpt);
+
+ // Attempt to send the same time stamp again.
+ displayIdOpt = mHwc.onVsync(kHwcDisplayId, kTimestamp);
+ EXPECT_FALSE(displayIdOpt);
+}
+
+TEST_F(HWComposerTest, onVsyncInvalid) {
+ constexpr hal::HWDisplayId kInvalidHwcDisplayId = 2;
+ constexpr nsecs_t kTimestamp = 1;
+ const auto displayIdOpt = mHwc.onVsync(kInvalidHwcDisplayId, kTimestamp);
+ EXPECT_FALSE(displayIdOpt);
+}
+
struct MockHWC2ComposerCallback final : StrictMock<HWC2::ComposerCallback> {
MOCK_METHOD2(onComposerHalHotplug, void(hal::HWDisplayId, hal::Connection));
MOCK_METHOD1(onComposerHalRefresh, void(hal::HWDisplayId));
diff --git a/services/surfaceflinger/tests/unittests/LayerTestUtils.cpp b/services/surfaceflinger/tests/unittests/LayerTestUtils.cpp
index ee42e19..23506b1 100644
--- a/services/surfaceflinger/tests/unittests/LayerTestUtils.cpp
+++ b/services/surfaceflinger/tests/unittests/LayerTestUtils.cpp
@@ -64,7 +64,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/MessageQueueTest.cpp b/services/surfaceflinger/tests/unittests/MessageQueueTest.cpp
index 7aa5201..8f1b450 100644
--- a/services/surfaceflinger/tests/unittests/MessageQueueTest.cpp
+++ b/services/surfaceflinger/tests/unittests/MessageQueueTest.cpp
@@ -67,12 +67,12 @@
struct MessageQueueTest : testing::Test {
void SetUp() override {
- EXPECT_CALL(mVSyncDispatch, registerCallback(_, "sf")).WillOnce(Return(mCallbackToken));
+ EXPECT_CALL(*mVSyncDispatch, registerCallback(_, "sf")).WillOnce(Return(mCallbackToken));
EXPECT_NO_FATAL_FAILURE(mEventQueue.initVsync(mVSyncDispatch, mTokenManager, kDuration));
- EXPECT_CALL(mVSyncDispatch, unregisterCallback(mCallbackToken)).Times(1);
+ EXPECT_CALL(*mVSyncDispatch, unregisterCallback(mCallbackToken)).Times(1);
}
- mock::VSyncDispatch mVSyncDispatch;
+ std::shared_ptr<mock::VSyncDispatch> mVSyncDispatch = std::make_shared<mock::VSyncDispatch>();
MockTokenManager mTokenManager;
TestableMessageQueue mEventQueue;
@@ -90,7 +90,7 @@
.earliestVsync = 0};
EXPECT_FALSE(mEventQueue.getScheduledFrameTime());
- EXPECT_CALL(mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(1234));
+ EXPECT_CALL(*mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(1234));
EXPECT_NO_FATAL_FAILURE(mEventQueue.scheduleFrame());
ASSERT_TRUE(mEventQueue.getScheduledFrameTime());
@@ -103,13 +103,13 @@
.readyDuration = 0,
.earliestVsync = 0};
- EXPECT_CALL(mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(1234));
+ EXPECT_CALL(*mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(1234));
EXPECT_NO_FATAL_FAILURE(mEventQueue.scheduleFrame());
ASSERT_TRUE(mEventQueue.getScheduledFrameTime());
EXPECT_EQ(1234, mEventQueue.getScheduledFrameTime()->time_since_epoch().count());
- EXPECT_CALL(mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(4567));
+ EXPECT_CALL(*mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(4567));
EXPECT_NO_FATAL_FAILURE(mEventQueue.scheduleFrame());
ASSERT_TRUE(mEventQueue.getScheduledFrameTime());
@@ -122,7 +122,7 @@
.readyDuration = 0,
.earliestVsync = 0};
- EXPECT_CALL(mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(1234));
+ EXPECT_CALL(*mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(1234));
EXPECT_NO_FATAL_FAILURE(mEventQueue.scheduleFrame());
ASSERT_TRUE(mEventQueue.getScheduledFrameTime());
@@ -149,7 +149,7 @@
.readyDuration = 0,
.earliestVsync = kPresentTime.ns()};
- EXPECT_CALL(mVSyncDispatch, schedule(mCallbackToken, timingAfterCallback)).WillOnce(Return(0));
+ EXPECT_CALL(*mVSyncDispatch, schedule(mCallbackToken, timingAfterCallback)).WillOnce(Return(0));
EXPECT_NO_FATAL_FAILURE(mEventQueue.scheduleFrame());
}
@@ -161,7 +161,7 @@
.readyDuration = 0,
.earliestVsync = 0};
- EXPECT_CALL(mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(0));
+ EXPECT_CALL(*mVSyncDispatch, schedule(mCallbackToken, timing)).WillOnce(Return(0));
EXPECT_NO_FATAL_FAILURE(mEventQueue.scheduleFrame());
}
diff --git a/services/surfaceflinger/tests/unittests/SchedulerTest.cpp b/services/surfaceflinger/tests/unittests/SchedulerTest.cpp
index 4b15385..f0dd06d 100644
--- a/services/surfaceflinger/tests/unittests/SchedulerTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SchedulerTest.cpp
@@ -14,6 +14,7 @@
* limitations under the License.
*/
+#include <ftl/fake_guard.h>
#include <gmock/gmock.h>
#include <gtest/gtest.h>
#include <log/log.h>
@@ -176,7 +177,7 @@
ASSERT_EQ(0u, mScheduler->getNumActiveLayers());
constexpr hal::PowerMode kPowerModeOn = hal::PowerMode::ON;
- mScheduler->setDisplayPowerMode(kPowerModeOn);
+ FTL_FAKE_GUARD(kMainThreadContext, mScheduler->setDisplayPowerMode(kDisplayId1, kPowerModeOn));
constexpr uint32_t kDisplayArea = 999'999;
mScheduler->onActiveDisplayAreaChanged(kDisplayArea);
@@ -248,7 +249,7 @@
mScheduler->recordLayerHistory(layer.get(), 0, LayerHistory::LayerUpdateType::Buffer);
constexpr hal::PowerMode kPowerModeOn = hal::PowerMode::ON;
- mScheduler->setDisplayPowerMode(kPowerModeOn);
+ FTL_FAKE_GUARD(kMainThreadContext, mScheduler->setDisplayPowerMode(kDisplayId1, kPowerModeOn));
constexpr uint32_t kDisplayArea = 999'999;
mScheduler->onActiveDisplayAreaChanged(kDisplayArea);
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_DisplayModeSwitching.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_DisplayModeSwitching.cpp
index ad3bd35..31f948f 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_DisplayModeSwitching.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_DisplayModeSwitching.cpp
@@ -100,7 +100,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp
index f553a23..98644aa 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp
@@ -44,7 +44,10 @@
// We expect a scheduled commit for the display transaction.
EXPECT_CALL(*mFlinger.scheduler(), scheduleFrame()).Times(1);
- EXPECT_CALL(*mVSyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
+ EXPECT_CALL(static_cast<mock::VSyncTracker&>(
+ mFlinger.scheduler()->getVsyncSchedule()->getTracker()),
+ nextAnticipatedVSyncTimeFrom(_))
+ .WillRepeatedly(Return(0));
// --------------------------------------------------------------------
// Invocation
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_PowerHintTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_PowerHintTest.cpp
index 622717f..2a0f2ef 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_PowerHintTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_PowerHintTest.cpp
@@ -116,7 +116,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp
index ab732ed..80ad22c 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp
@@ -61,7 +61,7 @@
struct EventThreadBaseSupportedVariant {
static void setupVsyncAndEventThreadNoCallExpectations(DisplayTransactionTest* test) {
// The callback should not be notified to toggle VSYNC.
- EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(_)).Times(0);
+ EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(_, _)).Times(0);
// The event thread should not be notified.
EXPECT_CALL(*test->mEventThread, onScreenReleased()).Times(0);
@@ -71,24 +71,28 @@
struct EventThreadNotSupportedVariant : public EventThreadBaseSupportedVariant {
static void setupAcquireAndEnableVsyncCallExpectations(DisplayTransactionTest* test) {
- // These calls are only expected for the primary display.
+ // The callback should be notified to enable VSYNC.
+ EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(_, true)).Times(1);
- // Instead expect no calls.
- setupVsyncAndEventThreadNoCallExpectations(test);
+ // The event thread should not be notified.
+ EXPECT_CALL(*test->mEventThread, onScreenReleased()).Times(0);
+ EXPECT_CALL(*test->mEventThread, onScreenAcquired()).Times(0);
}
static void setupReleaseAndDisableVsyncCallExpectations(DisplayTransactionTest* test) {
- // These calls are only expected for the primary display.
+ // The callback should be notified to disable VSYNC.
+ EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(_, false)).Times(1);
- // Instead expect no calls.
- setupVsyncAndEventThreadNoCallExpectations(test);
+ // The event thread should not be notified.
+ EXPECT_CALL(*test->mEventThread, onScreenReleased()).Times(0);
+ EXPECT_CALL(*test->mEventThread, onScreenAcquired()).Times(0);
}
};
struct EventThreadIsSupportedVariant : public EventThreadBaseSupportedVariant {
static void setupAcquireAndEnableVsyncCallExpectations(DisplayTransactionTest* test) {
// The callback should be notified to enable VSYNC.
- EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(true)).Times(1);
+ EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(_, true)).Times(1);
// The event thread should be notified that the screen was acquired.
EXPECT_CALL(*test->mEventThread, onScreenAcquired()).Times(1);
@@ -96,7 +100,7 @@
static void setupReleaseAndDisableVsyncCallExpectations(DisplayTransactionTest* test) {
// The callback should be notified to disable VSYNC.
- EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(false)).Times(1);
+ EXPECT_CALL(test->mFlinger.mockSchedulerCallback(), setVsyncEnabled(_, false)).Times(1);
// The event thread should not be notified that the screen was released.
EXPECT_CALL(*test->mEventThread, onScreenReleased()).Times(1);
@@ -105,8 +109,12 @@
struct DispSyncIsSupportedVariant {
static void setupResetModelCallExpectations(DisplayTransactionTest* test) {
- EXPECT_CALL(*test->mVsyncController, startPeriodTransition(DEFAULT_VSYNC_PERIOD)).Times(1);
- EXPECT_CALL(*test->mVSyncTracker, resetModel()).Times(1);
+ auto vsyncSchedule = test->mFlinger.scheduler()->getVsyncSchedule();
+ EXPECT_CALL(static_cast<mock::VsyncController&>(vsyncSchedule->getController()),
+ startPeriodTransition(DEFAULT_VSYNC_PERIOD, false))
+ .Times(1);
+ EXPECT_CALL(static_cast<mock::VSyncTracker&>(vsyncSchedule->getTracker()), resetModel())
+ .Times(1);
}
};
@@ -262,8 +270,9 @@
return display;
}
- static void setInitialPrimaryHWVsyncEnabled(DisplayTransactionTest* test, bool enabled) {
- test->mFlinger.scheduler()->mutablePrimaryHWVsyncEnabled() = enabled;
+ static void setInitialHwVsyncEnabled(DisplayTransactionTest* test, PhysicalDisplayId id,
+ bool enabled) {
+ test->mFlinger.scheduler()->setInitialHwVsyncEnabled(id, enabled);
}
static void setupRepaintEverythingCallExpectations(DisplayTransactionTest* test) {
@@ -300,6 +309,11 @@
// A sample configuration for the external display.
// In addition to not having event thread support, we emulate not having doze
// support.
+// FIXME (b/267483230): ExternalDisplay supports the features tracked in
+// DispSyncIsSupportedVariant, but is the follower, so the
+// expectations set by DispSyncIsSupportedVariant don't match (wrong schedule).
+// We need a way to retrieve the proper DisplayId from
+// setupResetModelCallExpectations (or pass it in).
template <typename TransitionVariant>
using ExternalDisplayPowerCase =
DisplayPowerCase<ExternalDisplayVariant, DozeNotSupportedVariant<ExternalDisplayVariant>,
@@ -329,9 +343,12 @@
Case::Doze::setupComposerCallExpectations(this);
auto display =
Case::injectDisplayWithInitialPowerMode(this, Case::Transition::INITIAL_POWER_MODE);
- Case::setInitialPrimaryHWVsyncEnabled(this,
- PowerModeInitialVSyncEnabled<
- Case::Transition::INITIAL_POWER_MODE>::value);
+ auto displayId = display->getId();
+ if (auto physicalDisplayId = PhysicalDisplayId::tryCast(displayId)) {
+ Case::setInitialHwVsyncEnabled(this, *physicalDisplayId,
+ PowerModeInitialVSyncEnabled<
+ Case::Transition::INITIAL_POWER_MODE>::value);
+ }
// --------------------------------------------------------------------
// Call Expectations
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_UpdateLayerMetadataSnapshotTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_UpdateLayerMetadataSnapshotTest.cpp
index fed6a1a..7e14588 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_UpdateLayerMetadataSnapshotTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_UpdateLayerMetadataSnapshotTest.cpp
@@ -34,7 +34,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/TestableScheduler.h b/services/surfaceflinger/tests/unittests/TestableScheduler.h
index 6cf6141..c360f93 100644
--- a/services/surfaceflinger/tests/unittests/TestableScheduler.h
+++ b/services/surfaceflinger/tests/unittests/TestableScheduler.h
@@ -37,19 +37,16 @@
public:
TestableScheduler(RefreshRateSelectorPtr selectorPtr, ISchedulerCallback& callback)
: TestableScheduler(std::make_unique<mock::VsyncController>(),
- std::make_unique<mock::VSyncTracker>(), std::move(selectorPtr),
+ std::make_shared<mock::VSyncTracker>(), std::move(selectorPtr),
/* modulatorPtr */ nullptr, callback) {}
TestableScheduler(std::unique_ptr<VsyncController> controller,
- std::unique_ptr<VSyncTracker> tracker, RefreshRateSelectorPtr selectorPtr,
+ std::shared_ptr<VSyncTracker> tracker, RefreshRateSelectorPtr selectorPtr,
sp<VsyncModulator> modulatorPtr, ISchedulerCallback& callback)
: Scheduler(*this, callback, Feature::kContentDetection, std::move(modulatorPtr)) {
- mVsyncSchedule.emplace(VsyncSchedule(std::move(tracker),
- std::make_unique<mock::VSyncDispatch>(),
- std::move(controller)));
-
const auto displayId = selectorPtr->getActiveMode().modePtr->getPhysicalDisplayId();
- registerDisplay(displayId, std::move(selectorPtr));
+ registerDisplay(displayId, std::move(selectorPtr), std::move(controller),
+ std::move(tracker));
ON_CALL(*this, postMessage).WillByDefault([](sp<MessageHandler>&& handler) {
// Execute task to prevent broken promise exception on destruction.
@@ -66,13 +63,6 @@
return Scheduler::createConnection(std::move(eventThread));
}
- /* ------------------------------------------------------------------------
- * Read-write access to private data to set up preconditions and assert
- * post-conditions.
- */
- auto& mutablePrimaryHWVsyncEnabled() { return mPrimaryHWVsyncEnabled; }
- auto& mutableHWVsyncAvailable() { return mHWVsyncAvailable; }
-
auto refreshRateSelector() { return leaderSelectorPtr(); }
const auto& refreshRateSelectors() const NO_THREAD_SAFETY_ANALYSIS {
@@ -80,8 +70,21 @@
}
void registerDisplay(PhysicalDisplayId displayId, RefreshRateSelectorPtr selectorPtr) {
+ registerDisplay(displayId, std::move(selectorPtr),
+ std::make_unique<mock::VsyncController>(),
+ std::make_shared<mock::VSyncTracker>());
+ }
+
+ void registerDisplay(PhysicalDisplayId displayId, RefreshRateSelectorPtr selectorPtr,
+ std::unique_ptr<VsyncController> controller,
+ std::shared_ptr<VSyncTracker> tracker) {
ftl::FakeGuard guard(kMainThreadContext);
- Scheduler::registerDisplay(displayId, std::move(selectorPtr));
+ Scheduler::registerDisplayInternal(displayId, std::move(selectorPtr),
+ std::shared_ptr<VsyncSchedule>(
+ new VsyncSchedule(displayId, std::move(tracker),
+ std::make_shared<
+ mock::VSyncDispatch>(),
+ std::move(controller))));
}
void unregisterDisplay(PhysicalDisplayId displayId) {
@@ -157,6 +160,13 @@
Scheduler::onNonPrimaryDisplayModeChanged(handle, mode);
}
+ void setInitialHwVsyncEnabled(PhysicalDisplayId id, bool enabled) {
+ auto schedule = getVsyncSchedule(id);
+ std::lock_guard<std::mutex> lock(schedule->mHwVsyncLock);
+ schedule->mHwVsyncState = enabled ? VsyncSchedule::HwVsyncState::Enabled
+ : VsyncSchedule::HwVsyncState::Disabled;
+ }
+
private:
// ICompositor overrides:
void configure() override {}
diff --git a/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h b/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h
index 68c738f..63b79a4 100644
--- a/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h
+++ b/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h
@@ -201,7 +201,7 @@
std::variant<OneDisplayMode, TwoDisplayModes, RefreshRateSelectorPtr>;
void setupScheduler(std::unique_ptr<scheduler::VsyncController> vsyncController,
- std::unique_ptr<scheduler::VSyncTracker> vsyncTracker,
+ std::shared_ptr<scheduler::VSyncTracker> vsyncTracker,
std::unique_ptr<EventThread> appEventThread,
std::unique_ptr<EventThread> sfEventThread,
SchedulerCallbackImpl callbackImpl = SchedulerCallbackImpl::kNoOp,
@@ -253,7 +253,7 @@
std::move(modulatorPtr), callback);
}
- mScheduler->initVsync(mScheduler->getVsyncSchedule().getDispatch(), *mTokenManager, 0ms);
+ mScheduler->initVsync(mScheduler->getVsyncSchedule()->getDispatch(), *mTokenManager, 0ms);
mFlinger->mAppConnectionHandle = mScheduler->createConnection(std::move(appEventThread));
mFlinger->mSfConnectionHandle = mScheduler->createConnection(std::move(sfEventThread));
diff --git a/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp b/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp
index 859f702..a9a617b 100644
--- a/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp
+++ b/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp
@@ -83,15 +83,14 @@
mFlinger.setupComposer(std::make_unique<Hwc2::mock::Composer>());
mFlinger.setupScheduler(std::unique_ptr<mock::VsyncController>(mVsyncController),
- std::unique_ptr<mock::VSyncTracker>(mVSyncTracker),
- std::move(eventThread), std::move(sfEventThread));
+ mVSyncTracker, std::move(eventThread), std::move(sfEventThread));
mFlinger.flinger()->addTransactionReadyFilters();
}
TestableSurfaceFlinger mFlinger;
mock::VsyncController* mVsyncController = new mock::VsyncController();
- mock::VSyncTracker* mVSyncTracker = new mock::VSyncTracker();
+ std::shared_ptr<mock::VSyncTracker> mVSyncTracker = std::make_shared<mock::VSyncTracker>();
struct TransactionInfo {
Vector<ComposerState> states;
diff --git a/services/surfaceflinger/tests/unittests/TransactionFrameTracerTest.cpp b/services/surfaceflinger/tests/unittests/TransactionFrameTracerTest.cpp
index 1173d1c..b228bcb 100644
--- a/services/surfaceflinger/tests/unittests/TransactionFrameTracerTest.cpp
+++ b/services/surfaceflinger/tests/unittests/TransactionFrameTracerTest.cpp
@@ -85,7 +85,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/TransactionSurfaceFrameTest.cpp b/services/surfaceflinger/tests/unittests/TransactionSurfaceFrameTest.cpp
index ae03db4..bfebecd 100644
--- a/services/surfaceflinger/tests/unittests/TransactionSurfaceFrameTest.cpp
+++ b/services/surfaceflinger/tests/unittests/TransactionSurfaceFrameTest.cpp
@@ -84,7 +84,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp b/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp
index da87f1d..aa33716 100644
--- a/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp
+++ b/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp
@@ -117,7 +117,7 @@
ResyncCallback())));
auto vsyncController = std::make_unique<mock::VsyncController>();
- auto vsyncTracker = std::make_unique<mock::VSyncTracker>();
+ auto vsyncTracker = std::make_shared<mock::VSyncTracker>();
EXPECT_CALL(*vsyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
EXPECT_CALL(*vsyncTracker, currentPeriod())
diff --git a/services/surfaceflinger/tests/unittests/VSyncDispatchRealtimeTest.cpp b/services/surfaceflinger/tests/unittests/VSyncDispatchRealtimeTest.cpp
index 47c2dee..fcd2f56 100644
--- a/services/surfaceflinger/tests/unittests/VSyncDispatchRealtimeTest.cpp
+++ b/services/surfaceflinger/tests/unittests/VSyncDispatchRealtimeTest.cpp
@@ -109,7 +109,8 @@
class RepeatingCallbackReceiver {
public:
- RepeatingCallbackReceiver(VSyncDispatch& dispatch, nsecs_t workload, nsecs_t readyDuration)
+ RepeatingCallbackReceiver(std::shared_ptr<VSyncDispatch> dispatch, nsecs_t workload,
+ nsecs_t readyDuration)
: mWorkload(workload),
mReadyDuration(readyDuration),
mCallback(
@@ -166,9 +167,10 @@
};
TEST_F(VSyncDispatchRealtimeTest, triple_alarm) {
- FixedRateIdealStubTracker tracker;
- VSyncDispatchTimerQueue dispatch(std::make_unique<Timer>(), tracker, mDispatchGroupThreshold,
- mVsyncMoveThreshold);
+ auto tracker = std::make_shared<FixedRateIdealStubTracker>();
+ auto dispatch =
+ std::make_shared<VSyncDispatchTimerQueue>(std::make_unique<Timer>(), tracker,
+ mDispatchGroupThreshold, mVsyncMoveThreshold);
static size_t constexpr num_clients = 3;
std::array<RepeatingCallbackReceiver, num_clients>
@@ -195,14 +197,15 @@
// starts at 333hz, slides down to 43hz
TEST_F(VSyncDispatchRealtimeTest, vascillating_vrr) {
auto next_vsync_interval = toNs(3ms);
- VRRStubTracker tracker(next_vsync_interval);
- VSyncDispatchTimerQueue dispatch(std::make_unique<Timer>(), tracker, mDispatchGroupThreshold,
- mVsyncMoveThreshold);
+ auto tracker = std::make_shared<VRRStubTracker>(next_vsync_interval);
+ auto dispatch =
+ std::make_shared<VSyncDispatchTimerQueue>(std::make_unique<Timer>(), tracker,
+ mDispatchGroupThreshold, mVsyncMoveThreshold);
RepeatingCallbackReceiver cb_receiver(dispatch, toNs(1ms), toNs(5ms));
auto const on_each_frame = [&](nsecs_t last_known) {
- tracker.set_interval(next_vsync_interval += toNs(1ms), last_known);
+ tracker->set_interval(next_vsync_interval += toNs(1ms), last_known);
};
std::thread eventThread([&] { cb_receiver.repeatedly_schedule(mIterations, on_each_frame); });
@@ -213,9 +216,10 @@
// starts at 333hz, jumps to 200hz at frame 10
TEST_F(VSyncDispatchRealtimeTest, fixed_jump) {
- VRRStubTracker tracker(toNs(3ms));
- VSyncDispatchTimerQueue dispatch(std::make_unique<Timer>(), tracker, mDispatchGroupThreshold,
- mVsyncMoveThreshold);
+ auto tracker = std::make_shared<VRRStubTracker>(toNs(3ms));
+ auto dispatch =
+ std::make_shared<VSyncDispatchTimerQueue>(std::make_unique<Timer>(), tracker,
+ mDispatchGroupThreshold, mVsyncMoveThreshold);
RepeatingCallbackReceiver cb_receiver(dispatch, toNs(1ms), toNs(5ms));
@@ -223,7 +227,7 @@
auto constexpr jump_frame_at = 10u;
auto const on_each_frame = [&](nsecs_t last_known) {
if (jump_frame_counter++ == jump_frame_at) {
- tracker.set_interval(toNs(5ms), last_known);
+ tracker->set_interval(toNs(5ms), last_known);
}
};
std::thread eventThread([&] { cb_receiver.repeatedly_schedule(mIterations, on_each_frame); });
diff --git a/services/surfaceflinger/tests/unittests/VSyncDispatchTimerQueueTest.cpp b/services/surfaceflinger/tests/unittests/VSyncDispatchTimerQueueTest.cpp
index 14a2860..82daffd 100644
--- a/services/surfaceflinger/tests/unittests/VSyncDispatchTimerQueueTest.cpp
+++ b/services/surfaceflinger/tests/unittests/VSyncDispatchTimerQueueTest.cpp
@@ -116,13 +116,14 @@
class CountingCallback {
public:
- CountingCallback(VSyncDispatch& dispatch)
- : mDispatch(dispatch),
- mToken(dispatch.registerCallback(std::bind(&CountingCallback::counter, this,
- std::placeholders::_1, std::placeholders::_2,
- std::placeholders::_3),
- "test")) {}
- ~CountingCallback() { mDispatch.unregisterCallback(mToken); }
+ CountingCallback(std::shared_ptr<VSyncDispatch> dispatch)
+ : mDispatch(std::move(dispatch)),
+ mToken(mDispatch->registerCallback(std::bind(&CountingCallback::counter, this,
+ std::placeholders::_1,
+ std::placeholders::_2,
+ std::placeholders::_3),
+ "test")) {}
+ ~CountingCallback() { mDispatch->unregisterCallback(mToken); }
operator VSyncDispatch::CallbackToken() const { return mToken; }
@@ -132,7 +133,7 @@
mReadyTime.push_back(readyTime);
}
- VSyncDispatch& mDispatch;
+ std::shared_ptr<VSyncDispatch> mDispatch;
VSyncDispatch::CallbackToken mToken;
std::vector<nsecs_t> mCalls;
std::vector<nsecs_t> mWakeupTime;
@@ -141,12 +142,12 @@
class PausingCallback {
public:
- PausingCallback(VSyncDispatch& dispatch, std::chrono::milliseconds pauseAmount)
- : mDispatch(dispatch),
- mToken(dispatch.registerCallback(std::bind(&PausingCallback::pause, this,
- std::placeholders::_1,
- std::placeholders::_2),
- "test")),
+ PausingCallback(std::shared_ptr<VSyncDispatch> dispatch, std::chrono::milliseconds pauseAmount)
+ : mDispatch(std::move(dispatch)),
+ mToken(mDispatch->registerCallback(std::bind(&PausingCallback::pause, this,
+ std::placeholders::_1,
+ std::placeholders::_2),
+ "test")),
mRegistered(true),
mPauseAmount(pauseAmount) {}
~PausingCallback() { unregister(); }
@@ -181,12 +182,12 @@
void unregister() {
if (mRegistered) {
- mDispatch.unregisterCallback(mToken);
+ mDispatch->unregisterCallback(mToken);
mRegistered = false;
}
}
- VSyncDispatch& mDispatch;
+ std::shared_ptr<VSyncDispatch> mDispatch;
VSyncDispatch::CallbackToken mToken;
bool mRegistered = true;
@@ -231,22 +232,26 @@
static nsecs_t constexpr mDispatchGroupThreshold = 5;
nsecs_t const mPeriod = 1000;
nsecs_t const mVsyncMoveThreshold = 300;
- NiceMock<MockVSyncTracker> mStubTracker{mPeriod};
- VSyncDispatchTimerQueue mDispatch{createTimeKeeper(), mStubTracker, mDispatchGroupThreshold,
- mVsyncMoveThreshold};
+ std::shared_ptr<NiceMock<MockVSyncTracker>> mStubTracker =
+ std::make_shared<NiceMock<MockVSyncTracker>>(mPeriod);
+ std::shared_ptr<VSyncDispatch> mDispatch =
+ std::make_shared<VSyncDispatchTimerQueue>(createTimeKeeper(), mStubTracker,
+ mDispatchGroupThreshold, mVsyncMoveThreshold);
};
TEST_F(VSyncDispatchTimerQueueTest, unregistersSetAlarmOnDestruction) {
EXPECT_CALL(mMockClock, alarmAt(_, 900));
EXPECT_CALL(mMockClock, alarmCancel());
{
- VSyncDispatchTimerQueue mDispatch{createTimeKeeper(), mStubTracker, mDispatchGroupThreshold,
- mVsyncMoveThreshold};
+ std::shared_ptr<VSyncDispatch> mDispatch =
+ std::make_shared<VSyncDispatchTimerQueue>(createTimeKeeper(), mStubTracker,
+ mDispatchGroupThreshold,
+ mVsyncMoveThreshold);
CountingCallback cb(mDispatch);
- const auto result = mDispatch.schedule(cb,
- {.workDuration = 100,
- .readyDuration = 0,
- .earliestVsync = 1000});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(900, *result);
}
@@ -257,10 +262,10 @@
EXPECT_CALL(mMockClock, alarmAt(_, 900));
CountingCallback cb(mDispatch);
- const auto result = mDispatch.schedule(cb,
- {.workDuration = 100,
- .readyDuration = 0,
- .earliestVsync = intended});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = intended});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(900, *result);
@@ -277,14 +282,15 @@
EXPECT_CALL(mMockClock, alarmAt(_, 700)).InSequence(seq);
CountingCallback cb(mDispatch);
- auto result = mDispatch.schedule(cb,
- {.workDuration = 100,
- .readyDuration = 0,
- .earliestVsync = intended});
+ auto result = mDispatch->schedule(cb,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = intended});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(900, *result);
- result = mDispatch.update(cb,
+ result =
+ mDispatch->update(cb,
{.workDuration = 300, .readyDuration = 0, .earliestVsync = intended});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(700, *result);
@@ -302,17 +308,17 @@
CountingCallback cb(mDispatch);
const auto result =
- mDispatch.update(cb,
- {.workDuration = 300, .readyDuration = 0, .earliestVsync = intended});
+ mDispatch->update(cb,
+ {.workDuration = 300, .readyDuration = 0, .earliestVsync = intended});
EXPECT_FALSE(result.has_value());
}
TEST_F(VSyncDispatchTimerQueueTest, basicAlarmSettingFutureWithAdjustmentToTrueVsync) {
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000)).WillOnce(Return(1150));
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(1000)).WillOnce(Return(1150));
EXPECT_CALL(mMockClock, alarmAt(_, 1050));
CountingCallback cb(mDispatch);
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
advanceToNextCallback();
ASSERT_THAT(cb.mCalls.size(), Eq(1));
@@ -323,15 +329,15 @@
auto const now = 234;
mMockClock.advanceBy(234);
auto const workDuration = 10 * mPeriod;
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(now + workDuration))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(now + workDuration))
.WillOnce(Return(mPeriod * 11));
EXPECT_CALL(mMockClock, alarmAt(_, mPeriod));
CountingCallback cb(mDispatch);
- const auto result = mDispatch.schedule(cb,
- {.workDuration = workDuration,
- .readyDuration = 0,
- .earliestVsync = mPeriod});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = workDuration,
+ .readyDuration = 0,
+ .earliestVsync = mPeriod});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(mPeriod, *result);
}
@@ -341,12 +347,13 @@
EXPECT_CALL(mMockClock, alarmCancel());
CountingCallback cb(mDispatch);
- const auto result =
- mDispatch.schedule(cb,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = mPeriod});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(mPeriod - 100, *result);
- EXPECT_EQ(mDispatch.cancel(cb), CancelResult::Cancelled);
+ EXPECT_EQ(mDispatch->cancel(cb), CancelResult::Cancelled);
}
TEST_F(VSyncDispatchTimerQueueTest, basicAlarmCancelTooLate) {
@@ -354,13 +361,14 @@
EXPECT_CALL(mMockClock, alarmCancel());
CountingCallback cb(mDispatch);
- const auto result =
- mDispatch.schedule(cb,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = mPeriod});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(mPeriod - 100, *result);
mMockClock.advanceBy(950);
- EXPECT_EQ(mDispatch.cancel(cb), CancelResult::TooLate);
+ EXPECT_EQ(mDispatch->cancel(cb), CancelResult::TooLate);
}
TEST_F(VSyncDispatchTimerQueueTest, basicAlarmCancelTooLateWhenRunning) {
@@ -368,15 +376,16 @@
EXPECT_CALL(mMockClock, alarmCancel());
PausingCallback cb(mDispatch, std::chrono::duration_cast<std::chrono::milliseconds>(1s));
- const auto result =
- mDispatch.schedule(cb,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = mPeriod});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(mPeriod - 100, *result);
std::thread pausingThread([&] { mMockClock.advanceToNextCallback(); });
EXPECT_TRUE(cb.waitForPause());
- EXPECT_EQ(mDispatch.cancel(cb), CancelResult::TooLate);
+ EXPECT_EQ(mDispatch->cancel(cb), CancelResult::TooLate);
cb.unpause();
pausingThread.join();
}
@@ -389,9 +398,10 @@
PausingCallback cb(mDispatch, 50ms);
cb.stashResource(resource);
- const auto result =
- mDispatch.schedule(cb,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = mPeriod});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(mPeriod - 100, *result);
@@ -408,7 +418,7 @@
}
TEST_F(VSyncDispatchTimerQueueTest, basicTwoAlarmSetting) {
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(1000))
.Times(4)
.WillOnce(Return(1055))
.WillOnce(Return(1063))
@@ -423,8 +433,8 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
- mDispatch.schedule(cb1, {.workDuration = 250, .readyDuration = 0, .earliestVsync = mPeriod});
+ mDispatch->schedule(cb0, {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod});
+ mDispatch->schedule(cb1, {.workDuration = 250, .readyDuration = 0, .earliestVsync = mPeriod});
advanceToNextCallback();
advanceToNextCallback();
@@ -436,7 +446,7 @@
}
TEST_F(VSyncDispatchTimerQueueTest, noCloseCallbacksAfterPeriodChange) {
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(_))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(_))
.Times(4)
.WillOnce(Return(1000))
.WillOnce(Return(2000))
@@ -450,21 +460,21 @@
CountingCallback cb(mDispatch);
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 0});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 0});
advanceToNextCallback();
ASSERT_THAT(cb.mCalls.size(), Eq(1));
EXPECT_THAT(cb.mCalls[0], Eq(1000));
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
ASSERT_THAT(cb.mCalls.size(), Eq(2));
EXPECT_THAT(cb.mCalls[1], Eq(2000));
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
advanceToNextCallback();
@@ -473,7 +483,7 @@
}
TEST_F(VSyncDispatchTimerQueueTest, rearmsFaroutTimeoutWhenCancellingCloseOne) {
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(_))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(_))
.Times(4)
.WillOnce(Return(10000))
.WillOnce(Return(1000))
@@ -488,10 +498,10 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod * 10});
- mDispatch.schedule(cb1, {.workDuration = 250, .readyDuration = 0, .earliestVsync = mPeriod});
- mDispatch.cancel(cb1);
+ mDispatch->schedule(cb0,
+ {.workDuration = 100, .readyDuration = 0, .earliestVsync = mPeriod * 10});
+ mDispatch->schedule(cb1, {.workDuration = 250, .readyDuration = 0, .earliestVsync = mPeriod});
+ mDispatch->cancel(cb1);
}
TEST_F(VSyncDispatchTimerQueueTest, noUnnecessaryRearmsWhenRescheduling) {
@@ -502,9 +512,9 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 300, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 300, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
}
@@ -517,9 +527,9 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
}
@@ -537,10 +547,10 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1,
- {.workDuration = closeOffset, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1,
+ {.workDuration = closeOffset, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
ASSERT_THAT(cb0.mCalls.size(), Eq(1));
@@ -548,9 +558,11 @@
ASSERT_THAT(cb1.mCalls.size(), Eq(1));
EXPECT_THAT(cb1.mCalls[0], Eq(mPeriod));
- mDispatch.schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 2000});
- mDispatch.schedule(cb1,
- {.workDuration = notCloseOffset, .readyDuration = 0, .earliestVsync = 2000});
+ mDispatch->schedule(cb0, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 2000});
+ mDispatch->schedule(cb1,
+ {.workDuration = notCloseOffset,
+ .readyDuration = 0,
+ .earliestVsync = 2000});
advanceToNextCallback();
ASSERT_THAT(cb1.mCalls.size(), Eq(2));
EXPECT_THAT(cb1.mCalls[1], Eq(2000));
@@ -570,32 +582,32 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
- EXPECT_EQ(mDispatch.cancel(cb0), CancelResult::Cancelled);
+ EXPECT_EQ(mDispatch->cancel(cb0), CancelResult::Cancelled);
}
TEST_F(VSyncDispatchTimerQueueTest, setAlarmCallsAtCorrectTimeWithChangingVsync) {
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(_))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(_))
.Times(3)
.WillOnce(Return(950))
.WillOnce(Return(1975))
.WillOnce(Return(2950));
CountingCallback cb(mDispatch);
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 920});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 920});
mMockClock.advanceBy(850);
EXPECT_THAT(cb.mCalls.size(), Eq(1));
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1900});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1900});
mMockClock.advanceBy(900);
EXPECT_THAT(cb.mCalls.size(), Eq(1));
mMockClock.advanceBy(125);
EXPECT_THAT(cb.mCalls.size(), Eq(2));
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2900});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2900});
mMockClock.advanceBy(975);
EXPECT_THAT(cb.mCalls.size(), Eq(3));
}
@@ -606,48 +618,48 @@
EXPECT_CALL(mMockClock, alarmAt(_, 1900)).InSequence(seq);
VSyncDispatch::CallbackToken tmp;
- tmp = mDispatch.registerCallback(
+ tmp = mDispatch->registerCallback(
[&](auto, auto, auto) {
- mDispatch.schedule(tmp,
- {.workDuration = 100,
- .readyDuration = 0,
- .earliestVsync = 2000});
+ mDispatch->schedule(tmp,
+ {.workDuration = 100,
+ .readyDuration = 0,
+ .earliestVsync = 2000});
},
"o.o");
- mDispatch.schedule(tmp, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(tmp, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
}
TEST_F(VSyncDispatchTimerQueueTest, callbackReentrantWithPastWakeup) {
VSyncDispatch::CallbackToken tmp;
std::optional<nsecs_t> lastTarget;
- tmp = mDispatch.registerCallback(
+ tmp = mDispatch->registerCallback(
[&](auto timestamp, auto, auto) {
auto result =
- mDispatch.schedule(tmp,
- {.workDuration = 400,
- .readyDuration = 0,
- .earliestVsync = timestamp - mVsyncMoveThreshold});
- EXPECT_TRUE(result.has_value());
- EXPECT_EQ(mPeriod + timestamp - 400, *result);
- result = mDispatch.schedule(tmp,
+ mDispatch->schedule(tmp,
{.workDuration = 400,
.readyDuration = 0,
- .earliestVsync = timestamp});
+ .earliestVsync = timestamp - mVsyncMoveThreshold});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(mPeriod + timestamp - 400, *result);
- result = mDispatch.schedule(tmp,
- {.workDuration = 400,
- .readyDuration = 0,
- .earliestVsync = timestamp + mVsyncMoveThreshold});
+ result = mDispatch->schedule(tmp,
+ {.workDuration = 400,
+ .readyDuration = 0,
+ .earliestVsync = timestamp});
+ EXPECT_TRUE(result.has_value());
+ EXPECT_EQ(mPeriod + timestamp - 400, *result);
+ result = mDispatch->schedule(tmp,
+ {.workDuration = 400,
+ .readyDuration = 0,
+ .earliestVsync = timestamp + mVsyncMoveThreshold});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(mPeriod + timestamp - 400, *result);
lastTarget = timestamp;
},
"oo");
- mDispatch.schedule(tmp, {.workDuration = 999, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(tmp, {.workDuration = 999, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
EXPECT_THAT(lastTarget, Eq(1000));
@@ -663,16 +675,16 @@
EXPECT_CALL(mMockClock, alarmAt(_, 1900)).InSequence(seq);
CountingCallback cb(mDispatch);
- mDispatch.schedule(cb, {.workDuration = 0, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb, {.workDuration = 0, .readyDuration = 0, .earliestVsync = 1000});
mMockClock.advanceBy(750);
- mDispatch.schedule(cb, {.workDuration = 50, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb, {.workDuration = 50, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
- mDispatch.schedule(cb, {.workDuration = 50, .readyDuration = 0, .earliestVsync = 2000});
+ mDispatch->schedule(cb, {.workDuration = 50, .readyDuration = 0, .earliestVsync = 2000});
mMockClock.advanceBy(800);
- mDispatch.schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
+ mDispatch->schedule(cb, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
}
TEST_F(VSyncDispatchTimerQueueTest, lateModifications) {
@@ -685,12 +697,12 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
- mDispatch.schedule(cb0, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 2000});
- mDispatch.schedule(cb1, {.workDuration = 150, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0, {.workDuration = 200, .readyDuration = 0, .earliestVsync = 2000});
+ mDispatch->schedule(cb1, {.workDuration = 150, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
advanceToNextCallback();
@@ -702,8 +714,8 @@
CountingCallback cb0(mDispatch);
CountingCallback cb1(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb1, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 20000});
+ mDispatch->schedule(cb0, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 20000});
}
TEST_F(VSyncDispatchTimerQueueTest, setsTimerAfterCancellation) {
@@ -713,29 +725,30 @@
EXPECT_CALL(mMockClock, alarmAt(_, 900)).InSequence(seq);
CountingCallback cb0(mDispatch);
- mDispatch.schedule(cb0, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.cancel(cb0);
- mDispatch.schedule(cb0, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0, {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->cancel(cb0);
+ mDispatch->schedule(cb0, {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
}
TEST_F(VSyncDispatchTimerQueueTest, makingUpIdsError) {
VSyncDispatch::CallbackToken token(100);
- EXPECT_FALSE(mDispatch
- .schedule(token,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000})
- .has_value());
- EXPECT_THAT(mDispatch.cancel(token), Eq(CancelResult::Error));
+ EXPECT_FALSE(
+ mDispatch
+ ->schedule(token,
+ {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000})
+ .has_value());
+ EXPECT_THAT(mDispatch->cancel(token), Eq(CancelResult::Error));
}
TEST_F(VSyncDispatchTimerQueueTest, canMoveCallbackBackwardsInTime) {
CountingCallback cb0(mDispatch);
auto result =
- mDispatch.schedule(cb0,
- {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0,
+ {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(500, *result);
- result = mDispatch.schedule(cb0,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
+ result = mDispatch->schedule(cb0,
+ {.workDuration = 100, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(900, *result);
}
@@ -745,14 +758,14 @@
EXPECT_CALL(mMockClock, alarmAt(_, 500));
CountingCallback cb(mDispatch);
auto result =
- mDispatch.schedule(cb,
- {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb,
+ {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(500, *result);
mMockClock.advanceBy(400);
- result = mDispatch.schedule(cb,
- {.workDuration = 800, .readyDuration = 0, .earliestVsync = 1000});
+ result = mDispatch->schedule(cb,
+ {.workDuration = 800, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(1200, *result);
advanceToNextCallback();
@@ -760,19 +773,19 @@
}
TEST_F(VSyncDispatchTimerQueueTest, targetOffsetMovingBackALittleCanStillSchedule) {
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(1000))
.Times(2)
.WillOnce(Return(1000))
.WillOnce(Return(1002));
CountingCallback cb(mDispatch);
auto result =
- mDispatch.schedule(cb,
- {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb,
+ {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(500, *result);
mMockClock.advanceBy(400);
- result = mDispatch.schedule(cb,
- {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ result = mDispatch->schedule(cb,
+ {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(602, *result);
}
@@ -780,13 +793,13 @@
TEST_F(VSyncDispatchTimerQueueTest, canScheduleNegativeOffsetAgainstDifferentPeriods) {
CountingCallback cb0(mDispatch);
auto result =
- mDispatch.schedule(cb0,
- {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0,
+ {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(500, *result);
advanceToNextCallback();
- result = mDispatch.schedule(cb0,
- {.workDuration = 1100, .readyDuration = 0, .earliestVsync = 2000});
+ result = mDispatch->schedule(cb0,
+ {.workDuration = 1100, .readyDuration = 0, .earliestVsync = 2000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(900, *result);
}
@@ -797,13 +810,13 @@
EXPECT_CALL(mMockClock, alarmAt(_, 1100)).InSequence(seq);
CountingCallback cb0(mDispatch);
auto result =
- mDispatch.schedule(cb0,
- {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb0,
+ {.workDuration = 500, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(500, *result);
advanceToNextCallback();
- result = mDispatch.schedule(cb0,
- {.workDuration = 1900, .readyDuration = 0, .earliestVsync = 2000});
+ result = mDispatch->schedule(cb0,
+ {.workDuration = 1900, .readyDuration = 0, .earliestVsync = 2000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(1100, *result);
}
@@ -813,13 +826,13 @@
CountingCallback cb(mDispatch);
auto result =
- mDispatch.schedule(cb,
- {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb,
+ {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(600, *result);
- result = mDispatch.schedule(cb,
- {.workDuration = 1400, .readyDuration = 0, .earliestVsync = 1000});
+ result = mDispatch->schedule(cb,
+ {.workDuration = 1400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(600, *result);
@@ -865,16 +878,16 @@
CountingCallback cb2(mDispatch);
auto result =
- mDispatch.schedule(cb1,
- {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1,
+ {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(600, *result);
mMockClock.setLag(100);
mMockClock.advanceBy(620);
- result = mDispatch.schedule(cb2,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
+ result = mDispatch->schedule(cb2,
+ {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(1900, *result);
mMockClock.advanceBy(80);
@@ -893,16 +906,16 @@
CountingCallback cb(mDispatch);
auto result =
- mDispatch.schedule(cb,
- {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb,
+ {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(600, *result);
mMockClock.setLag(100);
mMockClock.advanceBy(620);
- result = mDispatch.schedule(cb,
- {.workDuration = 370, .readyDuration = 0, .earliestVsync = 2000});
+ result = mDispatch->schedule(cb,
+ {.workDuration = 370, .readyDuration = 0, .earliestVsync = 2000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(1630, *result);
mMockClock.advanceBy(80);
@@ -919,19 +932,19 @@
CountingCallback cb2(mDispatch);
auto result =
- mDispatch.schedule(cb1,
- {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1,
+ {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(600, *result);
- result = mDispatch.schedule(cb2,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
+ result = mDispatch->schedule(cb2,
+ {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(1900, *result);
mMockClock.setLag(100);
mMockClock.advanceBy(620);
- EXPECT_EQ(mDispatch.cancel(cb2), CancelResult::Cancelled);
+ EXPECT_EQ(mDispatch->cancel(cb2), CancelResult::Cancelled);
mMockClock.advanceBy(80);
@@ -948,19 +961,19 @@
CountingCallback cb2(mDispatch);
auto result =
- mDispatch.schedule(cb1,
- {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1,
+ {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(600, *result);
- result = mDispatch.schedule(cb2,
- {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
+ result = mDispatch->schedule(cb2,
+ {.workDuration = 100, .readyDuration = 0, .earliestVsync = 2000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(1900, *result);
mMockClock.setLag(100);
mMockClock.advanceBy(620);
- EXPECT_EQ(mDispatch.cancel(cb1), CancelResult::Cancelled);
+ EXPECT_EQ(mDispatch->cancel(cb1), CancelResult::Cancelled);
EXPECT_THAT(cb1.mCalls.size(), Eq(0));
EXPECT_THAT(cb2.mCalls.size(), Eq(0));
@@ -975,21 +988,21 @@
CountingCallback cb2(mDispatch);
Sequence seq;
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(1000))
.InSequence(seq)
.WillOnce(Return(1000));
EXPECT_CALL(mMockClock, alarmAt(_, 600)).InSequence(seq);
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(1000))
.InSequence(seq)
.WillOnce(Return(1000));
auto result =
- mDispatch.schedule(cb1,
- {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb1,
+ {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(600, *result);
- result = mDispatch.schedule(cb2,
- {.workDuration = 390, .readyDuration = 0, .earliestVsync = 1000});
+ result = mDispatch->schedule(cb2,
+ {.workDuration = 390, .readyDuration = 0, .earliestVsync = 1000});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(610, *result);
@@ -1011,10 +1024,10 @@
EXPECT_CALL(mMockClock, alarmAt(_, 900));
CountingCallback cb(mDispatch);
- const auto result = mDispatch.schedule(cb,
- {.workDuration = 70,
- .readyDuration = 30,
- .earliestVsync = intended});
+ const auto result = mDispatch->schedule(cb,
+ {.workDuration = 70,
+ .readyDuration = 30,
+ .earliestVsync = intended});
EXPECT_TRUE(result.has_value());
EXPECT_EQ(900, *result);
advanceToNextCallback();
@@ -1033,8 +1046,8 @@
CountingCallback cb(mDispatch);
- mDispatch.schedule(cb, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
- mDispatch.schedule(cb, {.workDuration = 1400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb, {.workDuration = 400, .readyDuration = 0, .earliestVsync = 1000});
+ mDispatch->schedule(cb, {.workDuration = 1400, .readyDuration = 0, .earliestVsync = 1000});
advanceToNextCallback();
@@ -1052,7 +1065,8 @@
protected:
nsecs_t const mPeriod = 1000;
nsecs_t const mVsyncMoveThreshold = 200;
- NiceMock<MockVSyncTracker> mStubTracker{mPeriod};
+ std::shared_ptr<NiceMock<MockVSyncTracker>> mStubTracker =
+ std::make_shared<NiceMock<MockVSyncTracker>>(mPeriod);
};
TEST_F(VSyncDispatchTimerQueueEntryTest, stateAfterInitialization) {
@@ -1070,7 +1084,7 @@
EXPECT_FALSE(entry.wakeupTime());
EXPECT_TRUE(entry.schedule({.workDuration = 100, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
auto const wakeup = entry.wakeupTime();
ASSERT_TRUE(wakeup);
@@ -1084,7 +1098,7 @@
auto const duration = 500;
auto const now = 8750;
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(now + duration))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(now + duration))
.Times(1)
.WillOnce(Return(10000));
VSyncDispatchTimerQueueEntry entry(
@@ -1092,7 +1106,7 @@
EXPECT_FALSE(entry.wakeupTime());
EXPECT_TRUE(entry.schedule({.workDuration = 500, .readyDuration = 0, .earliestVsync = 994},
- mStubTracker, now)
+ *mStubTracker.get(), now)
.has_value());
auto const wakeup = entry.wakeupTime();
ASSERT_TRUE(wakeup);
@@ -1115,7 +1129,7 @@
mVsyncMoveThreshold);
EXPECT_TRUE(entry.schedule({.workDuration = 100, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
auto const wakeup = entry.wakeupTime();
ASSERT_TRUE(wakeup);
@@ -1137,7 +1151,7 @@
}
TEST_F(VSyncDispatchTimerQueueEntryTest, updateCallback) {
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(_))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(_))
.Times(2)
.WillOnce(Return(1000))
.WillOnce(Return(1020));
@@ -1146,17 +1160,17 @@
"test", [](auto, auto, auto) {}, mVsyncMoveThreshold);
EXPECT_FALSE(entry.wakeupTime());
- entry.update(mStubTracker, 0);
+ entry.update(*mStubTracker.get(), 0);
EXPECT_FALSE(entry.wakeupTime());
EXPECT_TRUE(entry.schedule({.workDuration = 100, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
auto wakeup = entry.wakeupTime();
ASSERT_TRUE(wakeup);
EXPECT_THAT(wakeup, Eq(900));
- entry.update(mStubTracker, 0);
+ entry.update(*mStubTracker.get(), 0);
wakeup = entry.wakeupTime();
ASSERT_TRUE(wakeup);
EXPECT_THAT(*wakeup, Eq(920));
@@ -1166,9 +1180,9 @@
VSyncDispatchTimerQueueEntry entry(
"test", [](auto, auto, auto) {}, mVsyncMoveThreshold);
EXPECT_TRUE(entry.schedule({.workDuration = 100, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
- entry.update(mStubTracker, 0);
+ entry.update(*mStubTracker.get(), 0);
auto const wakeup = entry.wakeupTime();
ASSERT_TRUE(wakeup);
@@ -1179,24 +1193,24 @@
VSyncDispatchTimerQueueEntry entry(
"test", [](auto, auto, auto) {}, mVsyncMoveThreshold);
EXPECT_TRUE(entry.schedule({.workDuration = 100, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
entry.executing(); // 1000 is executing
// had 1000 not been executing, this could have been scheduled for time 800.
EXPECT_TRUE(entry.schedule({.workDuration = 200, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
EXPECT_THAT(*entry.wakeupTime(), Eq(1800));
EXPECT_THAT(*entry.readyTime(), Eq(2000));
EXPECT_TRUE(entry.schedule({.workDuration = 50, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
EXPECT_THAT(*entry.wakeupTime(), Eq(1950));
EXPECT_THAT(*entry.readyTime(), Eq(2000));
EXPECT_TRUE(entry.schedule({.workDuration = 200, .readyDuration = 0, .earliestVsync = 1001},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
EXPECT_THAT(*entry.wakeupTime(), Eq(1800));
EXPECT_THAT(*entry.readyTime(), Eq(2000));
@@ -1208,24 +1222,24 @@
"test", [](auto, auto, auto) {}, mVsyncMoveThreshold);
Sequence seq;
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(500))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(500))
.InSequence(seq)
.WillOnce(Return(1000));
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(500))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(500))
.InSequence(seq)
.WillOnce(Return(1000));
- EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000 + mVsyncMoveThreshold))
+ EXPECT_CALL(*mStubTracker.get(), nextAnticipatedVSyncTimeFrom(1000 + mVsyncMoveThreshold))
.InSequence(seq)
.WillOnce(Return(2000));
EXPECT_TRUE(entry.schedule({.workDuration = 100, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
entry.executing(); // 1000 is executing
EXPECT_TRUE(entry.schedule({.workDuration = 200, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
}
@@ -1233,16 +1247,16 @@
VSyncDispatchTimerQueueEntry entry(
"test", [](auto, auto, auto) {}, mVsyncMoveThreshold);
EXPECT_TRUE(entry.schedule({.workDuration = 100, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
EXPECT_TRUE(entry.schedule({.workDuration = 200, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
EXPECT_TRUE(entry.schedule({.workDuration = 50, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
EXPECT_TRUE(entry.schedule({.workDuration = 1200, .readyDuration = 0, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
}
@@ -1255,7 +1269,7 @@
entry.addPendingWorkloadUpdate(
{.workDuration = effectualOffset, .readyDuration = 0, .earliestVsync = 400});
EXPECT_TRUE(entry.hasPendingWorkloadUpdate());
- entry.update(mStubTracker, 0);
+ entry.update(*mStubTracker.get(), 0);
EXPECT_FALSE(entry.hasPendingWorkloadUpdate());
EXPECT_THAT(*entry.wakeupTime(), Eq(mPeriod - effectualOffset));
}
@@ -1276,7 +1290,7 @@
mVsyncMoveThreshold);
EXPECT_TRUE(entry.schedule({.workDuration = 70, .readyDuration = 30, .earliestVsync = 500},
- mStubTracker, 0)
+ *mStubTracker.get(), 0)
.has_value());
auto const wakeup = entry.wakeupTime();
ASSERT_TRUE(wakeup);
diff --git a/services/surfaceflinger/tests/unittests/VSyncPredictorTest.cpp b/services/surfaceflinger/tests/unittests/VSyncPredictorTest.cpp
index 3095e8a..7947a5e 100644
--- a/services/surfaceflinger/tests/unittests/VSyncPredictorTest.cpp
+++ b/services/surfaceflinger/tests/unittests/VSyncPredictorTest.cpp
@@ -55,7 +55,7 @@
static constexpr size_t kOutlierTolerancePercent = 25;
static constexpr nsecs_t mMaxRoundingError = 100;
- VSyncPredictor tracker{mPeriod, kHistorySize, kMinimumSamplesForPrediction,
+ VSyncPredictor tracker{"tracker", mPeriod, kHistorySize, kMinimumSamplesForPrediction,
kOutlierTolerancePercent};
};
@@ -376,7 +376,8 @@
// See b/151146131
TEST_F(VSyncPredictorTest, hasEnoughPrecision) {
- VSyncPredictor tracker{mPeriod, 20, kMinimumSamplesForPrediction, kOutlierTolerancePercent};
+ VSyncPredictor tracker{"tracker", mPeriod, 20, kMinimumSamplesForPrediction,
+ kOutlierTolerancePercent};
std::vector<nsecs_t> const simulatedVsyncs{840873348817, 840890049444, 840906762675,
840923581635, 840940161584, 840956868096,
840973702473, 840990256277, 841007116851,
diff --git a/services/surfaceflinger/tests/unittests/VSyncReactorTest.cpp b/services/surfaceflinger/tests/unittests/VSyncReactorTest.cpp
index 1fb2709..a2de136 100644
--- a/services/surfaceflinger/tests/unittests/VSyncReactorTest.cpp
+++ b/services/surfaceflinger/tests/unittests/VSyncReactorTest.cpp
@@ -96,8 +96,8 @@
VSyncReactorTest()
: mMockTracker(std::make_shared<NiceMock<MockVSyncTracker>>()),
mMockClock(std::make_shared<NiceMock<MockClock>>()),
- mReactor(std::make_unique<ClockWrapper>(mMockClock), *mMockTracker, kPendingLimit,
- false /* supportKernelIdleTimer */) {
+ mReactor("reactor", std::make_unique<ClockWrapper>(mMockClock), *mMockTracker,
+ kPendingLimit, false /* supportKernelIdleTimer */) {
ON_CALL(*mMockClock, now()).WillByDefault(Return(mFakeNow));
ON_CALL(*mMockTracker, currentPeriod()).WillByDefault(Return(period));
}
@@ -192,7 +192,7 @@
mReactor.setIgnorePresentFences(true);
nsecs_t const newPeriod = 5000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(0, std::nullopt, &periodFlushed));
EXPECT_FALSE(periodFlushed);
@@ -205,7 +205,7 @@
TEST_F(VSyncReactorTest, setPeriodCalledOnceConfirmedChange) {
nsecs_t const newPeriod = 5000;
EXPECT_CALL(*mMockTracker, setPeriod(_)).Times(0);
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
bool periodFlushed = true;
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(10000, std::nullopt, &periodFlushed));
@@ -224,7 +224,7 @@
TEST_F(VSyncReactorTest, changingPeriodBackAbortsConfirmationProcess) {
nsecs_t sampleTime = 0;
nsecs_t const newPeriod = 5000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
bool periodFlushed = true;
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(sampleTime += period, std::nullopt, &periodFlushed));
EXPECT_FALSE(periodFlushed);
@@ -232,7 +232,7 @@
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(sampleTime += period, std::nullopt, &periodFlushed));
EXPECT_FALSE(periodFlushed);
- mReactor.startPeriodTransition(period);
+ mReactor.startPeriodTransition(period, false);
EXPECT_FALSE(mReactor.addHwVsyncTimestamp(sampleTime += period, std::nullopt, &periodFlushed));
EXPECT_FALSE(periodFlushed);
}
@@ -242,13 +242,13 @@
nsecs_t const secondPeriod = 5000;
nsecs_t const thirdPeriod = 2000;
- mReactor.startPeriodTransition(secondPeriod);
+ mReactor.startPeriodTransition(secondPeriod, false);
bool periodFlushed = true;
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(sampleTime += period, std::nullopt, &periodFlushed));
EXPECT_FALSE(periodFlushed);
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(sampleTime += period, std::nullopt, &periodFlushed));
EXPECT_FALSE(periodFlushed);
- mReactor.startPeriodTransition(thirdPeriod);
+ mReactor.startPeriodTransition(thirdPeriod, false);
EXPECT_TRUE(
mReactor.addHwVsyncTimestamp(sampleTime += secondPeriod, std::nullopt, &periodFlushed));
EXPECT_FALSE(periodFlushed);
@@ -289,14 +289,14 @@
TEST_F(VSyncReactorTest, presentFenceAdditionDoesNotInterruptConfirmationProcess) {
nsecs_t const newPeriod = 5000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
EXPECT_TRUE(mReactor.addPresentFence(generateSignalledFenceWithTime(0)));
}
TEST_F(VSyncReactorTest, setPeriodCalledFirstTwoEventsNewPeriod) {
nsecs_t const newPeriod = 5000;
EXPECT_CALL(*mMockTracker, setPeriod(_)).Times(0);
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
bool periodFlushed = true;
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(5000, std::nullopt, &periodFlushed));
@@ -321,7 +321,7 @@
bool periodFlushed = false;
nsecs_t const newPeriod = 4000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
auto time = 0;
auto constexpr numTimestampSubmissions = 10;
@@ -346,7 +346,7 @@
bool periodFlushed = false;
nsecs_t const newPeriod = 4000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
auto time = 0;
// If the power mode is not DOZE or DOZE_SUSPEND, it is still collecting timestamps.
@@ -363,7 +363,7 @@
auto time = 0;
bool periodFlushed = false;
nsecs_t const newPeriod = 4000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
time += period;
mReactor.addHwVsyncTimestamp(time, std::nullopt, &periodFlushed);
@@ -379,7 +379,7 @@
auto time = 0;
bool periodFlushed = false;
nsecs_t const newPeriod = 4000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
static auto constexpr numSamplesWithNewPeriod = 4;
Sequence seq;
@@ -406,7 +406,7 @@
auto time = 0;
bool periodFlushed = false;
nsecs_t const newPeriod = 4000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
Sequence seq;
EXPECT_CALL(*mMockTracker, needsMoreSamples())
@@ -426,7 +426,7 @@
nsecs_t const newPeriod1 = 4000;
nsecs_t const newPeriod2 = 7000;
- mReactor.startPeriodTransition(newPeriod1);
+ mReactor.startPeriodTransition(newPeriod1, false);
Sequence seq;
EXPECT_CALL(*mMockTracker, needsMoreSamples())
@@ -445,7 +445,7 @@
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(time += newPeriod1, std::nullopt, &periodFlushed));
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(time += newPeriod1, std::nullopt, &periodFlushed));
- mReactor.startPeriodTransition(newPeriod2);
+ mReactor.startPeriodTransition(newPeriod2, false);
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(time += newPeriod1, std::nullopt, &periodFlushed));
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(time += newPeriod2, std::nullopt, &periodFlushed));
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(time += newPeriod2, std::nullopt, &periodFlushed));
@@ -458,7 +458,7 @@
mReactor.setIgnorePresentFences(true);
nsecs_t const newPeriod = 5000;
- mReactor.startPeriodTransition(newPeriod);
+ mReactor.startPeriodTransition(newPeriod, false);
EXPECT_TRUE(mReactor.addHwVsyncTimestamp(0, 0, &periodFlushed));
EXPECT_FALSE(periodFlushed);
@@ -472,8 +472,9 @@
TEST_F(VSyncReactorTest, periodIsMeasuredIfIgnoringComposer) {
// Create a reactor which supports the kernel idle timer
- auto idleReactor = VSyncReactor(std::make_unique<ClockWrapper>(mMockClock), *mMockTracker,
- kPendingLimit, true /* supportKernelIdleTimer */);
+ auto idleReactor =
+ VSyncReactor("reactor", std::make_unique<ClockWrapper>(mMockClock), *mMockTracker,
+ kPendingLimit, true /* supportKernelIdleTimer */);
bool periodFlushed = true;
EXPECT_CALL(*mMockTracker, addVsyncTimestamp(_)).Times(4);
@@ -481,7 +482,7 @@
// First, set the same period, which should only be confirmed when we receive two
// matching callbacks
- idleReactor.startPeriodTransition(10000);
+ idleReactor.startPeriodTransition(10000, false);
EXPECT_TRUE(idleReactor.addHwVsyncTimestamp(0, 0, &periodFlushed));
EXPECT_FALSE(periodFlushed);
// Correct period but incorrect timestamp delta
@@ -494,7 +495,7 @@
// Then, set a new period, which should be confirmed as soon as we receive a callback
// reporting the new period
nsecs_t const newPeriod = 5000;
- idleReactor.startPeriodTransition(newPeriod);
+ idleReactor.startPeriodTransition(newPeriod, false);
// Incorrect timestamp delta and period
EXPECT_TRUE(idleReactor.addHwVsyncTimestamp(20000, 10000, &periodFlushed));
EXPECT_FALSE(periodFlushed);
diff --git a/services/surfaceflinger/tests/unittests/mock/MockEventThread.h b/services/surfaceflinger/tests/unittests/mock/MockEventThread.h
index f8567bd..3a6068a 100644
--- a/services/surfaceflinger/tests/unittests/mock/MockEventThread.h
+++ b/services/surfaceflinger/tests/unittests/mock/MockEventThread.h
@@ -29,27 +29,29 @@
EventThread();
~EventThread() override;
- MOCK_CONST_METHOD2(createEventConnection,
- sp<EventThreadConnection>(ResyncCallback, EventRegistrationFlags));
- MOCK_METHOD0(onScreenReleased, void());
- MOCK_METHOD0(onScreenAcquired, void());
- MOCK_METHOD2(onHotplugReceived, void(PhysicalDisplayId, bool));
- MOCK_METHOD1(onModeChanged, void(const scheduler::FrameRateMode &));
- MOCK_METHOD2(onFrameRateOverridesChanged,
- void(PhysicalDisplayId, std::vector<FrameRateOverride>));
- MOCK_CONST_METHOD1(dump, void(std::string&));
- MOCK_METHOD2(setDuration,
- void(std::chrono::nanoseconds workDuration,
- std::chrono::nanoseconds readyDuration));
- MOCK_METHOD1(registerDisplayEventConnection,
- status_t(const sp<android::EventThreadConnection> &));
- MOCK_METHOD2(setVsyncRate, void(uint32_t, const sp<android::EventThreadConnection> &));
- MOCK_METHOD1(requestNextVsync, void(const sp<android::EventThreadConnection> &));
+ MOCK_METHOD(sp<EventThreadConnection>, createEventConnection,
+ (ResyncCallback, EventRegistrationFlags), (const, override));
+ MOCK_METHOD(void, onScreenReleased, (), (override));
+ MOCK_METHOD(void, onScreenAcquired, (), (override));
+ MOCK_METHOD(void, onHotplugReceived, (PhysicalDisplayId, bool), (override));
+ MOCK_METHOD(void, onModeChanged, (const scheduler::FrameRateMode&), (override));
+ MOCK_METHOD(void, onFrameRateOverridesChanged,
+ (PhysicalDisplayId, std::vector<FrameRateOverride>), (override));
+ MOCK_METHOD(void, dump, (std::string&), (const, override));
+ MOCK_METHOD(void, setDuration,
+ (std::chrono::nanoseconds workDuration, std::chrono::nanoseconds readyDuration),
+ (override));
+ MOCK_METHOD(status_t, registerDisplayEventConnection,
+ (const sp<android::EventThreadConnection>&), (override));
+ MOCK_METHOD(void, setVsyncRate, (uint32_t, const sp<android::EventThreadConnection>&),
+ (override));
+ MOCK_METHOD(void, requestNextVsync, (const sp<android::EventThreadConnection>&), (override));
MOCK_METHOD(VsyncEventData, getLatestVsyncEventData,
- (const sp<android::EventThreadConnection> &), (const));
- MOCK_METHOD1(requestLatestConfig, void(const sp<android::EventThreadConnection> &));
- MOCK_METHOD1(pauseVsyncCallback, void(bool));
- MOCK_METHOD0(getEventThreadConnectionCount, size_t());
+ (const sp<android::EventThreadConnection>&), (const, override));
+ MOCK_METHOD(void, requestLatestConfig, (const sp<android::EventThreadConnection>&));
+ MOCK_METHOD(void, pauseVsyncCallback, (bool));
+ MOCK_METHOD(size_t, getEventThreadConnectionCount, (), (override));
+ MOCK_METHOD(void, onNewVsyncSchedule, (std::shared_ptr<scheduler::VsyncSchedule>), (override));
};
} // namespace android::mock
diff --git a/services/surfaceflinger/tests/unittests/mock/MockSchedulerCallback.h b/services/surfaceflinger/tests/unittests/mock/MockSchedulerCallback.h
index 7d4b159..a8eca21 100644
--- a/services/surfaceflinger/tests/unittests/mock/MockSchedulerCallback.h
+++ b/services/surfaceflinger/tests/unittests/mock/MockSchedulerCallback.h
@@ -18,19 +18,19 @@
#include <gmock/gmock.h>
-#include "Scheduler/Scheduler.h"
+#include "Scheduler/ISchedulerCallback.h"
namespace android::scheduler::mock {
struct SchedulerCallback final : ISchedulerCallback {
- MOCK_METHOD(void, setVsyncEnabled, (bool), (override));
+ MOCK_METHOD(void, setVsyncEnabled, (PhysicalDisplayId, bool), (override));
MOCK_METHOD(void, requestDisplayModes, (std::vector<display::DisplayModeRequest>), (override));
MOCK_METHOD(void, kernelTimerChanged, (bool), (override));
MOCK_METHOD(void, triggerOnFrameRateOverridesChanged, (), (override));
};
struct NoOpSchedulerCallback final : ISchedulerCallback {
- void setVsyncEnabled(bool) override {}
+ void setVsyncEnabled(PhysicalDisplayId, bool) override {}
void requestDisplayModes(std::vector<display::DisplayModeRequest>) override {}
void kernelTimerChanged(bool) override {}
void triggerOnFrameRateOverridesChanged() override {}
diff --git a/services/surfaceflinger/tests/unittests/mock/MockVsyncController.h b/services/surfaceflinger/tests/unittests/mock/MockVsyncController.h
index 4ef91da..69ec60a 100644
--- a/services/surfaceflinger/tests/unittests/mock/MockVsyncController.h
+++ b/services/surfaceflinger/tests/unittests/mock/MockVsyncController.h
@@ -28,12 +28,12 @@
~VsyncController() override;
MOCK_METHOD(bool, addPresentFence, (std::shared_ptr<FenceTime>), (override));
- MOCK_METHOD3(addHwVsyncTimestamp, bool(nsecs_t, std::optional<nsecs_t>, bool*));
- MOCK_METHOD1(startPeriodTransition, void(nsecs_t));
- MOCK_METHOD1(setIgnorePresentFences, void(bool));
+ MOCK_METHOD(bool, addHwVsyncTimestamp, (nsecs_t, std::optional<nsecs_t>, bool*), (override));
+ MOCK_METHOD(void, startPeriodTransition, (nsecs_t, bool), (override));
+ MOCK_METHOD(void, setIgnorePresentFences, (bool), (override));
MOCK_METHOD(void, setDisplayPowerMode, (hal::PowerMode), (override));
- MOCK_CONST_METHOD1(dump, void(std::string&));
+ MOCK_METHOD(void, dump, (std::string&), (const, override));
};
} // namespace android::mock