Retain stream time offsets through codecs

ExoPlayer applies a large time offset to buffers so that, if the input has negative timestamps, generally buffers seen by the decoders should have positive timestamps. Modify how the offset is handled in `Transformer` so that decoders and encoders generally see positive timestamps, by leaving the offset on samples when reading them in the base renderer (remove the code that removed the offset), and then removing the offset when muxing. Also update the frame processor chain and slow motion flattening code to retain the existing behavior after this change (these both need original media presentation timestamps)

Tested via existing end-to-end tests and manually verified that the overlay frame processor shows the expected original media timestamps.

Aside: we don't need the same logic as ExoPlayer to track stream offsets across the decoder yet, because we don't try to handle stream changes during playback in single asset editing. (There is an edge case of multi-period DASH that may not work but I doubt anyone will use that as input to `Transformer` before we change the code to handle multi-asset properly.) In future we should try to refactor interaction with the decoder to use the same code for Transformer and ExoPlayer.
PiperOrigin-RevId: 451846055
This commit is contained in:
andrewlewis 2022-05-30 11:04:24 +00:00 committed by Marc Baechinger
parent cad1440e66
commit b25d00a795
10 changed files with 69 additions and 32 deletions

View File

@ -309,6 +309,7 @@ public final class FrameProcessorChainPixelTest {
pixelWidthHeightRatio, pixelWidthHeightRatio,
inputWidth, inputWidth,
inputHeight, inputHeight,
/* streamOffsetUs= */ 0L,
effects, effects,
/* enableExperimentalHdrEditing= */ false); /* enableExperimentalHdrEditing= */ false);
Size outputSize = frameProcessorChain.getOutputSize(); Size outputSize = frameProcessorChain.getOutputSize();

View File

@ -130,6 +130,7 @@ public final class FrameProcessorChainTest {
pixelWidthHeightRatio, pixelWidthHeightRatio,
inputSize.getWidth(), inputSize.getWidth(),
inputSize.getHeight(), inputSize.getHeight(),
/* streamOffsetUs= */ 0L,
effects.build(), effects.build(),
/* enableExperimentalHdrEditing= */ false); /* enableExperimentalHdrEditing= */ false);
} }

View File

@ -56,6 +56,7 @@ import org.checkerframework.dataflow.qual.Pure;
public AudioTranscodingSamplePipeline( public AudioTranscodingSamplePipeline(
Format inputFormat, Format inputFormat,
long streamOffsetUs,
TransformationRequest transformationRequest, TransformationRequest transformationRequest,
Codec.DecoderFactory decoderFactory, Codec.DecoderFactory decoderFactory,
Codec.EncoderFactory encoderFactory, Codec.EncoderFactory encoderFactory,
@ -108,6 +109,9 @@ import org.checkerframework.dataflow.qual.Pure;
fallbackListener.onTransformationRequestFinalized( fallbackListener.onTransformationRequestFinalized(
createFallbackTransformationRequest( createFallbackTransformationRequest(
transformationRequest, requestedOutputFormat, encoder.getConfigurationFormat())); transformationRequest, requestedOutputFormat, encoder.getConfigurationFormat()));
// Use the same stream offset as the input stream for encoder input buffers.
nextEncoderInputBufferTimeUs = streamOffsetUs;
} }
@Override @Override

View File

@ -101,6 +101,7 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
float pixelWidthHeightRatio, float pixelWidthHeightRatio,
int inputWidth, int inputWidth,
int inputHeight, int inputHeight,
long streamOffsetUs,
List<GlEffect> effects, List<GlEffect> effects,
boolean enableExperimentalHdrEditing) boolean enableExperimentalHdrEditing)
throws FrameProcessingException { throws FrameProcessingException {
@ -119,6 +120,7 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
pixelWidthHeightRatio, pixelWidthHeightRatio,
inputWidth, inputWidth,
inputHeight, inputHeight,
streamOffsetUs,
effects, effects,
enableExperimentalHdrEditing, enableExperimentalHdrEditing,
singleThreadExecutorService)) singleThreadExecutorService))
@ -145,6 +147,7 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
float pixelWidthHeightRatio, float pixelWidthHeightRatio,
int inputWidth, int inputWidth,
int inputHeight, int inputHeight,
long streamOffsetUs,
List<GlEffect> effects, List<GlEffect> effects,
boolean enableExperimentalHdrEditing, boolean enableExperimentalHdrEditing,
ExecutorService singleThreadExecutorService) ExecutorService singleThreadExecutorService)
@ -190,6 +193,7 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
eglContext, eglContext,
singleThreadExecutorService, singleThreadExecutorService,
inputExternalTexId, inputExternalTexId,
streamOffsetUs,
framebuffers, framebuffers,
textureProcessors, textureProcessors,
listener, listener,
@ -252,6 +256,11 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
private final EGLContext eglContext; private final EGLContext eglContext;
/** Some OpenGL commands may block, so all OpenGL commands are run on a background thread. */ /** Some OpenGL commands may block, so all OpenGL commands are run on a background thread. */
private final ExecutorService singleThreadExecutorService; private final ExecutorService singleThreadExecutorService;
/**
* Offset compared to original media presentation time that has been added to incoming frame
* timestamps, in microseconds.
*/
private final long streamOffsetUs;
/** Futures corresponding to the executor service's pending tasks. */ /** Futures corresponding to the executor service's pending tasks. */
private final ConcurrentLinkedQueue<Future<?>> futures; private final ConcurrentLinkedQueue<Future<?>> futures;
/** Number of frames {@linkplain #registerInputFrame() registered} but not fully processed. */ /** Number of frames {@linkplain #registerInputFrame() registered} but not fully processed. */
@ -308,6 +317,7 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
EGLContext eglContext, EGLContext eglContext,
ExecutorService singleThreadExecutorService, ExecutorService singleThreadExecutorService,
int inputExternalTexId, int inputExternalTexId,
long streamOffsetUs,
int[] framebuffers, int[] framebuffers,
ImmutableList<SingleFrameGlTextureProcessor> textureProcessors, ImmutableList<SingleFrameGlTextureProcessor> textureProcessors,
Listener listener, Listener listener,
@ -317,6 +327,7 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
this.eglDisplay = eglDisplay; this.eglDisplay = eglDisplay;
this.eglContext = eglContext; this.eglContext = eglContext;
this.singleThreadExecutorService = singleThreadExecutorService; this.singleThreadExecutorService = singleThreadExecutorService;
this.streamOffsetUs = streamOffsetUs;
this.framebuffers = framebuffers; this.framebuffers = framebuffers;
this.textureProcessors = textureProcessors; this.textureProcessors = textureProcessors;
this.listener = listener; this.listener = listener;
@ -476,8 +487,9 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
} }
inputSurfaceTexture.updateTexImage(); inputSurfaceTexture.updateTexImage();
long presentationTimeNs = inputSurfaceTexture.getTimestamp(); long inputFrameTimeNs = inputSurfaceTexture.getTimestamp();
presentationTimeUs = presentationTimeNs / 1000; // Correct for the stream offset so processors see original media presentation timestamps.
presentationTimeUs = inputFrameTimeNs / 1000 - streamOffsetUs;
inputSurfaceTexture.getTransformMatrix(textureTransformMatrix); inputSurfaceTexture.getTransformMatrix(textureTransformMatrix);
((ExternalTextureProcessor) textureProcessors.get(0)) ((ExternalTextureProcessor) textureProcessors.get(0))
.setTextureTransformMatrix(textureTransformMatrix); .setTextureTransformMatrix(textureTransformMatrix);
@ -502,7 +514,7 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
clearOutputFrame(); clearOutputFrame();
getLast(textureProcessors).drawFrame(presentationTimeUs); getLast(textureProcessors).drawFrame(presentationTimeUs);
EGLExt.eglPresentationTimeANDROID(eglDisplay, outputEglSurface, presentationTimeNs); EGLExt.eglPresentationTimeANDROID(eglDisplay, outputEglSurface, inputFrameTimeNs);
EGL14.eglSwapBuffers(eglDisplay, outputEglSurface); EGL14.eglSwapBuffers(eglDisplay, outputEglSurface);
if (debugSurfaceViewWrapper != null) { if (debugSurfaceViewWrapper != null) {

View File

@ -25,17 +25,14 @@ import androidx.media3.decoder.DecoderInputBuffer;
private final DecoderInputBuffer buffer; private final DecoderInputBuffer buffer;
private final Format format; private final Format format;
private final long outputPresentationTimeOffsetUs;
private boolean hasPendingBuffer; private boolean hasPendingBuffer;
public PassthroughSamplePipeline( public PassthroughSamplePipeline(
Format format, Format format,
long outputPresentationTimeOffsetUs,
TransformationRequest transformationRequest, TransformationRequest transformationRequest,
FallbackListener fallbackListener) { FallbackListener fallbackListener) {
this.format = format; this.format = format;
this.outputPresentationTimeOffsetUs = outputPresentationTimeOffsetUs;
buffer = new DecoderInputBuffer(DecoderInputBuffer.BUFFER_REPLACEMENT_MODE_DIRECT); buffer = new DecoderInputBuffer(DecoderInputBuffer.BUFFER_REPLACEMENT_MODE_DIRECT);
hasPendingBuffer = false; hasPendingBuffer = false;
fallbackListener.onTransformationRequestFinalized(transformationRequest); fallbackListener.onTransformationRequestFinalized(transformationRequest);
@ -49,7 +46,6 @@ import androidx.media3.decoder.DecoderInputBuffer;
@Override @Override
public void queueInputBuffer() { public void queueInputBuffer() {
buffer.timeUs -= outputPresentationTimeOffsetUs;
hasPendingBuffer = true; hasPendingBuffer = true;
} }

View File

@ -28,7 +28,6 @@ import androidx.media3.common.Format;
import androidx.media3.common.Metadata; import androidx.media3.common.Metadata;
import androidx.media3.common.MimeTypes; import androidx.media3.common.MimeTypes;
import androidx.media3.common.util.Util; import androidx.media3.common.util.Util;
import androidx.media3.decoder.DecoderInputBuffer;
import androidx.media3.extractor.metadata.mp4.SlowMotionData; import androidx.media3.extractor.metadata.mp4.SlowMotionData;
import androidx.media3.extractor.metadata.mp4.SmtaMetadataEntry; import androidx.media3.extractor.metadata.mp4.SmtaMetadataEntry;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
@ -106,9 +105,15 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
* segments into account, in microseconds. * segments into account, in microseconds.
*/ */
private long frameTimeDeltaUs; private long frameTimeDeltaUs;
/**
* The presentation time for the last {@linkplain #dropOrTransformSample(ByteBuffer, long)
* processed sample}.
*/
private long lastSamplePresentationTimeUs;
public SefSlowMotionFlattener(Format format) { public SefSlowMotionFlattener(Format format) {
scratch = new byte[NAL_START_CODE_LENGTH]; scratch = new byte[NAL_START_CODE_LENGTH];
lastSamplePresentationTimeUs = C.TIME_UNSET;
MetadataInfo metadataInfo = getMetadataInfo(format.metadata); MetadataInfo metadataInfo = getMetadataInfo(format.metadata);
slowMotionData = metadataInfo.slowMotionData; slowMotionData = metadataInfo.slowMotionData;
List<SlowMotionData.Segment> segments = List<SlowMotionData.Segment> segments =
@ -132,36 +137,47 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
* Applies slow motion flattening by either indicating that the buffer's data should be dropped or * Applies slow motion flattening by either indicating that the buffer's data should be dropped or
* transforming it in place. * transforming it in place.
* *
* <p>After calling this method, call {@link #getSamplePresentationTimeUs()} to get the new
* presentation time for the sample (whether it was dropped or not).
*
* @return Whether the buffer should be dropped. * @return Whether the buffer should be dropped.
*/ */
@RequiresNonNull("#1.data") public boolean dropOrTransformSample(ByteBuffer buffer, long bufferTimeUs) {
public boolean dropOrTransformSample(DecoderInputBuffer buffer) {
if (slowMotionData == null) { if (slowMotionData == null) {
// The input is not an SEF slow motion video. // The input is not an SEF slow motion video.
lastSamplePresentationTimeUs = bufferTimeUs;
return false; return false;
} }
ByteBuffer data = buffer.data; int originalPosition = buffer.position();
int originalPosition = data.position(); buffer.position(originalPosition + NAL_START_CODE_LENGTH);
data.position(originalPosition + NAL_START_CODE_LENGTH); buffer.get(scratch, 0, 4); // Read nal_unit_header_svc_extension.
data.get(scratch, 0, 4); // Read nal_unit_header_svc_extension.
int nalUnitType = scratch[0] & 0x1F; int nalUnitType = scratch[0] & 0x1F;
boolean svcExtensionFlag = ((scratch[1] & 0xFF) >> 7) == 1; boolean svcExtensionFlag = ((scratch[1] & 0xFF) >> 7) == 1;
checkState( checkState(
nalUnitType == NAL_UNIT_TYPE_PREFIX && svcExtensionFlag, nalUnitType == NAL_UNIT_TYPE_PREFIX && svcExtensionFlag,
"Missing SVC extension prefix NAL unit."); "Missing SVC extension prefix NAL unit.");
int layer = (scratch[3] & 0xFF) >> 5; int layer = (scratch[3] & 0xFF) >> 5;
boolean shouldKeepFrame = processCurrentFrame(layer, buffer.timeUs); boolean shouldKeepFrame = processCurrentFrame(layer, bufferTimeUs);
// Update the timestamp regardless of whether the buffer is dropped as the timestamp may be // Update the timestamp regardless of whether the buffer is dropped as the timestamp may be
// reused for the empty end-of-stream buffer. // reused for the empty end-of-stream buffer.
buffer.timeUs = getCurrentFrameOutputTimeUs(/* inputTimeUs= */ buffer.timeUs); lastSamplePresentationTimeUs = getCurrentFrameOutputTimeUs(bufferTimeUs);
if (shouldKeepFrame) { if (shouldKeepFrame) {
data.position(originalPosition); buffer.position(originalPosition);
return false; return false;
} }
return true; return true;
} }
/**
* Returns the new presentation time for the last sample handled via {@link
* #dropOrTransformSample(ByteBuffer, long)}.
*/
public long getSamplePresentationTimeUs() {
checkState(lastSamplePresentationTimeUs != C.TIME_UNSET);
return lastSamplePresentationTimeUs;
}
/** /**
* Processes the current frame and returns whether it should be kept. * Processes the current frame and returns whether it should be kept.
* *

View File

@ -70,12 +70,12 @@ import androidx.media3.extractor.metadata.mp4.SlowMotionData;
Format inputFormat = checkNotNull(formatHolder.format); Format inputFormat = checkNotNull(formatHolder.format);
if (shouldPassthrough(inputFormat)) { if (shouldPassthrough(inputFormat)) {
samplePipeline = samplePipeline =
new PassthroughSamplePipeline( new PassthroughSamplePipeline(inputFormat, transformationRequest, fallbackListener);
inputFormat, startPositionOffsetUs, transformationRequest, fallbackListener);
} else { } else {
samplePipeline = samplePipeline =
new AudioTranscodingSamplePipeline( new AudioTranscodingSamplePipeline(
inputFormat, inputFormat,
streamOffsetUs,
transformationRequest, transformationRequest,
decoderFactory, decoderFactory,
encoderFactory, encoderFactory,

View File

@ -45,7 +45,7 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
protected boolean muxerWrapperTrackAdded; protected boolean muxerWrapperTrackAdded;
protected boolean muxerWrapperTrackEnded; protected boolean muxerWrapperTrackEnded;
protected long streamOffsetUs; protected long streamOffsetUs;
protected long startPositionOffsetUs; protected long streamStartPositionUs;
protected @MonotonicNonNull SamplePipeline samplePipeline; protected @MonotonicNonNull SamplePipeline samplePipeline;
public TransformerBaseRenderer( public TransformerBaseRenderer(
@ -110,7 +110,7 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
@Override @Override
protected final void onStreamChanged(Format[] formats, long startPositionUs, long offsetUs) { protected final void onStreamChanged(Format[] formats, long startPositionUs, long offsetUs) {
this.streamOffsetUs = offsetUs; this.streamOffsetUs = offsetUs;
this.startPositionOffsetUs = startPositionUs - offsetUs; this.streamStartPositionUs = startPositionUs;
} }
@Override @Override
@ -178,11 +178,14 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
return false; return false;
} }
long samplePresentationTimeUs = samplePipelineOutputBuffer.timeUs - streamStartPositionUs;
// TODO(b/204892224): Consider subtracting the first sample timestamp from the sample pipeline
// buffer from all samples so that they are guaranteed to start from zero in the output file.
if (!muxerWrapper.writeSample( if (!muxerWrapper.writeSample(
getTrackType(), getTrackType(),
checkStateNotNull(samplePipelineOutputBuffer.data), checkStateNotNull(samplePipelineOutputBuffer.data),
samplePipelineOutputBuffer.isKeyFrame(), samplePipelineOutputBuffer.isKeyFrame(),
samplePipelineOutputBuffer.timeUs)) { samplePresentationTimeUs)) {
return false; return false;
} }
samplePipeline.releaseOutputBuffer(); samplePipeline.releaseOutputBuffer();
@ -212,7 +215,6 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
return false; return false;
} }
mediaClock.updateTimeForTrackType(getTrackType(), samplePipelineInputBuffer.timeUs); mediaClock.updateTimeForTrackType(getTrackType(), samplePipelineInputBuffer.timeUs);
samplePipelineInputBuffer.timeUs -= streamOffsetUs;
checkStateNotNull(samplePipelineInputBuffer.data); checkStateNotNull(samplePipelineInputBuffer.data);
maybeQueueSampleToPipeline(samplePipelineInputBuffer); maybeQueueSampleToPipeline(samplePipelineInputBuffer);
return true; return true;

View File

@ -89,14 +89,13 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
Format inputFormat = checkNotNull(formatHolder.format); Format inputFormat = checkNotNull(formatHolder.format);
if (shouldPassthrough(inputFormat)) { if (shouldPassthrough(inputFormat)) {
samplePipeline = samplePipeline =
new PassthroughSamplePipeline( new PassthroughSamplePipeline(inputFormat, transformationRequest, fallbackListener);
inputFormat, startPositionOffsetUs, transformationRequest, fallbackListener);
} else { } else {
samplePipeline = samplePipeline =
new VideoTranscodingSamplePipeline( new VideoTranscodingSamplePipeline(
context, context,
inputFormat, inputFormat,
startPositionOffsetUs, streamOffsetUs,
transformationRequest, transformationRequest,
effects, effects,
decoderFactory, decoderFactory,
@ -113,7 +112,7 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
} }
private boolean shouldPassthrough(Format inputFormat) { private boolean shouldPassthrough(Format inputFormat) {
if (startPositionOffsetUs != 0 && !clippingStartsAtKeyFrame) { if ((streamStartPositionUs - streamOffsetUs) != 0 && !clippingStartsAtKeyFrame) {
return false; return false;
} }
if (encoderFactory.videoNeedsEncoding()) { if (encoderFactory.videoNeedsEncoding()) {
@ -166,9 +165,16 @@ import org.checkerframework.checker.nullness.qual.RequiresNonNull;
@RequiresNonNull({"samplePipeline", "#1.data"}) @RequiresNonNull({"samplePipeline", "#1.data"})
protected void maybeQueueSampleToPipeline(DecoderInputBuffer inputBuffer) protected void maybeQueueSampleToPipeline(DecoderInputBuffer inputBuffer)
throws TransformationException { throws TransformationException {
if (sefSlowMotionFlattener == null) {
samplePipeline.queueInputBuffer();
return;
}
ByteBuffer data = inputBuffer.data; ByteBuffer data = inputBuffer.data;
long presentationTimeUs = inputBuffer.timeUs - streamOffsetUs;
boolean shouldDropSample = boolean shouldDropSample =
sefSlowMotionFlattener != null && sefSlowMotionFlattener.dropOrTransformSample(inputBuffer); sefSlowMotionFlattener.dropOrTransformSample(data, presentationTimeUs);
inputBuffer.timeUs = streamOffsetUs + sefSlowMotionFlattener.getSamplePresentationTimeUs();
if (shouldDropSample) { if (shouldDropSample) {
data.clear(); data.clear();
} else { } else {

View File

@ -36,7 +36,6 @@ import org.checkerframework.dataflow.qual.Pure;
*/ */
/* package */ final class VideoTranscodingSamplePipeline implements SamplePipeline { /* package */ final class VideoTranscodingSamplePipeline implements SamplePipeline {
private final int outputRotationDegrees; private final int outputRotationDegrees;
private final long outputPresentationTimeOffsetUs;
private final int maxPendingFrameCount; private final int maxPendingFrameCount;
private final DecoderInputBuffer decoderInputBuffer; private final DecoderInputBuffer decoderInputBuffer;
@ -53,7 +52,7 @@ import org.checkerframework.dataflow.qual.Pure;
public VideoTranscodingSamplePipeline( public VideoTranscodingSamplePipeline(
Context context, Context context,
Format inputFormat, Format inputFormat,
long outputPresentationTimeOffsetUs, long streamOffsetUs,
TransformationRequest transformationRequest, TransformationRequest transformationRequest,
ImmutableList<GlEffect> effects, ImmutableList<GlEffect> effects,
Codec.DecoderFactory decoderFactory, Codec.DecoderFactory decoderFactory,
@ -63,7 +62,6 @@ import org.checkerframework.dataflow.qual.Pure;
FrameProcessorChain.Listener frameProcessorChainListener, FrameProcessorChain.Listener frameProcessorChainListener,
Transformer.DebugViewProvider debugViewProvider) Transformer.DebugViewProvider debugViewProvider)
throws TransformationException { throws TransformationException {
this.outputPresentationTimeOffsetUs = outputPresentationTimeOffsetUs;
decoderInputBuffer = decoderInputBuffer =
new DecoderInputBuffer(DecoderInputBuffer.BUFFER_REPLACEMENT_MODE_DISABLED); new DecoderInputBuffer(DecoderInputBuffer.BUFFER_REPLACEMENT_MODE_DISABLED);
encoderOutputBuffer = encoderOutputBuffer =
@ -102,6 +100,7 @@ import org.checkerframework.dataflow.qual.Pure;
inputFormat.pixelWidthHeightRatio, inputFormat.pixelWidthHeightRatio,
/* inputWidth= */ decodedWidth, /* inputWidth= */ decodedWidth,
/* inputHeight= */ decodedHeight, /* inputHeight= */ decodedHeight,
streamOffsetUs,
effectsListBuilder.build(), effectsListBuilder.build(),
transformationRequest.enableHdrEditing); transformationRequest.enableHdrEditing);
} catch (FrameProcessingException e) { } catch (FrameProcessingException e) {
@ -202,7 +201,7 @@ import org.checkerframework.dataflow.qual.Pure;
return null; return null;
} }
MediaCodec.BufferInfo bufferInfo = checkNotNull(encoder.getOutputBufferInfo()); MediaCodec.BufferInfo bufferInfo = checkNotNull(encoder.getOutputBufferInfo());
encoderOutputBuffer.timeUs = bufferInfo.presentationTimeUs - outputPresentationTimeOffsetUs; encoderOutputBuffer.timeUs = bufferInfo.presentationTimeUs;
encoderOutputBuffer.setFlags(bufferInfo.flags); encoderOutputBuffer.setFlags(bufferInfo.flags);
return encoderOutputBuffer; return encoderOutputBuffer;
} }