Merge "BufferQueue: Allow detaching/reattaching buffers"
diff --git a/include/gui/BufferQueue.h b/include/gui/BufferQueue.h
index 7eaf842..f230e55 100644
--- a/include/gui/BufferQueue.h
+++ b/include/gui/BufferQueue.h
@@ -28,10 +28,43 @@
#include <binder/IBinder.h>
namespace android {
-// ----------------------------------------------------------------------------
-class BufferQueue : public BnGraphicBufferProducer,
- public BnGraphicBufferConsumer,
+// BQProducer and BQConsumer are thin shim classes to allow methods with the
+// same signature in both IGraphicBufferProducer and IGraphicBufferConsumer.
+// This will stop being an issue when we deprecate creating BufferQueues
+// directly (as opposed to using the *Producer and *Consumer interfaces).
+class BQProducer : public BnGraphicBufferProducer {
+public:
+ virtual status_t detachProducerBuffer(int slot) = 0;
+ virtual status_t attachProducerBuffer(int* slot,
+ const sp<GraphicBuffer>& buffer) = 0;
+
+ virtual status_t detachBuffer(int slot) {
+ return detachProducerBuffer(slot);
+ }
+
+ virtual status_t attachBuffer(int* slot, const sp<GraphicBuffer>& buffer) {
+ return attachProducerBuffer(slot, buffer);
+ }
+};
+
+class BQConsumer : public BnGraphicBufferConsumer {
+public:
+ virtual status_t detachConsumerBuffer(int slot) = 0;
+ virtual status_t attachConsumerBuffer(int* slot,
+ const sp<GraphicBuffer>& buffer) = 0;
+
+ virtual status_t detachBuffer(int slot) {
+ return detachConsumerBuffer(slot);
+ }
+
+ virtual status_t attachBuffer(int* slot, const sp<GraphicBuffer>& buffer) {
+ return attachConsumerBuffer(slot, buffer);
+ }
+};
+
+class BufferQueue : public BQProducer,
+ public BQConsumer,
private IBinder::DeathRecipient {
public:
// BufferQueue will keep track of at most this value of buffers.
@@ -73,6 +106,10 @@
wp<ConsumerListener> mConsumerListener;
};
+ static void createBufferQueue(sp<BnGraphicBufferProducer>* outProducer,
+ sp<BnGraphicBufferConsumer>* outConsumer,
+ const sp<IGraphicBufferAlloc>& allocator = NULL);
+
// BufferQueue manages a pool of gralloc memory slots to be used by
// producers and consumers. allocator is used to allocate all the
// needed gralloc buffers.
@@ -157,6 +194,13 @@
virtual status_t dequeueBuffer(int *buf, sp<Fence>* fence, bool async,
uint32_t width, uint32_t height, uint32_t format, uint32_t usage);
+ // See IGraphicBufferProducer::detachBuffer
+ virtual status_t detachProducerBuffer(int slot);
+
+ // See IGraphicBufferProducer::attachBuffer
+ virtual status_t attachProducerBuffer(int* slot,
+ const sp<GraphicBuffer>& buffer);
+
// queueBuffer returns a filled buffer to the BufferQueue.
//
// Additional data is provided in the QueueBufferInput struct. Notably,
@@ -223,6 +267,13 @@
// is CLOCK_MONOTONIC.
virtual status_t acquireBuffer(BufferItem* buffer, nsecs_t presentWhen);
+ // See IGraphicBufferConsumer::detachBuffer
+ virtual status_t detachConsumerBuffer(int slot);
+
+ // See IGraphicBufferConsumer::attachBuffer
+ virtual status_t attachConsumerBuffer(int* slot,
+ const sp<GraphicBuffer>& buffer);
+
// releaseBuffer releases a buffer slot from the consumer back to the
// BufferQueue. This may be done while the buffer's contents are still
// being accessed. The fence will signal when the buffer is no longer
diff --git a/include/gui/BufferQueueConsumer.h b/include/gui/BufferQueueConsumer.h
index 20db98c..71105df 100644
--- a/include/gui/BufferQueueConsumer.h
+++ b/include/gui/BufferQueueConsumer.h
@@ -49,6 +49,12 @@
virtual status_t acquireBuffer(BufferItem* outBuffer,
nsecs_t expectedPresent);
+ // See IGraphicBufferConsumer::detachBuffer
+ virtual status_t detachBuffer(int slot);
+
+ // See IGraphicBufferConsumer::attachBuffer
+ virtual status_t attachBuffer(int* slot, const sp<GraphicBuffer>& buffer);
+
// releaseBuffer releases a buffer slot from the consumer back to the
// BufferQueue. This may be done while the buffer's contents are still
// being accessed. The fence will signal when the buffer is no longer
diff --git a/include/gui/BufferQueueProducer.h b/include/gui/BufferQueueProducer.h
index 8912097..2eeb979 100644
--- a/include/gui/BufferQueueProducer.h
+++ b/include/gui/BufferQueueProducer.h
@@ -96,6 +96,12 @@
virtual status_t dequeueBuffer(int *outSlot, sp<Fence>* outFence, bool async,
uint32_t width, uint32_t height, uint32_t format, uint32_t usage);
+ // See IGraphicBufferProducer::detachBuffer
+ virtual status_t detachBuffer(int slot);
+
+ // See IGraphicBufferProducer::attachBuffer
+ virtual status_t attachBuffer(int* outSlot, const sp<GraphicBuffer>& buffer);
+
// queueBuffer returns a filled buffer to the BufferQueue.
//
// Additional data is provided in the QueueBufferInput struct. Notably,
@@ -151,6 +157,14 @@
// This is required by the IBinder::DeathRecipient interface
virtual void binderDied(const wp<IBinder>& who);
+ // waitForFreeSlotThenRelock finds the oldest slot in the FREE state. It may
+ // block if there are no available slots and we are not in non-blocking
+ // mode (producer and consumer controlled by the application). If it blocks,
+ // it will release mCore->mMutex while blocked so that other operations on
+ // the BufferQueue may succeed.
+ status_t waitForFreeSlotThenRelock(const char* caller, bool async,
+ int* found, status_t* returnFlags) const;
+
sp<BufferQueueCore> mCore;
// This references mCore->mSlots. Lock mCore->mMutex while accessing.
diff --git a/include/gui/BufferSlot.h b/include/gui/BufferSlot.h
index 2c4b43f..6085e11 100644
--- a/include/gui/BufferSlot.h
+++ b/include/gui/BufferSlot.h
@@ -38,7 +38,8 @@
mFrameNumber(0),
mEglFence(EGL_NO_SYNC_KHR),
mAcquireCalled(false),
- mNeedsCleanupOnRelease(false) {
+ mNeedsCleanupOnRelease(false),
+ mAttachedByConsumer(false) {
}
// mGraphicBuffer points to the buffer allocated for this slot or is NULL
@@ -129,6 +130,11 @@
// consumer. This is set when a buffer in ACQUIRED state is freed.
// It causes releaseBuffer to return STALE_BUFFER_SLOT.
bool mNeedsCleanupOnRelease;
+
+ // Indicates whether the buffer was attached on the consumer side.
+ // If so, it needs to set the BUFFER_NEEDS_REALLOCATION flag when dequeued
+ // to prevent the producer from using a stale cached buffer.
+ bool mAttachedByConsumer;
};
} // namespace android
diff --git a/include/gui/IGraphicBufferConsumer.h b/include/gui/IGraphicBufferConsumer.h
index 9a6645c..5e97854 100644
--- a/include/gui/IGraphicBufferConsumer.h
+++ b/include/gui/IGraphicBufferConsumer.h
@@ -139,6 +139,36 @@
// * INVALID_OPERATION - too many buffers have been acquired
virtual status_t acquireBuffer(BufferItem* buffer, nsecs_t presentWhen) = 0;
+ // detachBuffer attempts to remove all ownership of the buffer in the given
+ // slot from the buffer queue. If this call succeeds, the slot will be
+ // freed, and there will be no way to obtain the buffer from this interface.
+ // The freed slot will remain unallocated until either it is selected to
+ // hold a freshly allocated buffer in dequeueBuffer or a buffer is attached
+ // to the slot. The buffer must have already been acquired.
+ //
+ // Return of a value other than NO_ERROR means an error has occurred:
+ // * BAD_VALUE - the given slot number is invalid, either because it is
+ // out of the range [0, NUM_BUFFER_SLOTS) or because the slot
+ // it refers to is not currently acquired.
+ virtual status_t detachBuffer(int slot) = 0;
+
+ // attachBuffer attempts to transfer ownership of a buffer to the buffer
+ // queue. If this call succeeds, it will be as if this buffer was acquired
+ // from the returned slot number. As such, this call will fail if attaching
+ // this buffer would cause too many buffers to be simultaneously acquired.
+ //
+ // If the buffer is successfully attached, its frameNumber is initialized
+ // to 0. This must be passed into the releaseBuffer call or else the buffer
+ // will be deallocated as stale.
+ //
+ // Return of a value other than NO_ERROR means an error has occurred:
+ // * BAD_VALUE - outSlot or buffer were NULL
+ // * INVALID_OPERATION - cannot attach the buffer because it would cause too
+ // many buffers to be acquired.
+ // * NO_MEMORY - no free slots available
+ virtual status_t attachBuffer(int *outSlot,
+ const sp<GraphicBuffer>& buffer) = 0;
+
// releaseBuffer releases a buffer slot from the consumer back to the
// BufferQueue. This may be done while the buffer's contents are still
// being accessed. The fence will signal when the buffer is no longer
diff --git a/include/gui/IGraphicBufferProducer.h b/include/gui/IGraphicBufferProducer.h
index 7002530..25a86a5 100644
--- a/include/gui/IGraphicBufferProducer.h
+++ b/include/gui/IGraphicBufferProducer.h
@@ -150,12 +150,14 @@
//
// Return of a negative means an error has occurred:
// * NO_INIT - the buffer queue has been abandoned.
- // * BAD_VALUE - one of the below conditions occurred:
- // * both in async mode and buffer count was less than the
- // max numbers of buffers that can be allocated at once
- // * attempting dequeue more than one buffer at a time
- // without setting the buffer count with setBufferCount()
- // * -EBUSY - attempting to dequeue too many buffers at a time
+ // * BAD_VALUE - both in async mode and buffer count was less than the
+ // max numbers of buffers that can be allocated at once.
+ // * INVALID_OPERATION - cannot attach the buffer because it would cause
+ // too many buffers to be dequeued, either because
+ // the producer already has a single buffer dequeued
+ // and did not set a buffer count, or because a
+ // buffer count was set and this call would cause
+ // it to be exceeded.
// * WOULD_BLOCK - no buffer is currently available, and blocking is disabled
// since both the producer/consumer are controlled by app
// * NO_MEMORY - out of memory, cannot allocate the graphics buffer.
@@ -165,6 +167,49 @@
virtual status_t dequeueBuffer(int* slot, sp<Fence>* fence, bool async,
uint32_t w, uint32_t h, uint32_t format, uint32_t usage) = 0;
+ // detachBuffer attempts to remove all ownership of the buffer in the given
+ // slot from the buffer queue. If this call succeeds, the slot will be
+ // freed, and there will be no way to obtain the buffer from this interface.
+ // The freed slot will remain unallocated until either it is selected to
+ // hold a freshly allocated buffer in dequeueBuffer or a buffer is attached
+ // to the slot. The buffer must have already been dequeued, and the caller
+ // must already possesses the sp<GraphicBuffer> (i.e., must have called
+ // requestBuffer).
+ //
+ // Return of a value other than NO_ERROR means an error has occurred:
+ // * NO_INIT - the buffer queue has been abandoned.
+ // * BAD_VALUE - the given slot number is invalid, either because it is
+ // out of the range [0, NUM_BUFFER_SLOTS), or because the slot
+ // it refers to is not currently dequeued and requested.
+ virtual status_t detachBuffer(int slot) = 0;
+
+ // attachBuffer attempts to transfer ownership of a buffer to the buffer
+ // queue. If this call succeeds, it will be as if this buffer was dequeued
+ // from the returned slot number. As such, this call will fail if attaching
+ // this buffer would cause too many buffers to be simultaneously dequeued.
+ //
+ // If attachBuffer returns the RELEASE_ALL_BUFFERS flag, the caller is
+ // expected to release all of the mirrored slot->buffer mappings.
+ //
+ // A non-negative value with flags set (see above) will be returned upon
+ // success.
+ //
+ // Return of a negative value means an error has occurred:
+ // * NO_INIT - the buffer queue has been abandoned.
+ // * BAD_VALUE - outSlot or buffer were NULL or invalid combination of
+ // async mode and buffer count override.
+ // * INVALID_OPERATION - cannot attach the buffer because it would cause
+ // too many buffers to be dequeued, either because
+ // the producer already has a single buffer dequeued
+ // and did not set a buffer count, or because a
+ // buffer count was set and this call would cause
+ // it to be exceeded.
+ // * WOULD_BLOCK - no buffer slot is currently available, and blocking is
+ // disabled since both the producer/consumer are
+ // controlled by the app.
+ virtual status_t attachBuffer(int* outSlot,
+ const sp<GraphicBuffer>& buffer) = 0;
+
// queueBuffer indicates that the client has finished filling in the
// contents of the buffer associated with slot and transfers ownership of
// that slot back to the server.
diff --git a/libs/gui/BufferQueue.cpp b/libs/gui/BufferQueue.cpp
index af857fd..26e215b 100644
--- a/libs/gui/BufferQueue.cpp
+++ b/libs/gui/BufferQueue.cpp
@@ -43,6 +43,19 @@
}
}
+void BufferQueue::createBufferQueue(sp<BnGraphicBufferProducer>* outProducer,
+ sp<BnGraphicBufferConsumer>* outConsumer,
+ const sp<IGraphicBufferAlloc>& allocator) {
+ LOG_ALWAYS_FATAL_IF(outProducer == NULL,
+ "BufferQueue: outProducer must not be NULL");
+ LOG_ALWAYS_FATAL_IF(outConsumer == NULL,
+ "BufferQueue: outConsumer must not be NULL");
+
+ sp<BufferQueueCore> core(new BufferQueueCore(allocator));
+ *outProducer = new BufferQueueProducer(core);
+ *outConsumer = new BufferQueueConsumer(core);
+}
+
BufferQueue::BufferQueue(const sp<IGraphicBufferAlloc>& allocator) :
mProducer(),
mConsumer()
@@ -75,6 +88,15 @@
return mProducer->dequeueBuffer(outBuf, outFence, async, w, h, format, usage);
}
+status_t BufferQueue::detachProducerBuffer(int slot) {
+ return mProducer->detachBuffer(slot);
+}
+
+status_t BufferQueue::attachProducerBuffer(int* slot,
+ const sp<GraphicBuffer>& buffer) {
+ return mProducer->attachBuffer(slot, buffer);
+}
+
status_t BufferQueue::queueBuffer(int buf,
const QueueBufferInput& input, QueueBufferOutput* output) {
return mProducer->queueBuffer(buf, input, output);
@@ -97,6 +119,15 @@
return mConsumer->acquireBuffer(buffer, presentWhen);
}
+status_t BufferQueue::detachConsumerBuffer(int slot) {
+ return mConsumer->detachBuffer(slot);
+}
+
+status_t BufferQueue::attachConsumerBuffer(int* slot,
+ const sp<GraphicBuffer>& buffer) {
+ return mConsumer->attachBuffer(slot, buffer);
+}
+
status_t BufferQueue::releaseBuffer(
int buf, uint64_t frameNumber, EGLDisplay display,
EGLSyncKHR eglFence, const sp<Fence>& fence) {
diff --git a/libs/gui/BufferQueueConsumer.cpp b/libs/gui/BufferQueueConsumer.cpp
index dc7aa15..66fdab3 100644
--- a/libs/gui/BufferQueueConsumer.cpp
+++ b/libs/gui/BufferQueueConsumer.cpp
@@ -168,13 +168,96 @@
return NO_ERROR;
}
+status_t BufferQueueConsumer::detachBuffer(int slot) {
+ ATRACE_CALL();
+ ATRACE_BUFFER_INDEX(slot);
+ BQ_LOGV("detachBuffer(C): slot %d", slot);
+ Mutex::Autolock lock(mCore->mMutex);
+
+ if (mCore->mIsAbandoned) {
+ BQ_LOGE("detachBuffer(C): BufferQueue has been abandoned");
+ return NO_INIT;
+ }
+
+ if (slot < 0 || slot >= BufferQueueDefs::NUM_BUFFER_SLOTS) {
+ BQ_LOGE("detachBuffer(C): slot index %d out of range [0, %d)",
+ slot, BufferQueueDefs::NUM_BUFFER_SLOTS);
+ return BAD_VALUE;
+ } else if (mSlots[slot].mBufferState != BufferSlot::ACQUIRED) {
+ BQ_LOGE("detachBuffer(C): slot %d is not owned by the consumer "
+ "(state = %d)", slot, mSlots[slot].mBufferState);
+ return BAD_VALUE;
+ }
+
+ mCore->freeBufferLocked(slot);
+ mCore->mDequeueCondition.broadcast();
+
+ return NO_ERROR;
+}
+
+status_t BufferQueueConsumer::attachBuffer(int* outSlot,
+ const sp<android::GraphicBuffer>& buffer) {
+ ATRACE_CALL();
+
+ if (outSlot == NULL) {
+ BQ_LOGE("attachBuffer(P): outSlot must not be NULL");
+ return BAD_VALUE;
+ } else if (buffer == NULL) {
+ BQ_LOGE("attachBuffer(P): cannot attach NULL buffer");
+ return BAD_VALUE;
+ }
+
+ Mutex::Autolock lock(mCore->mMutex);
+
+ // Make sure we don't have too many acquired buffers and find a free slot
+ // to put the buffer into (the oldest if there are multiple).
+ int numAcquiredBuffers = 0;
+ int found = BufferQueueCore::INVALID_BUFFER_SLOT;
+ for (int s = 0; s < BufferQueueDefs::NUM_BUFFER_SLOTS; ++s) {
+ if (mSlots[s].mBufferState == BufferSlot::ACQUIRED) {
+ ++numAcquiredBuffers;
+ } else if (mSlots[s].mBufferState == BufferSlot::FREE) {
+ if (found == BufferQueueCore::INVALID_BUFFER_SLOT ||
+ mSlots[s].mFrameNumber < mSlots[found].mFrameNumber) {
+ found = s;
+ }
+ }
+ }
+
+ if (numAcquiredBuffers >= mCore->mMaxAcquiredBufferCount + 1) {
+ BQ_LOGE("attachBuffer(P): max acquired buffer count reached: %d "
+ "(max %d)", numAcquiredBuffers,
+ mCore->mMaxAcquiredBufferCount);
+ return INVALID_OPERATION;
+ }
+ if (found == BufferQueueCore::INVALID_BUFFER_SLOT) {
+ BQ_LOGE("attachBuffer(P): could not find free buffer slot");
+ return NO_MEMORY;
+ }
+
+ *outSlot = found;
+ ATRACE_BUFFER_INDEX(*outSlot);
+ BQ_LOGV("attachBuffer(C): returning slot %d", *outSlot);
+
+ mSlots[*outSlot].mGraphicBuffer = buffer;
+ mSlots[*outSlot].mBufferState = BufferSlot::ACQUIRED;
+ mSlots[*outSlot].mAttachedByConsumer = true;
+ mSlots[*outSlot].mAcquireCalled = true;
+ mSlots[*outSlot].mNeedsCleanupOnRelease = false;
+ mSlots[*outSlot].mFence = Fence::NO_FENCE;
+ mSlots[*outSlot].mFrameNumber = 0;
+
+ return NO_ERROR;
+}
+
status_t BufferQueueConsumer::releaseBuffer(int slot, uint64_t frameNumber,
const sp<Fence>& releaseFence, EGLDisplay eglDisplay,
EGLSyncKHR eglFence) {
ATRACE_CALL();
ATRACE_BUFFER_INDEX(slot);
- if (slot == BufferQueueCore::INVALID_BUFFER_SLOT || releaseFence == NULL) {
+ if (slot < 0 || slot >= BufferQueueDefs::NUM_BUFFER_SLOTS ||
+ releaseFence == NULL) {
return BAD_VALUE;
}
@@ -192,7 +275,7 @@
if (current->mSlot == slot) {
BQ_LOGE("releaseBuffer: buffer slot %d pending release is "
"currently queued", slot);
- return -EINVAL;
+ return BAD_VALUE;
}
++current;
}
@@ -210,7 +293,7 @@
} else {
BQ_LOGV("releaseBuffer: attempted to release buffer slot %d "
"but its state was %d", slot, mSlots[slot].mBufferState);
- return -EINVAL;
+ return BAD_VALUE;
}
mCore->mDequeueCondition.broadcast();
@@ -252,7 +335,7 @@
if (mCore->mConsumerListener == NULL) {
BQ_LOGE("disconnect(C): no consumer is connected");
- return -EINVAL;
+ return BAD_VALUE;
}
mCore->mIsAbandoned = true;
diff --git a/libs/gui/BufferQueueProducer.cpp b/libs/gui/BufferQueueProducer.cpp
index 0e9de88..58e19f0 100644
--- a/libs/gui/BufferQueueProducer.cpp
+++ b/libs/gui/BufferQueueProducer.cpp
@@ -86,7 +86,7 @@
for (int s = 0; s < BufferQueueDefs::NUM_BUFFER_SLOTS; ++s) {
if (mSlots[s].mBufferState == BufferSlot::DEQUEUED) {
BQ_LOGE("setBufferCount: buffer owned by producer");
- return -EINVAL;
+ return BAD_VALUE;
}
}
@@ -121,6 +121,110 @@
return NO_ERROR;
}
+status_t BufferQueueProducer::waitForFreeSlotThenRelock(const char* caller,
+ bool async, int* found, status_t* returnFlags) const {
+ bool tryAgain = true;
+ while (tryAgain) {
+ if (mCore->mIsAbandoned) {
+ BQ_LOGE("%s: BufferQueue has been abandoned", caller);
+ return NO_INIT;
+ }
+
+ const int maxBufferCount = mCore->getMaxBufferCountLocked(async);
+ if (async && mCore->mOverrideMaxBufferCount) {
+ // FIXME: Some drivers are manually setting the buffer count
+ // (which they shouldn't), so we do this extra test here to
+ // handle that case. This is TEMPORARY until we get this fixed.
+ if (mCore->mOverrideMaxBufferCount < maxBufferCount) {
+ BQ_LOGE("%s: async mode is invalid with buffer count override",
+ caller);
+ return BAD_VALUE;
+ }
+ }
+
+ // Free up any buffers that are in slots beyond the max buffer count
+ for (int s = maxBufferCount; s < BufferQueueDefs::NUM_BUFFER_SLOTS; ++s) {
+ assert(mSlots[s].mBufferState == BufferSlot::FREE);
+ if (mSlots[s].mGraphicBuffer != NULL) {
+ mCore->freeBufferLocked(s);
+ *returnFlags |= RELEASE_ALL_BUFFERS;
+ }
+ }
+
+ // Look for a free buffer to give to the client
+ *found = BufferQueueCore::INVALID_BUFFER_SLOT;
+ int dequeuedCount = 0;
+ int acquiredCount = 0;
+ for (int s = 0; s < maxBufferCount; ++s) {
+ switch (mSlots[s].mBufferState) {
+ case BufferSlot::DEQUEUED:
+ ++dequeuedCount;
+ break;
+ case BufferSlot::ACQUIRED:
+ ++acquiredCount;
+ break;
+ case BufferSlot::FREE:
+ // We return the oldest of the free buffers to avoid
+ // stalling the producer if possible, since the consumer
+ // may still have pending reads of in-flight buffers
+ if (*found == BufferQueueCore::INVALID_BUFFER_SLOT ||
+ mSlots[s].mFrameNumber < mSlots[*found].mFrameNumber) {
+ *found = s;
+ }
+ break;
+ default:
+ break;
+ }
+ }
+
+ // Producers are not allowed to dequeue more than one buffer if they
+ // did not set a buffer count
+ if (!mCore->mOverrideMaxBufferCount && dequeuedCount) {
+ BQ_LOGE("%s: can't dequeue multiple buffers without setting the "
+ "buffer count", caller);
+ return INVALID_OPERATION;
+ }
+
+ // See whether a buffer has been queued since the last
+ // setBufferCount so we know whether to perform the min undequeued
+ // buffers check below
+ if (mCore->mBufferHasBeenQueued) {
+ // Make sure the producer is not trying to dequeue more buffers
+ // than allowed
+ const int newUndequeuedCount =
+ maxBufferCount - (dequeuedCount + 1);
+ const int minUndequeuedCount =
+ mCore->getMinUndequeuedBufferCountLocked(async);
+ if (newUndequeuedCount < minUndequeuedCount) {
+ BQ_LOGE("%s: min undequeued buffer count (%d) exceeded "
+ "(dequeued=%d undequeued=%d)",
+ caller, minUndequeuedCount,
+ dequeuedCount, newUndequeuedCount);
+ return INVALID_OPERATION;
+ }
+ }
+
+ // If no buffer is found, wait for a buffer to be released or for
+ // the max buffer count to change
+ tryAgain = (*found == BufferQueueCore::INVALID_BUFFER_SLOT);
+ if (tryAgain) {
+ // Return an error if we're in non-blocking mode (producer and
+ // consumer are controlled by the application).
+ // However, the consumer is allowed to briefly acquire an extra
+ // buffer (which could cause us to have to wait here), which is
+ // okay, since it is only used to implement an atomic acquire +
+ // release (e.g., in GLConsumer::updateTexImage())
+ if (mCore->mDequeueBufferCannotBlock &&
+ (acquiredCount <= mCore->mMaxAcquiredBufferCount)) {
+ return WOULD_BLOCK;
+ }
+ mCore->mDequeueCondition.wait(mCore->mMutex);
+ }
+ } // while (tryAgain)
+
+ return NO_ERROR;
+}
+
status_t BufferQueueProducer::dequeueBuffer(int *outSlot,
sp<android::Fence> *outFence, bool async,
uint32_t width, uint32_t height, uint32_t format, uint32_t usage) {
@@ -141,6 +245,7 @@
status_t returnFlags = NO_ERROR;
EGLDisplay eglDisplay = EGL_NO_DISPLAY;
EGLSyncKHR eglFence = EGL_NO_SYNC_KHR;
+ bool attachedByConsumer = false;
{ // Autolock scope
Mutex::Autolock lock(mCore->mMutex);
@@ -152,105 +257,12 @@
// Enable the usage bits the consumer requested
usage |= mCore->mConsumerUsageBits;
- int found = -1;
- bool tryAgain = true;
- while (tryAgain) {
- if (mCore->mIsAbandoned) {
- BQ_LOGE("dequeueBuffer: BufferQueue has been abandoned");
- return NO_INIT;
- }
-
- const int maxBufferCount = mCore->getMaxBufferCountLocked(async);
- if (async && mCore->mOverrideMaxBufferCount) {
- // FIXME: Some drivers are manually setting the buffer count
- // (which they shouldn't), so we do this extra test here to
- // handle that case. This is TEMPORARY until we get this fixed.
- if (mCore->mOverrideMaxBufferCount < maxBufferCount) {
- BQ_LOGE("dequeueBuffer: async mode is invalid with "
- "buffer count override");
- return BAD_VALUE;
- }
- }
-
- // Free up any buffers that are in slots beyond the max buffer count
- for (int s = maxBufferCount; s < BufferQueueDefs::NUM_BUFFER_SLOTS; ++s) {
- assert(mSlots[s].mBufferState == BufferSlot::FREE);
- if (mSlots[s].mGraphicBuffer != NULL) {
- mCore->freeBufferLocked(s);
- returnFlags |= RELEASE_ALL_BUFFERS;
- }
- }
-
- // Look for a free buffer to give to the client
- found = BufferQueueCore::INVALID_BUFFER_SLOT;
- int dequeuedCount = 0;
- int acquiredCount = 0;
- for (int s = 0; s < maxBufferCount; ++s) {
- switch (mSlots[s].mBufferState) {
- case BufferSlot::DEQUEUED:
- ++dequeuedCount;
- break;
- case BufferSlot::ACQUIRED:
- ++acquiredCount;
- break;
- case BufferSlot::FREE:
- // We return the oldest of the free buffers to avoid
- // stalling the producer if possible, since the consumer
- // may still have pending reads of in-flight buffers
- if (found == BufferQueueCore::INVALID_BUFFER_SLOT ||
- mSlots[s].mFrameNumber < mSlots[found].mFrameNumber) {
- found = s;
- }
- break;
- default:
- break;
- }
- }
-
- // Producers are not allowed to dequeue more than one buffer if they
- // did not set a buffer count
- if (!mCore->mOverrideMaxBufferCount && dequeuedCount) {
- BQ_LOGE("dequeueBuffer: can't dequeue multiple buffers "
- "without setting the buffer count");
- return -EINVAL;
- }
-
- // See whether a buffer has been queued since the last
- // setBufferCount so we know whether to perform the min undequeued
- // buffers check below
- if (mCore->mBufferHasBeenQueued) {
- // Make sure the producer is not trying to dequeue more buffers
- // than allowed
- const int newUndequeuedCount =
- maxBufferCount - (dequeuedCount + 1);
- const int minUndequeuedCount =
- mCore->getMinUndequeuedBufferCountLocked(async);
- if (newUndequeuedCount < minUndequeuedCount) {
- BQ_LOGE("dequeueBuffer: min undequeued buffer count (%d) "
- "exceeded (dequeued=%d undequeued=%d)",
- minUndequeuedCount, dequeuedCount,
- newUndequeuedCount);
- return -EBUSY;
- }
- }
-
- // If no buffer is found, wait for a buffer to be released or for
- // the max buffer count to change
- tryAgain = (found == BufferQueueCore::INVALID_BUFFER_SLOT);
- if (tryAgain) {
- // Return an error if we're in non-blocking mode (producer and
- // consumer are controlled by the application).
- // However, the consumer is allowed to briefly acquire an extra
- // buffer (which could cause us to have to wait here), which is
- // okay, since it is only used to implement an atomic acquire +
- // release (e.g., in GLConsumer::updateTexImage())
- if (mCore->mDequeueBufferCannotBlock &&
- (acquiredCount <= mCore->mMaxAcquiredBufferCount)) {
- return WOULD_BLOCK;
- }
- mCore->mDequeueCondition.wait(mCore->mMutex);
- }
- } // while (tryAgain)
+ int found;
+ status_t status = waitForFreeSlotThenRelock("dequeueBuffer", async,
+ &found, &returnFlags);
+ if (status != NO_ERROR) {
+ return status;
+ }
// This should not happen
if (found == BufferQueueCore::INVALID_BUFFER_SLOT) {
@@ -261,6 +273,8 @@
*outSlot = found;
ATRACE_BUFFER_INDEX(found);
+ attachedByConsumer = mSlots[found].mAttachedByConsumer;
+
const bool useDefaultSize = !width && !height;
if (useDefaultSize) {
width = mCore->mDefaultWidth;
@@ -316,11 +330,15 @@
return NO_INIT;
}
- mSlots[*outSlot].mFrameNumber = ~0;
+ mSlots[*outSlot].mFrameNumber = UINT32_MAX;
mSlots[*outSlot].mGraphicBuffer = graphicBuffer;
} // Autolock scope
}
+ if (attachedByConsumer) {
+ returnFlags |= BUFFER_NEEDS_REALLOCATION;
+ }
+
if (eglFence != EGL_NO_SYNC_KHR) {
EGLint result = eglClientWaitSyncKHR(eglDisplay, eglFence, 0,
1000000000);
@@ -343,6 +361,81 @@
return returnFlags;
}
+status_t BufferQueueProducer::detachBuffer(int slot) {
+ ATRACE_CALL();
+ ATRACE_BUFFER_INDEX(slot);
+ BQ_LOGV("detachBuffer(P): slot %d", slot);
+ Mutex::Autolock lock(mCore->mMutex);
+
+ if (mCore->mIsAbandoned) {
+ BQ_LOGE("detachBuffer(P): BufferQueue has been abandoned");
+ return NO_INIT;
+ }
+
+ if (slot < 0 || slot >= BufferQueueDefs::NUM_BUFFER_SLOTS) {
+ BQ_LOGE("detachBuffer(P): slot index %d out of range [0, %d)",
+ slot, BufferQueueDefs::NUM_BUFFER_SLOTS);
+ return BAD_VALUE;
+ } else if (mSlots[slot].mBufferState != BufferSlot::DEQUEUED) {
+ BQ_LOGE("detachBuffer(P): slot %d is not owned by the producer "
+ "(state = %d)", slot, mSlots[slot].mBufferState);
+ return BAD_VALUE;
+ } else if (!mSlots[slot].mRequestBufferCalled) {
+ BQ_LOGE("detachBuffer(P): buffer in slot %d has not been requested",
+ slot);
+ return BAD_VALUE;
+ }
+
+ mCore->freeBufferLocked(slot);
+ mCore->mDequeueCondition.broadcast();
+
+ return NO_ERROR;
+}
+
+status_t BufferQueueProducer::attachBuffer(int* outSlot,
+ const sp<android::GraphicBuffer>& buffer) {
+ ATRACE_CALL();
+
+ if (outSlot == NULL) {
+ BQ_LOGE("attachBuffer(P): outSlot must not be NULL");
+ return BAD_VALUE;
+ } else if (buffer == NULL) {
+ BQ_LOGE("attachBuffer(P): cannot attach NULL buffer");
+ return BAD_VALUE;
+ }
+
+ Mutex::Autolock lock(mCore->mMutex);
+
+ status_t returnFlags = NO_ERROR;
+ int found;
+ // TODO: Should we provide an async flag to attachBuffer? It seems
+ // unlikely that buffers which we are attaching to a BufferQueue will
+ // be asynchronous (droppable), but it may not be impossible.
+ status_t status = waitForFreeSlotThenRelock("attachBuffer(P)", false,
+ &found, &returnFlags);
+ if (status != NO_ERROR) {
+ return status;
+ }
+
+ // This should not happen
+ if (found == BufferQueueCore::INVALID_BUFFER_SLOT) {
+ BQ_LOGE("attachBuffer(P): no available buffer slots");
+ return -EBUSY;
+ }
+
+ *outSlot = found;
+ ATRACE_BUFFER_INDEX(*outSlot);
+ BQ_LOGV("attachBuffer(P): returning slot %d flags=%#x",
+ *outSlot, returnFlags);
+
+ mSlots[*outSlot].mGraphicBuffer = buffer;
+ mSlots[*outSlot].mBufferState = BufferSlot::DEQUEUED;
+ mSlots[*outSlot].mEglFence = EGL_NO_SYNC_KHR;
+ mSlots[*outSlot].mFence = Fence::NO_FENCE;
+
+ return returnFlags;
+}
+
status_t BufferQueueProducer::queueBuffer(int slot,
const QueueBufferInput &input, QueueBufferOutput *output) {
ATRACE_CALL();
@@ -371,7 +464,7 @@
break;
default:
BQ_LOGE("queueBuffer: unknown scaling mode %d", scalingMode);
- return -EINVAL;
+ return BAD_VALUE;
}
sp<IConsumerListener> listener;
@@ -398,15 +491,15 @@
if (slot < 0 || slot >= maxBufferCount) {
BQ_LOGE("queueBuffer: slot index %d out of range [0, %d)",
slot, maxBufferCount);
- return -EINVAL;
+ return BAD_VALUE;
} else if (mSlots[slot].mBufferState != BufferSlot::DEQUEUED) {
BQ_LOGE("queueBuffer: slot %d is not owned by the producer "
"(state = %d)", slot, mSlots[slot].mBufferState);
- return -EINVAL;
+ return BAD_VALUE;
} else if (!mSlots[slot].mRequestBufferCalled) {
BQ_LOGE("queueBuffer: slot %d was queued without requesting "
"a buffer", slot);
- return -EINVAL;
+ return BAD_VALUE;
}
BQ_LOGV("queueBuffer: slot=%d/%llu time=%llu crop=[%d,%d,%d,%d] "
@@ -422,7 +515,7 @@
if (croppedRect != crop) {
BQ_LOGE("queueBuffer: crop rect is not contained within the "
"buffer in slot %d", slot);
- return -EINVAL;
+ return BAD_VALUE;
}
mSlots[slot].mFence = fence;
@@ -679,12 +772,12 @@
} else {
BQ_LOGE("disconnect(P): connected to another API "
"(cur=%d req=%d)", mCore->mConnectedApi, api);
- status = -EINVAL;
+ status = BAD_VALUE;
}
break;
default:
BQ_LOGE("disconnect(P): unknown API %d", api);
- status = -EINVAL;
+ status = BAD_VALUE;
break;
}
} // Autolock scope
diff --git a/libs/gui/IGraphicBufferConsumer.cpp b/libs/gui/IGraphicBufferConsumer.cpp
index 876c895..8d64380 100644
--- a/libs/gui/IGraphicBufferConsumer.cpp
+++ b/libs/gui/IGraphicBufferConsumer.cpp
@@ -183,6 +183,8 @@
enum {
ACQUIRE_BUFFER = IBinder::FIRST_CALL_TRANSACTION,
+ DETACH_BUFFER,
+ ATTACH_BUFFER,
RELEASE_BUFFER,
CONSUMER_CONNECT,
CONSUMER_DISCONNECT,
@@ -222,6 +224,31 @@
return reply.readInt32();
}
+ virtual status_t detachBuffer(int slot) {
+ Parcel data, reply;
+ data.writeInterfaceToken(IGraphicBufferConsumer::getInterfaceDescriptor());
+ data.writeInt32(slot);
+ status_t result = remote()->transact(DETACH_BUFFER, data, &reply);
+ if (result != NO_ERROR) {
+ return result;
+ }
+ result = reply.readInt32();
+ return result;
+ }
+
+ virtual status_t attachBuffer(int* slot, const sp<GraphicBuffer>& buffer) {
+ Parcel data, reply;
+ data.writeInterfaceToken(IGraphicBufferConsumer::getInterfaceDescriptor());
+ data.write(*buffer.get());
+ status_t result = remote()->transact(ATTACH_BUFFER, data, &reply);
+ if (result != NO_ERROR) {
+ return result;
+ }
+ *slot = reply.readInt32();
+ result = reply.readInt32();
+ return result;
+ }
+
virtual status_t releaseBuffer(int buf, uint64_t frameNumber,
EGLDisplay display __attribute__((unused)), EGLSyncKHR fence __attribute__((unused)),
const sp<Fence>& releaseFence) {
@@ -382,6 +409,23 @@
reply->writeInt32(result);
return NO_ERROR;
} break;
+ case DETACH_BUFFER: {
+ CHECK_INTERFACE(IGraphicBufferConsumer, data, reply);
+ int slot = data.readInt32();
+ int result = detachBuffer(slot);
+ reply->writeInt32(result);
+ return NO_ERROR;
+ } break;
+ case ATTACH_BUFFER: {
+ CHECK_INTERFACE(IGraphicBufferConsumer, data, reply);
+ sp<GraphicBuffer> buffer = new GraphicBuffer();
+ data.read(*buffer.get());
+ int slot;
+ int result = attachBuffer(&slot, buffer);
+ reply->writeInt32(slot);
+ reply->writeInt32(result);
+ return NO_ERROR;
+ } break;
case RELEASE_BUFFER: {
CHECK_INTERFACE(IGraphicBufferConsumer, data, reply);
int buf = data.readInt32();
diff --git a/libs/gui/IGraphicBufferProducer.cpp b/libs/gui/IGraphicBufferProducer.cpp
index 0f461e5..16d8c05 100644
--- a/libs/gui/IGraphicBufferProducer.cpp
+++ b/libs/gui/IGraphicBufferProducer.cpp
@@ -34,6 +34,8 @@
REQUEST_BUFFER = IBinder::FIRST_CALL_TRANSACTION,
SET_BUFFER_COUNT,
DEQUEUE_BUFFER,
+ DETACH_BUFFER,
+ ATTACH_BUFFER,
QUEUE_BUFFER,
CANCEL_BUFFER,
QUERY,
@@ -106,6 +108,31 @@
return result;
}
+ virtual status_t detachBuffer(int slot) {
+ Parcel data, reply;
+ data.writeInterfaceToken(IGraphicBufferProducer::getInterfaceDescriptor());
+ data.writeInt32(slot);
+ status_t result = remote()->transact(DETACH_BUFFER, data, &reply);
+ if (result != NO_ERROR) {
+ return result;
+ }
+ result = reply.readInt32();
+ return result;
+ }
+
+ virtual status_t attachBuffer(int* slot, const sp<GraphicBuffer>& buffer) {
+ Parcel data, reply;
+ data.writeInterfaceToken(IGraphicBufferProducer::getInterfaceDescriptor());
+ data.write(*buffer.get());
+ status_t result = remote()->transact(ATTACH_BUFFER, data, &reply);
+ if (result != NO_ERROR) {
+ return result;
+ }
+ *slot = reply.readInt32();
+ result = reply.readInt32();
+ return result;
+ }
+
virtual status_t queueBuffer(int buf,
const QueueBufferInput& input, QueueBufferOutput* output) {
Parcel data, reply;
@@ -216,6 +243,23 @@
reply->writeInt32(result);
return NO_ERROR;
} break;
+ case DETACH_BUFFER: {
+ CHECK_INTERFACE(IGraphicBufferProducer, data, reply);
+ int slot = data.readInt32();
+ int result = detachBuffer(slot);
+ reply->writeInt32(result);
+ return NO_ERROR;
+ } break;
+ case ATTACH_BUFFER: {
+ CHECK_INTERFACE(IGraphicBufferProducer, data, reply);
+ sp<GraphicBuffer> buffer = new GraphicBuffer();
+ data.read(*buffer.get());
+ int slot;
+ int result = attachBuffer(&slot, buffer);
+ reply->writeInt32(slot);
+ reply->writeInt32(result);
+ return NO_ERROR;
+ } break;
case QUEUE_BUFFER: {
CHECK_INTERFACE(IGraphicBufferProducer, data, reply);
int buf = data.readInt32();
diff --git a/libs/gui/tests/BufferQueue_test.cpp b/libs/gui/tests/BufferQueue_test.cpp
index aafa6ea..b859fcb 100644
--- a/libs/gui/tests/BufferQueue_test.cpp
+++ b/libs/gui/tests/BufferQueue_test.cpp
@@ -24,27 +24,31 @@
#include <ui/GraphicBuffer.h>
+#include <binder/IServiceManager.h>
#include <gui/BufferQueue.h>
namespace android {
class BufferQueueTest : public ::testing::Test {
+
+public:
+ static const String16 PRODUCER_NAME;
+ static const String16 CONSUMER_NAME;
+
protected:
-
- BufferQueueTest() {}
-
- virtual void SetUp() {
+ BufferQueueTest() {
const ::testing::TestInfo* const testInfo =
::testing::UnitTest::GetInstance()->current_test_info();
ALOGV("Begin test: %s.%s", testInfo->test_case_name(),
testInfo->name());
- mBQ = new BufferQueue();
+ BufferQueue::createBufferQueue(&mProducer, &mConsumer);
+ sp<IServiceManager> serviceManager = defaultServiceManager();
+ serviceManager->addService(PRODUCER_NAME, mProducer.get());
+ serviceManager->addService(CONSUMER_NAME, mConsumer.get());
}
- virtual void TearDown() {
- mBQ.clear();
-
+ ~BufferQueueTest() {
const ::testing::TestInfo* const testInfo =
::testing::UnitTest::GetInstance()->current_test_info();
ALOGV("End test: %s.%s", testInfo->test_case_name(),
@@ -52,14 +56,19 @@
}
void GetMinUndequeuedBufferCount(int* bufferCount) {
- ASSERT_NE((void*)NULL, bufferCount);
- ASSERT_EQ(OK, mBQ->query(NATIVE_WINDOW_MIN_UNDEQUEUED_BUFFERS, bufferCount));
- ASSERT_LE(0, *bufferCount); // non-negative
+ ASSERT_TRUE(bufferCount != NULL);
+ ASSERT_EQ(OK, mProducer->query(NATIVE_WINDOW_MIN_UNDEQUEUED_BUFFERS,
+ bufferCount));
+ ASSERT_GE(*bufferCount, 0);
}
- sp<BufferQueue> mBQ;
+ sp<BnGraphicBufferProducer> mProducer;
+ sp<BnGraphicBufferConsumer> mConsumer;
};
+const String16 BufferQueueTest::PRODUCER_NAME = String16("BQTestProducer");
+const String16 BufferQueueTest::CONSUMER_NAME = String16("BQTestConsumer");
+
struct DummyConsumer : public BnConsumerListener {
virtual void onFrameAvailable() {}
virtual void onBuffersReleased() {}
@@ -67,10 +76,10 @@
TEST_F(BufferQueueTest, AcquireBuffer_ExceedsMaxAcquireCount_Fails) {
sp<DummyConsumer> dc(new DummyConsumer);
- mBQ->consumerConnect(dc, false);
+ mConsumer->consumerConnect(dc, false);
IGraphicBufferProducer::QueueBufferOutput qbo;
- mBQ->connect(NULL, NATIVE_WINDOW_API_CPU, false, &qbo);
- mBQ->setBufferCount(4);
+ mProducer->connect(NULL, NATIVE_WINDOW_API_CPU, false, &qbo);
+ mProducer->setBufferCount(4);
int slot;
sp<Fence> fence;
@@ -81,50 +90,198 @@
for (int i = 0; i < 2; i++) {
ASSERT_EQ(IGraphicBufferProducer::BUFFER_NEEDS_REALLOCATION,
- mBQ->dequeueBuffer(&slot, &fence, false, 1, 1, 0,
+ mProducer->dequeueBuffer(&slot, &fence, false, 1, 1, 0,
GRALLOC_USAGE_SW_READ_OFTEN));
- ASSERT_EQ(OK, mBQ->requestBuffer(slot, &buf));
- ASSERT_EQ(OK, mBQ->queueBuffer(slot, qbi, &qbo));
- ASSERT_EQ(OK, mBQ->acquireBuffer(&item, 0));
+ ASSERT_EQ(OK, mProducer->requestBuffer(slot, &buf));
+ ASSERT_EQ(OK, mProducer->queueBuffer(slot, qbi, &qbo));
+ ASSERT_EQ(OK, mConsumer->acquireBuffer(&item, 0));
}
ASSERT_EQ(IGraphicBufferProducer::BUFFER_NEEDS_REALLOCATION,
- mBQ->dequeueBuffer(&slot, &fence, false, 1, 1, 0,
+ mProducer->dequeueBuffer(&slot, &fence, false, 1, 1, 0,
GRALLOC_USAGE_SW_READ_OFTEN));
- ASSERT_EQ(OK, mBQ->requestBuffer(slot, &buf));
- ASSERT_EQ(OK, mBQ->queueBuffer(slot, qbi, &qbo));
+ ASSERT_EQ(OK, mProducer->requestBuffer(slot, &buf));
+ ASSERT_EQ(OK, mProducer->queueBuffer(slot, qbi, &qbo));
// Acquire the third buffer, which should fail.
- ASSERT_EQ(INVALID_OPERATION, mBQ->acquireBuffer(&item, 0));
+ ASSERT_EQ(INVALID_OPERATION, mConsumer->acquireBuffer(&item, 0));
}
TEST_F(BufferQueueTest, SetMaxAcquiredBufferCountWithIllegalValues_ReturnsError) {
sp<DummyConsumer> dc(new DummyConsumer);
- mBQ->consumerConnect(dc, false);
+ mConsumer->consumerConnect(dc, false);
int minBufferCount;
ASSERT_NO_FATAL_FAILURE(GetMinUndequeuedBufferCount(&minBufferCount));
- EXPECT_EQ(BAD_VALUE, mBQ->setMaxAcquiredBufferCount(minBufferCount - 1));
+ EXPECT_EQ(BAD_VALUE, mConsumer->setMaxAcquiredBufferCount(
+ minBufferCount - 1));
- EXPECT_EQ(BAD_VALUE, mBQ->setMaxAcquiredBufferCount(0));
- EXPECT_EQ(BAD_VALUE, mBQ->setMaxAcquiredBufferCount(-3));
- EXPECT_EQ(BAD_VALUE, mBQ->setMaxAcquiredBufferCount(
+ EXPECT_EQ(BAD_VALUE, mConsumer->setMaxAcquiredBufferCount(0));
+ EXPECT_EQ(BAD_VALUE, mConsumer->setMaxAcquiredBufferCount(-3));
+ EXPECT_EQ(BAD_VALUE, mConsumer->setMaxAcquiredBufferCount(
BufferQueue::MAX_MAX_ACQUIRED_BUFFERS+1));
- EXPECT_EQ(BAD_VALUE, mBQ->setMaxAcquiredBufferCount(100));
+ EXPECT_EQ(BAD_VALUE, mConsumer->setMaxAcquiredBufferCount(100));
}
TEST_F(BufferQueueTest, SetMaxAcquiredBufferCountWithLegalValues_Succeeds) {
sp<DummyConsumer> dc(new DummyConsumer);
- mBQ->consumerConnect(dc, false);
+ mConsumer->consumerConnect(dc, false);
int minBufferCount;
ASSERT_NO_FATAL_FAILURE(GetMinUndequeuedBufferCount(&minBufferCount));
- EXPECT_EQ(OK, mBQ->setMaxAcquiredBufferCount(1));
- EXPECT_EQ(OK, mBQ->setMaxAcquiredBufferCount(2));
- EXPECT_EQ(OK, mBQ->setMaxAcquiredBufferCount(minBufferCount));
- EXPECT_EQ(OK, mBQ->setMaxAcquiredBufferCount(
+ EXPECT_EQ(OK, mConsumer->setMaxAcquiredBufferCount(1));
+ EXPECT_EQ(OK, mConsumer->setMaxAcquiredBufferCount(2));
+ EXPECT_EQ(OK, mConsumer->setMaxAcquiredBufferCount(minBufferCount));
+ EXPECT_EQ(OK, mConsumer->setMaxAcquiredBufferCount(
BufferQueue::MAX_MAX_ACQUIRED_BUFFERS));
}
+TEST_F(BufferQueueTest, DetachAndReattachOnProducerSide) {
+ sp<DummyConsumer> dc(new DummyConsumer);
+ ASSERT_EQ(OK, mConsumer->consumerConnect(dc, false));
+ IGraphicBufferProducer::QueueBufferOutput output;
+ ASSERT_EQ(OK,
+ mProducer->connect(NULL, NATIVE_WINDOW_API_CPU, false, &output));
+
+ ASSERT_EQ(BAD_VALUE, mProducer->detachBuffer(-1)); // Index too low
+ ASSERT_EQ(BAD_VALUE, mProducer->detachBuffer(
+ BufferQueueDefs::NUM_BUFFER_SLOTS)); // Index too high
+ ASSERT_EQ(BAD_VALUE, mProducer->detachBuffer(0)); // Not dequeued
+
+ int slot;
+ sp<Fence> fence;
+ sp<GraphicBuffer> buffer;
+ ASSERT_EQ(IGraphicBufferProducer::BUFFER_NEEDS_REALLOCATION,
+ mProducer->dequeueBuffer(&slot, &fence, false, 0, 0, 0,
+ GRALLOC_USAGE_SW_WRITE_OFTEN));
+ ASSERT_EQ(BAD_VALUE, mProducer->detachBuffer(slot)); // Not requested
+ ASSERT_EQ(OK, mProducer->requestBuffer(slot, &buffer));
+ ASSERT_EQ(OK, mProducer->detachBuffer(slot));
+ ASSERT_EQ(BAD_VALUE, mProducer->detachBuffer(slot)); // Not dequeued
+
+ sp<GraphicBuffer> safeToClobberBuffer;
+ // Can no longer request buffer from this slot
+ ASSERT_EQ(BAD_VALUE, mProducer->requestBuffer(slot, &safeToClobberBuffer));
+
+ uint32_t* dataIn;
+ ASSERT_EQ(OK, buffer->lock(GraphicBuffer::USAGE_SW_WRITE_OFTEN,
+ reinterpret_cast<void**>(&dataIn)));
+ *dataIn = 0x12345678;
+ ASSERT_EQ(OK, buffer->unlock());
+
+ int newSlot;
+ ASSERT_EQ(BAD_VALUE, mProducer->attachBuffer(NULL, safeToClobberBuffer));
+ ASSERT_EQ(BAD_VALUE, mProducer->attachBuffer(&newSlot, NULL));
+
+ ASSERT_EQ(OK, mProducer->attachBuffer(&newSlot, buffer));
+ IGraphicBufferProducer::QueueBufferInput input(0, false, Rect(0, 0, 1, 1),
+ NATIVE_WINDOW_SCALING_MODE_FREEZE, 0, false, Fence::NO_FENCE);
+ ASSERT_EQ(OK, mProducer->queueBuffer(newSlot, input, &output));
+
+ IGraphicBufferConsumer::BufferItem item;
+ ASSERT_EQ(OK, mConsumer->acquireBuffer(&item, static_cast<nsecs_t>(0)));
+
+ uint32_t* dataOut;
+ ASSERT_EQ(OK, item.mGraphicBuffer->lock(GraphicBuffer::USAGE_SW_READ_OFTEN,
+ reinterpret_cast<void**>(&dataOut)));
+ ASSERT_EQ(*dataOut, 0x12345678);
+}
+
+TEST_F(BufferQueueTest, DetachAndReattachOnConsumerSide) {
+ sp<DummyConsumer> dc(new DummyConsumer);
+ ASSERT_EQ(OK, mConsumer->consumerConnect(dc, false));
+ IGraphicBufferProducer::QueueBufferOutput output;
+ ASSERT_EQ(OK,
+ mProducer->connect(NULL, NATIVE_WINDOW_API_CPU, false, &output));
+
+ int slot;
+ sp<Fence> fence;
+ sp<GraphicBuffer> buffer;
+ ASSERT_EQ(IGraphicBufferProducer::BUFFER_NEEDS_REALLOCATION,
+ mProducer->dequeueBuffer(&slot, &fence, false, 0, 0, 0,
+ GRALLOC_USAGE_SW_WRITE_OFTEN));
+ ASSERT_EQ(OK, mProducer->requestBuffer(slot, &buffer));
+ IGraphicBufferProducer::QueueBufferInput input(0, false, Rect(0, 0, 1, 1),
+ NATIVE_WINDOW_SCALING_MODE_FREEZE, 0, false, Fence::NO_FENCE);
+ ASSERT_EQ(OK, mProducer->queueBuffer(slot, input, &output));
+
+ ASSERT_EQ(BAD_VALUE, mConsumer->detachBuffer(-1)); // Index too low
+ ASSERT_EQ(BAD_VALUE, mConsumer->detachBuffer(
+ BufferQueueDefs::NUM_BUFFER_SLOTS)); // Index too high
+ ASSERT_EQ(BAD_VALUE, mConsumer->detachBuffer(0)); // Not acquired
+
+ IGraphicBufferConsumer::BufferItem item;
+ ASSERT_EQ(OK, mConsumer->acquireBuffer(&item, static_cast<nsecs_t>(0)));
+
+ ASSERT_EQ(OK, mConsumer->detachBuffer(item.mBuf));
+ ASSERT_EQ(BAD_VALUE, mConsumer->detachBuffer(item.mBuf)); // Not acquired
+
+ uint32_t* dataIn;
+ ASSERT_EQ(OK, item.mGraphicBuffer->lock(
+ GraphicBuffer::USAGE_SW_WRITE_OFTEN,
+ reinterpret_cast<void**>(&dataIn)));
+ *dataIn = 0x12345678;
+ ASSERT_EQ(OK, item.mGraphicBuffer->unlock());
+
+ int newSlot;
+ sp<GraphicBuffer> safeToClobberBuffer;
+ ASSERT_EQ(BAD_VALUE, mConsumer->attachBuffer(NULL, safeToClobberBuffer));
+ ASSERT_EQ(BAD_VALUE, mConsumer->attachBuffer(&newSlot, NULL));
+ ASSERT_EQ(OK, mConsumer->attachBuffer(&newSlot, item.mGraphicBuffer));
+
+ ASSERT_EQ(OK, mConsumer->releaseBuffer(item.mBuf, 0, EGL_NO_DISPLAY,
+ EGL_NO_SYNC_KHR, Fence::NO_FENCE));
+
+ ASSERT_EQ(IGraphicBufferProducer::BUFFER_NEEDS_REALLOCATION,
+ mProducer->dequeueBuffer(&slot, &fence, false, 0, 0, 0,
+ GRALLOC_USAGE_SW_WRITE_OFTEN));
+ ASSERT_EQ(OK, mProducer->requestBuffer(slot, &buffer));
+
+ uint32_t* dataOut;
+ ASSERT_EQ(OK, buffer->lock(GraphicBuffer::USAGE_SW_READ_OFTEN,
+ reinterpret_cast<void**>(&dataOut)));
+ ASSERT_EQ(*dataOut, 0x12345678);
+}
+
+TEST_F(BufferQueueTest, MoveFromConsumerToProducer) {
+ sp<DummyConsumer> dc(new DummyConsumer);
+ ASSERT_EQ(OK, mConsumer->consumerConnect(dc, false));
+ IGraphicBufferProducer::QueueBufferOutput output;
+ ASSERT_EQ(OK,
+ mProducer->connect(NULL, NATIVE_WINDOW_API_CPU, false, &output));
+
+ int slot;
+ sp<Fence> fence;
+ sp<GraphicBuffer> buffer;
+ ASSERT_EQ(IGraphicBufferProducer::BUFFER_NEEDS_REALLOCATION,
+ mProducer->dequeueBuffer(&slot, &fence, false, 0, 0, 0,
+ GRALLOC_USAGE_SW_WRITE_OFTEN));
+ ASSERT_EQ(OK, mProducer->requestBuffer(slot, &buffer));
+
+ uint32_t* dataIn;
+ ASSERT_EQ(OK, buffer->lock(GraphicBuffer::USAGE_SW_WRITE_OFTEN,
+ reinterpret_cast<void**>(&dataIn)));
+ *dataIn = 0x12345678;
+ ASSERT_EQ(OK, buffer->unlock());
+
+ IGraphicBufferProducer::QueueBufferInput input(0, false, Rect(0, 0, 1, 1),
+ NATIVE_WINDOW_SCALING_MODE_FREEZE, 0, false, Fence::NO_FENCE);
+ ASSERT_EQ(OK, mProducer->queueBuffer(slot, input, &output));
+
+ IGraphicBufferConsumer::BufferItem item;
+ ASSERT_EQ(OK, mConsumer->acquireBuffer(&item, static_cast<nsecs_t>(0)));
+ ASSERT_EQ(OK, mConsumer->detachBuffer(item.mBuf));
+
+ int newSlot;
+ ASSERT_EQ(OK, mProducer->attachBuffer(&newSlot, item.mGraphicBuffer));
+ ASSERT_EQ(OK, mProducer->queueBuffer(newSlot, input, &output));
+ ASSERT_EQ(OK, mConsumer->acquireBuffer(&item, static_cast<nsecs_t>(0)));
+
+ uint32_t* dataOut;
+ ASSERT_EQ(OK, item.mGraphicBuffer->lock(GraphicBuffer::USAGE_SW_READ_OFTEN,
+ reinterpret_cast<void**>(&dataOut)));
+ ASSERT_EQ(*dataOut, 0x12345678);
+}
+
} // namespace android
diff --git a/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.cpp b/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.cpp
index a1820ab..de0d16d 100644
--- a/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.cpp
+++ b/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.cpp
@@ -374,6 +374,15 @@
return result;
}
+status_t VirtualDisplaySurface::detachBuffer(int slot) {
+ return mSource[SOURCE_SINK]->detachBuffer(slot);
+}
+
+status_t VirtualDisplaySurface::attachBuffer(int* outSlot,
+ const sp<GraphicBuffer>& buffer) {
+ return mSource[SOURCE_SINK]->attachBuffer(outSlot, buffer);
+}
+
status_t VirtualDisplaySurface::queueBuffer(int pslot,
const QueueBufferInput& input, QueueBufferOutput* output) {
VDS_LOGW_IF(mDbgState != DBG_STATE_GLES,
diff --git a/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.h b/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.h
index 6899904..cd9a5b0 100644
--- a/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.h
+++ b/services/surfaceflinger/DisplayHardware/VirtualDisplaySurface.h
@@ -98,6 +98,8 @@
virtual status_t setBufferCount(int bufferCount);
virtual status_t dequeueBuffer(int* pslot, sp<Fence>* fence, bool async,
uint32_t w, uint32_t h, uint32_t format, uint32_t usage);
+ virtual status_t detachBuffer(int slot);
+ virtual status_t attachBuffer(int* slot, const sp<GraphicBuffer>& buffer);
virtual status_t queueBuffer(int pslot,
const QueueBufferInput& input, QueueBufferOutput* output);
virtual void cancelBuffer(int pslot, const sp<Fence>& fence);