Allow re-configuring DVFP in a (mostly) non-blocking manner

The first call to method `registerInputStream` doesn't block.

Later successive calls to the method blocks until the previous register call
finishes.

PiperOrigin-RevId: 559694490
This commit is contained in:
claincly 2023-08-24 02:53:42 -07:00 committed by Copybara-Service
parent 80495ddf9c
commit 930e538aca
15 changed files with 515 additions and 141 deletions

View File

@ -162,6 +162,9 @@ This release includes the following changes since
* Effect:
* Add `VideoFrameProcessor.queueInputBitmap(Bitmap, Iterator<Long>)`
queuing bitmap input by timestamp.
* Change `VideoFrameProcessor.registerInputStream()` to be non-blocking.
Apps must implement
`VideoFrameProcessor.Listener#onInputStreamRegistered()`.
* UI:
* Add a `Player.Listener` implementation for Wear OS devices that handles
playback suppression due to

View File

@ -111,6 +111,22 @@ public interface VideoFrameProcessor {
*/
interface Listener {
/**
* Called when the {@link VideoFrameProcessor} finishes {@linkplain #registerInputStream(int,
* List, FrameInfo) registering an input stream}.
*
* <p>The {@link VideoFrameProcessor} is now ready to accept new input {@linkplain
* VideoFrameProcessor#registerInputFrame frames}, {@linkplain
* VideoFrameProcessor#queueInputBitmap(Bitmap, TimestampIterator) bitmaps} or {@linkplain
* VideoFrameProcessor#queueInputTexture(int, long) textures}.
*
* @param inputType The {@link InputType} of the new input stream.
* @param effects The list of {@link Effect effects} to apply to the new input stream.
* @param frameInfo The {@link FrameInfo} of the new input stream.
*/
void onInputStreamRegistered(
@InputType int inputType, List<Effect> effects, FrameInfo frameInfo);
/**
* Called when the output size changes.
*
@ -161,10 +177,12 @@ public interface VideoFrameProcessor {
* @param inputBitmap The {@link Bitmap} queued to the {@code VideoFrameProcessor}.
* @param inStreamOffsetsUs The times within the current stream that the bitmap should be shown
* at. The timestamps should be monotonically increasing.
* @return Whether the {@link Bitmap} was successfully queued. A return value of {@code false}
* indicates the {@code VideoFrameProcessor} is not ready to accept input.
* @throws UnsupportedOperationException If the {@code VideoFrameProcessor} does not accept
* {@linkplain #INPUT_TYPE_BITMAP bitmap input}.
*/
void queueInputBitmap(Bitmap inputBitmap, TimestampIterator inStreamOffsetsUs);
boolean queueInputBitmap(Bitmap inputBitmap, TimestampIterator inStreamOffsetsUs);
/**
* Provides an input texture ID to the {@code VideoFrameProcessor}.
@ -176,8 +194,11 @@ public interface VideoFrameProcessor {
*
* @param textureId The ID of the texture queued to the {@code VideoFrameProcessor}.
* @param presentationTimeUs The presentation time of the queued texture, in microseconds.
* @return Whether the texture was successfully queued. A return value of {@code false} indicates
* the {@code VideoFrameProcessor} is not ready to accept input.
*/
void queueInputTexture(int textureId, long presentationTimeUs);
// TODO - b/294369303: Remove polling API.
boolean queueInputTexture(int textureId, long presentationTimeUs);
/**
* Sets the {@link OnInputFrameProcessedListener}.
@ -207,6 +228,17 @@ public interface VideoFrameProcessor {
* Informs the {@code VideoFrameProcessor} that a new input stream will be queued with the list of
* {@link Effect Effects} to apply to the new input stream.
*
* <p>After registering the first input stream, this method must only be called after the last
* frame of the already-registered input stream has been {@linkplain #registerInputFrame
* registered}, last bitmap {@link #queueInputBitmap queued} or last texture id {@linkplain
* #queueInputTexture queued}.
*
* <p>This method blocks the calling thread until the previous calls to this method finish, that
* is when {@link Listener#onInputStreamRegistered(int, List, FrameInfo)} is called after the
* underlying processing pipeline has been adapted to the registered input stream.
*
* <p>Can be called on any thread.
*
* @param inputType The {@link InputType} of the new input stream.
* @param effects The list of {@link Effect effects} to apply to the new input stream.
* @param frameInfo The {@link FrameInfo} of the new input stream.
@ -217,16 +249,22 @@ public interface VideoFrameProcessor {
* Informs the {@code VideoFrameProcessor} that a frame will be queued to its {@linkplain
* #getInputSurface() input surface}.
*
* <p>Must be called before rendering a frame to the input surface.
* <p>Must be called before rendering a frame to the input surface. The caller must not render
* frames to the {@linkplain #getInputSurface input surface} when {@code false} is returned.
*
* <p>Can be called on any thread.
*
* @return Whether the input frame was successfully registered. If {@link
* #registerInputStream(int, List, FrameInfo)} is called, this method returns {@code false}
* until {@link Listener#onInputStreamRegistered(int, List, FrameInfo)} is called. Otherwise,
* a return value of {@code false} indicates the {@code VideoFrameProcessor} is not ready to
* accept input.
* @throws UnsupportedOperationException If the {@code VideoFrameProcessor} does not accept
* {@linkplain #INPUT_TYPE_SURFACE surface input}.
* @throws IllegalStateException If called after {@link #signalEndOfInput()} or before {@link
* #registerInputStream}.
*/
void registerInputFrame();
boolean registerInputFrame();
/**
* Returns the number of input frames that have been made available to the {@code

View File

@ -168,32 +168,6 @@ public class DefaultVideoFrameProcessorImageFrameOutputTest {
.inOrder();
}
@Test
@RequiresNonNull({"framesProduced", "testId"})
public void
imageInput_queueEndAndQueueAgain_outputsFirstSetOfFramesOnlyAtTheCorrectPresentationTimesUs()
throws Exception {
Queue<Long> actualPresentationTimesUs = new ConcurrentLinkedQueue<>();
videoFrameProcessorTestRunner =
getDefaultFrameProcessorTestRunnerBuilder(testId)
.setOnOutputFrameAvailableForRenderingListener(actualPresentationTimesUs::add)
.build();
videoFrameProcessorTestRunner.queueInputBitmap(
readBitmap(ORIGINAL_PNG_ASSET_PATH),
/* durationUs= */ C.MICROS_PER_SECOND,
/* offsetToAddUs= */ 0L,
/* frameRate= */ 2);
videoFrameProcessorTestRunner.endFrameProcessing();
videoFrameProcessorTestRunner.queueInputBitmap(
readBitmap(ORIGINAL_PNG_ASSET_PATH),
/* durationUs= */ 2 * C.MICROS_PER_SECOND,
/* offsetToAddUs= */ 0L,
/* frameRate= */ 3);
assertThat(actualPresentationTimesUs).containsExactly(0L, C.MICROS_PER_SECOND / 2).inOrder();
}
@Test
@RequiresNonNull({"framesProduced", "testId"})
public void queueBitmapsWithTimestamps_outputsFramesAtTheCorrectPresentationTimesUs()

View File

@ -0,0 +1,230 @@
/*
* Copyright 2023 The Android Open Source Project
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* https://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.
*/
package androidx.media3.effect;
import static androidx.media3.common.util.Assertions.checkNotNull;
import static androidx.test.core.app.ApplicationProvider.getApplicationContext;
import static com.google.common.truth.Truth.assertThat;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import androidx.annotation.Nullable;
import androidx.media3.common.ColorInfo;
import androidx.media3.common.DebugViewProvider;
import androidx.media3.common.Effect;
import androidx.media3.common.FrameInfo;
import androidx.media3.common.VideoFrameProcessingException;
import androidx.media3.common.VideoFrameProcessor;
import androidx.media3.common.util.Util;
import androidx.test.ext.junit.runners.AndroidJUnit4;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.MoreExecutors;
import java.util.List;
import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
/** Test for {@link DefaultVideoFrameProcessor}. */
@RunWith(AndroidJUnit4.class)
public class DefaultVideoFrameProcessorTest {
private static final long INPUT_REGISTRATION_TIMEOUT_MS = 1_000L;
private DefaultVideoFrameProcessor.@MonotonicNonNull Factory factory;
@Before
public void setUp() {
factory = new DefaultVideoFrameProcessor.Factory.Builder().build();
}
@Test
public void registerInputStream_withBlockingVideoFrameProcessorConfiguration_succeeds()
throws Exception {
AtomicReference<Exception> videoFrameProcessingException = new AtomicReference<>();
CountDownLatch inputStreamRegisteredCountDownLatch = new CountDownLatch(1);
DefaultVideoFrameProcessor defaultVideoFrameProcessor =
createDefaultVideoFrameProcessor(
new VideoFrameProcessor.Listener() {
@Override
public void onInputStreamRegistered(
@VideoFrameProcessor.InputType int inputType,
List<Effect> effects,
FrameInfo frameInfo) {
inputStreamRegisteredCountDownLatch.countDown();
}
@Override
public void onOutputSizeChanged(int width, int height) {}
@Override
public void onOutputFrameAvailableForRendering(long presentationTimeUs) {}
@Override
public void onError(VideoFrameProcessingException exception) {
videoFrameProcessingException.set(exception);
}
@Override
public void onEnded() {}
});
CountDownLatch videoFrameProcessorConfigurationCountDownLatch = new CountDownLatch(1);
// Blocks VideoFrameProcessor configuration.
defaultVideoFrameProcessor
.getTaskExecutor()
.submit(
() -> {
try {
videoFrameProcessorConfigurationCountDownLatch.await();
} catch (InterruptedException e) {
throw new VideoFrameProcessingException(e);
}
});
defaultVideoFrameProcessor.registerInputStream(
VideoFrameProcessor.INPUT_TYPE_BITMAP,
ImmutableList.of(),
new FrameInfo.Builder(/* width= */ 100, /* height= */ 100).build());
assertThat(defaultVideoFrameProcessor.getPendingInputFrameCount()).isEqualTo(0);
// Unblocks configuration.
videoFrameProcessorConfigurationCountDownLatch.countDown();
assertThat(
inputStreamRegisteredCountDownLatch.await(INPUT_REGISTRATION_TIMEOUT_MS, MILLISECONDS))
.isTrue();
assertThat(videoFrameProcessingException.get()).isNull();
}
@Test
public void
registerInputStream_threeTimesConsecutively_onInputStreamRegisteredIsInvokedCorrectly()
throws Exception {
AtomicReference<Exception> videoFrameProcessingException = new AtomicReference<>();
CountDownLatch countDownLatch = new CountDownLatch(3);
Queue<InputStreamInfo> registeredInputStreamInfoWidths = new ConcurrentLinkedQueue<>();
DefaultVideoFrameProcessor defaultVideoFrameProcessor =
createDefaultVideoFrameProcessor(
new VideoFrameProcessor.Listener() {
@Override
public void onInputStreamRegistered(
@VideoFrameProcessor.InputType int inputType,
List<Effect> effects,
FrameInfo frameInfo) {
registeredInputStreamInfoWidths.add(
new InputStreamInfo(inputType, effects, frameInfo));
countDownLatch.countDown();
}
@Override
public void onOutputSizeChanged(int width, int height) {}
@Override
public void onOutputFrameAvailableForRendering(long presentationTimeUs) {}
@Override
public void onError(VideoFrameProcessingException exception) {
videoFrameProcessingException.set(exception);
}
@Override
public void onEnded() {}
});
InputStreamInfo stream1 =
new InputStreamInfo(
VideoFrameProcessor.INPUT_TYPE_BITMAP,
ImmutableList.of(),
new FrameInfo.Builder(/* width= */ 100, /* height= */ 100).build());
InputStreamInfo stream2 =
new InputStreamInfo(
VideoFrameProcessor.INPUT_TYPE_BITMAP,
ImmutableList.of(new Contrast(.5f)),
new FrameInfo.Builder(/* width= */ 200, /* height= */ 200).build());
InputStreamInfo stream3 =
new InputStreamInfo(
VideoFrameProcessor.INPUT_TYPE_BITMAP,
ImmutableList.of(),
new FrameInfo.Builder(/* width= */ 300, /* height= */ 300).build());
registerInputStream(defaultVideoFrameProcessor, stream1);
registerInputStream(defaultVideoFrameProcessor, stream2);
registerInputStream(defaultVideoFrameProcessor, stream3);
assertThat(countDownLatch.await(INPUT_REGISTRATION_TIMEOUT_MS, MILLISECONDS)).isTrue();
assertThat(videoFrameProcessingException.get()).isNull();
assertThat(registeredInputStreamInfoWidths)
.containsExactly(stream1, stream2, stream3)
.inOrder();
}
private DefaultVideoFrameProcessor createDefaultVideoFrameProcessor(
VideoFrameProcessor.Listener listener) throws Exception {
return checkNotNull(factory)
.create(
getApplicationContext(),
DebugViewProvider.NONE,
/* inputColorInfo= */ ColorInfo.SDR_BT709_LIMITED,
/* outputColorInfo= */ ColorInfo.SDR_BT709_LIMITED,
/* renderFramesAutomatically= */ true,
/* listenerExecutor= */ MoreExecutors.directExecutor(),
listener);
}
private static void registerInputStream(
DefaultVideoFrameProcessor defaultVideoFrameProcessor, InputStreamInfo inputStreamInfo) {
defaultVideoFrameProcessor.registerInputStream(
inputStreamInfo.inputType, inputStreamInfo.effects, inputStreamInfo.frameInfo);
}
private static final class InputStreamInfo {
public final @VideoFrameProcessor.InputType int inputType;
public final List<Effect> effects;
public final FrameInfo frameInfo;
private InputStreamInfo(
@VideoFrameProcessor.InputType int inputType, List<Effect> effects, FrameInfo frameInfo) {
this.inputType = inputType;
this.effects = effects;
this.frameInfo = frameInfo;
}
@Override
public boolean equals(@Nullable Object o) {
if (this == o) {
return true;
}
if (!(o instanceof InputStreamInfo)) {
return false;
}
InputStreamInfo that = (InputStreamInfo) o;
return inputType == that.inputType
&& Util.areEqual(this.effects, that.effects)
&& Util.areEqual(this.frameInfo, that.frameInfo);
}
@Override
public int hashCode() {
int result = 17;
result = 31 * result + inputType;
result = 31 * result + effects.hashCode();
result = 31 * result + frameInfo.hashCode();
return result;
}
}
}

View File

@ -27,6 +27,7 @@ import android.media.ImageReader;
import androidx.annotation.Nullable;
import androidx.media3.common.ColorInfo;
import androidx.media3.common.DebugViewProvider;
import androidx.media3.common.Effect;
import androidx.media3.common.FrameInfo;
import androidx.media3.common.SurfaceInfo;
import androidx.media3.common.VideoFrameProcessingException;
@ -282,6 +283,7 @@ public final class DefaultVideoFrameProcessorVideoFrameRenderingTest {
AtomicReference<@NullableType VideoFrameProcessingException>
videoFrameProcessingExceptionReference = new AtomicReference<>();
BlankFrameProducer blankFrameProducer = new BlankFrameProducer(WIDTH, HEIGHT);
CountDownLatch videoFrameProcessorReadyCountDownLatch = new CountDownLatch(1);
CountDownLatch videoFrameProcessingEndedCountDownLatch = new CountDownLatch(1);
defaultVideoFrameProcessor =
checkNotNull(
@ -295,6 +297,14 @@ public final class DefaultVideoFrameProcessorVideoFrameRenderingTest {
renderFramesAutomatically,
MoreExecutors.directExecutor(),
new VideoFrameProcessor.Listener() {
@Override
public void onInputStreamRegistered(
@VideoFrameProcessor.InputType int inputType,
List<Effect> effects,
FrameInfo frameInfo) {
videoFrameProcessorReadyCountDownLatch.countDown();
}
@Override
public void onOutputSizeChanged(int width, int height) {
ImageReader outputImageReader =
@ -340,6 +350,7 @@ public final class DefaultVideoFrameProcessorVideoFrameRenderingTest {
INPUT_TYPE_SURFACE,
/* effects= */ ImmutableList.of((GlEffect) (context, useHdr) -> blankFrameProducer),
new FrameInfo.Builder(WIDTH, HEIGHT).build());
videoFrameProcessorReadyCountDownLatch.await();
blankFrameProducer.produceBlankFrames(inputPresentationTimesUs);
defaultVideoFrameProcessor.signalEndOfInput();
videoFrameProcessingEndedCountDownLatch.await();

View File

@ -34,6 +34,7 @@ import android.text.style.TypefaceSpan;
import androidx.annotation.Nullable;
import androidx.media3.common.ColorInfo;
import androidx.media3.common.DebugViewProvider;
import androidx.media3.common.Effect;
import androidx.media3.common.FrameInfo;
import androidx.media3.common.VideoFrameProcessingException;
import androidx.media3.common.VideoFrameProcessor;
@ -153,6 +154,7 @@ public class FrameDropTest {
videoFrameProcessingExceptionReference = new AtomicReference<>();
BlankFrameProducer blankFrameProducer =
new BlankFrameProducer(BLANK_FRAME_WIDTH, BLANK_FRAME_HEIGHT);
CountDownLatch videoFrameProcessorReadyCountDownLatch = new CountDownLatch(1);
CountDownLatch videoFrameProcessingEndedCountDownLatch = new CountDownLatch(1);
ImmutableList.Builder<Long> actualPresentationTimesUs = new ImmutableList.Builder<>();
@ -175,6 +177,14 @@ public class FrameDropTest {
/* renderFramesAutomatically= */ true,
MoreExecutors.directExecutor(),
new VideoFrameProcessor.Listener() {
@Override
public void onInputStreamRegistered(
@VideoFrameProcessor.InputType int inputType,
List<Effect> effects,
FrameInfo frameInfo) {
videoFrameProcessorReadyCountDownLatch.countDown();
}
@Override
public void onOutputSizeChanged(int width, int height) {}
@ -231,6 +241,7 @@ public class FrameDropTest {
})),
frameDropEffect),
new FrameInfo.Builder(BLANK_FRAME_WIDTH, BLANK_FRAME_HEIGHT).build());
videoFrameProcessorReadyCountDownLatch.await();
blankFrameProducer.produceBlankFrames(inputPresentationTimesUs);
defaultVideoFrameProcessor.signalEndOfInput();
videoFrameProcessingEndedCountDownLatch.await();

View File

@ -20,7 +20,6 @@ import static androidx.media3.common.util.Assertions.checkArgument;
import static androidx.media3.common.util.Assertions.checkNotNull;
import static androidx.media3.common.util.Assertions.checkState;
import static androidx.media3.common.util.Assertions.checkStateNotNull;
import static androidx.media3.effect.DebugTraceUtil.EVENT_VFP_FINISH_PROCESSING_INPUT_STREAM;
import static androidx.media3.effect.DebugTraceUtil.EVENT_VFP_RECEIVE_END_OF_INPUT;
import static androidx.media3.effect.DebugTraceUtil.EVENT_VFP_REGISTER_NEW_INPUT_STREAM;
import static androidx.media3.effect.DebugTraceUtil.EVENT_VFP_SIGNAL_ENDED;
@ -50,6 +49,7 @@ import androidx.media3.common.OnInputFrameProcessedListener;
import androidx.media3.common.SurfaceInfo;
import androidx.media3.common.VideoFrameProcessingException;
import androidx.media3.common.VideoFrameProcessor;
import androidx.media3.common.util.ConditionVariable;
import androidx.media3.common.util.GlUtil;
import androidx.media3.common.util.Log;
import androidx.media3.common.util.TimestampIterator;
@ -336,17 +336,23 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
// Shader programs that apply Effects.
private final List<GlShaderProgram> intermediateGlShaderPrograms;
private final ConditionVariable inputStreamRegisteredCondition;
// Whether DefaultVideoFrameProcessor is currently processing an input stream.
/**
* The input stream that is {@linkplain #registerInputStream(int, List, FrameInfo) registered},
* but the pipeline has not adapted to processing it.
*/
@GuardedBy("lock")
private boolean processingInput;
@Nullable
private InputStreamInfo pendingInputStreamInfo;
@GuardedBy("lock")
private boolean registeredFirstInputStream;
private final List<Effect> activeEffects;
private final Object lock;
private final ColorInfo outputColorInfo;
// CountDownLatch to wait for the current input stream to finish processing.
private volatile @MonotonicNonNull CountDownLatch latch;
private volatile @MonotonicNonNull FrameInfo nextInputFrameInfo;
private volatile boolean inputStreamEnded;
@ -375,24 +381,27 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
this.lock = new Object();
this.outputColorInfo = outputColorInfo;
this.finalShaderProgramWrapper = finalShaderProgramWrapper;
finalShaderProgramWrapper.setOnInputStreamProcessedListener(
this.intermediateGlShaderPrograms = new ArrayList<>();
this.inputStreamRegisteredCondition = new ConditionVariable();
inputStreamRegisteredCondition.open();
this.finalShaderProgramWrapper.setOnInputStreamProcessedListener(
() -> {
logEvent(EVENT_VFP_FINISH_PROCESSING_INPUT_STREAM, C.TIME_END_OF_SOURCE);
boolean inputEndedAfterThisInputStream;
synchronized (lock) {
processingInput = false;
// inputStreamEnded could be overwritten right after counting down the latch.
inputEndedAfterThisInputStream = this.inputStreamEnded;
if (latch != null) {
latch.countDown();
}
}
if (inputEndedAfterThisInputStream) {
if (inputStreamEnded) {
listenerExecutor.execute(listener::onEnded);
logEvent(EVENT_VFP_SIGNAL_ENDED, C.TIME_END_OF_SOURCE);
} else {
synchronized (lock) {
if (pendingInputStreamInfo != null) {
InputStreamInfo pendingInputStreamInfo = this.pendingInputStreamInfo;
videoFrameProcessingTaskExecutor.submit(
() -> {
configureEffects(pendingInputStreamInfo, /* forceReconfigure= */ false);
});
this.pendingInputStreamInfo = null;
}
}
}
});
this.intermediateGlShaderPrograms = new ArrayList<>();
}
/** Returns the task executor that runs video frame processing tasks. */
@ -417,11 +426,14 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
* @param height The default height for input buffers, in pixels.
*/
public void setInputDefaultBufferSize(int width, int height) {
inputSwitcher.activeTextureManager().setDefaultBufferSize(width, height);
inputSwitcher.setInputDefaultBufferSize(width, height);
}
@Override
public void queueInputBitmap(Bitmap inputBitmap, TimestampIterator inStreamOffsetsUs) {
public boolean queueInputBitmap(Bitmap inputBitmap, TimestampIterator inStreamOffsetsUs) {
if (!inputStreamRegisteredCondition.isOpen()) {
return false;
}
FrameInfo frameInfo = checkNotNull(this.nextInputFrameInfo);
inputSwitcher
.activeTextureManager()
@ -430,16 +442,21 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
new FrameInfo.Builder(frameInfo).setOffsetToAddUs(frameInfo.offsetToAddUs).build(),
inStreamOffsetsUs,
/* useHdr= */ false);
return true;
}
@Override
public void queueInputTexture(int textureId, long presentationTimeUs) {
public boolean queueInputTexture(int textureId, long presentationTimeUs) {
if (!inputStreamRegisteredCondition.isOpen()) {
return false;
}
inputSwitcher.activeTextureManager().queueInputTexture(textureId, presentationTimeUs);
return true;
}
@Override
public void setOnInputFrameProcessedListener(OnInputFrameProcessedListener listener) {
inputSwitcher.activeTextureManager().setOnInputFrameProcessedListener(listener);
inputSwitcher.setOnInputFrameProcessedListener(listener);
}
@Override
@ -450,6 +467,8 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
@Override
public void registerInputStream(
@InputType int inputType, List<Effect> effects, FrameInfo frameInfo) {
// This method is only called after all samples in the current input stream are registered or
// queued.
logEvent(
EVENT_VFP_REGISTER_NEW_INPUT_STREAM,
/* presentationTimeUs= */ frameInfo.offsetToAddUs,
@ -457,52 +476,54 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
"InputType %s - %dx%d",
getInputTypeString(inputType), frameInfo.width, frameInfo.height));
nextInputFrameInfo = adjustForPixelWidthHeightRatio(frameInfo);
synchronized (lock) {
if (!processingInput) {
videoFrameProcessingTaskExecutor.submitAndBlock(() -> configureEffects(effects));
inputSwitcher.switchToInput(inputType, nextInputFrameInfo);
inputSwitcher.activeTextureManager().setInputFrameInfo(nextInputFrameInfo);
processingInput = true;
return;
}
}
// Wait until the current input stream is processed before continuing to the next input.
latch = new CountDownLatch(1);
inputSwitcher.activeTextureManager().signalEndOfCurrentInputStream();
try {
latch.await();
// Blocks until the previous input stream registration completes.
// TODO: b/296897956 - Handle multiple thread unblocking at the same time.
inputStreamRegisteredCondition.block();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
listenerExecutor.execute(() -> listener.onError(VideoFrameProcessingException.from(e)));
}
synchronized (lock) {
processingInput = true;
// An input stream is pending until its effects are configured.
InputStreamInfo pendingInputStreamInfo = new InputStreamInfo(inputType, effects, frameInfo);
if (!registeredFirstInputStream) {
registeredFirstInputStream = true;
inputStreamRegisteredCondition.close();
videoFrameProcessingTaskExecutor.submit(
() -> configureEffects(pendingInputStreamInfo, /* forceReconfigure= */ true));
} else {
// Rejects further inputs after signaling EOS and before the next input stream is fully
// configured.
this.pendingInputStreamInfo = pendingInputStreamInfo;
inputStreamRegisteredCondition.close();
inputSwitcher.activeTextureManager().signalEndOfCurrentInputStream();
}
}
if (!activeEffects.equals(effects)) {
// TODO(b/269424561) Investigate non blocking re-configuration.
// Shader program recreation must be on GL thread. Currently the calling thread is blocked
// until all shader programs are recreated, so that DefaultVideoFrameProcessor doesn't receive
// a new frame from the new input stream prematurely.
videoFrameProcessingTaskExecutor.submitAndBlock(() -> configureEffects(effects));
}
inputSwitcher.switchToInput(inputType, nextInputFrameInfo);
}
@Override
public void registerInputFrame() {
public boolean registerInputFrame() {
checkState(!inputStreamEnded);
checkStateNotNull(
nextInputFrameInfo, "registerInputStream must be called before registering input frames");
if (!inputStreamRegisteredCondition.isOpen()) {
return false;
}
inputSwitcher.activeTextureManager().registerInputFrame(nextInputFrameInfo);
return true;
}
@Override
public int getPendingInputFrameCount() {
return inputSwitcher.activeTextureManager().getPendingFrameCount();
if (inputSwitcher.hasActiveInput()) {
return inputSwitcher.activeTextureManager().getPendingFrameCount();
}
// Return zero when InputSwitcher is not set up, i.e. before VideoFrameProcessor finishes its
// first configuration.
return 0;
}
/**
@ -536,7 +557,7 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
logEvent(EVENT_VFP_RECEIVE_END_OF_INPUT, C.TIME_END_OF_SOURCE);
checkState(!inputStreamEnded);
inputStreamEnded = true;
inputSwitcher.signalEndOfCurrentInputStream();
inputSwitcher.signalEndOfInputStream();
}
@Override
@ -792,31 +813,48 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
}
}
/** Configures the {@link GlShaderProgram} instances for {@code effects}. */
private void configureEffects(List<Effect> effects) throws VideoFrameProcessingException {
if (!intermediateGlShaderPrograms.isEmpty()) {
for (int i = 0; i < intermediateGlShaderPrograms.size(); i++) {
intermediateGlShaderPrograms.get(i).release();
/**
* Configures the {@link GlShaderProgram} instances for {@code effects}.
*
* <p>The pipeline will only re-configure if the {@link InputStreamInfo#effects new effects}
* doesn't match the {@link #activeEffects}, or when {@code forceReconfigure} is set to {@code
* true}.
*/
private void configureEffects(InputStreamInfo inputStreamInfo, boolean forceReconfigure)
throws VideoFrameProcessingException {
if (forceReconfigure || !activeEffects.equals(inputStreamInfo.effects)) {
if (!intermediateGlShaderPrograms.isEmpty()) {
for (int i = 0; i < intermediateGlShaderPrograms.size(); i++) {
intermediateGlShaderPrograms.get(i).release();
}
intermediateGlShaderPrograms.clear();
}
intermediateGlShaderPrograms.clear();
// The GlShaderPrograms that should be inserted in between InputSwitcher and
// FinalShaderProgramWrapper.
intermediateGlShaderPrograms.addAll(
createGlShaderPrograms(
context, inputStreamInfo.effects, outputColorInfo, finalShaderProgramWrapper));
inputSwitcher.setDownstreamShaderProgram(
getFirst(intermediateGlShaderPrograms, /* defaultValue= */ finalShaderProgramWrapper));
chainShaderProgramsWithListeners(
glObjectsProvider,
intermediateGlShaderPrograms,
finalShaderProgramWrapper,
videoFrameProcessingTaskExecutor,
listener,
listenerExecutor);
activeEffects.clear();
activeEffects.addAll(inputStreamInfo.effects);
}
// The GlShaderPrograms that should be inserted in between InputSwitcher and
// FinalShaderProgramWrapper.
intermediateGlShaderPrograms.addAll(
createGlShaderPrograms(context, effects, outputColorInfo, finalShaderProgramWrapper));
inputSwitcher.setDownstreamShaderProgram(
getFirst(intermediateGlShaderPrograms, /* defaultValue= */ finalShaderProgramWrapper));
chainShaderProgramsWithListeners(
glObjectsProvider,
intermediateGlShaderPrograms,
finalShaderProgramWrapper,
videoFrameProcessingTaskExecutor,
listener,
listenerExecutor);
activeEffects.clear();
activeEffects.addAll(effects);
inputSwitcher.switchToInput(inputStreamInfo.inputType, inputStreamInfo.frameInfo);
inputStreamRegisteredCondition.open();
listenerExecutor.execute(
() ->
listener.onInputStreamRegistered(
inputStreamInfo.inputType, inputStreamInfo.effects, inputStreamInfo.frameInfo));
}
/**
@ -842,4 +880,16 @@ public final class DefaultVideoFrameProcessor implements VideoFrameProcessor {
}
}
}
private static final class InputStreamInfo {
public final @InputType int inputType;
public final List<Effect> effects;
public final FrameInfo frameInfo;
public InputStreamInfo(@InputType int inputType, List<Effect> effects, FrameInfo frameInfo) {
this.inputType = inputType;
this.effects = effects;
this.frameInfo = frameInfo;
}
}
}

View File

@ -32,6 +32,7 @@ import androidx.media3.common.ColorInfo;
import androidx.media3.common.FrameInfo;
import androidx.media3.common.GlObjectsProvider;
import androidx.media3.common.GlTextureInfo;
import androidx.media3.common.OnInputFrameProcessedListener;
import androidx.media3.common.VideoFrameProcessingException;
import androidx.media3.common.VideoFrameProcessor;
import com.google.common.collect.ImmutableList;
@ -190,20 +191,26 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
checkNotNull(activeTextureManager).setInputFrameInfo(inputFrameInfo);
}
/** Returns whether the {@code InputSwitcher} is connected to an active input. */
public boolean hasActiveInput() {
return activeTextureManager != null;
}
/**
* Returns the {@link TextureManager} that is currently being used.
*
* <p>Must call {@link #switchToInput} before calling this method.
* @throws IllegalStateException If the {@code InputSwitcher} is not connected to an {@linkplain
* #hasActiveInput() input}.
*/
public TextureManager activeTextureManager() {
return checkNotNull(activeTextureManager);
return checkStateNotNull(activeTextureManager);
}
/**
* Invokes {@link TextureManager#signalEndOfCurrentInputStream} on the active {@link
* TextureManager}.
*/
public void signalEndOfCurrentInputStream() {
public void signalEndOfInputStream() {
checkNotNull(activeTextureManager).signalEndOfCurrentInputStream();
}
@ -220,6 +227,28 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
return inputs.get(INPUT_TYPE_SURFACE).textureManager.getInputSurface();
}
/**
* See {@link DefaultVideoFrameProcessor#setInputDefaultBufferSize}.
*
* @throws IllegalStateException If {@link VideoFrameProcessor#INPUT_TYPE_SURFACE} is not
* {@linkplain #registerInput registered}.
*/
public void setInputDefaultBufferSize(int width, int height) {
checkState(containsKey(inputs, INPUT_TYPE_SURFACE));
inputs.get(INPUT_TYPE_SURFACE).textureManager.setDefaultBufferSize(width, height);
}
/**
* Sets the {@link OnInputFrameProcessedListener}.
*
* @throws IllegalStateException If {@link VideoFrameProcessor#INPUT_TYPE_TEXTURE_ID} is not
* {@linkplain #registerInput registered}.
*/
public void setOnInputFrameProcessedListener(OnInputFrameProcessedListener listener) {
checkState(containsKey(inputs, INPUT_TYPE_TEXTURE_ID));
inputs.get(INPUT_TYPE_TEXTURE_ID).textureManager.setOnInputFrameProcessedListener(listener);
}
/** Releases the resources. */
public void release() throws VideoFrameProcessingException {
for (int i = 0; i < inputs.size(); i++) {

View File

@ -25,7 +25,6 @@ import androidx.media3.common.util.UnstableApi;
import java.util.ArrayDeque;
import java.util.Queue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
@ -109,25 +108,6 @@ import java.util.concurrent.RejectedExecutionException;
}
}
/** Submits the given {@link Task} to execute, and returns after the task is executed. */
public void submitAndBlock(Task task) {
synchronized (lock) {
if (shouldCancelTasks) {
return;
}
}
Future<?> future = wrapTaskAndSubmitToExecutorService(task, /* isFlushOrReleaseTask= */ false);
try {
future.get();
} catch (ExecutionException e) {
handleException(e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
handleException(e);
}
}
/**
* Submits the given {@link Task} to be executed after the currently running task and all
* previously submitted high-priority tasks have completed.

View File

@ -334,7 +334,9 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
>= videoFrameProcessorMaxPendingFrameCount) {
return C.TIME_UNSET;
}
videoFrameProcessor.registerInputFrame();
if (!videoFrameProcessor.registerInputFrame()) {
return C.TIME_UNSET;
}
// The sink takes in frames with monotonically increasing, non-offset frame
// timestamps. That is, with two ten-second long videos, the first frame of the second video
// should bear a timestamp of 10s seen from VideoFrameProcessor; while in ExoPlayer, the
@ -407,6 +409,12 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
// VideoFrameProcessor.Listener impl
@Override
public void onInputStreamRegistered(
@VideoFrameProcessor.InputType int inputType, List<Effect> effects, FrameInfo frameInfo) {
// Do nothing.
}
@Override
public void onOutputSizeChanged(int width, int height) {
VideoSize newVideoSize = new VideoSize(width, height);

View File

@ -17,6 +17,7 @@ package androidx.media3.exoplayer.video;
import static com.google.common.truth.Truth.assertThat;
import static org.junit.Assert.assertThrows;
import static org.mockito.Mockito.when;
import android.content.Context;
import androidx.media3.common.ColorInfo;
@ -148,6 +149,7 @@ public final class CompositingVideoSinkProviderTest {
Executor listenerExecutor,
VideoFrameProcessor.Listener listener)
throws VideoFrameProcessingException {
when(videoFrameProcessor.registerInputFrame()).thenReturn(true);
return videoFrameProcessor;
}
}

View File

@ -19,6 +19,7 @@ import static androidx.media3.common.VideoFrameProcessor.INPUT_TYPE_BITMAP;
import static androidx.media3.common.VideoFrameProcessor.INPUT_TYPE_SURFACE;
import static androidx.media3.common.VideoFrameProcessor.INPUT_TYPE_TEXTURE_ID;
import static androidx.media3.common.util.Assertions.checkNotNull;
import static androidx.media3.common.util.Assertions.checkState;
import static androidx.media3.common.util.Assertions.checkStateNotNull;
import static androidx.media3.test.utils.BitmapPixelTestUtil.createArgb8888BitmapFromRgba8888Image;
import static androidx.media3.test.utils.BitmapPixelTestUtil.maybeSaveTestBitmap;
@ -44,6 +45,7 @@ import androidx.media3.common.GlTextureInfo;
import androidx.media3.common.SurfaceInfo;
import androidx.media3.common.VideoFrameProcessingException;
import androidx.media3.common.VideoFrameProcessor;
import androidx.media3.common.util.ConditionVariable;
import androidx.media3.common.util.ConstantRateTimestampIterator;
import androidx.media3.common.util.GlUtil;
import androidx.media3.common.util.TimestampIterator;
@ -251,6 +253,7 @@ public final class VideoFrameProcessorTestRunner {
private final @MonotonicNonNull String videoAssetPath;
private final String outputFileLabel;
private final float pixelWidthHeightRatio;
private final ConditionVariable videoFrameProcessorReadyCondition;
private final @MonotonicNonNull CountDownLatch videoFrameProcessingEndedLatch;
private final AtomicReference<VideoFrameProcessingException> videoFrameProcessingException;
private final VideoFrameProcessor videoFrameProcessor;
@ -275,6 +278,7 @@ public final class VideoFrameProcessorTestRunner {
this.videoAssetPath = videoAssetPath;
this.outputFileLabel = outputFileLabel;
this.pixelWidthHeightRatio = pixelWidthHeightRatio;
videoFrameProcessorReadyCondition = new ConditionVariable();
videoFrameProcessingEndedLatch = new CountDownLatch(1);
videoFrameProcessingException = new AtomicReference<>();
@ -287,6 +291,14 @@ public final class VideoFrameProcessorTestRunner {
/* renderFramesAutomatically= */ true,
MoreExecutors.directExecutor(),
new VideoFrameProcessor.Listener() {
@Override
public void onInputStreamRegistered(
@VideoFrameProcessor.InputType int inputType,
List<Effect> effects,
FrameInfo frameInfo) {
videoFrameProcessorReadyCondition.open();
}
@Override
public void onOutputSizeChanged(int width, int height) {
boolean useHighPrecisionColorComponents = ColorInfo.isTransferHdr(outputColorInfo);
@ -327,6 +339,7 @@ public final class VideoFrameProcessorTestRunner {
new DecodeOneFrameUtil.Listener() {
@Override
public void onContainerExtracted(MediaFormat mediaFormat) {
videoFrameProcessorReadyCondition.close();
videoFrameProcessor.registerInputStream(
INPUT_TYPE_SURFACE,
effects,
@ -335,7 +348,13 @@ public final class VideoFrameProcessorTestRunner {
mediaFormat.getInteger(MediaFormat.KEY_HEIGHT))
.setPixelWidthHeightRatio(pixelWidthHeightRatio)
.build());
videoFrameProcessor.registerInputFrame();
try {
videoFrameProcessorReadyCondition.block();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new IllegalStateException(e);
}
checkState(videoFrameProcessor.registerInputFrame());
}
@Override
@ -348,7 +367,9 @@ public final class VideoFrameProcessorTestRunner {
}
public void queueInputBitmap(
Bitmap inputBitmap, long durationUs, long offsetToAddUs, float frameRate) {
Bitmap inputBitmap, long durationUs, long offsetToAddUs, float frameRate)
throws InterruptedException {
videoFrameProcessorReadyCondition.close();
videoFrameProcessor.registerInputStream(
INPUT_TYPE_BITMAP,
effects,
@ -356,23 +377,28 @@ public final class VideoFrameProcessorTestRunner {
.setPixelWidthHeightRatio(pixelWidthHeightRatio)
.setOffsetToAddUs(offsetToAddUs)
.build());
videoFrameProcessor.queueInputBitmap(
inputBitmap, new ConstantRateTimestampIterator(durationUs, frameRate));
videoFrameProcessorReadyCondition.block();
checkState(
videoFrameProcessor.queueInputBitmap(
inputBitmap, new ConstantRateTimestampIterator(durationUs, frameRate)));
}
public void queueInputBitmaps(int width, int height, Pair<Bitmap, TimestampIterator>... frames) {
public void queueInputBitmaps(int width, int height, Pair<Bitmap, TimestampIterator>... frames)
throws InterruptedException {
videoFrameProcessorReadyCondition.close();
videoFrameProcessor.registerInputStream(
INPUT_TYPE_BITMAP,
effects,
new FrameInfo.Builder(width, height)
.setPixelWidthHeightRatio(pixelWidthHeightRatio)
.build());
videoFrameProcessorReadyCondition.block();
for (Pair<Bitmap, TimestampIterator> frame : frames) {
videoFrameProcessor.queueInputBitmap(frame.first, frame.second);
}
}
public void queueInputTexture(GlTextureInfo inputTexture, long pts) {
public void queueInputTexture(GlTextureInfo inputTexture, long pts) throws InterruptedException {
videoFrameProcessor.registerInputStream(
INPUT_TYPE_TEXTURE_ID,
effects,
@ -388,7 +414,8 @@ public final class VideoFrameProcessorTestRunner {
throw new VideoFrameProcessingException(e);
}
});
videoFrameProcessor.queueInputTexture(inputTexture.texId, pts);
videoFrameProcessorReadyCondition.block();
checkState(videoFrameProcessor.queueInputTexture(inputTexture.texId, pts));
}
/** {@link #endFrameProcessing(long)} with {@link #VIDEO_FRAME_PROCESSING_WAIT_MS} applied. */

View File

@ -687,7 +687,7 @@ public final class DefaultVideoCompositorPixelTest {
* Queues {@code durationSec} bitmaps, with one bitmap per second, starting from and including
* {@code 0} seconds. Sources have a {@code frameRate} of {@code 1}.
*/
public void queueBitmapToAllInputs(int durationSec) throws IOException {
public void queueBitmapToAllInputs(int durationSec) throws IOException, InterruptedException {
for (int i = 0; i < inputVideoFrameProcessorTestRunners.size(); i++) {
queueBitmapToInput(
/* inputId= */ i, durationSec, /* offsetToAddSec= */ 0L, /* frameRate= */ 1f);
@ -700,7 +700,8 @@ public final class DefaultVideoCompositorPixelTest {
* sources have a {@code frameRate} of {@code secondarySourceFrameRate}.
*/
public void queueBitmapToInput(
int inputId, int durationSec, long offsetToAddSec, float frameRate) throws IOException {
int inputId, int durationSec, long offsetToAddSec, float frameRate)
throws IOException, InterruptedException {
inputVideoFrameProcessorTestRunners
.get(inputId)
.queueInputBitmap(

View File

@ -553,7 +553,12 @@ public final class DefaultVideoFrameProcessorTextureOutputPixelTest {
.setEffects(effects)
.build();
GlUtil.awaitSyncObject(syncObject);
videoFrameProcessorTestRunner.queueInputTexture(texture, presentationTimeUs);
try {
videoFrameProcessorTestRunner.queueInputTexture(texture, presentationTimeUs);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw VideoFrameProcessingException.from(e);
}
videoFrameProcessorTestRunner.endFrameProcessing(VIDEO_FRAME_PROCESSING_WAIT_MS / 2);
releaseOutputTextureCallback.release(presentationTimeUs);
}

View File

@ -120,6 +120,14 @@ import java.util.concurrent.atomic.AtomicLong;
new VideoFrameProcessor.Listener() {
private long lastProcessedFramePresentationTimeUs;
@Override
public void onInputStreamRegistered(
@VideoFrameProcessor.InputType int inputType,
List<Effect> effects,
FrameInfo frameInfo) {
// Do nothing.
}
@Override
public void onOutputSizeChanged(int width, int height) {
// TODO: b/289986435 - Allow setting output surface info on VideoGraph.
@ -170,8 +178,7 @@ import java.util.concurrent.atomic.AtomicLong;
@Override
public boolean queueInputBitmap(Bitmap inputBitmap, TimestampIterator inStreamOffsetsUs) {
videoFrameProcessor.queueInputBitmap(inputBitmap, inStreamOffsetsUs);
return true;
return videoFrameProcessor.queueInputBitmap(inputBitmap, inStreamOffsetsUs);
}
@Override
@ -181,8 +188,7 @@ import java.util.concurrent.atomic.AtomicLong;
@Override
public boolean queueInputTexture(int texId, long presentationTimeUs) {
videoFrameProcessor.queueInputTexture(texId, presentationTimeUs);
return true;
return videoFrameProcessor.queueInputTexture(texId, presentationTimeUs);
}
@Override
@ -202,8 +208,7 @@ import java.util.concurrent.atomic.AtomicLong;
@Override
public boolean registerVideoFrame(long presentationTimeUs) {
videoFrameProcessor.registerInputFrame();
return true;
return videoFrameProcessor.registerInputFrame();
}
@Override