Reland "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 many current 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. 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.
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.
This reverts commit db16a2be4054d0dd7d940f4fa0a2ced0575fd2e7 aka
I0757a7df540fad316b2db42e4c77f1c73bc49420, which reverted the original
CL, If60218e85292c786b9fa70ecb33ee374d3a385e0. Notable differences from
the original CL:
- Logical pieces have been separated and landed in the following CLs:
- I21be76e20776d8a3f49e5bd295a0042de9e2dde9
- I062002245db8fd817e145a3aaf197bb874b00636
- I54a1304a3428968134cc707b24d5b325927c31df
- Remove IEventThreadCallback changes. These are ultimately orthogonal
to the motivation of this CL.
- Store the ID instead of the name and use ftl::Concat, as tracked in
b/266817103.
- Rename MessageQueue::updateVsyncRegistration to onNewVsyncSchedule, to
match the name in EventThread. These methods do essentially the same
thing, so use the same name.
- Crucially, I54a1304a3428968134cc707b24d5b325927c31df restores the
pending HWC Vsync state, which was originally removed in
If60218e85292c786b9fa70ecb33ee374d3a385e0. The removal seems to have
been responsible for b/267562341.
Bug: 255601557
Bug: 256196556
Bug: 241285473
Bug: 241286146
Bug: 268366385
Fixes: 266817103
Test: libsurfaceflinger_unittest
Test: manual (look at perfetto traces)
Change-Id: Icdb80253436b4d0034fc20fcae8583efb7c30292
diff --git a/services/surfaceflinger/Scheduler/EventThread.cpp b/services/surfaceflinger/Scheduler/EventThread.cpp
index b78b92b..57661f1 100644
--- a/services/surfaceflinger/Scheduler/EventThread.cpp
+++ b/services/surfaceflinger/Scheduler/EventThread.cpp
@@ -238,7 +238,7 @@
namespace impl {
-EventThread::EventThread(const char* name, scheduler::VsyncSchedule& vsyncSchedule,
+EventThread::EventThread(const char* name, std::shared_ptr<scheduler::VsyncSchedule> vsyncSchedule,
android::frametimeline::TokenManager* tokenManager,
ThrottleVsyncCallback throttleVsyncCallback,
GetVsyncPeriodFunction getVsyncPeriodFunction,
@@ -248,13 +248,8 @@
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)) {
@@ -375,7 +370,7 @@
vsyncEventData.frameInterval = frameInterval;
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()};
}();
@@ -533,7 +528,7 @@
const auto throttleVsync = [&] {
const auto& vsyncData = event.vsync.vsyncData;
if (connection->frameRate.isValid()) {
- return !mVsyncSchedule.getTracker()
+ return !mVsyncSchedule->getTracker()
.isVSyncInPhase(vsyncData.preferredExpectedPresentationTime(),
connection->frameRate);
}
@@ -696,6 +691,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 5037860..87e20a0 100644
--- a/services/surfaceflinger/Scheduler/EventThread.h
+++ b/services/surfaceflinger/Scheduler/EventThread.h
@@ -133,6 +133,8 @@
// 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 {
@@ -142,8 +144,8 @@
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,
+ EventThread(const char* name, std::shared_ptr<scheduler::VsyncSchedule>,
+ frametimeline::TokenManager*, ThrottleVsyncCallback, GetVsyncPeriodFunction,
std::chrono::nanoseconds workDuration, std::chrono::nanoseconds readyDuration);
~EventThread();
@@ -172,6 +174,8 @@
size_t getEventThreadConnectionCount() override;
+ void onNewVsyncSchedule(std::shared_ptr<scheduler::VsyncSchedule>) override;
+
private:
friend EventThreadTest;
@@ -195,11 +199,13 @@
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;
diff --git a/services/surfaceflinger/Scheduler/ISchedulerCallback.h b/services/surfaceflinger/Scheduler/ISchedulerCallback.h
index c4de749..92c2189 100644
--- a/services/surfaceflinger/Scheduler/ISchedulerCallback.h
+++ b/services/surfaceflinger/Scheduler/ISchedulerCallback.h
@@ -18,12 +18,14 @@
#include <vector>
+#include <ui/DisplayId.h>
+
#include "Display/DisplayModeRequest.h"
namespace android::scheduler {
struct ISchedulerCallback {
- virtual void setVsyncEnabled(bool) = 0;
+ virtual void setVsyncEnabled(PhysicalDisplayId, bool) = 0;
virtual void requestDisplayModes(std::vector<display::DisplayModeRequest>) = 0;
virtual void kernelTimerChanged(bool expired) = 0;
virtual void triggerOnFrameRateOverridesChanged() = 0;
diff --git a/services/surfaceflinger/Scheduler/MessageQueue.cpp b/services/surfaceflinger/Scheduler/MessageQueue.cpp
index dec8f59..7457b84 100644
--- a/services/surfaceflinger/Scheduler/MessageQueue.cpp
+++ b/services/surfaceflinger/Scheduler/MessageQueue.cpp
@@ -75,19 +75,36 @@
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;
+ onNewVsyncScheduleLocked(std::move(dispatch));
+}
+
+void MessageQueue::onNewVsyncSchedule(std::shared_ptr<scheduler::VSyncDispatch> dispatch) {
+ std::lock_guard lock(mVsync.mutex);
+ onNewVsyncScheduleLocked(std::move(dispatch));
+}
+
+void MessageQueue::onNewVsyncScheduleLocked(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..9c9b2f3 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 onNewVsyncSchedule(std::shared_ptr<scheduler::VSyncDispatch>) EXCLUDES(mVsync.mutex);
+
private:
virtual void onFrameSignal(ICompositor&, VsyncId, TimePoint expectedVsyncTime) = 0;
@@ -127,10 +129,12 @@
Vsync mVsync;
+ void onNewVsyncScheduleLocked(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 0daabe2..064f853 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.cpp
+++ b/services/surfaceflinger/Scheduler/Scheduler.cpp
@@ -114,10 +114,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();
}
@@ -127,6 +135,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
@@ -154,10 +163,6 @@
compositor.sample();
}
-void Scheduler::createVsyncSchedule(FeatureFlags features) {
- mVsyncSchedule = std::make_unique<VsyncSchedule>(features);
-}
-
std::optional<Fps> Scheduler::getFrameRateOverride(uid_t uid) const {
const bool supportsFrameRateOverrideByContent =
leaderSelectorPtr()->supportsAppFrameRateOverrideByContent();
@@ -172,11 +177,11 @@
}
ATRACE_FORMAT("%s uid: %d frameRate: %s", __func__, uid, to_string(*frameRate).c_str());
- return mVsyncSchedule->getTracker().isVSyncInPhase(expectedVsyncTimestamp.ns(), *frameRate);
+ return getVsyncSchedule()->getTracker().isVSyncInPhase(expectedVsyncTimestamp.ns(), *frameRate);
}
bool Scheduler::isVsyncInPhase(TimePoint timePoint, const Fps frameRate) const {
- return mVsyncSchedule->getTracker().isVSyncInPhase(timePoint.ns(), frameRate);
+ return getVsyncSchedule()->getTracker().isVSyncInPhase(timePoint.ns(), frameRate);
}
impl::EventThread::ThrottleVsyncCallback Scheduler::makeThrottleVsyncCallback() const {
@@ -188,7 +193,8 @@
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 nsecs_t currentPeriod =
+ getVsyncSchedule()->period().ns() ?: refreshRate.getPeriodNsecs();
const auto frameRate = getFrameRateOverride(uid);
if (!frameRate.has_value()) {
@@ -208,7 +214,7 @@
std::chrono::nanoseconds workDuration,
std::chrono::nanoseconds readyDuration) {
auto eventThread = std::make_unique<impl::EventThread>(cycle == Cycle::Render ? "app" : "appSf",
- *mVsyncSchedule, tokenManager,
+ getVsyncSchedule(), tokenManager,
makeThrottleVsyncCallback(),
makeGetVsyncPeriodFunction(),
workDuration, readyDuration);
@@ -385,32 +391,59 @@
setDuration(config.sfWorkDuration);
}
-void Scheduler::enableHardwareVsync() {
- mVsyncSchedule->enableHardwareVsync(mSchedulerCallback);
+void Scheduler::enableHardwareVsync(PhysicalDisplayId id) {
+ auto schedule = getVsyncSchedule(id);
+ schedule->enableHardwareVsync(mSchedulerCallback);
}
-void Scheduler::disableHardwareVsync(bool disallow) {
- mVsyncSchedule->disableHardwareVsync(mSchedulerCallback, disallow);
+void Scheduler::disableHardwareVsync(PhysicalDisplayId id, bool disallow) {
+ auto schedule = getVsyncSchedule(id);
+ schedule->disableHardwareVsync(mSchedulerCallback, disallow);
}
-void Scheduler::resyncToHardwareVsync(bool allowToEnable, Fps refreshRate) {
- if (mVsyncSchedule->isHardwareVsyncAllowed(allowToEnable) && refreshRate.isValid()) {
- mVsyncSchedule->startPeriodTransition(mSchedulerCallback, refreshRate.getPeriod());
+void Scheduler::resyncAllToHardwareVsync(bool allowToEnable) {
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+
+ for (const auto& [id, _] : mRefreshRateSelectors) {
+ resyncToHardwareVsyncLocked(id, allowToEnable);
}
}
-void Scheduler::setRenderRate(Fps renderFrameRate) {
- const auto mode = leaderSelectorPtr()->getActiveMode();
+void Scheduler::resyncToHardwareVsyncLocked(PhysicalDisplayId id, bool allowToEnable,
+ std::optional<Fps> refreshRate) {
+ auto schedule = getVsyncScheduleLocked(id);
+ if (schedule->isHardwareVsyncAllowed(allowToEnable)) {
+ if (!refreshRate) {
+ auto selectorPtr = mRefreshRateSelectors.get(id);
+ LOG_ALWAYS_FATAL_IF(!selectorPtr);
+ refreshRate = selectorPtr->get()->getActiveMode().modePtr->getFps();
+ }
+ if (refreshRate->isValid()) {
+ schedule->startPeriodTransition(mSchedulerCallback, refreshRate->getPeriod(),
+ false /* force */);
+ }
+ }
+}
+
+void Scheduler::setRenderRate(PhysicalDisplayId id, Fps renderFrameRate) {
+ std::scoped_lock lock(mDisplayLock);
+ ftl::FakeGuard guard(kMainThreadContext);
+
+ 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());
- mVsyncSchedule->getTracker().setRenderRate(renderFrameRate);
+ getVsyncScheduleLocked(id)->getTracker().setRenderRate(renderFrameRate);
}
void Scheduler::resync() {
@@ -420,24 +453,26 @@
const nsecs_t last = mLastResyncTime.exchange(now);
if (now - last > kIgnoreDelay) {
- const auto refreshRate = leaderSelectorPtr()->getActiveMode().modePtr->getFps();
- resyncToHardwareVsync(false, refreshRate);
+ resyncAllToHardwareVsync(false /* allowToEnable */);
}
}
-bool Scheduler::addResyncSample(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriodIn) {
+bool Scheduler::addResyncSample(PhysicalDisplayId id, nsecs_t timestamp,
+ std::optional<nsecs_t> hwcVsyncPeriodIn) {
const auto hwcVsyncPeriod = ftl::Optional(hwcVsyncPeriodIn).transform([](nsecs_t nanos) {
return Period::fromNs(nanos);
});
- return mVsyncSchedule->addResyncSample(mSchedulerCallback, TimePoint::fromNs(timestamp),
- hwcVsyncPeriod);
+ return getVsyncSchedule(id)->addResyncSample(mSchedulerCallback, TimePoint::fromNs(timestamp),
+ hwcVsyncPeriod);
}
-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 */);
}
}
@@ -489,12 +524,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();
@@ -505,6 +550,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));
@@ -519,12 +582,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);
@@ -581,7 +649,20 @@
}
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) {
@@ -601,6 +682,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); },
@@ -610,6 +692,18 @@
[this] { kernelIdleTimerCallback(TimerState::Expired); }}});
leaderPtr->startIdleTimer();
+
+ const Fps refreshRate = leaderPtr->getActiveMode().modePtr->getFps();
+ vsyncSchedule->startPeriodTransition(mSchedulerCallback, refreshRate.getPeriod(),
+ true /* force */);
+ }
+
+ onNewVsyncSchedule(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 67f4daa..7374054 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.h
+++ b/services/surfaceflinger/Scheduler/Scheduler.h
@@ -114,8 +114,6 @@
void run();
- void createVsyncSchedule(FeatureFlags);
-
using Impl::initVsync;
using Impl::getScheduledFrameTime;
@@ -169,9 +167,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());
}
@@ -180,24 +190,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 disallow);
+ void enableHardwareVsync(PhysicalDisplayId);
+ void disableHardwareVsync(PhysicalDisplayId, bool disallow);
// Resyncs the scheduler to hardware vsync.
// 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 allowToEnable, 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. Returns true if
// VsyncController detected that the vsync period changed and false
// otherwise.
- bool addResyncSample(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriod);
- void addPresentFence(std::shared_ptr<FenceTime>);
+ 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*);
@@ -215,20 +233,26 @@
// 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();
@@ -288,6 +312,10 @@
void touchTimerCallback(TimerState);
void displayPowerTimerCallback(TimerState);
+ void resyncToHardwareVsyncLocked(PhysicalDisplayId, bool allowToEnable,
+ std::optional<Fps> refreshRate = std::nullopt)
+ REQUIRES(kMainThreadContext, mDisplayLock);
+ void resyncAllToHardwareVsync(bool allowToEnable) EXCLUDES(mDisplayLock);
void setVsyncConfig(const VsyncConfig&, Period vsyncPeriod);
// Chooses a leader among the registered displays, unless `leaderIdOpt` is specified. The new
@@ -299,6 +327,10 @@
// 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);
+
struct Policy;
// Sets the S state of the policy to the T value under mPolicyLock, and chooses a display mode
@@ -355,7 +387,6 @@
std::atomic<nsecs_t> mLastResyncTime = 0;
const FeatureFlags mFeatures;
- std::unique_ptr<VsyncSchedule> mVsyncSchedule;
// Shifts the VSYNC phase during certain transactions and refresh rate changes.
const sp<VsyncModulator> mVsyncModulator;
@@ -380,6 +411,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);
@@ -399,6 +434,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;
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 fdeb310..e969fdc 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 <ftl/concat.h>
#include <gui/TraceUtils.h>
#include <utils/Log.h>
@@ -45,9 +46,10 @@
VSyncPredictor::~VSyncPredictor() = default;
-VSyncPredictor::VSyncPredictor(nsecs_t idealPeriod, size_t historySize,
+VSyncPredictor::VSyncPredictor(PhysicalDisplayId id, nsecs_t idealPeriod, size_t historySize,
size_t minimumSamplesForPrediction, uint32_t outlierTolerancePercent)
- : mTraceOn(property_get_bool("debug.sf.vsp_trace", false)),
+ : mId(id),
+ mTraceOn(property_get_bool("debug.sf.vsp_trace", false)),
kHistorySize(historySize),
kMinimumSamplesForPrediction(minimumSamplesForPrediction),
kOutlierTolerancePercent(std::min(outlierTolerancePercent, kMaxPercent)),
@@ -57,12 +59,12 @@
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);
+ ATRACE_INT64(ftl::Concat(ftl::truncated<14>(name), " ", mId.value).c_str(), value);
}
inline size_t VSyncPredictor::next(size_t i) const {
@@ -214,8 +216,8 @@
it->second = {anticipatedPeriod, intercept};
- ALOGV("model update ts: %" PRId64 " slope: %" PRId64 " intercept: %" PRId64, timestamp,
- anticipatedPeriod, intercept);
+ ALOGV("model update ts %" PRIu64 ": %" PRId64 " slope: %" PRId64 " intercept: %" PRId64,
+ mId.value, timestamp, anticipatedPeriod, intercept);
return true;
}
@@ -331,7 +333,7 @@
}
void VSyncPredictor::setRenderRate(Fps fps) {
- ALOGV("%s: %s", __func__, to_string(fps).c_str());
+ ALOGV("%s %s: %s", __func__, to_string(mId).c_str(), to_string(fps).c_str());
std::lock_guard lock(mMutex);
mRenderRate = fps;
}
@@ -347,7 +349,7 @@
}
void VSyncPredictor::setPeriod(nsecs_t period) {
- ATRACE_CALL();
+ ATRACE_FORMAT("%s %s", __func__, to_string(mId).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 cd5d9ef..c01c44d 100644
--- a/services/surfaceflinger/Scheduler/VSyncPredictor.h
+++ b/services/surfaceflinger/Scheduler/VSyncPredictor.h
@@ -21,6 +21,7 @@
#include <vector>
#include <android-base/thread_annotations.h>
+#include <ui/DisplayId.h>
#include "VSyncTracker.h"
@@ -29,14 +30,15 @@
class VSyncPredictor : public VSyncTracker {
public:
/*
+ * \param [in] PhysicalDisplayid 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(PhysicalDisplayId, nsecs_t idealPeriod, size_t historySize,
+ size_t minimumSamplesForPrediction, uint32_t outlierTolerancePercent);
~VSyncPredictor();
bool addVsyncTimestamp(nsecs_t timestamp) final EXCLUDES(mMutex);
@@ -76,6 +78,8 @@
VSyncPredictor& operator=(VSyncPredictor const&) = delete;
void clearTimestamps() REQUIRES(mMutex);
+ const PhysicalDisplayId mId;
+
inline void traceInt64If(const char* name, int64_t value) const;
inline void traceInt64(const char* name, int64_t value) const;
diff --git a/services/surfaceflinger/Scheduler/VSyncReactor.cpp b/services/surfaceflinger/Scheduler/VSyncReactor.cpp
index b5f212e..2938aa3 100644
--- a/services/surfaceflinger/Scheduler/VSyncReactor.cpp
+++ b/services/surfaceflinger/Scheduler/VSyncReactor.cpp
@@ -21,6 +21,8 @@
#include <assert.h>
#include <cutils/properties.h>
+#include <ftl/concat.h>
+#include <gui/TraceUtils.h>
#include <log/log.h>
#include <utils/Trace.h>
@@ -39,12 +41,13 @@
return systemTime(SYSTEM_TIME_MONOTONIC);
}
-VSyncReactor::VSyncReactor(std::unique_ptr<Clock> clock, VSyncTracker& tracker,
- size_t pendingFenceLimit, bool supportKernelIdleTimer)
- : mClock(std::move(clock)),
+VSyncReactor::VSyncReactor(PhysicalDisplayId id, std::unique_ptr<Clock> clock,
+ VSyncTracker& tracker, size_t pendingFenceLimit,
+ bool supportKernelIdleTimer)
+ : mId(id),
+ mClock(std::move(clock)),
mTracker(tracker),
mPendingLimit(pendingFenceLimit),
- // TODO(adyabr): change mSupportKernelIdleTimer when the active display changes
mSupportKernelIdleTimer(supportKernelIdleTimer) {}
VSyncReactor::~VSyncReactor() = default;
@@ -114,7 +117,7 @@
}
void VSyncReactor::startPeriodTransitionInternal(nsecs_t newPeriod) {
- ATRACE_CALL();
+ ATRACE_FORMAT("%s %" PRIu64, __func__, mId.value);
mPeriodConfirmationInProgress = true;
mPeriodTransitioningTo = newPeriod;
mMoreSamplesNeeded = true;
@@ -122,18 +125,18 @@
}
void VSyncReactor::endPeriodTransition() {
- ATRACE_CALL();
+ ATRACE_FORMAT("%s %" PRIu64, __func__, mId.value);
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) {
+ ATRACE_INT64(ftl::Concat("VSR-", __func__, " ", mId.value).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 +184,7 @@
std::lock_guard lock(mMutex);
if (periodConfirmed(timestamp, hwcVsyncPeriod)) {
- ATRACE_NAME("VSR: period confirmed");
+ ATRACE_FORMAT("VSR %" PRIu64 ": period confirmed", mId.value);
if (mPeriodTransitioningTo) {
mTracker.setPeriod(*mPeriodTransitioningTo);
*periodFlushed = true;
@@ -195,12 +198,12 @@
endPeriodTransition();
mMoreSamplesNeeded = mTracker.needsMoreSamples();
} else if (mPeriodConfirmationInProgress) {
- ATRACE_NAME("VSR: still confirming period");
+ ATRACE_FORMAT("VSR %" PRIu64 ": still confirming period", mId.value);
mLastHwVsync = timestamp;
mMoreSamplesNeeded = true;
*periodFlushed = false;
} else {
- ATRACE_NAME("VSR: adding sample");
+ ATRACE_FORMAT("VSR %" PRIu64 ": adding sample", mId.value);
*periodFlushed = false;
mTracker.addVsyncTimestamp(timestamp);
mMoreSamplesNeeded = mTracker.needsMoreSamples();
diff --git a/services/surfaceflinger/Scheduler/VSyncReactor.h b/services/surfaceflinger/Scheduler/VSyncReactor.h
index 4501487..f230242 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(PhysicalDisplayId, 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 PhysicalDisplayId mId;
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 5245556..62e37db 100644
--- a/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
+++ b/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
@@ -42,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();
}
@@ -54,16 +54,19 @@
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)),
mTracer(features.test(Feature::kTracePredictedVsync)
- ? std::make_unique<PredictedVsyncTracer>(*mDispatch)
+ ? std::make_unique<PredictedVsyncTracer>(mDispatch)
: nullptr) {}
-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)) {}
@@ -95,45 +98,46 @@
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>(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,
+ auto reactor = std::make_unique<VSyncReactor>(id, std::make_unique<SystemClock>(), tracker,
kMaxPendingFences, hasKernelIdleTimer);
reactor->setIgnorePresentFences(!features.test(Feature::kPresentFences));
return reactor;
}
-void VsyncSchedule::startPeriodTransition(ISchedulerCallback& callback, Period period) {
+void VsyncSchedule::startPeriodTransition(ISchedulerCallback& callback, Period period, bool force) {
std::lock_guard<std::mutex> lock(mHwVsyncLock);
- mController->startPeriodTransition(period.ns());
+ mController->startPeriodTransition(period.ns(), force);
enableHardwareVsyncLocked(callback);
}
@@ -165,7 +169,7 @@
void VsyncSchedule::enableHardwareVsyncLocked(ISchedulerCallback& callback) {
if (mHwVsyncState == HwVsyncState::Disabled) {
getTracker().resetModel();
- callback.setVsyncEnabled(true);
+ callback.setVsyncEnabled(mId, true);
mHwVsyncState = HwVsyncState::Enabled;
}
}
@@ -173,7 +177,7 @@
void VsyncSchedule::disableHardwareVsync(ISchedulerCallback& callback, bool disallow) {
std::lock_guard<std::mutex> lock(mHwVsyncLock);
if (mHwVsyncState == HwVsyncState::Enabled) {
- callback.setVsyncEnabled(false);
+ callback.setVsyncEnabled(mId, false);
}
mHwVsyncState = disallow ? HwVsyncState::Disallowed : HwVsyncState::Disabled;
}
diff --git a/services/surfaceflinger/Scheduler/VsyncSchedule.h b/services/surfaceflinger/Scheduler/VsyncSchedule.h
index a32acc7..763d058 100644
--- a/services/surfaceflinger/Scheduler/VsyncSchedule.h
+++ b/services/surfaceflinger/Scheduler/VsyncSchedule.h
@@ -25,6 +25,7 @@
#include <ftl/optional.h>
#include <scheduler/Features.h>
#include <scheduler/Time.h>
+#include <ui/DisplayId.h>
namespace android {
class EventThreadTest;
@@ -50,7 +51,7 @@
// Schedule that synchronizes to hardware VSYNC of a physical display.
class VsyncSchedule {
public:
- explicit VsyncSchedule(FeatureFlags);
+ VsyncSchedule(PhysicalDisplayId, FeatureFlags);
~VsyncSchedule();
Period period() const;
@@ -61,7 +62,9 @@
// enable hardware VSYNCs in order to calibrate.
//
// \param [in] period The period that the system is changing into.
- void startPeriodTransition(ISchedulerCallback&, Period period);
+ // \param [in] force True to force a transition even if it is not a
+ // change.
+ void startPeriodTransition(ISchedulerCallback&, Period period, bool force);
// Pass a VSYNC sample to VsyncController. Return true if
// VsyncController detected that the VSYNC period changed. Enable or disable
@@ -74,8 +77,13 @@
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;
@@ -84,7 +92,8 @@
void enableHardwareVsync(ISchedulerCallback&) EXCLUDES(mHwVsyncLock);
// Disable hardware VSYNCs. If `disallow` is true, future calls to
- // enableHardwareVsync are ineffective until allowHardwareVsync is called.
+ // enableHardwareVsync are ineffective until isHardwareVsyncAllowed is
+ // called with `makeAllowed` set to true.
void disableHardwareVsync(ISchedulerCallback&, bool disallow) EXCLUDES(mHwVsyncLock);
// If true, enableHardwareVsync can enable hardware VSYNC (if not already
@@ -95,22 +104,21 @@
bool getPendingHardwareVsyncState() const REQUIRES(kMainThreadContext);
+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::VsyncScheduleTest;
friend class android::fuzz::SchedulerFuzzer;
- using TrackerPtr = std::unique_ptr<VsyncTracker>;
- using DispatchPtr = std::unique_ptr<VsyncDispatch>;
- using ControllerPtr = std::unique_ptr<VsyncController>;
-
- // For tests.
- VsyncSchedule(TrackerPtr, DispatchPtr, ControllerPtr);
-
- static TrackerPtr createTracker();
- static DispatchPtr createDispatch(VsyncTracker&);
- static ControllerPtr createController(VsyncTracker&, FeatureFlags);
+ static TrackerPtr createTracker(PhysicalDisplayId);
+ static DispatchPtr createDispatch(TrackerPtr);
+ static ControllerPtr createController(PhysicalDisplayId, VsyncTracker&, FeatureFlags);
void enableHardwareVsyncLocked(ISchedulerCallback&) REQUIRES(mHwVsyncLock);
@@ -138,6 +146,7 @@
class PredictedVsyncTracer;
using TracerPtr = std::unique_ptr<PredictedVsyncTracer>;
+ const PhysicalDisplayId mId;
const TrackerPtr mTracker;
const DispatchPtr mDispatch;
const ControllerPtr mController;