Skip to content
Merged
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
@@ -1,7 +1,9 @@
package kafka.server.streamaspect

import com.automq.stream.api.exceptions.FastReadFailFastException
import com.automq.stream.s3.metrics.stats.NetworkStats
import com.automq.stream.s3.metrics.{MetricsLevel, TimerUtil}
import com.automq.stream.s3.network.{AsyncNetworkBandwidthLimiter, GlobalNetworkBandwidthLimiters, ThrottleStrategy}
import com.automq.stream.utils.FutureUtil
import com.automq.stream.utils.threads.S3StreamThreadPoolMonitor
import kafka.automq.interceptor.{ClientIdKey, ClientIdMetadata, TrafficInterceptor}
Expand Down Expand Up @@ -889,14 +891,16 @@ class ElasticReplicaManager(
}

var partitionIndex = 0;
while (remainingBytes.get() > 0 && partitionIndex < readPartitionInfo.size) {
while (remainingBytes.get() > 0 && partitionIndex < readPartitionInfo.size && fastReadFastFail.get() == null) {
// In each iteration, we read as many partitions as possible until we reach the maximum bytes limit.
val readCfArray = readFutureBuffer.get()
readCfArray.clear()
var assignedBytes = 0 // The total bytes we have assigned to the read requests.
val availableBytes = remainingBytes.get() // The remaining bytes we can assign to the read requests, used to control the following loop.

while (assignedBytes < availableBytes && partitionIndex < readPartitionInfo.size) {
while (assignedBytes < availableBytes && partitionIndex < readPartitionInfo.size
// When there is a fast read exception, quit the loop earlier.
&& fastReadFastFail.get() == null) {
// Iterate over the partitions.
val tp = readPartitionInfo(partitionIndex)._1
val partitionData = readPartitionInfo(partitionIndex)._2
Expand Down Expand Up @@ -971,9 +975,18 @@ class ElasticReplicaManager(
release()
throw fastReadFastFail.get()
}
acquireNetworkOutPermit(limitBytes - remainingBytes.get(), if (ReadHint.isFastRead) ThrottleStrategy.TAIL else ThrottleStrategy.CATCH_UP)
result
}

private def acquireNetworkOutPermit(size: Int, throttleStrategy: ThrottleStrategy): Unit = {
val start = time.nanoseconds()
GlobalNetworkBandwidthLimiters.instance().get(AsyncNetworkBandwidthLimiter.Type.OUTBOUND)
.consume(throttleStrategy, size).join()
val networkStats = NetworkStats.getInstance()
networkStats.networkLimiterQueueTimeStats(AsyncNetworkBandwidthLimiter.Type.OUTBOUND, throttleStrategy).record(time.nanoseconds() - start)
}

def handlePartitionFailure(partitionDir: String): Unit = {
warn(s"Stopping serving partition $partitionDir")
replicaStateChangeLock synchronized {
Expand Down
34 changes: 12 additions & 22 deletions s3stream/src/main/java/com/automq/stream/s3/S3Stream.java
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,6 @@
import com.automq.stream.s3.metrics.stats.NetworkStats;
import com.automq.stream.s3.metrics.stats.StreamOperationStats;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.network.AsyncNetworkBandwidthLimiter;
import com.automq.stream.s3.network.NetworkBandwidthLimiter;
import com.automq.stream.s3.network.ThrottleStrategy;
import com.automq.stream.s3.streams.StreamManager;
Expand Down Expand Up @@ -240,28 +239,19 @@ public CompletableFuture<FetchResult> fetch(FetchContext context,
readLock.lock();
try {
CompletableFuture<FetchResult> cf = exec(() -> fetch0(context, startOffset, endOffset, maxBytes), LOGGER, "fetch");
CompletableFuture<FetchResult> retCf = cf.thenCompose(rs -> {
if (networkOutboundLimiter != null) {
long totalSize = 0L;
for (RecordBatch recordBatch : rs.recordBatchList()) {
totalSize += recordBatch.rawPayload().remaining();
}
final long finalSize = totalSize;
long start = System.nanoTime();
ThrottleStrategy throttleStrategy = context.readOptions().prioritizedRead() ? ThrottleStrategy.BYPASS
: (context.readOptions().fastRead() ? ThrottleStrategy.TAIL : ThrottleStrategy.CATCH_UP);
return networkOutboundLimiter.consume(throttleStrategy, totalSize).thenApply(nil -> {
NetworkStats.getInstance().networkLimiterQueueTimeStats(AsyncNetworkBandwidthLimiter.Type.OUTBOUND, throttleStrategy)
.record(TimerUtil.timeElapsedSince(start, TimeUnit.NANOSECONDS));
if (context.readOptions().fastRead()) {
NetworkStats.getInstance().fastReadBytesStats(streamId).ifPresent(counter -> counter.inc(finalSize));
} else {
NetworkStats.getInstance().slowReadBytesStats(streamId).ifPresent(counter -> counter.inc(finalSize));
}
return rs;
});
CompletableFuture<FetchResult> retCf = cf.thenApply(rs -> {
// TODO: move the fast / slow read metrics to kafka module.
long totalSize = 0L;
for (RecordBatch recordBatch : rs.recordBatchList()) {
totalSize += recordBatch.rawPayload().remaining();
}
final long finalSize = totalSize;
if (context.readOptions().fastRead()) {
NetworkStats.getInstance().fastReadBytesStats(streamId).ifPresent(counter -> counter.inc(finalSize));
} else {
NetworkStats.getInstance().slowReadBytesStats(streamId).ifPresent(counter -> counter.inc(finalSize));
}
return CompletableFuture.completedFuture(rs);
return rs;
});
pendingFetches.add(retCf);
pendingFetchTimestamps.push(timerUtil.lastAs(TimeUnit.NANOSECONDS));
Expand Down
Loading