AIDL bufferpool implementation (just copy from HIDL impl)
Bug: 254050250
Change-Id: I48841189a8743dcd6533a9e61581e48f0f1717f1
diff --git a/media/bufferpool/aidl/default/tests/BufferpoolUnitTest.cpp b/media/bufferpool/aidl/default/tests/BufferpoolUnitTest.cpp
new file mode 100644
index 0000000..b448405
--- /dev/null
+++ b/media/bufferpool/aidl/default/tests/BufferpoolUnitTest.cpp
@@ -0,0 +1,541 @@
+/*
+ * Copyright (C) 2021 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 LOG_NDEBUG 0
+#define LOG_TAG "BufferpoolUnitTest"
+#include <utils/Log.h>
+
+#include <binder/ProcessState.h>
+#include <bufferpool/ClientManager.h>
+#include <gtest/gtest.h>
+#include <hidl/LegacySupport.h>
+#include <sys/types.h>
+#include <sys/wait.h>
+#include <unordered_set>
+#include <vector>
+#include "allocator.h"
+
+using android::hardware::configureRpcThreadpool;
+using android::hardware::media::bufferpool::BufferPoolData;
+using android::hardware::media::bufferpool::V2_0::IClientManager;
+using android::hardware::media::bufferpool::V2_0::ResultStatus;
+using android::hardware::media::bufferpool::V2_0::implementation::BufferId;
+using android::hardware::media::bufferpool::V2_0::implementation::ClientManager;
+using android::hardware::media::bufferpool::V2_0::implementation::ConnectionId;
+using android::hardware::media::bufferpool::V2_0::implementation::TransactionId;
+
+using namespace android;
+
+// communication message types between processes.
+enum PipeCommand : int32_t {
+ INIT,
+ TRANSFER,
+ STOP,
+
+ INIT_OK,
+ INIT_ERROR,
+ TRANSFER_OK,
+ TRANSFER_ERROR,
+ STOP_OK,
+ STOP_ERROR,
+};
+
+// communication message between processes.
+union PipeMessage {
+ struct {
+ int32_t command;
+ int32_t memsetValue;
+ BufferId bufferId;
+ ConnectionId connectionId;
+ TransactionId transactionId;
+ int64_t timestampUs;
+ } data;
+ char array[0];
+};
+
+static int32_t kNumIterationCount = 10;
+
+class BufferpoolTest {
+ public:
+ BufferpoolTest() : mConnectionValid(false), mManager(nullptr), mAllocator(nullptr) {
+ mConnectionId = -1;
+ mReceiverId = -1;
+ }
+
+ ~BufferpoolTest() {
+ if (mConnectionValid) {
+ mManager->close(mConnectionId);
+ }
+ }
+
+ protected:
+ bool mConnectionValid;
+ ConnectionId mConnectionId;
+ ConnectionId mReceiverId;
+
+ android::sp<ClientManager> mManager;
+ std::shared_ptr<BufferPoolAllocator> mAllocator;
+
+ void setupBufferpoolManager();
+};
+
+void BufferpoolTest::setupBufferpoolManager() {
+ // retrieving per process bufferpool object sp<ClientManager>
+ mManager = ClientManager::getInstance();
+ ASSERT_NE(mManager, nullptr) << "unable to get ClientManager\n";
+
+ mAllocator = std::make_shared<TestBufferPoolAllocator>();
+ ASSERT_NE(mAllocator, nullptr) << "unable to create TestBufferPoolAllocator\n";
+
+ // set-up local bufferpool connection for sender
+ ResultStatus status = mManager->create(mAllocator, &mConnectionId);
+ ASSERT_EQ(status, ResultStatus::OK)
+ << "unable to set-up local bufferpool connection for sender\n";
+ mConnectionValid = true;
+}
+
+class BufferpoolUnitTest : public BufferpoolTest, public ::testing::Test {
+ public:
+ virtual void SetUp() override { setupBufferpoolManager(); }
+
+ virtual void TearDown() override {}
+};
+
+class BufferpoolFunctionalityTest : public BufferpoolTest, public ::testing::Test {
+ public:
+ virtual void SetUp() override {
+ mReceiverPid = -1;
+
+ ASSERT_TRUE(pipe(mCommandPipeFds) == 0) << "pipe connection failed for commandPipe\n";
+ ASSERT_TRUE(pipe(mResultPipeFds) == 0) << "pipe connection failed for resultPipe\n";
+
+ mReceiverPid = fork();
+ ASSERT_TRUE(mReceiverPid >= 0) << "fork failed\n";
+
+ if (mReceiverPid == 0) {
+ doReceiver();
+ // In order to ignore gtest behaviour, wait for being killed from tearDown
+ pause();
+ }
+ setupBufferpoolManager();
+ }
+
+ virtual void TearDown() override {
+ if (mReceiverPid > 0) {
+ kill(mReceiverPid, SIGKILL);
+ int wstatus;
+ wait(&wstatus);
+ }
+ }
+
+ protected:
+ pid_t mReceiverPid;
+ int mCommandPipeFds[2];
+ int mResultPipeFds[2];
+
+ bool sendMessage(int* pipes, const PipeMessage& message) {
+ int ret = write(pipes[1], message.array, sizeof(PipeMessage));
+ return ret == sizeof(PipeMessage);
+ }
+
+ bool receiveMessage(int* pipes, PipeMessage* message) {
+ int ret = read(pipes[0], message->array, sizeof(PipeMessage));
+ return ret == sizeof(PipeMessage);
+ }
+
+ void doReceiver();
+};
+
+void BufferpoolFunctionalityTest::doReceiver() {
+ // Configures the threadpool used for handling incoming RPC calls in this process.
+ configureRpcThreadpool(1 /*threads*/, false /*willJoin*/);
+ bool receiverRunning = true;
+ while (receiverRunning) {
+ PipeMessage message;
+ receiveMessage(mCommandPipeFds, &message);
+ ResultStatus err = ResultStatus::OK;
+ switch (message.data.command) {
+ case PipeCommand::INIT: {
+ // receiver manager creation
+ mManager = ClientManager::getInstance();
+ if (!mManager) {
+ message.data.command = PipeCommand::INIT_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+
+ android::status_t status = mManager->registerAsService();
+ if (status != android::OK) {
+ message.data.command = PipeCommand::INIT_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ message.data.command = PipeCommand::INIT_OK;
+ sendMessage(mResultPipeFds, message);
+ break;
+ }
+ case PipeCommand::TRANSFER: {
+ native_handle_t* receiveHandle = nullptr;
+ std::shared_ptr<BufferPoolData> receiveBuffer;
+ err = mManager->receive(message.data.connectionId, message.data.transactionId,
+ message.data.bufferId, message.data.timestampUs,
+ &receiveHandle, &receiveBuffer);
+ if (err != ResultStatus::OK) {
+ message.data.command = PipeCommand::TRANSFER_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ if (!TestBufferPoolAllocator::Verify(receiveHandle, message.data.memsetValue)) {
+ message.data.command = PipeCommand::TRANSFER_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ if (receiveHandle) {
+ native_handle_close(receiveHandle);
+ native_handle_delete(receiveHandle);
+ }
+ receiveHandle = nullptr;
+ receiveBuffer.reset();
+ message.data.command = PipeCommand::TRANSFER_OK;
+ sendMessage(mResultPipeFds, message);
+ break;
+ }
+ case PipeCommand::STOP: {
+ err = mManager->close(message.data.connectionId);
+ if (err != ResultStatus::OK) {
+ message.data.command = PipeCommand::STOP_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ message.data.command = PipeCommand::STOP_OK;
+ sendMessage(mResultPipeFds, message);
+ receiverRunning = false;
+ break;
+ }
+ default:
+ ALOGE("unknown command. try again");
+ break;
+ }
+ }
+}
+
+// Buffer allocation test.
+// Check whether each buffer allocation is done successfully with unique buffer id.
+TEST_F(BufferpoolUnitTest, AllocateBuffer) {
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+
+ std::vector<std::shared_ptr<BufferPoolData>> buffers{};
+ std::vector<native_handle_t*> allocHandle{};
+ ResultStatus status;
+ for (int i = 0; i < kNumIterationCount; ++i) {
+ native_handle_t* handle = nullptr;
+ std::shared_ptr<BufferPoolData> buffer{};
+ status = mManager->allocate(mConnectionId, vecParams, &handle, &buffer);
+ ASSERT_EQ(status, ResultStatus::OK) << "allocate failed for " << i << "iteration";
+
+ buffers.push_back(std::move(buffer));
+ if (handle) {
+ allocHandle.push_back(std::move(handle));
+ }
+ }
+
+ for (int i = 0; i < kNumIterationCount; ++i) {
+ for (int j = i + 1; j < kNumIterationCount; ++j) {
+ ASSERT_TRUE(buffers[i]->mId != buffers[j]->mId) << "allocated buffers are not unique";
+ }
+ }
+ // delete the buffer handles
+ for (auto handle : allocHandle) {
+ native_handle_close(handle);
+ native_handle_delete(handle);
+ }
+ // clear the vectors
+ buffers.clear();
+ allocHandle.clear();
+}
+
+// Buffer recycle test.
+// Check whether de-allocated buffers are recycled.
+TEST_F(BufferpoolUnitTest, RecycleBuffer) {
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+
+ ResultStatus status;
+ std::vector<BufferId> bid{};
+ std::vector<native_handle_t*> allocHandle{};
+ for (int i = 0; i < kNumIterationCount; ++i) {
+ native_handle_t* handle = nullptr;
+ std::shared_ptr<BufferPoolData> buffer;
+ status = mManager->allocate(mConnectionId, vecParams, &handle, &buffer);
+ ASSERT_EQ(status, ResultStatus::OK) << "allocate failed for " << i << "iteration";
+
+ bid.push_back(buffer->mId);
+ if (handle) {
+ allocHandle.push_back(std::move(handle));
+ }
+ buffer.reset();
+ }
+
+ std::unordered_set<BufferId> set(bid.begin(), bid.end());
+ ASSERT_EQ(set.size(), 1) << "buffers are not recycled properly";
+
+ // delete the buffer handles
+ for (auto handle : allocHandle) {
+ native_handle_close(handle);
+ native_handle_delete(handle);
+ }
+ allocHandle.clear();
+}
+
+// Validate cache evict and invalidate APIs.
+TEST_F(BufferpoolUnitTest, FlushTest) {
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+
+ ResultStatus status = mManager->registerSender(mManager, mConnectionId, &mReceiverId);
+ ASSERT_TRUE(status == ResultStatus::ALREADY_EXISTS && mReceiverId == mConnectionId);
+
+ // testing empty flush
+ status = mManager->flush(mConnectionId);
+ ASSERT_EQ(status, ResultStatus::OK) << "failed to flush connection : " << mConnectionId;
+
+ std::vector<std::shared_ptr<BufferPoolData>> senderBuffer{};
+ std::vector<native_handle_t*> allocHandle{};
+ std::vector<TransactionId> tid{};
+ std::vector<int64_t> timestampUs{};
+
+ std::map<TransactionId, BufferId> bufferMap{};
+
+ for (int i = 0; i < kNumIterationCount; i++) {
+ int64_t postUs;
+ TransactionId transactionId;
+ native_handle_t* handle = nullptr;
+ std::shared_ptr<BufferPoolData> buffer{};
+ status = mManager->allocate(mConnectionId, vecParams, &handle, &buffer);
+ ASSERT_EQ(status, ResultStatus::OK) << "allocate failed for " << i << " iteration";
+
+ ASSERT_TRUE(TestBufferPoolAllocator::Fill(handle, i));
+
+ status = mManager->postSend(mReceiverId, buffer, &transactionId, &postUs);
+ ASSERT_EQ(status, ResultStatus::OK) << "unable to post send transaction on bufferpool";
+
+ timestampUs.push_back(postUs);
+ tid.push_back(transactionId);
+ bufferMap.insert({transactionId, buffer->mId});
+
+ senderBuffer.push_back(std::move(buffer));
+ if (handle) {
+ allocHandle.push_back(std::move(handle));
+ }
+ buffer.reset();
+ }
+
+ status = mManager->flush(mConnectionId);
+ ASSERT_EQ(status, ResultStatus::OK) << "failed to flush connection : " << mConnectionId;
+
+ std::shared_ptr<BufferPoolData> receiverBuffer{};
+ native_handle_t* recvHandle = nullptr;
+ for (int i = 0; i < kNumIterationCount; i++) {
+ status = mManager->receive(mReceiverId, tid[i], senderBuffer[i]->mId, timestampUs[i],
+ &recvHandle, &receiverBuffer);
+ ASSERT_EQ(status, ResultStatus::OK) << "receive failed for buffer " << senderBuffer[i]->mId;
+
+ // find the buffer id from transaction id
+ auto findIt = bufferMap.find(tid[i]);
+ ASSERT_NE(findIt, bufferMap.end()) << "inconsistent buffer mapping";
+
+ // buffer id received must be same as the buffer id sent
+ ASSERT_EQ(findIt->second, receiverBuffer->mId) << "invalid buffer received";
+
+ ASSERT_TRUE(TestBufferPoolAllocator::Verify(recvHandle, i))
+ << "Message received not same as that sent";
+
+ bufferMap.erase(findIt);
+ if (recvHandle) {
+ native_handle_close(recvHandle);
+ native_handle_delete(recvHandle);
+ }
+ recvHandle = nullptr;
+ receiverBuffer.reset();
+ }
+
+ ASSERT_EQ(bufferMap.size(), 0) << "buffers received is less than the number of buffers sent";
+
+ for (auto handle : allocHandle) {
+ native_handle_close(handle);
+ native_handle_delete(handle);
+ }
+ allocHandle.clear();
+ senderBuffer.clear();
+ timestampUs.clear();
+}
+
+// Buffer transfer test between processes.
+TEST_F(BufferpoolFunctionalityTest, TransferBuffer) {
+ // initialize the receiver
+ PipeMessage message;
+ message.data.command = PipeCommand::INIT;
+ sendMessage(mCommandPipeFds, message);
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message)) << "receiveMessage failed\n";
+ ASSERT_EQ(message.data.command, PipeCommand::INIT_OK) << "receiver init failed";
+
+ android::sp<IClientManager> receiver = IClientManager::getService();
+ ASSERT_NE(receiver, nullptr) << "getService failed for receiver\n";
+
+ ConnectionId receiverId;
+ ResultStatus status = mManager->registerSender(receiver, mConnectionId, &receiverId);
+ ASSERT_EQ(status, ResultStatus::OK)
+ << "registerSender failed for connection id " << mConnectionId << "\n";
+
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+
+ for (int i = 0; i < kNumIterationCount; ++i) {
+ native_handle_t* handle = nullptr;
+ std::shared_ptr<BufferPoolData> buffer;
+ status = mManager->allocate(mConnectionId, vecParams, &handle, &buffer);
+ ASSERT_EQ(status, ResultStatus::OK) << "allocate failed for " << i << "iteration";
+
+ ASSERT_TRUE(TestBufferPoolAllocator::Fill(handle, i))
+ << "Fill fail for buffer handle " << handle << "\n";
+
+ // send the buffer to the receiver
+ int64_t postUs;
+ TransactionId transactionId;
+ status = mManager->postSend(receiverId, buffer, &transactionId, &postUs);
+ ASSERT_EQ(status, ResultStatus::OK)
+ << "postSend failed for receiver " << receiverId << "\n";
+
+ // PipeMessage message;
+ message.data.command = PipeCommand::TRANSFER;
+ message.data.memsetValue = i;
+ message.data.bufferId = buffer->mId;
+ message.data.connectionId = receiverId;
+ message.data.transactionId = transactionId;
+ message.data.timestampUs = postUs;
+ sendMessage(mCommandPipeFds, message);
+ // delete buffer handle
+ if (handle) {
+ native_handle_close(handle);
+ native_handle_delete(handle);
+ }
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message)) << "receiveMessage failed\n";
+ ASSERT_EQ(message.data.command, PipeCommand::TRANSFER_OK)
+ << "received error during buffer transfer\n";
+ }
+ message.data.command = PipeCommand::STOP;
+ sendMessage(mCommandPipeFds, message);
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message)) << "receiveMessage failed\n";
+ ASSERT_EQ(message.data.command, PipeCommand::STOP_OK)
+ << "received error during buffer transfer\n";
+}
+
+/* Validate bufferpool for following corner cases:
+ 1. invalid connectionID
+ 2. invalid receiver
+ 3. when sender is not registered
+ 4. when connection is closed
+*/
+// TODO: Enable when the issue in b/212196495 is fixed
+TEST_F(BufferpoolFunctionalityTest, DISABLED_ValidityTest) {
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+
+ std::shared_ptr<BufferPoolData> senderBuffer;
+ native_handle_t* allocHandle = nullptr;
+
+ // call allocate() on a random connection id
+ ConnectionId randomId = rand();
+ ResultStatus status = mManager->allocate(randomId, vecParams, &allocHandle, &senderBuffer);
+ EXPECT_TRUE(status == ResultStatus::NOT_FOUND);
+
+ // initialize the receiver
+ PipeMessage message;
+ message.data.command = PipeCommand::INIT;
+ sendMessage(mCommandPipeFds, message);
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message)) << "receiveMessage failed\n";
+ ASSERT_EQ(message.data.command, PipeCommand::INIT_OK) << "receiver init failed";
+
+ allocHandle = nullptr;
+ senderBuffer.reset();
+ status = mManager->allocate(mConnectionId, vecParams, &allocHandle, &senderBuffer);
+
+ ASSERT_TRUE(TestBufferPoolAllocator::Fill(allocHandle, 0x77));
+
+ // send buffers w/o registering sender
+ int64_t postUs;
+ TransactionId transactionId;
+
+ // random receiver
+ status = mManager->postSend(randomId, senderBuffer, &transactionId, &postUs);
+ ASSERT_NE(status, ResultStatus::OK) << "bufferpool shouldn't allow send on random receiver";
+
+ // establish connection
+ android::sp<IClientManager> receiver = IClientManager::getService();
+ ASSERT_NE(receiver, nullptr) << "getService failed for receiver\n";
+
+ ConnectionId receiverId;
+ status = mManager->registerSender(receiver, mConnectionId, &receiverId);
+ ASSERT_EQ(status, ResultStatus::OK)
+ << "registerSender failed for connection id " << mConnectionId << "\n";
+
+ allocHandle = nullptr;
+ senderBuffer.reset();
+ status = mManager->allocate(mConnectionId, vecParams, &allocHandle, &senderBuffer);
+ ASSERT_EQ(status, ResultStatus::OK) << "allocate failed for connection " << mConnectionId;
+
+ ASSERT_TRUE(TestBufferPoolAllocator::Fill(allocHandle, 0x88));
+
+ // send the buffer to the receiver
+ status = mManager->postSend(receiverId, senderBuffer, &transactionId, &postUs);
+ ASSERT_EQ(status, ResultStatus::OK) << "postSend failed for receiver " << receiverId << "\n";
+
+ // PipeMessage message;
+ message.data.command = PipeCommand::TRANSFER;
+ message.data.memsetValue = 0x88;
+ message.data.bufferId = senderBuffer->mId;
+ message.data.connectionId = receiverId;
+ message.data.transactionId = transactionId;
+ message.data.timestampUs = postUs;
+ sendMessage(mCommandPipeFds, message);
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message)) << "receiveMessage failed\n";
+ ASSERT_EQ(message.data.command, PipeCommand::TRANSFER_OK)
+ << "received error during buffer transfer\n";
+
+ if (allocHandle) {
+ native_handle_close(allocHandle);
+ native_handle_delete(allocHandle);
+ }
+
+ message.data.command = PipeCommand::STOP;
+ sendMessage(mCommandPipeFds, message);
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message)) << "receiveMessage failed\n";
+ ASSERT_EQ(message.data.command, PipeCommand::STOP_OK)
+ << "received error during buffer transfer\n";
+
+ // try to send msg to closed connection
+ status = mManager->postSend(receiverId, senderBuffer, &transactionId, &postUs);
+ ASSERT_NE(status, ResultStatus::OK) << "bufferpool shouldn't allow send on closed connection";
+}
+
+int main(int argc, char** argv) {
+ android::hardware::details::setTrebleTestingOverride(true);
+ ::testing::InitGoogleTest(&argc, argv);
+ int status = RUN_ALL_TESTS();
+ ALOGV("Test result = %d\n", status);
+ return status;
+}
diff --git a/media/bufferpool/aidl/default/tests/allocator.cpp b/media/bufferpool/aidl/default/tests/allocator.cpp
new file mode 100644
index 0000000..25b08ef
--- /dev/null
+++ b/media/bufferpool/aidl/default/tests/allocator.cpp
@@ -0,0 +1,251 @@
+/*
+ * Copyright (C) 2018 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.
+ */
+
+#include <cutils/ashmem.h>
+#include <sys/mman.h>
+#include "allocator.h"
+
+union Params {
+ struct {
+ uint32_t capacity;
+ } data;
+ uint8_t array[0];
+ Params() : data{0} {}
+ Params(uint32_t size)
+ : data{size} {}
+};
+
+
+namespace {
+
+struct HandleAshmem : public native_handle_t {
+ HandleAshmem(int ashmemFd, size_t size)
+ : native_handle_t(cHeader),
+ mFds{ ashmemFd },
+ mInts{ int (size & 0xFFFFFFFF), int((uint64_t(size) >> 32) & 0xFFFFFFFF), kMagic } {}
+
+ int ashmemFd() const { return mFds.mAshmem; }
+ size_t size() const {
+ return size_t(unsigned(mInts.mSizeLo))
+ | size_t(uint64_t(unsigned(mInts.mSizeHi)) << 32);
+ }
+
+ static bool isValid(const native_handle_t * const o);
+
+protected:
+ struct {
+ int mAshmem;
+ } mFds;
+ struct {
+ int mSizeLo;
+ int mSizeHi;
+ int mMagic;
+ } mInts;
+
+private:
+ enum {
+ kMagic = 'ahm\x00',
+ numFds = sizeof(mFds) / sizeof(int),
+ numInts = sizeof(mInts) / sizeof(int),
+ version = sizeof(native_handle_t)
+ };
+ const static native_handle_t cHeader;
+};
+
+const native_handle_t HandleAshmem::cHeader = {
+ HandleAshmem::version,
+ HandleAshmem::numFds,
+ HandleAshmem::numInts,
+ {}
+};
+
+bool HandleAshmem::isValid(const native_handle_t * const o) {
+ if (!o || memcmp(o, &cHeader, sizeof(cHeader))) {
+ return false;
+ }
+ const HandleAshmem *other = static_cast<const HandleAshmem*>(o);
+ return other->mInts.mMagic == kMagic;
+}
+
+class AllocationAshmem {
+private:
+ AllocationAshmem(int ashmemFd, size_t capacity, bool res)
+ : mHandle(ashmemFd, capacity),
+ mInit(res) {}
+
+public:
+ static AllocationAshmem *Alloc(size_t size) {
+ constexpr static const char *kAllocationTag = "bufferpool_test";
+ int ashmemFd = ashmem_create_region(kAllocationTag, size);
+ return new AllocationAshmem(ashmemFd, size, ashmemFd >= 0);
+ }
+
+ ~AllocationAshmem() {
+ if (mInit) {
+ native_handle_close(&mHandle);
+ }
+ }
+
+ const HandleAshmem *handle() {
+ return &mHandle;
+ }
+
+private:
+ HandleAshmem mHandle;
+ bool mInit;
+ // TODO: mapping and map fd
+};
+
+struct AllocationDtor {
+ AllocationDtor(const std::shared_ptr<AllocationAshmem> &alloc)
+ : mAlloc(alloc) {}
+
+ void operator()(BufferPoolAllocation *poolAlloc) { delete poolAlloc; }
+
+ const std::shared_ptr<AllocationAshmem> mAlloc;
+};
+
+}
+
+void IpcMutex::init() {
+ pthread_mutexattr_t mattr;
+ pthread_mutexattr_init(&mattr);
+ pthread_mutexattr_setpshared(&mattr, PTHREAD_PROCESS_SHARED);
+ pthread_mutex_init(&lock, &mattr);
+ pthread_mutexattr_destroy(&mattr);
+
+ pthread_condattr_t cattr;
+ pthread_condattr_init(&cattr);
+ pthread_condattr_setpshared(&cattr, PTHREAD_PROCESS_SHARED);
+ pthread_cond_init(&cond, &cattr);
+ pthread_condattr_destroy(&cattr);
+}
+
+IpcMutex *IpcMutex::Import(void *pMutex) {
+ return reinterpret_cast<IpcMutex *>(pMutex);
+}
+
+
+ResultStatus TestBufferPoolAllocator::allocate(
+ const std::vector<uint8_t> ¶ms,
+ std::shared_ptr<BufferPoolAllocation> *alloc,
+ size_t *allocSize) {
+ Params ashmemParams;
+ memcpy(&ashmemParams, params.data(), std::min(sizeof(Params), params.size()));
+
+ std::shared_ptr<AllocationAshmem> ashmemAlloc =
+ std::shared_ptr<AllocationAshmem>(
+ AllocationAshmem::Alloc(ashmemParams.data.capacity));
+ if (ashmemAlloc) {
+ BufferPoolAllocation *ptr = new BufferPoolAllocation(ashmemAlloc->handle());
+ if (ptr) {
+ *alloc = std::shared_ptr<BufferPoolAllocation>(ptr, AllocationDtor(ashmemAlloc));
+ if (*alloc) {
+ *allocSize = ashmemParams.data.capacity;
+ return ResultStatus::OK;
+ }
+ delete ptr;
+ return ResultStatus::NO_MEMORY;
+ }
+ }
+ return ResultStatus::CRITICAL_ERROR;
+}
+
+bool TestBufferPoolAllocator::compatible(const std::vector<uint8_t> &newParams,
+ const std::vector<uint8_t> &oldParams) {
+ size_t newSize = newParams.size();
+ size_t oldSize = oldParams.size();
+ if (newSize == oldSize) {
+ for (size_t i = 0; i < newSize; ++i) {
+ if (newParams[i] != oldParams[i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+ return false;
+}
+
+bool TestBufferPoolAllocator::Fill(const native_handle_t *handle, const unsigned char val) {
+ if (!HandleAshmem::isValid(handle)) {
+ return false;
+ }
+ const HandleAshmem *o = static_cast<const HandleAshmem*>(handle);
+ unsigned char *ptr = (unsigned char *)mmap(
+ NULL, o->size(), PROT_READ|PROT_WRITE, MAP_SHARED, o->ashmemFd(), 0);
+
+ if (ptr != MAP_FAILED) {
+ for (size_t i = 0; i < o->size(); ++i) {
+ ptr[i] = val;
+ }
+ munmap(ptr, o->size());
+ return true;
+ }
+ return false;
+}
+
+bool TestBufferPoolAllocator::Verify(const native_handle_t *handle, const unsigned char val) {
+ if (!HandleAshmem::isValid(handle)) {
+ return false;
+ }
+ const HandleAshmem *o = static_cast<const HandleAshmem*>(handle);
+ unsigned char *ptr = (unsigned char *)mmap(
+ NULL, o->size(), PROT_READ, MAP_SHARED, o->ashmemFd(), 0);
+
+ if (ptr != MAP_FAILED) {
+ bool res = true;
+ for (size_t i = 0; i < o->size(); ++i) {
+ if (ptr[i] != val) {
+ res = false;
+ break;
+ }
+ }
+ munmap(ptr, o->size());
+ return res;
+ }
+ return false;
+}
+
+bool TestBufferPoolAllocator::MapMemoryForMutex(const native_handle_t *handle, void **mem) {
+ if (!HandleAshmem::isValid(handle)) {
+ return false;
+ }
+ const HandleAshmem *o = static_cast<const HandleAshmem*>(handle);
+ *mem = mmap(
+ NULL, o->size(), PROT_READ|PROT_WRITE, MAP_SHARED, o->ashmemFd(), 0);
+ if (*mem == MAP_FAILED || *mem == nullptr) {
+ return false;
+ }
+ return true;
+}
+
+bool TestBufferPoolAllocator::UnmapMemoryForMutex(void *mem) {
+ munmap(mem, sizeof(IpcMutex));
+ return true;
+}
+
+void getTestAllocatorParams(std::vector<uint8_t> *params) {
+ constexpr static int kAllocationSize = 1024 * 10;
+ Params ashmemParams(kAllocationSize);
+
+ params->assign(ashmemParams.array, ashmemParams.array + sizeof(ashmemParams));
+}
+
+void getIpcMutexParams(std::vector<uint8_t> *params) {
+ Params ashmemParams(sizeof(IpcMutex));
+
+ params->assign(ashmemParams.array, ashmemParams.array + sizeof(ashmemParams));
+}
diff --git a/media/bufferpool/aidl/default/tests/allocator.h b/media/bufferpool/aidl/default/tests/allocator.h
new file mode 100644
index 0000000..862d1a5
--- /dev/null
+++ b/media/bufferpool/aidl/default/tests/allocator.h
@@ -0,0 +1,68 @@
+/*
+ * Copyright (C) 2018 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.
+ */
+
+#ifndef VNDK_HIDL_BUFFERPOOL_V2_0_ALLOCATOR_H
+#define VNDK_HIDL_BUFFERPOOL_V2_0_ALLOCATOR_H
+
+#include <pthread.h>
+#include <bufferpool/BufferPoolTypes.h>
+
+using android::hardware::media::bufferpool::V2_0::ResultStatus;
+using android::hardware::media::bufferpool::V2_0::implementation::
+ BufferPoolAllocation;
+using android::hardware::media::bufferpool::V2_0::implementation::
+ BufferPoolAllocator;
+
+struct IpcMutex {
+ pthread_mutex_t lock;
+ pthread_cond_t cond;
+ int counter = 0;
+ bool signalled = false;
+
+ void init();
+
+ static IpcMutex *Import(void *mem);
+};
+
+// buffer allocator for the tests
+class TestBufferPoolAllocator : public BufferPoolAllocator {
+ public:
+ TestBufferPoolAllocator() {}
+
+ ~TestBufferPoolAllocator() override {}
+
+ ResultStatus allocate(const std::vector<uint8_t> ¶ms,
+ std::shared_ptr<BufferPoolAllocation> *alloc,
+ size_t *allocSize) override;
+
+ bool compatible(const std::vector<uint8_t> &newParams,
+ const std::vector<uint8_t> &oldParams) override;
+
+ static bool Fill(const native_handle_t *handle, const unsigned char val);
+
+ static bool Verify(const native_handle_t *handle, const unsigned char val);
+
+ static bool MapMemoryForMutex(const native_handle_t *handle, void **mem);
+
+ static bool UnmapMemoryForMutex(void *mem);
+};
+
+// retrieve buffer allocator paramters
+void getTestAllocatorParams(std::vector<uint8_t> *params);
+
+void getIpcMutexParams(std::vector<uint8_t> *params);
+
+#endif // VNDK_HIDL_BUFFERPOOL_V2_0_ALLOCATOR_H
diff --git a/media/bufferpool/aidl/default/tests/cond.cpp b/media/bufferpool/aidl/default/tests/cond.cpp
new file mode 100644
index 0000000..21beea8
--- /dev/null
+++ b/media/bufferpool/aidl/default/tests/cond.cpp
@@ -0,0 +1,269 @@
+/*
+ * Copyright (C) 2018 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 LOG_TAG "buffferpool_unit_test"
+
+#include <gtest/gtest.h>
+
+#include <android-base/logging.h>
+#include <binder/ProcessState.h>
+#include <bufferpool/ClientManager.h>
+#include <errno.h>
+#include <hidl/HidlSupport.h>
+#include <hidl/HidlTransportSupport.h>
+#include <hidl/LegacySupport.h>
+#include <hidl/Status.h>
+#include <signal.h>
+#include <sys/types.h>
+#include <sys/wait.h>
+#include <unistd.h>
+#include <iostream>
+#include <memory>
+#include <vector>
+#include "allocator.h"
+
+using android::hardware::configureRpcThreadpool;
+using android::hardware::hidl_handle;
+using android::hardware::media::bufferpool::V2_0::IClientManager;
+using android::hardware::media::bufferpool::V2_0::ResultStatus;
+using android::hardware::media::bufferpool::V2_0::implementation::BufferId;
+using android::hardware::media::bufferpool::V2_0::implementation::ClientManager;
+using android::hardware::media::bufferpool::V2_0::implementation::ConnectionId;
+using android::hardware::media::bufferpool::V2_0::implementation::TransactionId;
+using android::hardware::media::bufferpool::BufferPoolData;
+
+namespace {
+
+// communication message types between processes.
+enum PipeCommand : int32_t {
+ INIT_OK = 0,
+ INIT_ERROR,
+ SEND,
+ RECEIVE_OK,
+ RECEIVE_ERROR,
+};
+
+// communication message between processes.
+union PipeMessage {
+ struct {
+ int32_t command;
+ BufferId bufferId;
+ ConnectionId connectionId;
+ TransactionId transactionId;
+ int64_t timestampUs;
+ } data;
+ char array[0];
+};
+
+constexpr int kSignalInt = 200;
+
+// media.bufferpool test setup
+class BufferpoolMultiTest : public ::testing::Test {
+ public:
+ virtual void SetUp() override {
+ ResultStatus status;
+ mReceiverPid = -1;
+ mConnectionValid = false;
+
+ ASSERT_TRUE(pipe(mCommandPipeFds) == 0);
+ ASSERT_TRUE(pipe(mResultPipeFds) == 0);
+
+ mReceiverPid = fork();
+ ASSERT_TRUE(mReceiverPid >= 0);
+
+ if (mReceiverPid == 0) {
+ doReceiver();
+ // In order to ignore gtest behaviour, wait for being killed from
+ // tearDown
+ pause();
+ }
+
+ mManager = ClientManager::getInstance();
+ ASSERT_NE(mManager, nullptr);
+
+ mAllocator = std::make_shared<TestBufferPoolAllocator>();
+ ASSERT_TRUE((bool)mAllocator);
+
+ status = mManager->create(mAllocator, &mConnectionId);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ mConnectionValid = true;
+ }
+
+ virtual void TearDown() override {
+ if (mReceiverPid > 0) {
+ kill(mReceiverPid, SIGKILL);
+ int wstatus;
+ wait(&wstatus);
+ }
+
+ if (mConnectionValid) {
+ mManager->close(mConnectionId);
+ }
+ }
+
+ protected:
+ static void description(const std::string& description) {
+ RecordProperty("description", description);
+ }
+
+ android::sp<ClientManager> mManager;
+ std::shared_ptr<BufferPoolAllocator> mAllocator;
+ bool mConnectionValid;
+ ConnectionId mConnectionId;
+ pid_t mReceiverPid;
+ int mCommandPipeFds[2];
+ int mResultPipeFds[2];
+
+ bool sendMessage(int *pipes, const PipeMessage &message) {
+ int ret = write(pipes[1], message.array, sizeof(PipeMessage));
+ return ret == sizeof(PipeMessage);
+ }
+
+ bool receiveMessage(int *pipes, PipeMessage *message) {
+ int ret = read(pipes[0], message->array, sizeof(PipeMessage));
+ return ret == sizeof(PipeMessage);
+ }
+
+ void doReceiver() {
+ configureRpcThreadpool(1, false);
+ PipeMessage message;
+ mManager = ClientManager::getInstance();
+ if (!mManager) {
+ message.data.command = PipeCommand::INIT_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ android::status_t status = mManager->registerAsService();
+ if (status != android::OK) {
+ message.data.command = PipeCommand::INIT_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ message.data.command = PipeCommand::INIT_OK;
+ sendMessage(mResultPipeFds, message);
+
+ int val = 0;
+ receiveMessage(mCommandPipeFds, &message);
+ {
+ native_handle_t *rhandle = nullptr;
+ std::shared_ptr<BufferPoolData> rbuffer;
+ void *mem = nullptr;
+ IpcMutex *mutex = nullptr;
+ ResultStatus status = mManager->receive(
+ message.data.connectionId, message.data.transactionId,
+ message.data.bufferId, message.data.timestampUs, &rhandle, &rbuffer);
+ mManager->close(message.data.connectionId);
+ if (status != ResultStatus::OK) {
+ message.data.command = PipeCommand::RECEIVE_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ if (!TestBufferPoolAllocator::MapMemoryForMutex(rhandle, &mem)) {
+ message.data.command = PipeCommand::RECEIVE_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ mutex = IpcMutex::Import(mem);
+ pthread_mutex_lock(&(mutex->lock));
+ while (mutex->signalled != true) {
+ pthread_cond_wait(&(mutex->cond), &(mutex->lock));
+ }
+ val = mutex->counter;
+ pthread_mutex_unlock(&(mutex->lock));
+
+ (void)TestBufferPoolAllocator::UnmapMemoryForMutex(mem);
+ if (rhandle) {
+ native_handle_close(rhandle);
+ native_handle_delete(rhandle);
+ }
+ }
+ if (val == kSignalInt) {
+ message.data.command = PipeCommand::RECEIVE_OK;
+ } else {
+ message.data.command = PipeCommand::RECEIVE_ERROR;
+ }
+ sendMessage(mResultPipeFds, message);
+ }
+};
+
+// Buffer transfer test between processes.
+TEST_F(BufferpoolMultiTest, TransferBuffer) {
+ ResultStatus status;
+ PipeMessage message;
+
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message));
+
+ android::sp<IClientManager> receiver = IClientManager::getService();
+ ConnectionId receiverId;
+ ASSERT_TRUE((bool)receiver);
+
+ status = mManager->registerSender(receiver, mConnectionId, &receiverId);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ {
+ native_handle_t *shandle = nullptr;
+ std::shared_ptr<BufferPoolData> sbuffer;
+ TransactionId transactionId;
+ int64_t postUs;
+ std::vector<uint8_t> vecParams;
+ void *mem = nullptr;
+ IpcMutex *mutex = nullptr;
+
+ getIpcMutexParams(&vecParams);
+ status = mManager->allocate(mConnectionId, vecParams, &shandle, &sbuffer);
+ ASSERT_TRUE(status == ResultStatus::OK);
+
+ ASSERT_TRUE(TestBufferPoolAllocator::MapMemoryForMutex(shandle, &mem));
+
+ mutex = new(mem) IpcMutex();
+ mutex->init();
+
+ status = mManager->postSend(receiverId, sbuffer, &transactionId, &postUs);
+ ASSERT_TRUE(status == ResultStatus::OK);
+
+ message.data.command = PipeCommand::SEND;
+ message.data.bufferId = sbuffer->mId;
+ message.data.connectionId = receiverId;
+ message.data.transactionId = transactionId;
+ message.data.timestampUs = postUs;
+ sendMessage(mCommandPipeFds, message);
+ for (int i=0; i < 200000000; ++i) {
+ // no-op in order to ensure
+ // pthread_cond_wait is called before pthread_cond_signal
+ }
+ pthread_mutex_lock(&(mutex->lock));
+ mutex->counter = kSignalInt;
+ mutex->signalled = true;
+ pthread_cond_signal(&(mutex->cond));
+ pthread_mutex_unlock(&(mutex->lock));
+ (void)TestBufferPoolAllocator::UnmapMemoryForMutex(mem);
+ if (shandle) {
+ native_handle_close(shandle);
+ native_handle_delete(shandle);
+ }
+ }
+ EXPECT_TRUE(receiveMessage(mResultPipeFds, &message));
+ EXPECT_TRUE(message.data.command == PipeCommand::RECEIVE_OK);
+}
+
+} // anonymous namespace
+
+int main(int argc, char** argv) {
+ android::hardware::details::setTrebleTestingOverride(true);
+ ::testing::InitGoogleTest(&argc, argv);
+ int status = RUN_ALL_TESTS();
+ LOG(INFO) << "Test result = " << status;
+ return status;
+}
diff --git a/media/bufferpool/aidl/default/tests/multi.cpp b/media/bufferpool/aidl/default/tests/multi.cpp
new file mode 100644
index 0000000..43b0a8c
--- /dev/null
+++ b/media/bufferpool/aidl/default/tests/multi.cpp
@@ -0,0 +1,235 @@
+/*
+ * Copyright (C) 2018 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 LOG_TAG "buffferpool_unit_test"
+
+#include <gtest/gtest.h>
+
+#include <android-base/logging.h>
+#include <binder/ProcessState.h>
+#include <bufferpool/ClientManager.h>
+#include <hidl/HidlSupport.h>
+#include <hidl/HidlTransportSupport.h>
+#include <hidl/LegacySupport.h>
+#include <hidl/Status.h>
+#include <signal.h>
+#include <sys/types.h>
+#include <sys/wait.h>
+#include <unistd.h>
+#include <iostream>
+#include <memory>
+#include <vector>
+#include "allocator.h"
+
+using android::hardware::configureRpcThreadpool;
+using android::hardware::hidl_handle;
+using android::hardware::media::bufferpool::V2_0::IClientManager;
+using android::hardware::media::bufferpool::V2_0::ResultStatus;
+using android::hardware::media::bufferpool::V2_0::implementation::BufferId;
+using android::hardware::media::bufferpool::V2_0::implementation::ClientManager;
+using android::hardware::media::bufferpool::V2_0::implementation::ConnectionId;
+using android::hardware::media::bufferpool::V2_0::implementation::TransactionId;
+using android::hardware::media::bufferpool::BufferPoolData;
+
+namespace {
+
+// communication message types between processes.
+enum PipeCommand : int32_t {
+ INIT_OK = 0,
+ INIT_ERROR,
+ SEND,
+ RECEIVE_OK,
+ RECEIVE_ERROR,
+};
+
+// communication message between processes.
+union PipeMessage {
+ struct {
+ int32_t command;
+ BufferId bufferId;
+ ConnectionId connectionId;
+ TransactionId transactionId;
+ int64_t timestampUs;
+ } data;
+ char array[0];
+};
+
+// media.bufferpool test setup
+class BufferpoolMultiTest : public ::testing::Test {
+ public:
+ virtual void SetUp() override {
+ ResultStatus status;
+ mReceiverPid = -1;
+ mConnectionValid = false;
+
+ ASSERT_TRUE(pipe(mCommandPipeFds) == 0);
+ ASSERT_TRUE(pipe(mResultPipeFds) == 0);
+
+ mReceiverPid = fork();
+ ASSERT_TRUE(mReceiverPid >= 0);
+
+ if (mReceiverPid == 0) {
+ doReceiver();
+ // In order to ignore gtest behaviour, wait for being killed from
+ // tearDown
+ pause();
+ }
+
+ mManager = ClientManager::getInstance();
+ ASSERT_NE(mManager, nullptr);
+
+ mAllocator = std::make_shared<TestBufferPoolAllocator>();
+ ASSERT_TRUE((bool)mAllocator);
+
+ status = mManager->create(mAllocator, &mConnectionId);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ mConnectionValid = true;
+ }
+
+ virtual void TearDown() override {
+ if (mReceiverPid > 0) {
+ kill(mReceiverPid, SIGKILL);
+ int wstatus;
+ wait(&wstatus);
+ }
+
+ if (mConnectionValid) {
+ mManager->close(mConnectionId);
+ }
+ }
+
+ protected:
+ static void description(const std::string& description) {
+ RecordProperty("description", description);
+ }
+
+ android::sp<ClientManager> mManager;
+ std::shared_ptr<BufferPoolAllocator> mAllocator;
+ bool mConnectionValid;
+ ConnectionId mConnectionId;
+ pid_t mReceiverPid;
+ int mCommandPipeFds[2];
+ int mResultPipeFds[2];
+
+ bool sendMessage(int *pipes, const PipeMessage &message) {
+ int ret = write(pipes[1], message.array, sizeof(PipeMessage));
+ return ret == sizeof(PipeMessage);
+ }
+
+ bool receiveMessage(int *pipes, PipeMessage *message) {
+ int ret = read(pipes[0], message->array, sizeof(PipeMessage));
+ return ret == sizeof(PipeMessage);
+ }
+
+ void doReceiver() {
+ configureRpcThreadpool(1, false);
+ PipeMessage message;
+ mManager = ClientManager::getInstance();
+ if (!mManager) {
+ message.data.command = PipeCommand::INIT_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ android::status_t status = mManager->registerAsService();
+ if (status != android::OK) {
+ message.data.command = PipeCommand::INIT_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ message.data.command = PipeCommand::INIT_OK;
+ sendMessage(mResultPipeFds, message);
+
+ receiveMessage(mCommandPipeFds, &message);
+ {
+ native_handle_t *rhandle = nullptr;
+ std::shared_ptr<BufferPoolData> rbuffer;
+ ResultStatus status = mManager->receive(
+ message.data.connectionId, message.data.transactionId,
+ message.data.bufferId, message.data.timestampUs, &rhandle, &rbuffer);
+ mManager->close(message.data.connectionId);
+ if (status != ResultStatus::OK) {
+ message.data.command = PipeCommand::RECEIVE_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ if (!TestBufferPoolAllocator::Verify(rhandle, 0x77)) {
+ message.data.command = PipeCommand::RECEIVE_ERROR;
+ sendMessage(mResultPipeFds, message);
+ return;
+ }
+ if (rhandle) {
+ native_handle_close(rhandle);
+ native_handle_delete(rhandle);
+ }
+ }
+ message.data.command = PipeCommand::RECEIVE_OK;
+ sendMessage(mResultPipeFds, message);
+ }
+};
+
+// Buffer transfer test between processes.
+TEST_F(BufferpoolMultiTest, TransferBuffer) {
+ ResultStatus status;
+ PipeMessage message;
+
+ ASSERT_TRUE(receiveMessage(mResultPipeFds, &message));
+
+ android::sp<IClientManager> receiver = IClientManager::getService();
+ ConnectionId receiverId;
+ ASSERT_TRUE((bool)receiver);
+
+ status = mManager->registerSender(receiver, mConnectionId, &receiverId);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ {
+ native_handle_t *shandle = nullptr;
+ std::shared_ptr<BufferPoolData> sbuffer;
+ TransactionId transactionId;
+ int64_t postUs;
+ std::vector<uint8_t> vecParams;
+
+ getTestAllocatorParams(&vecParams);
+ status = mManager->allocate(mConnectionId, vecParams, &shandle, &sbuffer);
+ ASSERT_TRUE(status == ResultStatus::OK);
+
+ ASSERT_TRUE(TestBufferPoolAllocator::Fill(shandle, 0x77));
+ if (shandle) {
+ native_handle_close(shandle);
+ native_handle_delete(shandle);
+ }
+
+ status = mManager->postSend(receiverId, sbuffer, &transactionId, &postUs);
+ ASSERT_TRUE(status == ResultStatus::OK);
+
+ message.data.command = PipeCommand::SEND;
+ message.data.bufferId = sbuffer->mId;
+ message.data.connectionId = receiverId;
+ message.data.transactionId = transactionId;
+ message.data.timestampUs = postUs;
+ sendMessage(mCommandPipeFds, message);
+ }
+ EXPECT_TRUE(receiveMessage(mResultPipeFds, &message));
+ EXPECT_TRUE(message.data.command == PipeCommand::RECEIVE_OK);
+}
+
+} // anonymous namespace
+
+int main(int argc, char** argv) {
+ android::hardware::details::setTrebleTestingOverride(true);
+ ::testing::InitGoogleTest(&argc, argv);
+ int status = RUN_ALL_TESTS();
+ LOG(INFO) << "Test result = " << status;
+ return status;
+}
diff --git a/media/bufferpool/aidl/default/tests/single.cpp b/media/bufferpool/aidl/default/tests/single.cpp
new file mode 100644
index 0000000..1e9027b
--- /dev/null
+++ b/media/bufferpool/aidl/default/tests/single.cpp
@@ -0,0 +1,183 @@
+/*
+ * Copyright (C) 2018 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 LOG_TAG "buffferpool_unit_test"
+
+#include <gtest/gtest.h>
+
+#include <android-base/logging.h>
+#include <binder/ProcessState.h>
+#include <bufferpool/ClientManager.h>
+#include <hidl/HidlSupport.h>
+#include <hidl/HidlTransportSupport.h>
+#include <hidl/LegacySupport.h>
+#include <hidl/Status.h>
+#include <unistd.h>
+#include <iostream>
+#include <memory>
+#include <vector>
+#include "allocator.h"
+
+using android::hardware::hidl_handle;
+using android::hardware::media::bufferpool::V2_0::ResultStatus;
+using android::hardware::media::bufferpool::V2_0::implementation::BufferId;
+using android::hardware::media::bufferpool::V2_0::implementation::ClientManager;
+using android::hardware::media::bufferpool::V2_0::implementation::ConnectionId;
+using android::hardware::media::bufferpool::V2_0::implementation::TransactionId;
+using android::hardware::media::bufferpool::BufferPoolData;
+
+namespace {
+
+// Number of iteration for buffer allocation test.
+constexpr static int kNumAllocationTest = 3;
+
+// Number of iteration for buffer recycling test.
+constexpr static int kNumRecycleTest = 3;
+
+// media.bufferpool test setup
+class BufferpoolSingleTest : public ::testing::Test {
+ public:
+ virtual void SetUp() override {
+ ResultStatus status;
+ mConnectionValid = false;
+
+ mManager = ClientManager::getInstance();
+ ASSERT_NE(mManager, nullptr);
+
+ mAllocator = std::make_shared<TestBufferPoolAllocator>();
+ ASSERT_TRUE((bool)mAllocator);
+
+ status = mManager->create(mAllocator, &mConnectionId);
+ ASSERT_TRUE(status == ResultStatus::OK);
+
+ mConnectionValid = true;
+
+ status = mManager->registerSender(mManager, mConnectionId, &mReceiverId);
+ ASSERT_TRUE(status == ResultStatus::ALREADY_EXISTS &&
+ mReceiverId == mConnectionId);
+ }
+
+ virtual void TearDown() override {
+ if (mConnectionValid) {
+ mManager->close(mConnectionId);
+ }
+ }
+
+ protected:
+ static void description(const std::string& description) {
+ RecordProperty("description", description);
+ }
+
+ android::sp<ClientManager> mManager;
+ std::shared_ptr<BufferPoolAllocator> mAllocator;
+ bool mConnectionValid;
+ ConnectionId mConnectionId;
+ ConnectionId mReceiverId;
+
+};
+
+// Buffer allocation test.
+// Check whether each buffer allocation is done successfully with
+// unique buffer id.
+TEST_F(BufferpoolSingleTest, AllocateBuffer) {
+ ResultStatus status;
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+
+ std::shared_ptr<BufferPoolData> buffer[kNumAllocationTest];
+ native_handle_t *allocHandle = nullptr;
+ for (int i = 0; i < kNumAllocationTest; ++i) {
+ status = mManager->allocate(mConnectionId, vecParams, &allocHandle, &buffer[i]);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ if (allocHandle) {
+ native_handle_close(allocHandle);
+ native_handle_delete(allocHandle);
+ }
+ }
+ for (int i = 0; i < kNumAllocationTest; ++i) {
+ for (int j = i + 1; j < kNumAllocationTest; ++j) {
+ ASSERT_TRUE(buffer[i]->mId != buffer[j]->mId);
+ }
+ }
+ EXPECT_TRUE(kNumAllocationTest > 1);
+}
+
+// Buffer recycle test.
+// Check whether de-allocated buffers are recycled.
+TEST_F(BufferpoolSingleTest, RecycleBuffer) {
+ ResultStatus status;
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+
+ BufferId bid[kNumRecycleTest];
+ for (int i = 0; i < kNumRecycleTest; ++i) {
+ std::shared_ptr<BufferPoolData> buffer;
+ native_handle_t *allocHandle = nullptr;
+ status = mManager->allocate(mConnectionId, vecParams, &allocHandle, &buffer);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ bid[i] = buffer->mId;
+ if (allocHandle) {
+ native_handle_close(allocHandle);
+ native_handle_delete(allocHandle);
+ }
+ }
+ for (int i = 1; i < kNumRecycleTest; ++i) {
+ ASSERT_TRUE(bid[i - 1] == bid[i]);
+ }
+ EXPECT_TRUE(kNumRecycleTest > 1);
+}
+
+// Buffer transfer test.
+// Check whether buffer is transferred to another client successfully.
+TEST_F(BufferpoolSingleTest, TransferBuffer) {
+ ResultStatus status;
+ std::vector<uint8_t> vecParams;
+ getTestAllocatorParams(&vecParams);
+ std::shared_ptr<BufferPoolData> sbuffer, rbuffer;
+ native_handle_t *allocHandle = nullptr;
+ native_handle_t *recvHandle = nullptr;
+
+ TransactionId transactionId;
+ int64_t postUs;
+
+ status = mManager->allocate(mConnectionId, vecParams, &allocHandle, &sbuffer);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ ASSERT_TRUE(TestBufferPoolAllocator::Fill(allocHandle, 0x77));
+ status = mManager->postSend(mReceiverId, sbuffer, &transactionId, &postUs);
+ ASSERT_TRUE(status == ResultStatus::OK);
+ status = mManager->receive(mReceiverId, transactionId, sbuffer->mId, postUs,
+ &recvHandle, &rbuffer);
+ EXPECT_TRUE(status == ResultStatus::OK);
+ ASSERT_TRUE(TestBufferPoolAllocator::Verify(recvHandle, 0x77));
+
+ if (allocHandle) {
+ native_handle_close(allocHandle);
+ native_handle_delete(allocHandle);
+ }
+ if (recvHandle) {
+ native_handle_close(recvHandle);
+ native_handle_delete(recvHandle);
+ }
+}
+
+} // anonymous namespace
+
+int main(int argc, char** argv) {
+ ::testing::InitGoogleTest(&argc, argv);
+ int status = RUN_ALL_TESTS();
+ LOG(INFO) << "Test result = " << status;
+ return status;
+}