Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.LockSupport;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.HddsUtils;
Expand All @@ -63,6 +64,7 @@
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.util.Time;
import org.apache.ratis.protocol.exceptions.TimeoutIOException;
import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat;
import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
import org.apache.ratis.thirdparty.io.grpc.Status;
Expand Down Expand Up @@ -567,12 +569,38 @@ private XceiverClientReply sendCommandWithRetry(

@Override
public void streamRead(ContainerCommandRequestProto request,
StreamingReadResponse streamObserver) {
StreamingReadResponse streamObserver) throws IOException {
final ClientCallStreamObserver<ContainerCommandRequestProto> obs = streamObserver.getRequestObserver();

if (!obs.isReady()) {
LOG.debug("->{}: flow control stall (isReady=false) for block={} offset={} length={}. Waiting.",
streamObserver,
request.getReadBlock().getBlockID().getLocalID(),
request.getReadBlock().getOffset(),
request.getReadBlock().getLength());
final long now = System.nanoTime();
final long callerDeadlineNs = streamObserver.getReadDeadlineNs();
final long waitTimeoutNanos = callerDeadlineNs > 0 ? Math.max(0, callerDeadlineNs - now)
: TimeUnit.SECONDS.toNanos(timeout);
final long deadlineNs = callerDeadlineNs > 0 ? callerDeadlineNs : now + waitTimeoutNanos;
while (!obs.isReady() && System.nanoTime() < deadlineNs) {
LockSupport.parkNanos(10_000_000L);
if (Thread.currentThread().isInterrupted()) {
Thread.currentThread().interrupt();
throw new InterruptedIOException("Interrupted while waiting for stream to become ready: " + streamObserver);
}
}
if (!obs.isReady()) {
throw new TimeoutIOException("Timed out waiting for stream to become ready after "
+ TimeUnit.NANOSECONDS.toMillis(waitTimeoutNanos) + "ms");
}
}

if (LOG.isDebugEnabled()) {
LOG.debug("->{}, send onNext request {}",
streamObserver, TextFormat.shortDebugString(request.getReadBlock()));
}
streamObserver.getRequestObserver().onNext(request);
obs.onNext(request);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -322,8 +322,10 @@ synchronized void readBlockImpl(long length) throws IOException {
if (r == null) {
throw new IOException("Uninitialized StreamingReadResponse: " + blockID);
}
r.setReadDeadlineNs(System.nanoTime() + readTimeoutNanos);
xceiverClient.streamRead(ContainerProtocolCalls.buildReadBlockCommandProto(
blockID, requestedLength, length, responseDataSize, tokenRef.get(), pipelineRef.get()), r);
r.setReadDeadlineNs(System.nanoTime() + readTimeoutNanos);
}

private void handleExceptions(IOException cause) throws IOException {
Expand Down Expand Up @@ -406,7 +408,7 @@ void checkError() throws IOException {
}

ReadBlockResponseProto poll() throws IOException {
final long startTime = System.nanoTime();
final long deadlineNs = getReadDeadlineNs();
final long pollTimeoutNanos = Math.min(readTimeoutNanos / 10, 100_000_000);

while (true) {
Expand All @@ -430,15 +432,27 @@ ReadBlockResponseProto poll() throws IOException {
return null; // Stream ended, queue is empty
}

final long elapsedNanos = System.nanoTime() - startTime;
if (elapsedNanos >= readTimeoutNanos) {
if (System.nanoTime() >= deadlineNs) {
setFailedAndThrow(new TimeoutIOException(
"Timed out waiting for response after " + readTimeout));
return null;
}
}
}

private long getReadDeadlineNs() {
final StreamingReadResponse r = getResponse();
final long deadlineNs = r != null ? r.getReadDeadlineNs() : 0;
return deadlineNs > 0 ? deadlineNs : System.nanoTime() + readTimeoutNanos;
}

private void refreshReadDeadline() {
final StreamingReadResponse r = getResponse();
if (r != null) {
r.setReadDeadlineNs(System.nanoTime() + readTimeoutNanos);
}
}

private ByteBuffer read(int length, boolean preRead) throws IOException {
checkError();
if (future.isDone()) {
Expand All @@ -447,6 +461,7 @@ private ByteBuffer read(int length, boolean preRead) throws IOException {
return responseQueue.isEmpty() ? null : readFromQueue();
}

refreshReadDeadline();
readBlock(length, preRead);

while (true) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,11 +33,13 @@
import java.nio.ByteBuffer;
import java.time.Duration;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.DatanodeID;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumData;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
Expand Down Expand Up @@ -367,6 +369,109 @@ public void testReadDoesNotDropQueuedItemsWhenFutureIsDoneOnSecondCall() throws
}
}

@Test
public void testReadGetsFreshResponseTimeoutAfterStreamReadWait() throws Exception {
OzoneClientConfig clientConfig = newStreamReadConfig();
clientConfig.setStreamReadTimeout(Duration.ofMillis(500));
BlockID blockID = new BlockID(1L, 12L);
Pipeline pipeline = mockStandalonePipeline();
ClientCallStreamObserver<ContainerCommandRequestProto> requestObserver =
mock(ClientCallStreamObserver.class);
StreamingReadResponse streamingReadResponse = new StreamingReadResponse(
MockDatanodeDetails.randomDatanodeDetails(), requestObserver);

XceiverClientGrpc xceiverClient = mock(XceiverClientGrpc.class);
AtomicReference<StreamingReaderSpi> readerRef = new AtomicReference<>();
AtomicReference<Thread> responseThreadRef = new AtomicReference<>();
doAnswer(inv -> {
StreamingReaderSpi reader = inv.getArgument(1);
reader.setStreamingReadResponse(streamingReadResponse);
readerRef.set(reader);
return null;
}).when(xceiverClient).initStreamRead(any(BlockID.class), any());
doAnswer(inv -> {
Thread.sleep(450);
Thread responseThread = new Thread(() -> {
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
Thread.currentThread().interrupt();
}
readerRef.get().onNext(buildResponseProto(new byte[] {1}, 0));
});
responseThreadRef.set(responseThread);
responseThread.start();
return null;
}).when(xceiverClient).streamRead(any(), any());

XceiverClientFactory xceiverClientFactory = mock(XceiverClientFactory.class);
when(xceiverClientFactory.acquireClientForReadData(any(Pipeline.class)))
.thenReturn(xceiverClient);

try (StreamBlockInputStream sbis = new StreamBlockInputStream(
blockID, 1L, pipeline, null, xceiverClientFactory,
NO_REFRESH, clientConfig)) {
ByteBuffer buf = ByteBuffer.allocate(1);
assertEquals(1, sbis.read(buf));
responseThreadRef.get().join();
}
}

@Test
public void testReadWithoutNewRequestGetsFreshTimeoutBudget() throws Exception {
OzoneClientConfig clientConfig = newStreamReadConfig();
clientConfig.setStreamReadPreReadSize(10);
clientConfig.setStreamReadTimeout(Duration.ofMillis(500));
BlockID blockID = new BlockID(1L, 13L);
Pipeline pipeline = mockStandalonePipeline();
ClientCallStreamObserver<ContainerCommandRequestProto> requestObserver =
mock(ClientCallStreamObserver.class);
StreamingReadResponse streamingReadResponse = new StreamingReadResponse(
MockDatanodeDetails.randomDatanodeDetails(), requestObserver);

AtomicReference<StreamingReaderSpi> readerRef = new AtomicReference<>();
AtomicInteger streamReads = new AtomicInteger();
XceiverClientGrpc xceiverClient = mock(XceiverClientGrpc.class);
doAnswer(inv -> {
StreamingReaderSpi reader = inv.getArgument(1);
reader.setStreamingReadResponse(streamingReadResponse);
readerRef.set(reader);
return null;
}).when(xceiverClient).initStreamRead(any(BlockID.class), any());
doAnswer(inv -> {
streamReads.incrementAndGet();
readerRef.get().onNext(buildResponseProto(new byte[] {1}, 0));
return null;
}).when(xceiverClient).streamRead(any(), any());

XceiverClientFactory xceiverClientFactory = mock(XceiverClientFactory.class);
when(xceiverClientFactory.acquireClientForReadData(any(Pipeline.class)))
.thenReturn(xceiverClient);

try (StreamBlockInputStream sbis = new StreamBlockInputStream(
blockID, 2L, pipeline, null, xceiverClientFactory,
NO_REFRESH, clientConfig)) {
ByteBuffer first = ByteBuffer.allocate(1);
assertEquals(1, sbis.read(first));
Thread.sleep(600);

Thread delayedResponse = new Thread(() -> {
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
Thread.currentThread().interrupt();
}
readerRef.get().onNext(buildResponseProto(new byte[] {2}, 1));
});
delayedResponse.start();

ByteBuffer second = ByteBuffer.allocate(1);
assertEquals(1, sbis.readFully(second, false));
delayedResponse.join();
assertEquals(1, streamReads.get(), "second read should use data from the existing request");
}
}

private ReadBlockResponseProto buildReadBlockResponse(byte[] data) {
return ReadBlockResponseProto.newBuilder()
.setOffset(0)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,13 @@ public class StreamingReadResponse {
private final ClientCallStreamObserver<ContainerProtos.ContainerCommandRequestProto> requestObserver;
private final String name;

/**
* Deadline (System.nanoTime) for the current wait. Set before streamRead() to bound the
* isReady() flow-control wait, then refreshed afterwards to bound the response wait in poll().
* Zero falls back to the client-level read timeout.
*/
private volatile long readDeadlineNs;

public StreamingReadResponse(DatanodeDetails dn,
ClientCallStreamObserver<ContainerProtos.ContainerCommandRequestProto> requestObserver) {
this.dn = dn;
Expand All @@ -40,6 +47,14 @@ public StreamingReadResponse(DatanodeDetails dn,
this.name = "dn" + s.substring(s.lastIndexOf('-')) + "_stream";
}

public void setReadDeadlineNs(long deadlineNs) {
this.readDeadlineNs = deadlineNs;
}

public long getReadDeadlineNs() {
return readDeadlineNs;
}

public DatanodeDetails getDatanodeDetails() {
return dn;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,8 @@ public void initStreamRead(BlockID blockID, StreamingReaderSpi streamObserver) t
throw new UnsupportedOperationException("Stream read is not supported");
}

public void streamRead(ContainerCommandRequestProto request, StreamingReadResponse streamObserver) {
public void streamRead(ContainerCommandRequestProto request,
StreamingReadResponse streamObserver) throws IOException {
throw new UnsupportedOperationException("Stream read is not supported");
}

Expand Down
Loading