Skip to content

Commit 38b5b6f

Browse files
authored
feat(metadata): avoid serialize empty index (#1852)
Signed-off-by: Shichao Nie <[email protected]>
1 parent e815f82 commit 38b5b6f

File tree

1 file changed

+18
-5
lines changed

1 file changed

+18
-5
lines changed

s3stream/src/main/java/com/automq/stream/s3/index/LocalStreamRangeIndexCache.java

Lines changed: 18 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -223,17 +223,22 @@ private void evictIfNecessary() {
223223
}
224224
boolean evicted = false;
225225
boolean hasSufficientIndex = true;
226-
List<SparseRangeIndex> streamRangeIndexList = new ArrayList<>(streamRangeIndexMap.values());
226+
List<Map.Entry<Long, SparseRangeIndex>> streamRangeIndexList = new ArrayList<>(streamRangeIndexMap.entrySet());
227227
Collections.shuffle(streamRangeIndexList);
228228
while (totalSize > MAX_INDEX_SIZE) {
229229
// try to evict from each stream in round-robin manner
230-
for (SparseRangeIndex sparseRangeIndex : streamRangeIndexList) {
230+
for (Map.Entry<Long, SparseRangeIndex> entry : streamRangeIndexList) {
231+
long streamId = entry.getKey();
232+
SparseRangeIndex sparseRangeIndex = entry.getValue();
231233
if (sparseRangeIndex.length() <= 1 + COMPACT_NUM && hasSufficientIndex) {
232234
// skip evict if there is still sufficient stream to be evicted
233235
continue;
234236
}
235237
totalSize -= sparseRangeIndex.evictOnce();
236238
evicted = true;
239+
if (sparseRangeIndex.length() == 0) {
240+
streamRangeIndexMap.remove(streamId);
241+
}
237242
if (totalSize <= MAX_INDEX_SIZE) {
238243
break;
239244
}
@@ -303,6 +308,9 @@ public static ByteBuf toBuffer(Map<Long, SparseRangeIndex> streamRangeIndexMap)
303308
buffer.writeShort(VERSION);
304309
buffer.writeInt(streamRangeIndexMap.size());
305310
streamRangeIndexMap.forEach((streamId, sparseRangeIndex) -> {
311+
if (sparseRangeIndex == null || sparseRangeIndex.length() == 0) {
312+
return;
313+
}
306314
buffer.writeLong(streamId);
307315
buffer.writeInt(sparseRangeIndex.getRangeIndexList().size());
308316
sparseRangeIndex.getRangeIndexList().forEach(rangeIndex -> {
@@ -321,9 +329,14 @@ public static ByteBuf toBuffer(Map<Long, SparseRangeIndex> streamRangeIndexMap)
321329
private static int bufferSize(Map<Long, SparseRangeIndex> streamRangeIndexMap) {
322330
return Short.BYTES // version
323331
+ Integer.BYTES // stream num
324-
+ streamRangeIndexMap.values().stream().mapToInt(index -> Long.BYTES // stream id
325-
+ Integer.BYTES // range index num
326-
+ index.getRangeIndexList().size() * (3 * Long.BYTES)).sum();
332+
+ streamRangeIndexMap.values().stream().mapToInt(index -> {
333+
if (index == null || index.length() == 0) {
334+
return 0;
335+
}
336+
return Long.BYTES // stream id
337+
+ Integer.BYTES // range index num
338+
+ index.getRangeIndexList().size() * (3 * Long.BYTES);
339+
}).sum();
327340
}
328341

329342
public static Map<Long, List<RangeIndex>> fromBuffer(ByteBuf data) {

0 commit comments

Comments
 (0)