-
Notifications
You must be signed in to change notification settings - Fork 239
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(issues1087): add stream readers (#1116)
Signed-off-by: Robin Han <[email protected]>
- Loading branch information
Showing
8 changed files
with
272 additions
and
19 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
171 changes: 171 additions & 0 deletions
171
s3stream/src/main/java/com/automq/stream/s3/cache/blockcache/StreamReaders.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,171 @@ | ||
/* | ||
* Copyright 2024, AutoMQ CO.,LTD. | ||
* | ||
* Use of this software is governed by the Business Source License | ||
* included in the file BSL.md | ||
* | ||
* As of the Change Date specified in that file, in accordance with | ||
* the Business Source License, use of this software will be governed | ||
* by the Apache License, Version 2.0 | ||
*/ | ||
|
||
package com.automq.stream.s3.cache.blockcache; | ||
|
||
import com.automq.stream.s3.ObjectReader; | ||
import com.automq.stream.s3.cache.ObjectReaderLRUCache; | ||
import com.automq.stream.s3.cache.ReadDataBlock; | ||
import com.automq.stream.s3.cache.S3BlockCache; | ||
import com.automq.stream.s3.metadata.S3ObjectMetadata; | ||
import com.automq.stream.s3.objects.ObjectManager; | ||
import com.automq.stream.s3.operator.S3Operator; | ||
import com.automq.stream.s3.trace.context.TraceContext; | ||
import com.automq.stream.utils.FutureUtil; | ||
import com.automq.stream.utils.Systems; | ||
import com.automq.stream.utils.ThreadUtils; | ||
import io.netty.channel.DefaultEventLoop; | ||
import io.netty.channel.EventLoop; | ||
import java.util.HashMap; | ||
import java.util.Iterator; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.function.Function; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class StreamReaders implements S3BlockCache { | ||
private static final Logger LOGGER = LoggerFactory.getLogger(StreamReaders.class); | ||
private static final int MAX_OBJECT_READER_SIZE = 100 * 1024 * 1024; // 100MB; | ||
private static final long STREAM_READER_EXPIRED_MILLS = TimeUnit.MINUTES.toMillis(3); | ||
private static final long STREAM_READER_EXPIRED_CHECK_INTERVAL_MILLS = TimeUnit.MINUTES.toMillis(1); | ||
private final Cache[] caches; | ||
private final DataBlockCache dataBlockCache; | ||
private final ObjectReaderLRUCache objectReaders; | ||
private final ObjectReaderFactory objectReaderFactory; | ||
|
||
private final ObjectManager objectManager; | ||
private final S3Operator s3Operator; | ||
|
||
public StreamReaders(long size, ObjectManager objectManager, S3Operator s3Operator) { | ||
this(size, objectManager, s3Operator, Systems.CPU_CORES); | ||
} | ||
|
||
public StreamReaders(long size, ObjectManager objectManager, S3Operator s3Operator, int concurrency) { | ||
EventLoop[] eventLoops = new EventLoop[concurrency]; | ||
for (int i = 0; i < concurrency; i++) { | ||
eventLoops[i] = new DefaultEventLoop(ThreadUtils.createThreadFactory("stream-reader-" + i, true)); | ||
} | ||
this.caches = new Cache[concurrency]; | ||
for (int i = 0; i < concurrency; i++) { | ||
caches[i] = new Cache(eventLoops[i]); | ||
} | ||
this.dataBlockCache = new DataBlockCache(size, eventLoops); | ||
this.objectReaders = new ObjectReaderLRUCache(MAX_OBJECT_READER_SIZE); | ||
this.objectReaderFactory = new ObjectReaderFactory(); | ||
|
||
this.objectManager = objectManager; | ||
this.s3Operator = s3Operator; | ||
} | ||
|
||
@Override | ||
public CompletableFuture<ReadDataBlock> read(TraceContext context, long streamId, long startOffset, long endOffset, | ||
int maxBytes) { | ||
Cache cache = caches[Math.abs((int) (streamId % caches.length))]; | ||
return cache.read(streamId, startOffset, endOffset, maxBytes); | ||
} | ||
|
||
static class StreamReaderKey { | ||
final long streamId; | ||
final long startOffset; | ||
|
||
public StreamReaderKey(long streamId, long startOffset) { | ||
this.streamId = streamId; | ||
this.startOffset = startOffset; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) | ||
return true; | ||
if (o == null || getClass() != o.getClass()) | ||
return false; | ||
StreamReaderKey key = (StreamReaderKey) o; | ||
return streamId == key.streamId && startOffset == key.startOffset; | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(streamId, startOffset); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "StreamReaderKey{" + | ||
"streamId=" + streamId + | ||
", startOffset=" + startOffset + | ||
'}'; | ||
} | ||
} | ||
|
||
class Cache { | ||
private final EventLoop eventLoop; | ||
private final Map<StreamReaderKey, StreamReader> streamReaders = new HashMap<>(); | ||
private long lastStreamReaderExpiredCheckTime = System.currentTimeMillis(); | ||
|
||
public Cache(EventLoop eventLoop) { | ||
this.eventLoop = eventLoop; | ||
} | ||
|
||
public CompletableFuture<ReadDataBlock> read(long streamId, long startOffset, | ||
long endOffset, | ||
int maxBytes) { | ||
CompletableFuture<ReadDataBlock> cf = new CompletableFuture<>(); | ||
eventLoop.execute(() -> { | ||
cleanupExpiredStreamReader(); | ||
StreamReaderKey key = new StreamReaderKey(streamId, startOffset); | ||
StreamReader streamReader = streamReaders.computeIfAbsent(key, k -> new StreamReader(streamId, startOffset, eventLoop, objectManager, objectReaderFactory, dataBlockCache)); | ||
CompletableFuture<ReadDataBlock> streamReadCf = streamReader.read(startOffset, endOffset, maxBytes) | ||
.whenComplete((rst, ex) -> { | ||
if (ex != null) { | ||
LOGGER.error("read {} [{}, {}), maxBytes: {} from block cache fail", streamId, startOffset, endOffset, maxBytes, ex); | ||
} | ||
// when two stream read progress is the same, only one stream reader can be retained | ||
if (streamReaders.remove(key, streamReader) && ex == null) { | ||
streamReaders.put(new StreamReaderKey(streamId, streamReader.nextReadOffset()), streamReader); | ||
} | ||
}); | ||
FutureUtil.propagate(streamReadCf, cf); | ||
}); | ||
return cf; | ||
} | ||
|
||
private void cleanupExpiredStreamReader() { | ||
long now = System.currentTimeMillis(); | ||
if (now > lastStreamReaderExpiredCheckTime + STREAM_READER_EXPIRED_CHECK_INTERVAL_MILLS) { | ||
lastStreamReaderExpiredCheckTime = now; | ||
Iterator<Map.Entry<StreamReaderKey, StreamReader>> it = streamReaders.entrySet().iterator(); | ||
while (it.hasNext()) { | ||
Map.Entry<StreamReaderKey, StreamReader> entry = it.next(); | ||
StreamReader streamReader = entry.getValue(); | ||
if (now > streamReader.lastAccessTimestamp() + STREAM_READER_EXPIRED_MILLS) { | ||
streamReader.close(); | ||
it.remove(); | ||
} | ||
} | ||
} | ||
} | ||
} | ||
|
||
class ObjectReaderFactory implements Function<S3ObjectMetadata, ObjectReader> { | ||
@Override | ||
public synchronized ObjectReader apply(S3ObjectMetadata metadata) { | ||
ObjectReader objectReader = objectReaders.get(metadata.objectId()); | ||
if (objectReader == null) { | ||
objectReader = new ObjectReader(metadata, s3Operator); | ||
objectReaders.put(metadata.objectId(), objectReader); | ||
} | ||
return objectReader.retain(); | ||
} | ||
} | ||
} |
45 changes: 45 additions & 0 deletions
45
s3stream/src/main/java/com/automq/stream/utils/LogSuppressor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
/* | ||
* Copyright 2024, AutoMQ CO.,LTD. | ||
* | ||
* Use of this software is governed by the Business Source License | ||
* included in the file BSL.md | ||
* | ||
* As of the Change Date specified in that file, in accordance with | ||
* the Business Source License, use of this software will be governed | ||
* by the Apache License, Version 2.0 | ||
*/ | ||
|
||
package com.automq.stream.utils; | ||
|
||
import java.util.concurrent.atomic.AtomicInteger; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
import org.slf4j.Logger; | ||
|
||
public class LogSuppressor { | ||
private final Logger logger; | ||
private final long intervalMills; | ||
private final AtomicLong lastLogTime = new AtomicLong(System.currentTimeMillis()); | ||
private AtomicInteger supressedCount = new AtomicInteger(); | ||
|
||
public LogSuppressor(Logger logger, long intervalMills) { | ||
this.logger = logger; | ||
this.intervalMills = intervalMills; | ||
} | ||
|
||
public void warn(String message, Object... args) { | ||
long now = System.currentTimeMillis(); | ||
for (; ; ) { | ||
long last = lastLogTime.get(); | ||
if (now - last > intervalMills) { | ||
if (lastLogTime.compareAndSet(last, now)) { | ||
logger.warn("[SUPPRESSED_TIME=" + supressedCount.getAndSet(0) + "] " + message, args); | ||
break; | ||
} | ||
} else { | ||
supressedCount.incrementAndGet(); | ||
break; | ||
} | ||
} | ||
} | ||
|
||
} |
Oops, something went wrong.