Create VideoFrameRenderControl

Split CompositingVideoSinkProvider.VideoSinkImpl in two classes:
- VideoSinkImpl now only receives input from MediaCodecVideoRenderer and
  forwards frames to its connected VideoFrameProcessor
- VideoFrameRenderControl takes composited frames out of the VideoGraph
  and schedules the rendering of those.
- CompositingVideoSinkProvider connects VideoSinkImpl with
  VideoFramesRenderer.

PiperOrigin-RevId: 584605078
This commit is contained in:
christosts 2023-11-22 06:01:33 -08:00 committed by Copybara-Service
parent a063d137b4
commit 6435ddb89e
9 changed files with 988 additions and 364 deletions

View File

@ -53,4 +53,22 @@ public interface PreviewingVideoGraph extends VideoGraph {
long initialTimestampOffsetUs) long initialTimestampOffsetUs)
throws VideoFrameProcessingException; throws VideoFrameProcessingException;
} }
/**
* Renders the oldest unrendered output frame that has become {@linkplain
* Listener#onOutputFrameAvailableForRendering(long) available for rendering} at the given {@code
* renderTimeNs}.
*
* <p>This will either render the output frame to the {@linkplain #setOutputSurfaceInfo output
* surface}, or drop the frame, per {@code renderTimeNs}.
*
* <p>The {@code renderTimeNs} may be passed to {@link
* android.opengl.EGLExt#eglPresentationTimeANDROID} depending on the implementation.
*
* @param renderTimeNs The render time to use for the frame, in nanoseconds. The render time can
* be before or after the current system time. Use {@link
* VideoFrameProcessor#DROP_OUTPUT_FRAME} to drop the frame, or {@link
* VideoFrameProcessor#RENDER_OUTPUT_FRAME_IMMEDIATELY} to render the frame immediately.
*/
void renderOutputFrame(long renderTimeNs);
} }

View File

@ -98,4 +98,9 @@ public final class PreviewingSingleInputVideoGraph extends SingleInputVideoGraph
presentation, presentation,
initialTimestampOffsetUs); initialTimestampOffsetUs);
} }
@Override
public void renderOutputFrame(long renderTimeNs) {
getProcessor(SINGLE_INPUT_INDEX).renderOutputFrame(renderTimeNs);
}
} }

View File

@ -31,6 +31,7 @@ import androidx.media3.common.VideoFrameProcessingException;
import androidx.media3.common.VideoFrameProcessor; import androidx.media3.common.VideoFrameProcessor;
import androidx.media3.common.VideoGraph; import androidx.media3.common.VideoGraph;
import androidx.media3.common.util.UnstableApi; import androidx.media3.common.util.UnstableApi;
import com.google.common.util.concurrent.MoreExecutors;
import java.util.List; import java.util.List;
import java.util.concurrent.Executor; import java.util.concurrent.Executor;
@ -54,6 +55,7 @@ public abstract class SingleInputVideoGraph implements VideoGraph {
@Nullable private final Presentation presentation; @Nullable private final Presentation presentation;
@Nullable private VideoFrameProcessor videoFrameProcessor; @Nullable private VideoFrameProcessor videoFrameProcessor;
private boolean isEnded;
private boolean released; private boolean released;
private volatile boolean hasProducedFrameWithTimestampZero; private volatile boolean hasProducedFrameWithTimestampZero;
@ -112,7 +114,7 @@ public abstract class SingleInputVideoGraph implements VideoGraph {
inputColorInfo, inputColorInfo,
outputColorInfo, outputColorInfo,
renderFramesAutomatically, renderFramesAutomatically,
listenerExecutor, /* listenerExecutor= */ MoreExecutors.directExecutor(),
new VideoFrameProcessor.Listener() { new VideoFrameProcessor.Listener() {
private long lastProcessedFramePresentationTimeUs; private long lastProcessedFramePresentationTimeUs;
@ -129,6 +131,12 @@ public abstract class SingleInputVideoGraph implements VideoGraph {
@Override @Override
public void onOutputFrameAvailableForRendering(long presentationTimeUs) { public void onOutputFrameAvailableForRendering(long presentationTimeUs) {
if (isEnded) {
onError(
new VideoFrameProcessingException(
"onOutputFrameAvailableForRendering() received after onEnded()"));
return;
}
// Frames are rendered automatically. // Frames are rendered automatically.
if (presentationTimeUs == 0) { if (presentationTimeUs == 0) {
hasProducedFrameWithTimestampZero = true; hasProducedFrameWithTimestampZero = true;
@ -145,7 +153,13 @@ public abstract class SingleInputVideoGraph implements VideoGraph {
@Override @Override
public void onEnded() { public void onEnded() {
listener.onEnded(lastProcessedFramePresentationTimeUs); if (isEnded) {
onError(new VideoFrameProcessingException("onEnded() received multiple times"));
return;
}
isEnded = true;
listenerExecutor.execute(
() -> listener.onEnded(lastProcessedFramePresentationTimeUs));
} }
}); });
return SINGLE_INPUT_INDEX; return SINGLE_INPUT_INDEX;

View File

@ -15,16 +15,16 @@
*/ */
package androidx.media3.exoplayer.video; package androidx.media3.exoplayer.video;
import static androidx.media3.common.util.Assertions.checkArgument;
import static androidx.media3.common.util.Assertions.checkNotNull; import static androidx.media3.common.util.Assertions.checkNotNull;
import static androidx.media3.common.util.Assertions.checkState; import static androidx.media3.common.util.Assertions.checkState;
import static androidx.media3.common.util.Assertions.checkStateNotNull; import static androidx.media3.common.util.Assertions.checkStateNotNull;
import android.content.Context; import android.content.Context;
import android.graphics.Bitmap; import android.graphics.Bitmap;
import android.os.Handler; import android.os.Looper;
import android.util.Pair; import android.util.Pair;
import android.view.Surface; import android.view.Surface;
import androidx.annotation.FloatRange;
import androidx.annotation.Nullable; import androidx.annotation.Nullable;
import androidx.annotation.VisibleForTesting; import androidx.annotation.VisibleForTesting;
import androidx.media3.common.C; import androidx.media3.common.C;
@ -41,9 +41,8 @@ import androidx.media3.common.VideoFrameProcessor;
import androidx.media3.common.VideoGraph; import androidx.media3.common.VideoGraph;
import androidx.media3.common.VideoSize; import androidx.media3.common.VideoSize;
import androidx.media3.common.util.Clock; import androidx.media3.common.util.Clock;
import androidx.media3.common.util.LongArrayQueue; import androidx.media3.common.util.HandlerWrapper;
import androidx.media3.common.util.Size; import androidx.media3.common.util.Size;
import androidx.media3.common.util.TimedValueQueue;
import androidx.media3.common.util.TimestampIterator; import androidx.media3.common.util.TimestampIterator;
import androidx.media3.common.util.UnstableApi; import androidx.media3.common.util.UnstableApi;
import androidx.media3.common.util.Util; import androidx.media3.common.util.Util;
@ -53,6 +52,7 @@ import java.lang.reflect.Constructor;
import java.lang.reflect.Method; import java.lang.reflect.Method;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Objects;
import java.util.concurrent.Executor; import java.util.concurrent.Executor;
import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.initialization.qual.Initialized;
import org.checkerframework.checker.nullness.qual.EnsuresNonNull; import org.checkerframework.checker.nullness.qual.EnsuresNonNull;
@ -60,15 +60,27 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
/** Handles composition of video sinks. */ /** Handles composition of video sinks. */
@UnstableApi @UnstableApi
/* package */ final class CompositingVideoSinkProvider implements VideoSinkProvider { /* package */ final class CompositingVideoSinkProvider
implements VideoSinkProvider, VideoGraph.Listener, VideoFrameRenderControl.FrameRenderer {
private static final Executor NO_OP_EXECUTOR = runnable -> {};
private final Context context; private final Context context;
private final PreviewingVideoGraph.Factory previewingVideoGraphFactory; private final PreviewingVideoGraph.Factory previewingVideoGraphFactory;
private final VideoFrameReleaseControl videoFrameReleaseControl; private final VideoFrameReleaseControl videoFrameReleaseControl;
private final VideoFrameRenderControl videoFrameRenderControl;
@Nullable private VideoSinkImpl videoSinkImpl; private Clock clock;
@Nullable private List<Effect> videoEffects; private @MonotonicNonNull Format outputFormat;
@Nullable private VideoFrameMetadataListener videoFrameMetadataListener; private @MonotonicNonNull VideoFrameMetadataListener videoFrameMetadataListener;
private @MonotonicNonNull HandlerWrapper handler;
private @MonotonicNonNull PreviewingVideoGraph videoGraph;
private @MonotonicNonNull VideoSinkImpl videoSinkImpl;
private @MonotonicNonNull List<Effect> videoEffects;
@Nullable private Pair<Surface, Size> currentSurfaceAndSize;
private VideoSink.Listener listener;
private Executor listenerExecutor;
private int pendingFlushCount;
private boolean released; private boolean released;
/** Creates a new instance. */ /** Creates a new instance. */
@ -90,24 +102,54 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
this.context = context; this.context = context;
this.previewingVideoGraphFactory = previewingVideoGraphFactory; this.previewingVideoGraphFactory = previewingVideoGraphFactory;
this.videoFrameReleaseControl = videoFrameReleaseControl; this.videoFrameReleaseControl = videoFrameReleaseControl;
@SuppressWarnings("nullness:assignment")
VideoFrameRenderControl.@Initialized FrameRenderer thisRef = this;
videoFrameRenderControl =
new VideoFrameRenderControl(/* frameRenderer= */ thisRef, videoFrameReleaseControl);
clock = Clock.DEFAULT;
listener = VideoSink.Listener.NO_OP;
listenerExecutor = NO_OP_EXECUTOR;
} }
// VideoSinkProvider methods
@Override @Override
public void initialize(Format sourceFormat) throws VideoSink.VideoSinkException { public void initialize(Format sourceFormat) throws VideoSink.VideoSinkException {
checkState(!released && videoSinkImpl == null); checkState(!released && videoSinkImpl == null);
checkStateNotNull(videoEffects); checkStateNotNull(videoEffects);
// Lazily initialize the handler here so it's initialized on the playback looper.
handler = clock.createHandler(checkStateNotNull(Looper.myLooper()), /* callback= */ null);
ColorInfo inputColorInfo =
sourceFormat.colorInfo != null && ColorInfo.isTransferHdr(sourceFormat.colorInfo)
? sourceFormat.colorInfo
: ColorInfo.SDR_BT709_LIMITED;
ColorInfo outputColorInfo = inputColorInfo;
if (inputColorInfo.colorTransfer == C.COLOR_TRANSFER_HLG) {
// SurfaceView only supports BT2020 PQ input. Therefore, convert HLG to PQ.
outputColorInfo =
inputColorInfo.buildUpon().setColorTransfer(C.COLOR_TRANSFER_ST2084).build();
}
try { try {
@SuppressWarnings("nullness:assignment")
VideoGraph.@Initialized Listener thisRef = this;
videoGraph =
previewingVideoGraphFactory.create(
context,
inputColorInfo,
outputColorInfo,
DebugViewProvider.NONE,
/* listener= */ thisRef,
/* listenerExecutor= */ handler::post,
/* compositionEffects= */ ImmutableList.of(),
/* initialTimestampOffsetUs= */ 0);
videoSinkImpl = videoSinkImpl =
new VideoSinkImpl( new VideoSinkImpl(
context, previewingVideoGraphFactory, videoFrameReleaseControl, sourceFormat); context, /* compositingVideoSinkProvider= */ this, videoGraph, sourceFormat);
} catch (VideoFrameProcessingException e) { } catch (VideoFrameProcessingException e) {
throw new VideoSink.VideoSinkException(e, sourceFormat); throw new VideoSink.VideoSinkException(e, sourceFormat);
} }
if (videoFrameMetadataListener != null) {
videoSinkImpl.setVideoFrameMetadataListener(videoFrameMetadataListener);
}
videoSinkImpl.setVideoEffects(checkNotNull(videoEffects)); videoSinkImpl.setVideoEffects(checkNotNull(videoEffects));
} }
@ -122,10 +164,16 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
return; return;
} }
if (handler != null) {
handler.removeCallbacksAndMessages(/* token= */ null);
}
if (videoSinkImpl != null) { if (videoSinkImpl != null) {
videoSinkImpl.release(); videoSinkImpl.release();
videoSinkImpl = null;
} }
if (videoGraph != null) {
videoGraph.release();
}
currentSurfaceAndSize = null;
released = true; released = true;
} }
@ -157,138 +205,222 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
@Override @Override
public void setOutputSurfaceInfo(Surface outputSurface, Size outputResolution) { public void setOutputSurfaceInfo(Surface outputSurface, Size outputResolution) {
checkStateNotNull(videoSinkImpl).setOutputSurfaceInfo(outputSurface, outputResolution); if (currentSurfaceAndSize != null
&& currentSurfaceAndSize.first.equals(outputSurface)
&& currentSurfaceAndSize.second.equals(outputResolution)) {
return;
}
videoFrameReleaseControl.setOutputSurface(outputSurface);
currentSurfaceAndSize = Pair.create(outputSurface, outputResolution);
checkStateNotNull(videoGraph)
.setOutputSurfaceInfo(
new SurfaceInfo(
outputSurface, outputResolution.getWidth(), outputResolution.getHeight()));
} }
@Override @Override
public void clearOutputSurfaceInfo() { public void clearOutputSurfaceInfo() {
checkStateNotNull(videoSinkImpl).clearOutputSurfaceInfo(); checkStateNotNull(videoGraph).setOutputSurfaceInfo(/* outputSurfaceInfo= */ null);
currentSurfaceAndSize = null;
} }
@Override @Override
public void setVideoFrameMetadataListener(VideoFrameMetadataListener videoFrameMetadataListener) { public void setVideoFrameMetadataListener(VideoFrameMetadataListener videoFrameMetadataListener) {
this.videoFrameMetadataListener = videoFrameMetadataListener; this.videoFrameMetadataListener = videoFrameMetadataListener;
if (isInitialized()) { }
checkStateNotNull(videoSinkImpl).setVideoFrameMetadataListener(videoFrameMetadataListener);
@Override
public void setClock(Clock clock) {
checkState(!isInitialized());
this.clock = clock;
}
// VideoGraph.Listener
@Override
public void onOutputSizeChanged(int width, int height) {
// We forward output size changes to render control even if we are still flushing.
videoFrameRenderControl.onOutputSizeChanged(width, height);
}
@Override
public void onOutputFrameAvailableForRendering(long presentationTimeUs) {
if (pendingFlushCount > 0) {
// Ignore available frames while the sink provider is flushing
return;
}
videoFrameRenderControl.onOutputFrameAvailableForRendering(presentationTimeUs);
}
@Override
public void onEnded(long finalFramePresentationTimeUs) {
throw new UnsupportedOperationException();
}
@Override
public void onError(VideoFrameProcessingException exception) {
VideoSink.Listener currentListener = this.listener;
listenerExecutor.execute(
() -> {
VideoSinkImpl videoSink = checkStateNotNull(videoSinkImpl);
currentListener.onError(
videoSink,
new VideoSink.VideoSinkException(
exception, checkStateNotNull(videoSink.inputFormat)));
});
}
// FrameRenderer methods
@Override
public void onVideoSizeChanged(VideoSize videoSize) {
outputFormat =
new Format.Builder()
.setWidth(videoSize.width)
.setHeight(videoSize.height)
.setSampleMimeType(MimeTypes.VIDEO_RAW)
.build();
VideoSinkImpl videoSink = checkStateNotNull(videoSinkImpl);
VideoSink.Listener currentListener = this.listener;
listenerExecutor.execute(() -> currentListener.onVideoSizeChanged(videoSink, videoSize));
}
@Override
public void renderFrame(
long renderTimeNs, long bufferPresentationTimeUs, long streamOffsetUs, boolean isFirstFrame) {
if (isFirstFrame && listenerExecutor != NO_OP_EXECUTOR) {
VideoSinkImpl videoSink = checkStateNotNull(videoSinkImpl);
VideoSink.Listener currentListener = this.listener;
listenerExecutor.execute(() -> currentListener.onFirstFrameRendered(videoSink));
}
if (videoFrameMetadataListener != null) {
// TODO b/292111083 - outputFormat is initialized after the first frame is rendered because
// onVideoSizeChanged is announced after the first frame is available for rendering.
Format format = outputFormat == null ? new Format.Builder().build() : outputFormat;
videoFrameMetadataListener.onVideoFrameAboutToBeRendered(
/* presentationTimeUs= */ bufferPresentationTimeUs - streamOffsetUs,
clock.nanoTime(),
format,
/* mediaFormat= */ null);
}
checkStateNotNull(videoGraph).renderOutputFrame(renderTimeNs);
}
@Override
public void dropFrame() {
VideoSink.Listener currentListener = this.listener;
listenerExecutor.execute(
() -> currentListener.onFrameDropped(checkStateNotNull(videoSinkImpl)));
checkStateNotNull(videoGraph).renderOutputFrame(VideoFrameProcessor.DROP_OUTPUT_FRAME);
}
// Internal methods
private void setListener(VideoSink.Listener listener, Executor executor) {
if (Objects.equals(listener, this.listener)) {
checkState(Objects.equals(executor, listenerExecutor));
return;
}
this.listener = listener;
this.listenerExecutor = executor;
}
private boolean isReady() {
return pendingFlushCount == 0 && videoFrameRenderControl.isReady();
}
private boolean hasReleasedFrame(long presentationTimeUs) {
return pendingFlushCount == 0 && videoFrameRenderControl.hasReleasedFrame(presentationTimeUs);
}
private void render(long positionUs, long elapsedRealtimeUs) throws ExoPlaybackException {
if (pendingFlushCount == 0) {
videoFrameRenderControl.render(positionUs, elapsedRealtimeUs);
} }
} }
private static final class VideoSinkImpl implements VideoSink, VideoGraph.Listener { private void flush() {
pendingFlushCount++;
// Flush the render control now to ensure it has no data, eg calling isReady() must return false
// and
// render() should not render any frames.
videoFrameRenderControl.flush();
// Finish flushing after handling pending video graph callbacks to ensure video size changes
// reach the video render control.
checkStateNotNull(handler).post(this::flushInternal);
}
private void flushInternal() {
pendingFlushCount--;
if (pendingFlushCount > 0) {
// Another flush has been issued.
return;
} else if (pendingFlushCount < 0) {
throw new IllegalStateException(String.valueOf(pendingFlushCount));
}
// Flush the render control again.
videoFrameRenderControl.flush();
}
private void setPlaybackSpeed(float speed) {
videoFrameRenderControl.setPlaybackSpeed(speed);
}
private void onStreamOffsetChange(long bufferPresentationTimeUs, long streamOffsetUs) {
videoFrameRenderControl.onStreamOffsetChange(bufferPresentationTimeUs, streamOffsetUs);
}
/** Receives input from an ExoPlayer renderer and forwards it to the video graph. */
private static final class VideoSinkImpl implements VideoSink {
private final Context context; private final Context context;
private final VideoFrameReleaseControl videoFrameReleaseControl; private final CompositingVideoSinkProvider compositingVideoSinkProvider;
private final VideoFrameReleaseControl.FrameReleaseInfo videoFrameReleaseInfo;
private final VideoFrameProcessor videoFrameProcessor; private final VideoFrameProcessor videoFrameProcessor;
private final LongArrayQueue processedFramesBufferTimestampsUs;
private final TimedValueQueue<Long> streamOffsets;
private final TimedValueQueue<VideoSize> videoSizeChanges;
private final Handler handler;
private final int videoFrameProcessorMaxPendingFrameCount; private final int videoFrameProcessorMaxPendingFrameCount;
private final ArrayList<Effect> videoEffects; private final ArrayList<Effect> videoEffects;
@Nullable private final Effect rotationEffect; @Nullable private final Effect rotationEffect;
private VideoSink.@MonotonicNonNull Listener listener;
private @MonotonicNonNull Executor listenerExecutor;
@Nullable private VideoFrameMetadataListener videoFrameMetadataListener;
@Nullable private Format inputFormat; @Nullable private Format inputFormat;
@Nullable private Pair<Surface, Size> currentSurfaceAndSize;
/**
* Whether the last frame of the current stream is decoded and registered to {@link
* VideoFrameProcessor}.
*/
private boolean registeredLastFrame;
/**
* Whether the last frame of the current stream is processed by the {@link VideoFrameProcessor}.
*/
private boolean processedLastFrame;
/** Whether the last frame of the current stream is released to the output {@link Surface}. */
private boolean releasedLastFrame;
private long lastCodecBufferPresentationTimestampUs;
private VideoSize processedFrameSize;
private VideoSize reportedVideoSize;
private boolean pendingVideoSizeChange;
private long inputStreamOffsetUs; private long inputStreamOffsetUs;
private boolean pendingInputStreamOffsetChange; private boolean pendingInputStreamOffsetChange;
private long outputStreamOffsetUs;
// TODO b/292111083 - Remove the field and trigger the callback on every video size change. /** The buffer presentation time, in microseconds, of the final frame in the stream. */
private boolean onVideoSizeChangedCalled; private long finalBufferPresentationTimeUs;
/**
* The buffer presentation timestamp, in microseconds, of the most recently registered frame.
*/
private long lastBufferPresentationTimeUs;
private boolean hasRegisteredFirstInputStream; private boolean hasRegisteredFirstInputStream;
private boolean inputStreamRegistrationPending; private long pendingInputStreamBufferPresentationTimeUs;
private long lastFramePresentationTimeUs;
/** Creates a new instance. */ /** Creates a new instance. */
public VideoSinkImpl( public VideoSinkImpl(
Context context, Context context,
PreviewingVideoGraph.Factory previewingVideoGraphFactory, CompositingVideoSinkProvider compositingVideoSinkProvider,
VideoFrameReleaseControl videoFrameReleaseControl, PreviewingVideoGraph videoGraph,
Format sourceFormat) Format sourceFormat)
throws VideoFrameProcessingException { throws VideoFrameProcessingException {
this.context = context; this.context = context;
this.videoFrameReleaseControl = videoFrameReleaseControl; this.compositingVideoSinkProvider = compositingVideoSinkProvider;
videoFrameReleaseInfo = new VideoFrameReleaseControl.FrameReleaseInfo();
processedFramesBufferTimestampsUs = new LongArrayQueue();
streamOffsets = new TimedValueQueue<>();
videoSizeChanges = new TimedValueQueue<>();
// TODO b/226330223 - Investigate increasing frame count when frame dropping is // TODO b/226330223 - Investigate increasing frame count when frame dropping is
// allowed. // allowed.
// TODO b/278234847 - Evaluate whether limiting frame count when frame dropping is not allowed // TODO b/278234847 - Evaluate whether limiting frame count when frame dropping is not allowed
// reduces decoder timeouts, and consider restoring. // reduces decoder timeouts, and consider restoring.
videoFrameProcessorMaxPendingFrameCount = videoFrameProcessorMaxPendingFrameCount =
Util.getMaxPendingFramesCountForMediaCodecDecoders(context); Util.getMaxPendingFramesCountForMediaCodecDecoders(context);
lastCodecBufferPresentationTimestampUs = C.TIME_UNSET;
processedFrameSize = VideoSize.UNKNOWN;
reportedVideoSize = VideoSize.UNKNOWN;
// Playback thread handler.
handler = Util.createHandlerForCurrentLooper();
ColorInfo inputColorInfo =
sourceFormat.colorInfo != null && ColorInfo.isTransferHdr(sourceFormat.colorInfo)
? sourceFormat.colorInfo
: ColorInfo.SDR_BT709_LIMITED;
ColorInfo outputColorInfo = inputColorInfo;
if (inputColorInfo.colorTransfer == C.COLOR_TRANSFER_HLG) {
// SurfaceView only supports BT2020 PQ input. Therefore, convert HLG to PQ.
outputColorInfo =
inputColorInfo.buildUpon().setColorTransfer(C.COLOR_TRANSFER_ST2084).build();
}
@SuppressWarnings("nullness:assignment")
@Initialized
VideoSinkImpl thisRef = this;
PreviewingVideoGraph videoGraph =
previewingVideoGraphFactory.create(
context,
inputColorInfo,
outputColorInfo,
DebugViewProvider.NONE,
/* listener= */ thisRef,
/* listenerExecutor= */ handler::post,
/* compositionEffects= */ ImmutableList.of(),
/* initialTimestampOffsetUs= */ 0);
int videoGraphInputId = videoGraph.registerInput(); int videoGraphInputId = videoGraph.registerInput();
videoFrameProcessor = videoGraph.getProcessor(videoGraphInputId); videoFrameProcessor = videoGraph.getProcessor(videoGraphInputId);
if (currentSurfaceAndSize != null) {
Size outputSurfaceSize = currentSurfaceAndSize.second;
videoGraph.setOutputSurfaceInfo(
new SurfaceInfo(
currentSurfaceAndSize.first,
outputSurfaceSize.getWidth(),
outputSurfaceSize.getHeight()));
}
videoEffects = new ArrayList<>(); videoEffects = new ArrayList<>();
// MediaCodec applies rotation after API 21 // MediaCodec applies rotation after API 21.
rotationEffect = rotationEffect =
Util.SDK_INT < 21 && sourceFormat.rotationDegrees != 0 Util.SDK_INT < 21 && sourceFormat.rotationDegrees != 0
? ScaleAndRotateAccessor.createRotationEffect(sourceFormat.rotationDegrees) ? ScaleAndRotateAccessor.createRotationEffect(sourceFormat.rotationDegrees)
: null; : null;
lastFramePresentationTimeUs = C.TIME_UNSET; finalBufferPresentationTimeUs = C.TIME_UNSET;
lastBufferPresentationTimeUs = C.TIME_UNSET;
} }
// VideoSink impl // VideoSink impl
@ -296,27 +428,25 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
@Override @Override
public void flush() { public void flush() {
videoFrameProcessor.flush(); videoFrameProcessor.flush();
processedFramesBufferTimestampsUs.clear();
streamOffsets.clear();
handler.removeCallbacksAndMessages(/* token= */ null);
videoFrameReleaseControl.reset();
lastFramePresentationTimeUs = C.TIME_UNSET;
hasRegisteredFirstInputStream = false; hasRegisteredFirstInputStream = false;
if (registeredLastFrame) { finalBufferPresentationTimeUs = C.TIME_UNSET;
registeredLastFrame = false; lastBufferPresentationTimeUs = C.TIME_UNSET;
processedLastFrame = false; compositingVideoSinkProvider.flush();
releasedLastFrame = false; // Don't change input stream offset or reset the pending input stream offset change so that
} // it's announced with the next input frame.
// Don't reset pendingInputStreamBufferPresentationTimeUs because it's not guaranteed to
// receive a new input stream after seeking.
} }
@Override @Override
public boolean isReady() { public boolean isReady() {
return videoFrameReleaseControl.isReady(/* rendererReady= */ true); return compositingVideoSinkProvider.isReady();
} }
@Override @Override
public boolean isEnded() { public boolean isEnded() {
return releasedLastFrame; return finalBufferPresentationTimeUs != C.TIME_UNSET
&& compositingVideoSinkProvider.hasReleasedFrame(finalBufferPresentationTimeUs);
} }
@Override @Override
@ -329,29 +459,20 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
if (!hasRegisteredFirstInputStream) { if (!hasRegisteredFirstInputStream) {
maybeRegisterInputStream(); maybeRegisterInputStream();
hasRegisteredFirstInputStream = true; hasRegisteredFirstInputStream = true;
// If an input stream registration is pending and seek to another MediaItem, execution // If an input stream registration is pending and seek causes a format change, execution
// reaches here before registerInputFrame(), resetting inputStreamRegistrationPending to // reaches here before registerInputFrame(). Reset pendingInputStreamTimestampUs to
// avoid registering the same input stream again in registerInputFrame(). // avoid registering the same input stream again in registerInputFrame().
inputStreamRegistrationPending = false; pendingInputStreamBufferPresentationTimeUs = C.TIME_UNSET;
} else { } else {
inputStreamRegistrationPending = true; // If we reach this point, we must have registered at least one frame for processing.
} checkState(lastBufferPresentationTimeUs != C.TIME_UNSET);
pendingInputStreamBufferPresentationTimeUs = lastBufferPresentationTimeUs;
if (registeredLastFrame) {
registeredLastFrame = false;
processedLastFrame = false;
releasedLastFrame = false;
} }
} }
@Override @Override
public void setListener(Listener listener, Executor executor) { public void setListener(Listener listener, Executor executor) {
if (Util.areEqual(this.listener, listener)) { compositingVideoSinkProvider.setListener(listener, executor);
checkState(Util.areEqual(listenerExecutor, executor));
return;
}
this.listener = listener;
this.listenerExecutor = executor;
} }
@Override @Override
@ -370,12 +491,11 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
// An input stream is fully decoded, wait until all of its frames are released before queueing // An input stream is fully decoded, wait until all of its frames are released before queueing
// input frame from the next input stream. // input frame from the next input stream.
if (inputStreamRegistrationPending) { if (pendingInputStreamBufferPresentationTimeUs != C.TIME_UNSET) {
if (lastFramePresentationTimeUs == C.TIME_UNSET) { if (compositingVideoSinkProvider.hasReleasedFrame(
// A seek took place after signaling a new input stream, but the input stream is yet to be pendingInputStreamBufferPresentationTimeUs)) {
// registered.
maybeRegisterInputStream(); maybeRegisterInputStream();
inputStreamRegistrationPending = false; pendingInputStreamBufferPresentationTimeUs = C.TIME_UNSET;
} else { } else {
return C.TIME_UNSET; return C.TIME_UNSET;
} }
@ -388,7 +508,6 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
if (!videoFrameProcessor.registerInputFrame()) { if (!videoFrameProcessor.registerInputFrame()) {
return C.TIME_UNSET; return C.TIME_UNSET;
} }
lastFramePresentationTimeUs = framePresentationTimeUs;
// The sink takes in frames with monotonically increasing, non-offset frame // 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 // 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 // should bear a timestamp of 10s seen from VideoFrameProcessor; while in ExoPlayer, the
@ -397,12 +516,14 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
// handling of presentation timestamps in ExoPlayer and VideoFrameProcessor. // handling of presentation timestamps in ExoPlayer and VideoFrameProcessor.
long bufferPresentationTimeUs = framePresentationTimeUs + inputStreamOffsetUs; long bufferPresentationTimeUs = framePresentationTimeUs + inputStreamOffsetUs;
if (pendingInputStreamOffsetChange) { if (pendingInputStreamOffsetChange) {
streamOffsets.add(bufferPresentationTimeUs, inputStreamOffsetUs); compositingVideoSinkProvider.onStreamOffsetChange(
/* bufferPresentationTimeUs= */ bufferPresentationTimeUs,
/* streamOffsetUs= */ inputStreamOffsetUs);
pendingInputStreamOffsetChange = false; pendingInputStreamOffsetChange = false;
} }
lastBufferPresentationTimeUs = bufferPresentationTimeUs;
if (isLastFrame) { if (isLastFrame) {
registeredLastFrame = true; finalBufferPresentationTimeUs = bufferPresentationTimeUs;
lastCodecBufferPresentationTimestampUs = bufferPresentationTimeUs;
} }
return bufferPresentationTimeUs * 1000; return bufferPresentationTimeUs * 1000;
} }
@ -414,130 +535,27 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
@Override @Override
public void render(long positionUs, long elapsedRealtimeUs) throws VideoSinkException { public void render(long positionUs, long elapsedRealtimeUs) throws VideoSinkException {
while (!processedFramesBufferTimestampsUs.isEmpty()) {
long bufferPresentationTimeUs = processedFramesBufferTimestampsUs.element();
// check whether this buffer comes with a new stream offset.
if (maybeUpdateOutputStreamOffset(bufferPresentationTimeUs)) {
videoFrameReleaseControl.onProcessedStreamChange();
}
long framePresentationTimeUs = bufferPresentationTimeUs - outputStreamOffsetUs;
boolean isLastFrame = processedLastFrame && processedFramesBufferTimestampsUs.size() == 1;
@VideoFrameReleaseControl.FrameReleaseAction int frameReleaseAction;
try { try {
frameReleaseAction = compositingVideoSinkProvider.render(positionUs, elapsedRealtimeUs);
videoFrameReleaseControl.getFrameReleaseAction(
bufferPresentationTimeUs,
positionUs,
elapsedRealtimeUs,
outputStreamOffsetUs,
isLastFrame,
videoFrameReleaseInfo);
} catch (ExoPlaybackException e) { } catch (ExoPlaybackException e) {
throw new VideoSinkException( throw new VideoSinkException(
e, e, inputFormat != null ? inputFormat : new Format.Builder().build());
new Format.Builder()
.setSampleMimeType(MimeTypes.VIDEO_RAW)
.setWidth(processedFrameSize.width)
.setHeight(processedFrameSize.height)
.build());
}
switch (frameReleaseAction) {
case VideoFrameReleaseControl.FRAME_RELEASE_TRY_AGAIN_LATER:
return;
case VideoFrameReleaseControl.FRAME_RELEASE_SKIP:
case VideoFrameReleaseControl.FRAME_RELEASE_DROP:
dropFrame(isLastFrame);
break;
case VideoFrameReleaseControl.FRAME_RELEASE_IGNORE:
// TODO b/293873191 - Handle very late buffers and drop to key frame. Need to flush
// VideoFrameProcessor input frames in this case.
releaseProcessedFrameInternal(VideoFrameProcessor.DROP_OUTPUT_FRAME, isLastFrame);
break;
case VideoFrameReleaseControl.FRAME_RELEASE_IMMEDIATELY:
case VideoFrameReleaseControl.FRAME_RELEASE_SCHEDULED:
renderFrame(
framePresentationTimeUs, bufferPresentationTimeUs, frameReleaseAction, isLastFrame);
break;
default:
throw new IllegalStateException(String.valueOf(frameReleaseAction));
}
if (framePresentationTimeUs == lastFramePresentationTimeUs
&& inputStreamRegistrationPending) {
maybeRegisterInputStream();
inputStreamRegistrationPending = false;
}
}
}
@Override
public void setPlaybackSpeed(float speed) {
checkArgument(speed >= 0.0);
videoFrameReleaseControl.setPlaybackSpeed(speed);
}
// VideoGraph.Listener methods
@Override
public void onOutputSizeChanged(int width, int height) {
VideoSize newVideoSize = new VideoSize(width, height);
if (!processedFrameSize.equals(newVideoSize)) {
processedFrameSize = newVideoSize;
pendingVideoSizeChange = true;
} }
} }
@Override @Override
public void onOutputFrameAvailableForRendering(long presentationTimeUs) { public void setPlaybackSpeed(@FloatRange(from = 0, fromInclusive = false) float speed) {
if (pendingVideoSizeChange) { compositingVideoSinkProvider.setPlaybackSpeed(speed);
videoSizeChanges.add(presentationTimeUs, processedFrameSize);
pendingVideoSizeChange = false;
}
if (registeredLastFrame) {
checkState(lastCodecBufferPresentationTimestampUs != C.TIME_UNSET);
}
processedFramesBufferTimestampsUs.add(presentationTimeUs);
// TODO b/257464707 - Support extensively modified media.
if (registeredLastFrame && presentationTimeUs >= lastCodecBufferPresentationTimestampUs) {
processedLastFrame = true;
}
}
@Override
public void onError(VideoFrameProcessingException exception) {
if (listener == null || listenerExecutor == null) {
return;
}
listenerExecutor.execute(
() -> {
if (listener != null) {
listener.onError(
/* videoSink= */ this,
new VideoSink.VideoSinkException(
exception,
new Format.Builder()
.setSampleMimeType(MimeTypes.VIDEO_RAW)
.setWidth(processedFrameSize.width)
.setHeight(processedFrameSize.height)
.build()));
}
});
}
@Override
public void onEnded(long finalFramePresentationTimeUs) {
throw new IllegalStateException();
} }
// Other methods // Other methods
/** Releases the video sink. */
public void release() { public void release() {
videoFrameProcessor.release(); videoFrameProcessor.release();
handler.removeCallbacksAndMessages(/* token= */ null);
streamOffsets.clear();
processedFramesBufferTimestampsUs.clear();
} }
/** Sets the {@linkplain Effect video effects} to apply immediately. */ /** Sets the {@linkplain Effect video effects}. */
public void setVideoEffects(List<Effect> videoEffects) { public void setVideoEffects(List<Effect> videoEffects) {
setPendingVideoEffects(videoEffects); setPendingVideoEffects(videoEffects);
maybeRegisterInputStream(); maybeRegisterInputStream();
@ -552,16 +570,12 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
this.videoEffects.addAll(videoEffects); this.videoEffects.addAll(videoEffects);
} }
/** Sets the stream offset, in micro seconds. */
public void setStreamOffsetUs(long streamOffsetUs) { public void setStreamOffsetUs(long streamOffsetUs) {
pendingInputStreamOffsetChange = inputStreamOffsetUs != streamOffsetUs; pendingInputStreamOffsetChange = inputStreamOffsetUs != streamOffsetUs;
inputStreamOffsetUs = streamOffsetUs; inputStreamOffsetUs = streamOffsetUs;
} }
public void setVideoFrameMetadataListener(
VideoFrameMetadataListener videoFrameMetadataListener) {
this.videoFrameMetadataListener = videoFrameMetadataListener;
}
private void maybeRegisterInputStream() { private void maybeRegisterInputStream() {
if (inputFormat == null) { if (inputFormat == null) {
return; return;
@ -581,110 +595,6 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
.build()); .build());
} }
/**
* Sets the output surface info.
*
* @param outputSurface The {@link Surface} to which {@link VideoFrameProcessor} outputs.
* @param outputResolution The {@link Size} of the output resolution.
*/
public void setOutputSurfaceInfo(Surface outputSurface, Size outputResolution) {
if (currentSurfaceAndSize != null
&& currentSurfaceAndSize.first.equals(outputSurface)
&& currentSurfaceAndSize.second.equals(outputResolution)) {
return;
}
videoFrameReleaseControl.setOutputSurface(outputSurface);
currentSurfaceAndSize = Pair.create(outputSurface, outputResolution);
videoFrameProcessor.setOutputSurfaceInfo(
new SurfaceInfo(
outputSurface, outputResolution.getWidth(), outputResolution.getHeight()));
}
/** Clears the output surface info. */
public void clearOutputSurfaceInfo() {
videoFrameProcessor.setOutputSurfaceInfo(null);
currentSurfaceAndSize = null;
}
private boolean maybeUpdateOutputStreamOffset(long bufferPresentationTimeUs) {
boolean updatedOffset = false;
@Nullable Long newOutputStreamOffsetUs = streamOffsets.pollFloor(bufferPresentationTimeUs);
if (newOutputStreamOffsetUs != null && newOutputStreamOffsetUs != outputStreamOffsetUs) {
outputStreamOffsetUs = newOutputStreamOffsetUs;
updatedOffset = true;
}
return updatedOffset;
}
private void dropFrame(boolean isLastFrame) {
if (listenerExecutor != null) {
listenerExecutor.execute(
() -> {
if (listener != null) {
listener.onFrameDropped(this);
}
});
}
releaseProcessedFrameInternal(VideoFrameProcessor.DROP_OUTPUT_FRAME, isLastFrame);
}
private void renderFrame(
long framePresentationTimeUs,
long bufferPresentationTimeUs,
@VideoFrameReleaseControl.FrameReleaseAction int frameReleaseAction,
boolean isLastFrame) {
if (videoFrameMetadataListener != null) {
videoFrameMetadataListener.onVideoFrameAboutToBeRendered(
framePresentationTimeUs,
frameReleaseAction == VideoFrameReleaseControl.FRAME_RELEASE_IMMEDIATELY
? Clock.DEFAULT.nanoTime()
: videoFrameReleaseInfo.getReleaseTimeNs(),
checkNotNull(inputFormat),
/* mediaFormat= */ null);
}
if (videoFrameReleaseControl.onFrameReleasedIsFirstFrame() && listenerExecutor != null) {
listenerExecutor.execute(
() -> {
if (listener != null) {
listener.onFirstFrameRendered(/* videoSink= */ this);
}
});
}
releaseProcessedFrameInternal(
frameReleaseAction == VideoFrameReleaseControl.FRAME_RELEASE_IMMEDIATELY
? VideoFrameProcessor.RENDER_OUTPUT_FRAME_IMMEDIATELY
: videoFrameReleaseInfo.getReleaseTimeNs(),
isLastFrame);
maybeNotifyVideoSizeChanged(bufferPresentationTimeUs);
}
private void releaseProcessedFrameInternal(long releaseTimeNs, boolean isLastFrame) {
videoFrameProcessor.renderOutputFrame(releaseTimeNs);
processedFramesBufferTimestampsUs.remove();
if (isLastFrame) {
releasedLastFrame = true;
}
}
private void maybeNotifyVideoSizeChanged(long bufferPresentationTimeUs) {
if (onVideoSizeChangedCalled || listener == null) {
return;
}
@Nullable VideoSize videoSize = videoSizeChanges.pollFloor(bufferPresentationTimeUs);
if (videoSize == null) {
return;
}
if (!videoSize.equals(VideoSize.UNKNOWN) && !videoSize.equals(reportedVideoSize)) {
reportedVideoSize = videoSize;
checkNotNull(listenerExecutor)
.execute(() -> checkNotNull(listener).onVideoSizeChanged(this, videoSize));
}
onVideoSizeChangedCalled = true;
}
private static final class ScaleAndRotateAccessor { private static final class ScaleAndRotateAccessor {
private static @MonotonicNonNull Constructor<?> private static @MonotonicNonNull Constructor<?>
scaleAndRotateTransformationBuilderConstructor; scaleAndRotateTransformationBuilderConstructor;

View File

@ -56,6 +56,7 @@ import androidx.media3.common.PlaybackException;
import androidx.media3.common.VideoFrameProcessingException; import androidx.media3.common.VideoFrameProcessingException;
import androidx.media3.common.VideoFrameProcessor; import androidx.media3.common.VideoFrameProcessor;
import androidx.media3.common.VideoSize; import androidx.media3.common.VideoSize;
import androidx.media3.common.util.Clock;
import androidx.media3.common.util.Log; import androidx.media3.common.util.Log;
import androidx.media3.common.util.MediaFormatUtil; import androidx.media3.common.util.MediaFormatUtil;
import androidx.media3.common.util.Size; import androidx.media3.common.util.Size;
@ -393,7 +394,6 @@ public class MediaCodecVideoRenderer extends MediaCodecRenderer
assumedMinimumCodecOperatingRate); assumedMinimumCodecOperatingRate);
this.maxDroppedFramesToNotify = maxDroppedFramesToNotify; this.maxDroppedFramesToNotify = maxDroppedFramesToNotify;
this.context = context.getApplicationContext(); this.context = context.getApplicationContext();
@SuppressWarnings("nullness:assignment") @SuppressWarnings("nullness:assignment")
VideoFrameReleaseControl.@Initialized FrameTimingEvaluator thisRef = this; VideoFrameReleaseControl.@Initialized FrameTimingEvaluator thisRef = this;
videoFrameReleaseControl = videoFrameReleaseControl =
@ -617,7 +617,9 @@ public class MediaCodecVideoRenderer extends MediaCodecRenderer
@Override @Override
protected void onInit() { protected void onInit() {
super.onInit(); super.onInit();
videoFrameReleaseControl.setClock(getClock()); Clock clock = getClock();
videoFrameReleaseControl.setClock(clock);
videoSinkProvider.setClock(clock);
} }
@Override @Override

View File

@ -0,0 +1,276 @@
/*
* 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
*
* 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.
*/
package androidx.media3.exoplayer.video;
import static androidx.media3.common.util.Assertions.checkArgument;
import static androidx.media3.common.util.Assertions.checkNotNull;
import static androidx.media3.common.util.Assertions.checkStateNotNull;
import androidx.annotation.FloatRange;
import androidx.annotation.Nullable;
import androidx.media3.common.C;
import androidx.media3.common.VideoFrameProcessor;
import androidx.media3.common.VideoSize;
import androidx.media3.common.util.LongArrayQueue;
import androidx.media3.common.util.TimedValueQueue;
import androidx.media3.common.util.Util;
import androidx.media3.exoplayer.ExoPlaybackException;
/** Controls rendering of video frames. */
/* package */ final class VideoFrameRenderControl {
/** Receives frames from a {@link VideoFrameRenderControl}. */
interface FrameRenderer {
/**
* Called when the {@link VideoSize} changes. This method is called before the frame that
* changes the {@link VideoSize} is passed for render.
*/
void onVideoSizeChanged(VideoSize videoSize);
/**
* Called to release the {@linkplain
* VideoFrameRenderControl#onOutputFrameAvailableForRendering(long)} oldest frame that is
* available for rendering}.
*
* @param renderTimeNs The specific time, in nano seconds, that this frame should be rendered or
* {@link VideoFrameProcessor#RENDER_OUTPUT_FRAME_IMMEDIATELY} if the frame needs to be
* rendered immediately.
* @param presentationTimeUs The frame's presentation time, in microseconds, which was announced
* with {@link VideoFrameRenderControl#onOutputFrameAvailableForRendering(long)}.
* @param streamOffsetUs The stream offset, in microseconds, that is associated with this frame.
* @param isFirstFrame Whether this is the first frame of the stream.
*/
void renderFrame(
long renderTimeNs, long presentationTimeUs, long streamOffsetUs, boolean isFirstFrame);
/**
* Called to drop the {@linkplain
* VideoFrameRenderControl#onOutputFrameAvailableForRendering(long)} oldest frame that is
* available for rendering}.
*/
void dropFrame();
}
private final FrameRenderer frameRenderer;
private final VideoFrameReleaseControl videoFrameReleaseControl;
private final VideoFrameReleaseControl.FrameReleaseInfo videoFrameReleaseInfo;
private final TimedValueQueue<VideoSize> videoSizeChanges;
private final TimedValueQueue<Long> streamOffsets;
private final LongArrayQueue presentationTimestampsUs;
/**
* Stores a video size that is announced with {@link #onOutputSizeChanged(int, int)} until an
* output frame is made available. Once the next frame arrives, we associate the frame's timestamp
* with the video size change in {@link #videoSizeChanges} and clear this field.
*/
@Nullable private VideoSize pendingOutputVideoSize;
private VideoSize reportedVideoSize;
private long outputStreamOffsetUs;
// TODO b/292111083 - Remove the field and trigger the callback on every video size change.
private boolean reportedVideoSizeChange;
private long lastPresentationTimeUs;
/** Creates an instance. */
public VideoFrameRenderControl(
FrameRenderer frameRenderer, VideoFrameReleaseControl videoFrameReleaseControl) {
this.frameRenderer = frameRenderer;
this.videoFrameReleaseControl = videoFrameReleaseControl;
videoFrameReleaseInfo = new VideoFrameReleaseControl.FrameReleaseInfo();
videoSizeChanges = new TimedValueQueue<>();
streamOffsets = new TimedValueQueue<>();
presentationTimestampsUs = new LongArrayQueue();
reportedVideoSize = VideoSize.UNKNOWN;
lastPresentationTimeUs = C.TIME_UNSET;
}
/** Flushes the renderer. */
public void flush() {
presentationTimestampsUs.clear();
lastPresentationTimeUs = C.TIME_UNSET;
if (streamOffsets.size() > 0) {
// There is a pending streaming offset change. If seeking within the same stream, keep the
// pending offset with timestamp zero ensures the offset is applied on the frames after
// flushing. Otherwise if seeking to another stream, a new offset will be set before a new
// frame arrives so we'll be able to apply the new offset.
long lastStreamOffset = getLastAndClear(streamOffsets);
streamOffsets.add(/* timestamp= */ 0, lastStreamOffset);
}
if (pendingOutputVideoSize == null) {
if (videoSizeChanges.size() > 0) {
// Do not clear the last pending video size, we still want to report the size change after a
// flush. If after the flush, a new video size is announced, it will overwrite
// pendingOutputVideoSize. When the next frame is available for rendering, we will announce
// pendingOutputVideoSize.
pendingOutputVideoSize = getLastAndClear(videoSizeChanges);
}
} else {
// we keep the latest value of pendingOutputVideoSize
videoSizeChanges.clear();
}
// Do not clear reportedVideoSizeChange because we report a video size change at most once
// (b/292111083).
}
/** Returns whether the renderer is ready. */
public boolean isReady() {
return videoFrameReleaseControl.isReady(/* rendererReady= */ true);
}
/**
* Returns whether the renderer has released a frame after a specific presentation timestamp.
*
* @param presentationTimeUs The requested timestamp, in microseconds.
* @return Whether the renderer has released a frame with a timestamp greater than or equal to
* {@code presentationTimeUs}.
*/
public boolean hasReleasedFrame(long presentationTimeUs) {
return lastPresentationTimeUs != C.TIME_UNSET && lastPresentationTimeUs >= presentationTimeUs;
}
/** Sets the playback speed. */
public void setPlaybackSpeed(@FloatRange(from = 0, fromInclusive = false) float speed) {
checkArgument(speed > 0);
videoFrameReleaseControl.setPlaybackSpeed(speed);
}
/**
* Incrementally renders available video frames.
*
* @param positionUs The current playback position, in microseconds.
* @param elapsedRealtimeUs {@link android.os.SystemClock#elapsedRealtime()} in microseconds,
* taken approximately at the time the playback position was {@code positionUs}.
*/
public void render(long positionUs, long elapsedRealtimeUs) throws ExoPlaybackException {
while (!presentationTimestampsUs.isEmpty()) {
long presentationTimeUs = presentationTimestampsUs.element();
// Check whether this buffer comes with a new stream offset.
if (maybeUpdateOutputStreamOffset(presentationTimeUs)) {
videoFrameReleaseControl.onProcessedStreamChange();
}
@VideoFrameReleaseControl.FrameReleaseAction
int frameReleaseAction =
videoFrameReleaseControl.getFrameReleaseAction(
presentationTimeUs,
positionUs,
elapsedRealtimeUs,
outputStreamOffsetUs,
/* isLastFrame= */ false,
videoFrameReleaseInfo);
switch (frameReleaseAction) {
case VideoFrameReleaseControl.FRAME_RELEASE_TRY_AGAIN_LATER:
return;
case VideoFrameReleaseControl.FRAME_RELEASE_SKIP:
case VideoFrameReleaseControl.FRAME_RELEASE_DROP:
case VideoFrameReleaseControl.FRAME_RELEASE_IGNORE:
// TODO b/293873191 - Handle very late buffers and drop to key frame. Need to flush
// VideoGraph input frames in this case.
lastPresentationTimeUs = presentationTimeUs;
dropFrame();
break;
case VideoFrameReleaseControl.FRAME_RELEASE_IMMEDIATELY:
case VideoFrameReleaseControl.FRAME_RELEASE_SCHEDULED:
lastPresentationTimeUs = presentationTimeUs;
renderFrame(
/* shouldRenderImmediately= */ frameReleaseAction
== VideoFrameReleaseControl.FRAME_RELEASE_IMMEDIATELY);
break;
default:
throw new IllegalStateException(String.valueOf(frameReleaseAction));
}
}
}
/** Called when the size of the available frames has changed. */
public void onOutputSizeChanged(int width, int height) {
VideoSize newVideoSize = new VideoSize(width, height);
if (!Util.areEqual(pendingOutputVideoSize, newVideoSize)) {
pendingOutputVideoSize = newVideoSize;
}
}
/**
* Called when a frame is available for rendering.
*
* @param presentationTimeUs The frame's presentation timestamp, in microseconds.
*/
public void onOutputFrameAvailableForRendering(long presentationTimeUs) {
if (pendingOutputVideoSize != null) {
videoSizeChanges.add(presentationTimeUs, pendingOutputVideoSize);
pendingOutputVideoSize = null;
}
presentationTimestampsUs.add(presentationTimeUs);
// TODO b/257464707 - Support extensively modified media.
}
public void onStreamOffsetChange(long presentationTimeUs, long streamOffsetUs) {
streamOffsets.add(presentationTimeUs, streamOffsetUs);
}
private void dropFrame() {
checkStateNotNull(presentationTimestampsUs.remove());
frameRenderer.dropFrame();
}
private void renderFrame(boolean shouldRenderImmediately) {
long presentationTimeUs = checkStateNotNull(presentationTimestampsUs.remove());
boolean videoSizeUpdated = maybeUpdateVideoSize(presentationTimeUs);
if (videoSizeUpdated && !reportedVideoSizeChange) {
frameRenderer.onVideoSizeChanged(reportedVideoSize);
reportedVideoSizeChange = true;
}
long renderTimeNs =
shouldRenderImmediately
? VideoFrameProcessor.RENDER_OUTPUT_FRAME_IMMEDIATELY
: videoFrameReleaseInfo.getReleaseTimeNs();
frameRenderer.renderFrame(
renderTimeNs,
presentationTimeUs,
outputStreamOffsetUs,
videoFrameReleaseControl.onFrameReleasedIsFirstFrame());
}
private boolean maybeUpdateOutputStreamOffset(long presentationTimeUs) {
@Nullable Long newOutputStreamOffsetUs = streamOffsets.pollFloor(presentationTimeUs);
if (newOutputStreamOffsetUs != null && newOutputStreamOffsetUs != outputStreamOffsetUs) {
outputStreamOffsetUs = newOutputStreamOffsetUs;
return true;
}
return false;
}
private boolean maybeUpdateVideoSize(long presentationTimeUs) {
@Nullable VideoSize videoSize = videoSizeChanges.pollFloor(presentationTimeUs);
if (videoSize == null) {
return false;
}
if (!videoSize.equals(VideoSize.UNKNOWN) && !videoSize.equals(reportedVideoSize)) {
reportedVideoSize = videoSize;
return true;
}
return false;
}
private static <T> T getLastAndClear(TimedValueQueue<T> queue) {
checkArgument(queue.size() > 0);
while (queue.size() > 1) {
queue.pollFirst();
}
return checkNotNull(queue.pollFirst());
}
}

View File

@ -58,11 +58,30 @@ import java.util.concurrent.Executor;
/** Called when the sink dropped a frame. */ /** Called when the sink dropped a frame. */
void onFrameDropped(VideoSink videoSink); void onFrameDropped(VideoSink videoSink);
/** Called when the output video size changed. */ /**
* Called before a frame is rendered for the first time since setting the surface, and each time
* there's a change in the size, rotation or pixel aspect ratio of the video being rendered.
*/
void onVideoSizeChanged(VideoSink videoSink, VideoSize videoSize); void onVideoSizeChanged(VideoSink videoSink, VideoSize videoSize);
/** Called when the {@link VideoSink} encountered an error. */ /** Called when the {@link VideoSink} encountered an error. */
void onError(VideoSink videoSink, VideoSinkException videoSinkException); void onError(VideoSink videoSink, VideoSinkException videoSinkException);
/** A no-op listener implementation. */
Listener NO_OP =
new Listener() {
@Override
public void onFirstFrameRendered(VideoSink videoSink) {}
@Override
public void onFrameDropped(VideoSink videoSink) {}
@Override
public void onVideoSizeChanged(VideoSink videoSink, VideoSize videoSize) {}
@Override
public void onError(VideoSink videoSink, VideoSinkException videoSinkException) {}
};
} }
/** /**

View File

@ -19,6 +19,7 @@ package androidx.media3.exoplayer.video;
import android.view.Surface; import android.view.Surface;
import androidx.media3.common.Effect; import androidx.media3.common.Effect;
import androidx.media3.common.Format; import androidx.media3.common.Format;
import androidx.media3.common.util.Clock;
import androidx.media3.common.util.Size; import androidx.media3.common.util.Size;
import androidx.media3.common.util.UnstableApi; import androidx.media3.common.util.UnstableApi;
import java.util.List; import java.util.List;
@ -78,4 +79,11 @@ public interface VideoSinkProvider {
/** Sets a {@link VideoFrameMetadataListener} which is used in the returned {@link VideoSink}. */ /** Sets a {@link VideoFrameMetadataListener} which is used in the returned {@link VideoSink}. */
void setVideoFrameMetadataListener(VideoFrameMetadataListener videoFrameMetadataListener); void setVideoFrameMetadataListener(VideoFrameMetadataListener videoFrameMetadataListener);
/**
* Sets the {@link Clock} that the provider should use internally.
*
* <p>Must be called before the sink provider is {@linkplain #initialize(Format) initialized}.
*/
void setClock(Clock clock);
} }

View File

@ -0,0 +1,372 @@
/*
* 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
*
* 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.
*/
package androidx.media3.exoplayer.video;
import static com.google.common.truth.Truth.assertThat;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import androidx.media3.common.VideoSize;
import androidx.media3.exoplayer.ExoPlaybackException;
import androidx.media3.test.utils.FakeClock;
import androidx.test.core.app.ApplicationProvider;
import androidx.test.ext.junit.runners.AndroidJUnit4;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.InOrder;
import org.mockito.Mockito;
/** Tests for {@link VideoFrameRenderControl}. */
@RunWith(AndroidJUnit4.class)
public class VideoFrameRenderControlTest {
private static final int VIDEO_WIDTH = 640;
private static final int VIDEO_HEIGHT = 480;
@Test
public void isReady_afterInstantiation_returnsFalse() {
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(
mock(VideoFrameRenderControl.FrameRenderer.class), createVideoFrameReleaseControl());
assertThat(videoFrameRenderControl.isReady()).isFalse();
}
@Test
public void releaseFirstFrame() throws Exception {
VideoFrameRenderControl.FrameRenderer frameRenderer =
mock(VideoFrameRenderControl.FrameRenderer.class);
VideoFrameReleaseControl videoFrameReleaseControl = createVideoFrameReleaseControl();
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(frameRenderer, videoFrameReleaseControl);
videoFrameReleaseControl.onEnabled(/* releaseFirstFrameBeforeStarted= */ true);
videoFrameRenderControl.onOutputSizeChanged(
/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 0);
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
assertThat(videoFrameRenderControl.isReady()).isTrue();
InOrder inOrder = Mockito.inOrder(frameRenderer);
inOrder
.verify(frameRenderer)
.onVideoSizeChanged(new VideoSize(/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT));
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(0L),
/* streamOffsetUs= */ eq(0L),
/* isFirstFrame= */ eq(true));
}
@Test
public void releaseFirstAndSecondFrame() throws Exception {
VideoFrameRenderControl.FrameRenderer frameRenderer =
mock(VideoFrameRenderControl.FrameRenderer.class);
FakeClock clock = new FakeClock(/* isAutoAdvancing= */ false);
VideoFrameReleaseControl videoFrameReleaseControl = createVideoFrameReleaseControl();
videoFrameReleaseControl.setClock(clock);
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(frameRenderer, videoFrameReleaseControl);
videoFrameReleaseControl.onEnabled(/* releaseFirstFrameBeforeStarted= */ true);
videoFrameReleaseControl.onStarted();
videoFrameRenderControl.onOutputSizeChanged(
/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 0);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 10_000);
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
assertThat(videoFrameRenderControl.isReady()).isTrue();
InOrder inOrder = Mockito.inOrder(frameRenderer);
inOrder
.verify(frameRenderer)
.onVideoSizeChanged(new VideoSize(/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT));
// First frame.
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(0L),
/* streamOffsetUs= */ eq(0L),
/* isFirstFrame= */ eq(true));
inOrder.verifyNoMoreInteractions();
// 5 seconds pass
clock.advanceTime(/* timeDiffMs= */ 5);
videoFrameRenderControl.render(/* positionUs= */ 5_000, /* elapsedRealtimeUs= */ 5_000);
// Second frame
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(10_000L),
/* streamOffsetUs= */ eq(0L),
/* isFirstFrame= */ eq(false));
inOrder.verifyNoMoreInteractions();
}
@Test
public void renderFrames_withStreamOffsetSetChange_firstFrameAgain() throws Exception {
VideoFrameRenderControl.FrameRenderer frameRenderer =
mock(VideoFrameRenderControl.FrameRenderer.class);
FakeClock clock = new FakeClock(/* isAutoAdvancing= */ false);
VideoFrameReleaseControl videoFrameReleaseControl = createVideoFrameReleaseControl();
videoFrameReleaseControl.setClock(clock);
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(frameRenderer, videoFrameReleaseControl);
videoFrameReleaseControl.onEnabled(/* releaseFirstFrameBeforeStarted= */ true);
videoFrameReleaseControl.onStarted();
videoFrameRenderControl.onOutputSizeChanged(
/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT);
videoFrameRenderControl.onStreamOffsetChange(
/* presentationTimeUs= */ 0, /* streamOffsetUs= */ 10_000);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 0);
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
assertThat(videoFrameRenderControl.isReady()).isTrue();
InOrder inOrder = Mockito.inOrder(frameRenderer);
inOrder
.verify(frameRenderer)
.onVideoSizeChanged(new VideoSize(/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT));
// First frame has the first stream offset.
inOrder.verify(frameRenderer).renderFrame(anyLong(), eq(0L), eq(10_000L), eq(true));
inOrder.verifyNoMoreInteractions();
// 10 milliseconds pass
clock.advanceTime(/* timeDiffMs= */ 10);
videoFrameRenderControl.onStreamOffsetChange(
/* presentationTimeUs= */ 10_000, /* streamOffsetUs= */ 20_000);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 10_000);
videoFrameRenderControl.render(/* positionUs= */ 10_000, /* elapsedRealtimeUs= */ 0);
// Second frame has the second stream offset and it is also a first frame.
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(10_000L),
/* streamOffsetUs= */ eq(20_000L),
/* isFirstFrame= */ eq(true));
inOrder.verifyNoMoreInteractions();
}
@Test
public void dropFrames() throws Exception {
VideoFrameRenderControl.FrameRenderer frameRenderer =
mock(VideoFrameRenderControl.FrameRenderer.class);
FakeClock clock = new FakeClock(/* isAutoAdvancing= */ false);
VideoFrameReleaseControl videoFrameReleaseControl =
createVideoFrameReleaseControl(
new TestFrameTimingEvaluator(
/* shouldForceReleaseFrames= */ false,
/* shouldDropFrames= */ true,
/* shouldIgnoreFrames= */ false));
videoFrameReleaseControl.setClock(clock);
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(frameRenderer, videoFrameReleaseControl);
videoFrameReleaseControl.onEnabled(/* releaseFirstFrameBeforeStarted= */ true);
videoFrameReleaseControl.onStarted();
videoFrameRenderControl.onOutputSizeChanged(
/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 0);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 10_000);
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
InOrder inOrder = Mockito.inOrder(frameRenderer);
inOrder
.verify(frameRenderer)
.onVideoSizeChanged(new VideoSize(/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT));
// First frame was rendered because the fist frame is force released.
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(0L),
/* streamOffsetUs= */ eq(0L),
/* isFirstFrame= */ eq(true));
inOrder.verifyNoMoreInteractions();
clock.advanceTime(/* timeDiffMs= */ 100);
videoFrameRenderControl.render(/* positionUs= */ 100_000, /* elapsedRealtimeUs= */ 100_000);
// Second frame was dropped.
inOrder.verify(frameRenderer).dropFrame();
inOrder.verifyNoMoreInteractions();
}
@Test
public void flush_removesAvailableFramesForRendering_doesNotFlushOnVideoSizeChange()
throws Exception {
VideoFrameRenderControl.FrameRenderer frameRenderer =
mock(VideoFrameRenderControl.FrameRenderer.class);
VideoFrameReleaseControl videoFrameReleaseControl = createVideoFrameReleaseControl();
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(frameRenderer, videoFrameReleaseControl);
videoFrameReleaseControl.onEnabled(/* releaseFirstFrameBeforeStarted= */ true);
videoFrameReleaseControl.onStarted();
videoFrameRenderControl.onOutputSizeChanged(
/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 0);
videoFrameRenderControl.flush();
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
InOrder inOrder = Mockito.inOrder(frameRenderer);
inOrder.verifyNoMoreInteractions();
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 10_000);
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
// First frame was rendered with pending video size change.
inOrder
.verify(frameRenderer)
.onVideoSizeChanged(new VideoSize(/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT));
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(10_000L),
/* streamOffsetUs= */ eq(0L),
/* isFirstFrame= */ eq(true));
inOrder.verifyNoMoreInteractions();
}
@Test
public void hasReleasedFrame_noFrameReleased_returnsFalse() {
VideoFrameReleaseControl videoFrameReleaseControl = createVideoFrameReleaseControl();
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(
mock(VideoFrameRenderControl.FrameRenderer.class), videoFrameReleaseControl);
assertThat(videoFrameRenderControl.hasReleasedFrame(/* presentationTimeUs= */ 0)).isFalse();
}
@Test
public void hasReleasedFrame_frameIsReleased_returnsTrue() throws Exception {
VideoFrameRenderControl.FrameRenderer frameRenderer =
mock(VideoFrameRenderControl.FrameRenderer.class);
VideoFrameReleaseControl videoFrameReleaseControl = createVideoFrameReleaseControl();
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(frameRenderer, videoFrameReleaseControl);
videoFrameReleaseControl.onEnabled(/* releaseFirstFrameBeforeStarted= */ true);
videoFrameRenderControl.onOutputSizeChanged(
/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 0);
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
InOrder inOrder = Mockito.inOrder(frameRenderer);
inOrder
.verify(frameRenderer)
.onVideoSizeChanged(new VideoSize(/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT));
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(0L),
/* streamOffsetUs= */ eq(0L),
/* isFirstFrame= */ eq(true));
assertThat(videoFrameRenderControl.hasReleasedFrame(/* presentationTimeUs= */ 0)).isTrue();
}
@Test
public void hasReleasedFrame_frameIsReleasedAndFlushed_returnsFalse() throws Exception {
VideoFrameRenderControl.FrameRenderer frameRenderer =
mock(VideoFrameRenderControl.FrameRenderer.class);
VideoFrameReleaseControl videoFrameReleaseControl = createVideoFrameReleaseControl();
VideoFrameRenderControl videoFrameRenderControl =
new VideoFrameRenderControl(frameRenderer, videoFrameReleaseControl);
videoFrameReleaseControl.onEnabled(/* releaseFirstFrameBeforeStarted= */ true);
videoFrameRenderControl.onOutputSizeChanged(
/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT);
videoFrameRenderControl.onOutputFrameAvailableForRendering(/* presentationTimeUs= */ 0);
videoFrameRenderControl.render(/* positionUs= */ 0, /* elapsedRealtimeUs= */ 0);
InOrder inOrder = Mockito.inOrder(frameRenderer);
inOrder
.verify(frameRenderer)
.onVideoSizeChanged(new VideoSize(/* width= */ VIDEO_WIDTH, /* height= */ VIDEO_HEIGHT));
inOrder
.verify(frameRenderer)
.renderFrame(
/* renderTimeNs= */ anyLong(),
/* presentationTimeUs= */ eq(0L),
/* streamOffsetUs= */ eq(0L),
/* isFirstFrame= */ eq(true));
videoFrameRenderControl.flush();
assertThat(videoFrameRenderControl.hasReleasedFrame(/* presentationTimeUs= */ 0)).isFalse();
}
private static VideoFrameReleaseControl createVideoFrameReleaseControl() {
return createVideoFrameReleaseControl(
new TestFrameTimingEvaluator(
/* shouldForceReleaseFrames= */ false,
/* shouldDropFrames= */ false,
/* shouldIgnoreFrames= */ false));
}
private static VideoFrameReleaseControl createVideoFrameReleaseControl(
VideoFrameReleaseControl.FrameTimingEvaluator frameTimingEvaluator) {
return new VideoFrameReleaseControl(
ApplicationProvider.getApplicationContext(),
frameTimingEvaluator,
/* allowedJoiningTimeMs= */ 0);
}
private static class TestFrameTimingEvaluator
implements VideoFrameReleaseControl.FrameTimingEvaluator {
private final boolean shouldForceReleaseFrames;
private final boolean shouldDropFrames;
private final boolean shouldIgnoreFrames;
public TestFrameTimingEvaluator(
boolean shouldForceReleaseFrames, boolean shouldDropFrames, boolean shouldIgnoreFrames) {
this.shouldForceReleaseFrames = shouldForceReleaseFrames;
this.shouldDropFrames = shouldDropFrames;
this.shouldIgnoreFrames = shouldIgnoreFrames;
}
@Override
public boolean shouldForceReleaseFrame(long earlyUs, long elapsedSinceLastReleaseUs) {
return shouldForceReleaseFrames;
}
@Override
public boolean shouldDropFrame(long earlyUs, long elapsedRealtimeUs, boolean isLastFrame) {
return shouldDropFrames;
}
@Override
public boolean shouldIgnoreFrame(
long earlyUs,
long positionUs,
long elapsedRealtimeUs,
boolean isLastFrame,
boolean treatDroppedBuffersAsSkipped)
throws ExoPlaybackException {
return shouldIgnoreFrames;
}
}
}