SF: add VSyncDispatch
Add a low level timer primitive to arrange for calling a
callback relative to an anticipated vsync time.
This defines vsync offsets in terms of time remaining until the
targeted vsync event, which is a rate independent definition.
Bug: 140301853
Test: 31 new unit tests
Change-Id: I6ac8f05e4c2107755493b2709551773d0e79c17d
diff --git a/services/surfaceflinger/Android.bp b/services/surfaceflinger/Android.bp
index e5d23d0..4226e9a 100644
--- a/services/surfaceflinger/Android.bp
+++ b/services/surfaceflinger/Android.bp
@@ -165,6 +165,7 @@
"Scheduler/PhaseOffsets.cpp",
"Scheduler/Scheduler.cpp",
"Scheduler/SchedulerUtils.cpp",
+ "Scheduler/VSyncDispatch.cpp",
"Scheduler/VSyncModulator.cpp",
"StartPropertySetThread.cpp",
"SurfaceFlinger.cpp",
diff --git a/services/surfaceflinger/Scheduler/TimeKeeper.h b/services/surfaceflinger/Scheduler/TimeKeeper.h
new file mode 100644
index 0000000..699cd50
--- /dev/null
+++ b/services/surfaceflinger/Scheduler/TimeKeeper.h
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2019 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 <utils/Timers.h>
+#include <functional>
+
+namespace android::scheduler {
+
+/*
+ * TimeKeeper is the interface for a single-shot timer primitive.
+ */
+class TimeKeeper {
+public:
+ virtual ~TimeKeeper();
+
+ /*
+ * Arms callback to fired in time nanoseconds.
+ * There is only one timer, and subsequent calls will reset the callback function and the time.
+ */
+ virtual void alarmIn(std::function<void()> const& callback, nsecs_t time) = 0;
+
+ /*
+ * Cancels an existing pending callback
+ */
+ virtual void alarmCancel() = 0;
+
+ /*
+ * Returns the SYSTEM_TIME_MONOTONIC, used by testing infra to stub time.
+ */
+ virtual nsecs_t now() const = 0;
+
+protected:
+ TimeKeeper(TimeKeeper const&) = delete;
+ TimeKeeper& operator=(TimeKeeper const&) = delete;
+ TimeKeeper() = default;
+};
+
+} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatch.cpp b/services/surfaceflinger/Scheduler/VSyncDispatch.cpp
new file mode 100644
index 0000000..c9b2e77
--- /dev/null
+++ b/services/surfaceflinger/Scheduler/VSyncDispatch.cpp
@@ -0,0 +1,285 @@
+/*
+ * Copyright 2019 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.
+ */
+
+#define ATRACE_TAG ATRACE_TAG_GRAPHICS
+#include <utils/Trace.h>
+#include <vector>
+
+#include "TimeKeeper.h"
+#include "VSyncDispatch.h"
+#include "VSyncTracker.h"
+
+namespace android::scheduler {
+
+VSyncTracker::~VSyncTracker() = default;
+TimeKeeper::~TimeKeeper() = default;
+
+impl::VSyncDispatchEntry::VSyncDispatchEntry(std::string const& name,
+ std::function<void(nsecs_t)> const& cb)
+ : mName(name), mCallback(cb), mWorkDuration(0), mEarliestVsync(0) {}
+
+std::optional<nsecs_t> impl::VSyncDispatchEntry::lastExecutedVsyncTarget() const {
+ return mLastDispatchTime;
+}
+
+std::string_view impl::VSyncDispatchEntry::name() const {
+ return mName;
+}
+
+std::optional<nsecs_t> impl::VSyncDispatchEntry::wakeupTime() const {
+ if (!mArmedInfo) {
+ return {};
+ }
+ return {mArmedInfo->mActualWakeupTime};
+}
+
+nsecs_t impl::VSyncDispatchEntry::schedule(nsecs_t workDuration, nsecs_t earliestVsync,
+ VSyncTracker& tracker, nsecs_t now) {
+ mWorkDuration = workDuration;
+ mEarliestVsync = earliestVsync;
+ arm(tracker, now);
+ return mArmedInfo->mActualWakeupTime;
+}
+
+void impl::VSyncDispatchEntry::update(VSyncTracker& tracker, nsecs_t now) {
+ if (!mArmedInfo) {
+ return;
+ }
+ arm(tracker, now);
+}
+
+void impl::VSyncDispatchEntry::arm(VSyncTracker& tracker, nsecs_t now) {
+ auto const nextVsyncTime =
+ tracker.nextAnticipatedVSyncTimeFrom(std::max(mEarliestVsync, now + mWorkDuration));
+ mArmedInfo = {nextVsyncTime - mWorkDuration, nextVsyncTime};
+}
+
+void impl::VSyncDispatchEntry::disarm() {
+ mArmedInfo.reset();
+}
+
+nsecs_t impl::VSyncDispatchEntry::executing() {
+ mLastDispatchTime = mArmedInfo->mActualVsyncTime;
+ disarm();
+ return *mLastDispatchTime;
+}
+
+void impl::VSyncDispatchEntry::callback(nsecs_t t) {
+ {
+ std::lock_guard<std::mutex> lk(mRunningMutex);
+ mRunning = true;
+ }
+
+ mCallback(t);
+
+ std::lock_guard<std::mutex> lk(mRunningMutex);
+ mRunning = false;
+ mCv.notify_all();
+}
+
+void impl::VSyncDispatchEntry::ensureNotRunning() {
+ std::unique_lock<std::mutex> lk(mRunningMutex);
+ mCv.wait(lk, [this]() REQUIRES(mRunningMutex) { return !mRunning; });
+}
+
+VSyncDispatch::VSyncDispatch(std::unique_ptr<TimeKeeper> tk, VSyncTracker& tracker,
+ nsecs_t timerSlack)
+ : mTimeKeeper(std::move(tk)), mTracker(tracker), mTimerSlack(timerSlack) {}
+
+VSyncDispatch::~VSyncDispatch() {
+ std::lock_guard<decltype(mMutex)> lk(mMutex);
+ cancelTimer();
+}
+
+void VSyncDispatch::cancelTimer() {
+ mIntendedWakeupTime = kInvalidTime;
+ mTimeKeeper->alarmCancel();
+}
+
+void VSyncDispatch::setTimer(nsecs_t targetTime, nsecs_t now) {
+ mIntendedWakeupTime = targetTime;
+ mTimeKeeper->alarmIn(std::bind(&VSyncDispatch::timerCallback, this), targetTime - now);
+}
+
+void VSyncDispatch::rearmTimer(nsecs_t now) {
+ rearmTimerSkippingUpdateFor(now, mCallbacks.end());
+}
+
+void VSyncDispatch::rearmTimerSkippingUpdateFor(nsecs_t now,
+ CallbackMap::iterator const& skipUpdateIt) {
+ std::optional<nsecs_t> min;
+ for (auto it = mCallbacks.begin(); it != mCallbacks.end(); it++) {
+ auto& callback = it->second;
+ if (!callback->wakeupTime()) {
+ continue;
+ }
+
+ if (it != skipUpdateIt) {
+ callback->update(mTracker, now);
+ }
+ auto const wakeupTime = *callback->wakeupTime();
+ if (!min || (min && *min > wakeupTime)) {
+ min = wakeupTime;
+ }
+ }
+
+ if (min && (min < mIntendedWakeupTime)) {
+ setTimer(*min, now);
+ } else {
+ cancelTimer();
+ }
+}
+
+void VSyncDispatch::timerCallback() {
+ struct Invocation {
+ std::shared_ptr<impl::VSyncDispatchEntry> callback;
+ nsecs_t timestamp;
+ };
+ std::vector<Invocation> invocations;
+ {
+ std::lock_guard<decltype(mMutex)> lk(mMutex);
+ for (auto it = mCallbacks.begin(); it != mCallbacks.end(); it++) {
+ auto& callback = it->second;
+ auto const wakeupTime = callback->wakeupTime();
+ if (!wakeupTime) {
+ continue;
+ }
+
+ if (*wakeupTime < mIntendedWakeupTime + mTimerSlack) {
+ callback->executing();
+ invocations.emplace_back(
+ Invocation{callback, *callback->lastExecutedVsyncTarget()});
+ }
+ }
+
+ mIntendedWakeupTime = kInvalidTime;
+ rearmTimer(mTimeKeeper->now());
+ }
+
+ for (auto const& invocation : invocations) {
+ invocation.callback->callback(invocation.timestamp);
+ }
+}
+
+VSyncDispatch::CallbackToken VSyncDispatch::registerCallback(
+ std::function<void(nsecs_t)> const& callbackFn, std::string callbackName) {
+ std::lock_guard<decltype(mMutex)> lk(mMutex);
+ return CallbackToken{
+ mCallbacks
+ .emplace(++mCallbackToken,
+ std::make_shared<impl::VSyncDispatchEntry>(callbackName, callbackFn))
+ .first->first};
+}
+
+void VSyncDispatch::unregisterCallback(CallbackToken token) {
+ std::shared_ptr<impl::VSyncDispatchEntry> entry = nullptr;
+ {
+ std::lock_guard<decltype(mMutex)> lk(mMutex);
+ auto it = mCallbacks.find(token);
+ if (it != mCallbacks.end()) {
+ entry = it->second;
+ mCallbacks.erase(it);
+ }
+ }
+
+ if (entry) {
+ entry->ensureNotRunning();
+ }
+}
+
+ScheduleResult VSyncDispatch::schedule(CallbackToken token, nsecs_t workDuration,
+ nsecs_t earliestVsync) {
+ auto result = ScheduleResult::Error;
+ {
+ std::lock_guard<decltype(mMutex)> lk(mMutex);
+
+ auto it = mCallbacks.find(token);
+ if (it == mCallbacks.end()) {
+ return result;
+ }
+ auto& callback = it->second;
+ result = callback->wakeupTime() ? ScheduleResult::ReScheduled : ScheduleResult::Scheduled;
+
+ auto const now = mTimeKeeper->now();
+ auto const wakeupTime = callback->schedule(workDuration, earliestVsync, mTracker, now);
+
+ if (wakeupTime < now - mTimerSlack || callback->lastExecutedVsyncTarget() > wakeupTime) {
+ return ScheduleResult::CannotSchedule;
+ }
+
+ if (wakeupTime < mIntendedWakeupTime - mTimerSlack) {
+ rearmTimerSkippingUpdateFor(now, it);
+ }
+ }
+
+ return result;
+}
+
+CancelResult VSyncDispatch::cancel(CallbackToken token) {
+ std::lock_guard<decltype(mMutex)> lk(mMutex);
+
+ auto it = mCallbacks.find(token);
+ if (it == mCallbacks.end()) {
+ return CancelResult::Error;
+ }
+ auto& callback = it->second;
+
+ if (callback->wakeupTime()) {
+ callback->disarm();
+ mIntendedWakeupTime = kInvalidTime;
+ rearmTimer(mTimeKeeper->now());
+ return CancelResult::Cancelled;
+ }
+ return CancelResult::TooLate;
+}
+
+VSyncCallbackRegistration::VSyncCallbackRegistration(VSyncDispatch& dispatch,
+ std::function<void(nsecs_t)> const& callbackFn,
+ std::string const& callbackName)
+ : mDispatch(dispatch),
+ mToken(dispatch.registerCallback(callbackFn, callbackName)),
+ mValidToken(true) {}
+
+VSyncCallbackRegistration::VSyncCallbackRegistration(VSyncCallbackRegistration&& other)
+ : mDispatch(other.mDispatch),
+ mToken(std::move(other.mToken)),
+ mValidToken(std::move(other.mValidToken)) {
+ other.mValidToken = false;
+}
+
+VSyncCallbackRegistration& VSyncCallbackRegistration::operator=(VSyncCallbackRegistration&& other) {
+ mDispatch = std::move(other.mDispatch);
+ mToken = std::move(other.mToken);
+ mValidToken = std::move(other.mValidToken);
+ other.mValidToken = false;
+ return *this;
+}
+
+VSyncCallbackRegistration::~VSyncCallbackRegistration() {
+ if (mValidToken) mDispatch.get().unregisterCallback(mToken);
+}
+
+ScheduleResult VSyncCallbackRegistration::schedule(nsecs_t workDuration, nsecs_t earliestVsync) {
+ if (!mValidToken) return ScheduleResult::Error;
+ return mDispatch.get().schedule(mToken, workDuration, earliestVsync);
+}
+
+CancelResult VSyncCallbackRegistration::cancel() {
+ if (!mValidToken) return CancelResult::Error;
+ return mDispatch.get().cancel(mToken);
+}
+
+} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatch.h b/services/surfaceflinger/Scheduler/VSyncDispatch.h
new file mode 100644
index 0000000..0050495
--- /dev/null
+++ b/services/surfaceflinger/Scheduler/VSyncDispatch.h
@@ -0,0 +1,230 @@
+/*
+ * Copyright 2019 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 <android-base/thread_annotations.h>
+#include <utils/Timers.h>
+#include <functional>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <string_view>
+#include <unordered_map>
+
+#include "StrongTyping.h"
+
+namespace android::scheduler {
+class TimeKeeper;
+class VSyncTracker;
+
+enum class ScheduleResult { Scheduled, ReScheduled, CannotSchedule, Error };
+enum class CancelResult { Cancelled, TooLate, Error };
+
+namespace impl {
+
+// VSyncDispatchEntry is a helper class representing internal state for each entry in VSyncDispatch
+// hoisted to public for unit testing.
+class VSyncDispatchEntry {
+public:
+ // This is the state of the entry. There are 3 states, armed, running, disarmed.
+ // Valid transition: disarmed -> armed ( when scheduled )
+ // Valid transition: armed -> running -> disarmed ( when timer is called)
+ // Valid transition: armed -> disarmed ( when cancelled )
+ VSyncDispatchEntry(std::string const& name, std::function<void(nsecs_t)> const& fn);
+ std::string_view name() const;
+
+ // Start: functions that are not threadsafe.
+ // Return the last vsync time this callback was invoked.
+ std::optional<nsecs_t> lastExecutedVsyncTarget() const;
+
+ // This moves the state from disarmed->armed and will calculate the wakeupTime.
+ nsecs_t schedule(nsecs_t workDuration, nsecs_t earliestVsync, VSyncTracker& tracker,
+ nsecs_t now);
+ // This will update armed entries with the latest vsync information. Entry remains armed.
+ void update(VSyncTracker& tracker, nsecs_t now);
+
+ // This will return empty if not armed, or the next calculated wakeup time if armed.
+ // It will not update the wakeupTime.
+ std::optional<nsecs_t> wakeupTime() const;
+
+ // This moves state from armed->disarmed.
+ void disarm();
+
+ // This moves the state from armed->running.
+ // Store the timestamp that this was intended for as the last called timestamp.
+ nsecs_t executing();
+ // End: functions that are not threadsafe.
+
+ // Invoke the callback with the timestamp, moving the state from running->disarmed.
+ void callback(nsecs_t timestamp);
+ // Block calling thread while the callback is executing.
+ void ensureNotRunning();
+
+private:
+ void arm(VSyncTracker& tracker, nsecs_t now);
+ std::string const mName;
+ std::function<void(nsecs_t)> const mCallback;
+
+ nsecs_t mWorkDuration;
+ nsecs_t mEarliestVsync;
+
+ struct ArmingInfo {
+ nsecs_t mActualWakeupTime;
+ nsecs_t mActualVsyncTime;
+ };
+ std::optional<ArmingInfo> mArmedInfo;
+ std::optional<nsecs_t> mLastDispatchTime;
+
+ std::mutex mRunningMutex;
+ std::condition_variable mCv;
+ bool mRunning GUARDED_BY(mRunningMutex) = false;
+};
+
+} // namespace impl
+
+/*
+ * VSyncDispatch is a class that will dispatch callbacks relative to system vsync events.
+ */
+class VSyncDispatch {
+public:
+ using CallbackToken = StrongTyping<size_t, class CallbackTokenTag, Compare>;
+
+ /* creates a VsyncDispatch.
+ * \param [in] a timekeeper object for dispatching events.
+ * \param [in] a tracker object that is monitoring expected vsync events.
+ * \param [in] a tunable in nanoseconds that indicates when events that fall close together
+ * should be dispatched in one timer wakeup.
+ */
+ explicit VSyncDispatch(std::unique_ptr<TimeKeeper> tk, VSyncTracker& tracker,
+ nsecs_t timerSlack);
+ ~VSyncDispatch();
+
+ /*
+ * Registers a callback that will be called at designated points on the vsync timeline.
+ * The callback can be scheduled, rescheduled targeting vsync times, or cancelled.
+ * The token returned must be cleaned up via unregisterCallback.
+ *
+ * \param [in] callbackFn A function to schedule for callback. The resources needed to invoke
+ * callbackFn must have lifetimes encompassing the lifetime of the
+ * CallbackToken returned.
+ * \param [in] callbackName A human-readable, unique name to identify the callback.
+ * \return A token that can be used to schedule, reschedule, or cancel the
+ * invocation of callbackFn.
+ *
+ */
+ CallbackToken registerCallback(std::function<void(nsecs_t)> const& callbackFn,
+ std::string callbackName);
+
+ /*
+ * Unregisters a callback.
+ *
+ * \param [in] token The callback to unregister.
+ *
+ */
+ void unregisterCallback(CallbackToken token);
+
+ /*
+ * Schedules the registered callback to be dispatched.
+ *
+ * The callback will be dispatched at 'workDuration' nanoseconds before a vsync event.
+ *
+ * The caller designates the earliest vsync event that should be targeted by the earliestVsync
+ * parameter.
+ * The callback will be scheduled at (workDuration - predictedVsync), where predictedVsync
+ * is the first vsync event time where ( predictedVsync >= earliestVsync ).
+ *
+ * If (workDuration - earliestVsync) is in the past, or if a callback has already been
+ * dispatched for the predictedVsync, an error will be returned.
+ *
+ * It is valid to reschedule a callback to a different time.
+ *
+ * \param [in] token The callback to schedule.
+ * \param [in] workDuration The time before the actual vsync time to invoke the callback
+ * associated with token.
+ * \param [in] earliestVsync The targeted display time. This will be snapped to the closest
+ * predicted vsync time after earliestVsync.
+ * \return A ScheduleResult::Scheduled if callback was scheduled.
+ * A ScheduleResult::ReScheduled if callback was rescheduled.
+ * A ScheduleResult::CannotSchedule
+ * if (workDuration - earliestVsync) is in the past, or
+ * if a callback was dispatched for the predictedVsync already.
+ * A ScheduleResult::Error if there was another error.
+ */
+ ScheduleResult schedule(CallbackToken token, nsecs_t workDuration, nsecs_t earliestVsync);
+
+ /* Cancels a scheduled callback, if possible.
+ *
+ * \param [in] token The callback to cancel.
+ * \return A CancelResult::TooLate if the callback was already dispatched.
+ * A CancelResult::Cancelled if the callback was successfully cancelled.
+ * A CancelResult::Error if there was an pre-condition violation.
+ */
+ CancelResult cancel(CallbackToken token);
+
+private:
+ VSyncDispatch(VSyncDispatch const&) = delete;
+ VSyncDispatch& operator=(VSyncDispatch const&) = delete;
+
+ using CallbackMap = std::unordered_map<size_t, std::shared_ptr<impl::VSyncDispatchEntry>>;
+
+ void timerCallback();
+ void setTimer(nsecs_t, nsecs_t) REQUIRES(mMutex);
+ void rearmTimer(nsecs_t now) REQUIRES(mMutex);
+ void rearmTimerSkippingUpdateFor(nsecs_t now, CallbackMap::iterator const& skipUpdate)
+ REQUIRES(mMutex);
+ void cancelTimer() REQUIRES(mMutex);
+
+ static constexpr nsecs_t kInvalidTime = std::numeric_limits<int64_t>::max();
+ std::unique_ptr<TimeKeeper> const mTimeKeeper;
+ VSyncTracker& mTracker;
+ nsecs_t const mTimerSlack;
+
+ std::mutex mutable mMutex;
+ size_t mCallbackToken GUARDED_BY(mMutex) = 0;
+
+ CallbackMap mCallbacks GUARDED_BY(mMutex);
+ nsecs_t mIntendedWakeupTime GUARDED_BY(mMutex) = kInvalidTime;
+};
+
+/*
+ * Helper class to operate on registered callbacks. It is up to user of the class to ensure
+ * that VsyncDispatch lifetime exceeds the lifetime of VSyncCallbackRegistation.
+ */
+class VSyncCallbackRegistration {
+public:
+ VSyncCallbackRegistration(VSyncDispatch&, std::function<void(nsecs_t)> const& callbackFn,
+ std::string const& callbackName);
+ VSyncCallbackRegistration(VSyncCallbackRegistration&&);
+ VSyncCallbackRegistration& operator=(VSyncCallbackRegistration&&);
+ ~VSyncCallbackRegistration();
+
+ // See documentation for VSyncDispatch::schedule.
+ ScheduleResult schedule(nsecs_t workDuration, nsecs_t earliestVsync);
+
+ // See documentation for VSyncDispatch::cancel.
+ CancelResult cancel();
+
+private:
+ VSyncCallbackRegistration(VSyncCallbackRegistration const&) = delete;
+ VSyncCallbackRegistration& operator=(VSyncCallbackRegistration const&) = delete;
+
+ std::reference_wrapper<VSyncDispatch> mDispatch;
+ VSyncDispatch::CallbackToken mToken;
+ bool mValidToken;
+};
+
+} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/VSyncTracker.h b/services/surfaceflinger/Scheduler/VSyncTracker.h
new file mode 100644
index 0000000..97b9620
--- /dev/null
+++ b/services/surfaceflinger/Scheduler/VSyncTracker.h
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2019 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 <utils/Timers.h>
+#include "VSyncDispatch.h"
+
+namespace android::scheduler {
+/*
+ * VSyncTracker is an interface for providing estimates on future Vsync signal times based on
+ * historical vsync timing data.
+ */
+class VSyncTracker {
+public:
+ virtual ~VSyncTracker();
+
+ /*
+ * Adds a known timestamp from a vsync timing source (HWVsync signal, present fence)
+ * to the model.
+ *
+ * \param [in] timestamp The timestamp when the vsync signal was.
+ */
+ virtual void addVsyncTimestamp(nsecs_t timestamp) = 0;
+
+ /*
+ * Access the next anticipated vsync time such that the anticipated time >= timePoint.
+ * This will always give the best accurate at the time of calling; multiple
+ * calls with the same timePoint might give differing values if the internal model
+ * is updated.
+ *
+ * \param [in] timePoint The point in time after which to estimate a vsync event.
+ * \return A prediction of the timestamp of a vsync event.
+ */
+ virtual nsecs_t nextAnticipatedVSyncTimeFrom(nsecs_t timePoint) const = 0;
+
+protected:
+ VSyncTracker(VSyncTracker const&) = delete;
+ VSyncTracker& operator=(VSyncTracker const&) = delete;
+ VSyncTracker() = default;
+};
+
+} // namespace android::scheduler
diff --git a/services/surfaceflinger/tests/unittests/Android.bp b/services/surfaceflinger/tests/unittests/Android.bp
index f85da20..246a62f 100644
--- a/services/surfaceflinger/tests/unittests/Android.bp
+++ b/services/surfaceflinger/tests/unittests/Android.bp
@@ -54,6 +54,7 @@
"FrameTracerTest.cpp",
"TransactionApplicationTest.cpp",
"StrongTypingTest.cpp",
+ "VSyncDispatchTest.cpp",
"mock/DisplayHardware/MockComposer.cpp",
"mock/DisplayHardware/MockDisplay.cpp",
"mock/DisplayHardware/MockPowerAdvisor.cpp",
diff --git a/services/surfaceflinger/tests/unittests/VSyncDispatchTest.cpp b/services/surfaceflinger/tests/unittests/VSyncDispatchTest.cpp
new file mode 100644
index 0000000..d1ed7e3
--- /dev/null
+++ b/services/surfaceflinger/tests/unittests/VSyncDispatchTest.cpp
@@ -0,0 +1,680 @@
+/*
+ * Copyright 2019 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.
+ */
+
+#undef LOG_TAG
+#define LOG_TAG "LibSurfaceFlingerUnittests"
+#define LOG_NDEBUG 0
+
+#include "Scheduler/TimeKeeper.h"
+#include "Scheduler/VSyncDispatch.h"
+#include "Scheduler/VSyncTracker.h"
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+#include <thread>
+
+using namespace testing;
+using namespace std::literals;
+namespace android::scheduler {
+
+class MockVSyncTracker : public VSyncTracker {
+public:
+ MockVSyncTracker(nsecs_t period) : mPeriod{period} {
+ ON_CALL(*this, nextAnticipatedVSyncTimeFrom(_))
+ .WillByDefault(Invoke(this, &MockVSyncTracker::nextVSyncTime));
+ }
+
+ MOCK_METHOD1(addVsyncTimestamp, void(nsecs_t));
+ MOCK_CONST_METHOD1(nextAnticipatedVSyncTimeFrom, nsecs_t(nsecs_t));
+
+ nsecs_t nextVSyncTime(nsecs_t timePoint) const {
+ if (timePoint % mPeriod == 0) {
+ return timePoint;
+ }
+ return (timePoint - (timePoint % mPeriod) + mPeriod);
+ }
+
+protected:
+ nsecs_t const mPeriod;
+};
+
+class ControllableClock : public TimeKeeper {
+public:
+ ControllableClock() {
+ ON_CALL(*this, alarmIn(_, _))
+ .WillByDefault(Invoke(this, &ControllableClock::alarmInDefaultBehavior));
+ ON_CALL(*this, now()).WillByDefault(Invoke(this, &ControllableClock::fakeTime));
+ }
+
+ MOCK_CONST_METHOD0(now, nsecs_t());
+ MOCK_METHOD2(alarmIn, void(std::function<void()> const&, nsecs_t time));
+ MOCK_METHOD0(alarmCancel, void());
+
+ void alarmInDefaultBehavior(std::function<void()> const& callback, nsecs_t time) {
+ mCallback = callback;
+ mNextCallbackTime = time + mCurrentTime;
+ }
+
+ nsecs_t fakeTime() const { return mCurrentTime; }
+
+ void advanceToNextCallback() {
+ mCurrentTime = mNextCallbackTime;
+ if (mCallback) {
+ mCallback();
+ }
+ }
+
+ void advanceBy(nsecs_t advancement) {
+ mCurrentTime += advancement;
+ if (mCurrentTime >= mNextCallbackTime && mCallback) {
+ mCallback();
+ }
+ };
+
+private:
+ std::function<void()> mCallback;
+ nsecs_t mNextCallbackTime = 0;
+ nsecs_t mCurrentTime = 0;
+};
+
+class CountingCallback {
+public:
+ CountingCallback(VSyncDispatch& dispatch)
+ : mDispatch(dispatch),
+ mToken(dispatch.registerCallback(std::bind(&CountingCallback::counter, this,
+ std::placeholders::_1),
+ "test")) {}
+ ~CountingCallback() { mDispatch.unregisterCallback(mToken); }
+
+ operator VSyncDispatch::CallbackToken() const { return mToken; }
+
+ void counter(nsecs_t time) { mCalls.push_back(time); }
+
+ VSyncDispatch& mDispatch;
+ VSyncDispatch::CallbackToken mToken;
+ std::vector<nsecs_t> mCalls;
+};
+
+class PausingCallback {
+public:
+ PausingCallback(VSyncDispatch& dispatch, std::chrono::milliseconds pauseAmount)
+ : mDispatch(dispatch),
+ mToken(dispatch.registerCallback(std::bind(&PausingCallback::pause, this,
+ std::placeholders::_1),
+ "test")),
+ mRegistered(true),
+ mPauseAmount(pauseAmount) {}
+ ~PausingCallback() { unregister(); }
+
+ operator VSyncDispatch::CallbackToken() const { return mToken; }
+
+ void pause(nsecs_t) {
+ std::unique_lock<std::mutex> lk(mMutex);
+ mPause = true;
+ mCv.notify_all();
+
+ mCv.wait_for(lk, mPauseAmount, [this] { return !mPause; });
+
+ mResourcePresent = (mResource.lock() != nullptr);
+ }
+
+ bool waitForPause() {
+ std::unique_lock<std::mutex> lk(mMutex);
+ auto waiting = mCv.wait_for(lk, 10s, [this] { return mPause; });
+ return waiting;
+ }
+
+ void stashResource(std::weak_ptr<void> const& resource) { mResource = resource; }
+
+ bool resourcePresent() { return mResourcePresent; }
+
+ void unpause() {
+ std::unique_lock<std::mutex> lk(mMutex);
+ mPause = false;
+ mCv.notify_all();
+ }
+
+ void unregister() {
+ if (mRegistered) {
+ mDispatch.unregisterCallback(mToken);
+ mRegistered = false;
+ }
+ }
+
+ VSyncDispatch& mDispatch;
+ VSyncDispatch::CallbackToken mToken;
+ bool mRegistered = true;
+
+ std::mutex mMutex;
+ std::condition_variable mCv;
+ bool mPause = false;
+ std::weak_ptr<void> mResource;
+ bool mResourcePresent = false;
+ std::chrono::milliseconds const mPauseAmount;
+};
+
+class VSyncDispatchTest : public testing::Test {
+protected:
+ std::unique_ptr<TimeKeeper> createTimeKeeper() {
+ class TimeKeeperWrapper : public TimeKeeper {
+ public:
+ TimeKeeperWrapper(TimeKeeper& control) : mControllableClock(control) {}
+ void alarmIn(std::function<void()> const& callback, nsecs_t time) final {
+ mControllableClock.alarmIn(callback, time);
+ }
+ void alarmCancel() final { mControllableClock.alarmCancel(); }
+ nsecs_t now() const final { return mControllableClock.now(); }
+
+ private:
+ TimeKeeper& mControllableClock;
+ };
+ return std::make_unique<TimeKeeperWrapper>(mMockClock);
+ }
+
+ ~VSyncDispatchTest() {
+ // destructor of dispatch will cancelAlarm(). Ignore final cancel in common test.
+ Mock::VerifyAndClearExpectations(&mMockClock);
+ }
+
+ void advanceToNextCallback() { mMockClock.advanceToNextCallback(); }
+
+ NiceMock<ControllableClock> mMockClock;
+ static nsecs_t constexpr mDispatchGroupThreshold = 5;
+ nsecs_t const mPeriod = 1000;
+ NiceMock<MockVSyncTracker> mStubTracker{mPeriod};
+ VSyncDispatch mDispatch{createTimeKeeper(), mStubTracker, mDispatchGroupThreshold};
+};
+
+TEST_F(VSyncDispatchTest, unregistersSetAlarmOnDestruction) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 900));
+ EXPECT_CALL(mMockClock, alarmCancel());
+ {
+ VSyncDispatch mDispatch{createTimeKeeper(), mStubTracker, mDispatchGroupThreshold};
+ CountingCallback cb(mDispatch);
+ EXPECT_EQ(mDispatch.schedule(cb, 100, 1000), ScheduleResult::Scheduled);
+ }
+}
+
+TEST_F(VSyncDispatchTest, basicAlarmSettingFuture) {
+ auto intended = mPeriod - 230;
+ EXPECT_CALL(mMockClock, alarmIn(_, 900));
+
+ CountingCallback cb(mDispatch);
+ EXPECT_EQ(mDispatch.schedule(cb, 100, intended), ScheduleResult::Scheduled);
+ advanceToNextCallback();
+
+ ASSERT_THAT(cb.mCalls.size(), Eq(1));
+ EXPECT_THAT(cb.mCalls[0], Eq(mPeriod));
+}
+
+TEST_F(VSyncDispatchTest, basicAlarmSettingFutureWithAdjustmentToTrueVsync) {
+ EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000)).WillOnce(Return(1150));
+ EXPECT_CALL(mMockClock, alarmIn(_, 1050));
+
+ CountingCallback cb(mDispatch);
+ mDispatch.schedule(cb, 100, mPeriod);
+ advanceToNextCallback();
+
+ ASSERT_THAT(cb.mCalls.size(), Eq(1));
+ EXPECT_THAT(cb.mCalls[0], Eq(1150));
+}
+
+TEST_F(VSyncDispatchTest, basicAlarmSettingAdjustmentPast) {
+ auto const now = 234;
+ mMockClock.advanceBy(234);
+ auto const workDuration = 10 * mPeriod;
+ EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(now + workDuration))
+ .WillOnce(Return(mPeriod * 11));
+ EXPECT_CALL(mMockClock, alarmIn(_, mPeriod - now));
+
+ CountingCallback cb(mDispatch);
+ EXPECT_EQ(mDispatch.schedule(cb, workDuration, mPeriod), ScheduleResult::Scheduled);
+}
+
+TEST_F(VSyncDispatchTest, basicAlarmCancel) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 900));
+ EXPECT_CALL(mMockClock, alarmCancel());
+
+ CountingCallback cb(mDispatch);
+ EXPECT_EQ(mDispatch.schedule(cb, 100, mPeriod), ScheduleResult::Scheduled);
+ EXPECT_EQ(mDispatch.cancel(cb), CancelResult::Cancelled);
+}
+
+TEST_F(VSyncDispatchTest, basicAlarmCancelTooLate) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 900));
+ EXPECT_CALL(mMockClock, alarmCancel());
+
+ CountingCallback cb(mDispatch);
+ EXPECT_EQ(mDispatch.schedule(cb, 100, mPeriod), ScheduleResult::Scheduled);
+ mMockClock.advanceBy(950);
+ EXPECT_EQ(mDispatch.cancel(cb), CancelResult::TooLate);
+}
+
+TEST_F(VSyncDispatchTest, basicAlarmCancelTooLateWhenRunning) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 900));
+ EXPECT_CALL(mMockClock, alarmCancel());
+
+ PausingCallback cb(mDispatch, std::chrono::duration_cast<std::chrono::milliseconds>(1s));
+ EXPECT_EQ(mDispatch.schedule(cb, 100, mPeriod), ScheduleResult::Scheduled);
+
+ std::thread pausingThread([&] { mMockClock.advanceToNextCallback(); });
+ EXPECT_TRUE(cb.waitForPause());
+ EXPECT_EQ(mDispatch.cancel(cb), CancelResult::TooLate);
+ cb.unpause();
+ pausingThread.join();
+}
+
+TEST_F(VSyncDispatchTest, unregisterSynchronizes) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 900));
+ EXPECT_CALL(mMockClock, alarmCancel());
+
+ auto resource = std::make_shared<int>(110);
+
+ PausingCallback cb(mDispatch, 50ms);
+ cb.stashResource(resource);
+ EXPECT_EQ(mDispatch.schedule(cb, 100, mPeriod), ScheduleResult::Scheduled);
+
+ std::thread pausingThread([&] { mMockClock.advanceToNextCallback(); });
+ EXPECT_TRUE(cb.waitForPause());
+
+ cb.unregister();
+ resource.reset();
+
+ cb.unpause();
+ pausingThread.join();
+
+ EXPECT_TRUE(cb.resourcePresent());
+}
+
+TEST_F(VSyncDispatchTest, basicTwoAlarmSetting) {
+ EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(1000))
+ .Times(4)
+ .WillOnce(Return(1055))
+ .WillOnce(Return(1063))
+ .WillOnce(Return(1063))
+ .WillOnce(Return(1075));
+
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 955)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 813)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 162)).InSequence(seq);
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+
+ mDispatch.schedule(cb0, 100, mPeriod);
+ mDispatch.schedule(cb1, 250, mPeriod);
+
+ advanceToNextCallback();
+ advanceToNextCallback();
+
+ ASSERT_THAT(cb0.mCalls.size(), Eq(1));
+ EXPECT_THAT(cb0.mCalls[0], Eq(1075));
+ ASSERT_THAT(cb1.mCalls.size(), Eq(1));
+ EXPECT_THAT(cb1.mCalls[0], Eq(1063));
+}
+
+TEST_F(VSyncDispatchTest, rearmsFaroutTimeoutWhenCancellingCloseOne) {
+ EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(_))
+ .Times(4)
+ .WillOnce(Return(10000))
+ .WillOnce(Return(1000))
+ .WillOnce(Return(10000))
+ .WillOnce(Return(10000));
+
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 9900)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 750)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 9900)).InSequence(seq);
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+
+ mDispatch.schedule(cb0, 100, mPeriod * 10);
+ mDispatch.schedule(cb1, 250, mPeriod);
+ mDispatch.cancel(cb1);
+}
+
+TEST_F(VSyncDispatchTest, noUnnecessaryRearmsWhenRescheduling) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 600)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 100)).InSequence(seq);
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+
+ mDispatch.schedule(cb0, 400, 1000);
+ mDispatch.schedule(cb1, 200, 1000);
+ mDispatch.schedule(cb1, 300, 1000);
+ advanceToNextCallback();
+}
+
+TEST_F(VSyncDispatchTest, necessaryRearmsWhenModifying) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 600)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 500)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 100)).InSequence(seq);
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+
+ mDispatch.schedule(cb0, 400, 1000);
+ mDispatch.schedule(cb1, 200, 1000);
+ mDispatch.schedule(cb1, 500, 1000);
+ advanceToNextCallback();
+}
+
+TEST_F(VSyncDispatchTest, modifyIntoGroup) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 600)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 1000)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 990)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 10)).InSequence(seq);
+
+ auto offset = 400;
+ auto closeOffset = offset + mDispatchGroupThreshold - 1;
+ auto notCloseOffset = offset + 2 * mDispatchGroupThreshold;
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+
+ mDispatch.schedule(cb0, 400, 1000);
+ mDispatch.schedule(cb1, 200, 1000);
+ mDispatch.schedule(cb1, closeOffset, 1000);
+
+ advanceToNextCallback();
+ ASSERT_THAT(cb0.mCalls.size(), Eq(1));
+ EXPECT_THAT(cb0.mCalls[0], Eq(mPeriod));
+ ASSERT_THAT(cb1.mCalls.size(), Eq(1));
+ EXPECT_THAT(cb1.mCalls[0], Eq(mPeriod));
+
+ mDispatch.schedule(cb0, 400, 2000);
+ mDispatch.schedule(cb1, notCloseOffset, 2000);
+ advanceToNextCallback();
+ ASSERT_THAT(cb1.mCalls.size(), Eq(2));
+ EXPECT_THAT(cb1.mCalls[1], Eq(2000));
+
+ advanceToNextCallback();
+ ASSERT_THAT(cb0.mCalls.size(), Eq(2));
+ EXPECT_THAT(cb0.mCalls[1], Eq(2000));
+}
+
+TEST_F(VSyncDispatchTest, rearmsWhenEndingAndDoesntCancel) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 900));
+ EXPECT_CALL(mMockClock, alarmIn(_, 800));
+ EXPECT_CALL(mMockClock, alarmIn(_, 100));
+ EXPECT_CALL(mMockClock, alarmCancel());
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+
+ mDispatch.schedule(cb0, 100, 1000);
+ mDispatch.schedule(cb1, 200, 1000);
+ advanceToNextCallback();
+ EXPECT_EQ(mDispatch.cancel(cb0), CancelResult::Cancelled);
+}
+
+TEST_F(VSyncDispatchTest, setAlarmCallsAtCorrectTimeWithChangingVsync) {
+ EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(_))
+ .Times(3)
+ .WillOnce(Return(950))
+ .WillOnce(Return(1975))
+ .WillOnce(Return(2950));
+
+ CountingCallback cb(mDispatch);
+ mDispatch.schedule(cb, 100, 920);
+
+ mMockClock.advanceBy(850);
+ EXPECT_THAT(cb.mCalls.size(), Eq(1));
+
+ mDispatch.schedule(cb, 100, 1900);
+ mMockClock.advanceBy(900);
+ EXPECT_THAT(cb.mCalls.size(), Eq(1));
+ mMockClock.advanceBy(125);
+ EXPECT_THAT(cb.mCalls.size(), Eq(2));
+
+ mDispatch.schedule(cb, 100, 2900);
+ mMockClock.advanceBy(975);
+ EXPECT_THAT(cb.mCalls.size(), Eq(3));
+}
+
+TEST_F(VSyncDispatchTest, callbackReentrancy) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 900)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 1000)).InSequence(seq);
+
+ VSyncDispatch::CallbackToken tmp;
+ tmp = mDispatch.registerCallback([&](auto) { mDispatch.schedule(tmp, 100, 2000); }, "o.o");
+
+ mDispatch.schedule(tmp, 100, 1000);
+ advanceToNextCallback();
+}
+
+TEST_F(VSyncDispatchTest, callbackReentrantWithPastWakeup) {
+ VSyncDispatch::CallbackToken tmp;
+ tmp = mDispatch.registerCallback(
+ [&](auto) {
+ EXPECT_EQ(mDispatch.schedule(tmp, 400, 1000), ScheduleResult::CannotSchedule);
+ },
+ "oo");
+
+ mDispatch.schedule(tmp, 999, 1000);
+ advanceToNextCallback();
+}
+
+TEST_F(VSyncDispatchTest, modificationsAroundVsyncTime) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 1000)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 200)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 1000)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 150)).InSequence(seq);
+
+ CountingCallback cb(mDispatch);
+ mDispatch.schedule(cb, 0, 1000);
+
+ mMockClock.advanceBy(750);
+ mDispatch.schedule(cb, 50, 1000);
+
+ advanceToNextCallback();
+ mDispatch.schedule(cb, 50, 2000);
+
+ mMockClock.advanceBy(800);
+ mDispatch.schedule(cb, 100, 2000);
+}
+
+TEST_F(VSyncDispatchTest, lateModifications) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 500)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 400)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 350)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 950)).InSequence(seq);
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+
+ mDispatch.schedule(cb0, 500, 1000);
+ mDispatch.schedule(cb1, 100, 1000);
+
+ advanceToNextCallback();
+ mDispatch.schedule(cb0, 200, 2000);
+ mDispatch.schedule(cb1, 150, 1000);
+
+ advanceToNextCallback();
+ advanceToNextCallback();
+}
+
+TEST_F(VSyncDispatchTest, doesntCancelPriorValidTimerForFutureMod) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 500)).InSequence(seq);
+
+ CountingCallback cb0(mDispatch);
+ CountingCallback cb1(mDispatch);
+ mDispatch.schedule(cb0, 500, 1000);
+ mDispatch.schedule(cb1, 500, 20000);
+}
+
+TEST_F(VSyncDispatchTest, setsTimerAfterCancellation) {
+ Sequence seq;
+ EXPECT_CALL(mMockClock, alarmIn(_, 500)).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmCancel()).InSequence(seq);
+ EXPECT_CALL(mMockClock, alarmIn(_, 900)).InSequence(seq);
+
+ CountingCallback cb0(mDispatch);
+ mDispatch.schedule(cb0, 500, 1000);
+ mDispatch.cancel(cb0);
+ mDispatch.schedule(cb0, 100, 1000);
+}
+
+TEST_F(VSyncDispatchTest, makingUpIdsError) {
+ VSyncDispatch::CallbackToken token(100);
+ EXPECT_THAT(mDispatch.schedule(token, 100, 1000), Eq(ScheduleResult::Error));
+ EXPECT_THAT(mDispatch.cancel(token), Eq(CancelResult::Error));
+}
+
+TEST_F(VSyncDispatchTest, distinguishesScheduleAndReschedule) {
+ CountingCallback cb0(mDispatch);
+ EXPECT_EQ(mDispatch.schedule(cb0, 500, 1000), ScheduleResult::Scheduled);
+ EXPECT_EQ(mDispatch.schedule(cb0, 100, 1000), ScheduleResult::ReScheduled);
+}
+
+TEST_F(VSyncDispatchTest, helperMove) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 500)).Times(1);
+ EXPECT_CALL(mMockClock, alarmCancel()).Times(1);
+
+ VSyncCallbackRegistration cb(
+ mDispatch, [](auto) {}, "");
+ VSyncCallbackRegistration cb1(std::move(cb));
+ cb.schedule(100, 1000);
+ cb.cancel();
+
+ cb1.schedule(500, 1000);
+ cb1.cancel();
+}
+
+TEST_F(VSyncDispatchTest, helperMoveAssign) {
+ EXPECT_CALL(mMockClock, alarmIn(_, 500)).Times(1);
+ EXPECT_CALL(mMockClock, alarmCancel()).Times(1);
+
+ VSyncCallbackRegistration cb(
+ mDispatch, [](auto) {}, "");
+ VSyncCallbackRegistration cb1(
+ mDispatch, [](auto) {}, "");
+ cb1 = std::move(cb);
+ cb.schedule(100, 1000);
+ cb.cancel();
+
+ cb1.schedule(500, 1000);
+ cb1.cancel();
+}
+
+class VSyncDispatchEntryTest : public testing::Test {
+protected:
+ nsecs_t const mPeriod = 1000;
+ NiceMock<MockVSyncTracker> mStubTracker{mPeriod};
+};
+
+TEST_F(VSyncDispatchEntryTest, stateAfterInitialization) {
+ std::string name("basicname");
+ impl::VSyncDispatchEntry entry(name, [](auto) {});
+ EXPECT_THAT(entry.name(), Eq(name));
+ EXPECT_FALSE(entry.lastExecutedVsyncTarget());
+ EXPECT_FALSE(entry.wakeupTime());
+}
+
+TEST_F(VSyncDispatchEntryTest, stateScheduling) {
+ impl::VSyncDispatchEntry entry("test", [](auto) {});
+
+ EXPECT_FALSE(entry.wakeupTime());
+ auto const wakeup = entry.schedule(100, 500, mStubTracker, 0);
+ auto const queried = entry.wakeupTime();
+ ASSERT_TRUE(queried);
+ EXPECT_THAT(*queried, Eq(wakeup));
+ EXPECT_THAT(*queried, Eq(900));
+
+ entry.disarm();
+ EXPECT_FALSE(entry.wakeupTime());
+}
+
+TEST_F(VSyncDispatchEntryTest, stateSchedulingReallyLongWakeupLatency) {
+ auto const duration = 500;
+ auto const now = 8750;
+
+ EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(now + duration))
+ .Times(1)
+ .WillOnce(Return(10000));
+ impl::VSyncDispatchEntry entry("test", [](auto) {});
+
+ EXPECT_FALSE(entry.wakeupTime());
+ auto const wakeup = entry.schedule(500, 994, mStubTracker, now);
+ auto const queried = entry.wakeupTime();
+ ASSERT_TRUE(queried);
+ EXPECT_THAT(*queried, Eq(wakeup));
+ EXPECT_THAT(*queried, Eq(9500));
+}
+
+TEST_F(VSyncDispatchEntryTest, runCallback) {
+ auto callCount = 0;
+ auto calledTime = 0;
+ impl::VSyncDispatchEntry entry("test", [&](auto time) {
+ callCount++;
+ calledTime = time;
+ });
+
+ auto const wakeup = entry.schedule(100, 500, mStubTracker, 0);
+ EXPECT_THAT(wakeup, Eq(900));
+
+ entry.callback(entry.executing());
+
+ EXPECT_THAT(callCount, Eq(1));
+ EXPECT_THAT(calledTime, Eq(mPeriod));
+ EXPECT_FALSE(entry.wakeupTime());
+ auto lastCalledTarget = entry.lastExecutedVsyncTarget();
+ ASSERT_TRUE(lastCalledTarget);
+ EXPECT_THAT(*lastCalledTarget, Eq(mPeriod));
+}
+
+TEST_F(VSyncDispatchEntryTest, updateCallback) {
+ EXPECT_CALL(mStubTracker, nextAnticipatedVSyncTimeFrom(_))
+ .Times(2)
+ .WillOnce(Return(1000))
+ .WillOnce(Return(1020));
+
+ impl::VSyncDispatchEntry entry("test", [](auto) {});
+
+ EXPECT_FALSE(entry.wakeupTime());
+ entry.update(mStubTracker, 0);
+ EXPECT_FALSE(entry.wakeupTime());
+
+ auto const wakeup = entry.schedule(100, 500, mStubTracker, 0);
+ EXPECT_THAT(wakeup, Eq(900));
+
+ entry.update(mStubTracker, 0);
+ auto const queried = entry.wakeupTime();
+ ASSERT_TRUE(queried);
+ EXPECT_THAT(*queried, Eq(920));
+}
+
+TEST_F(VSyncDispatchEntryTest, skipsUpdateIfJustScheduled) {
+ impl::VSyncDispatchEntry entry("test", [](auto) {});
+ auto const wakeup = entry.schedule(100, 500, mStubTracker, 0);
+ entry.update(mStubTracker, 0);
+
+ auto const queried = entry.wakeupTime();
+ ASSERT_TRUE(queried);
+ EXPECT_THAT(*queried, Eq(wakeup));
+}
+
+} // namespace android::scheduler