objects() {
return objects;
}
+ @Override
+ public String toString() {
+ return "InRangeObjects{" +
+ "streamId=" + streamId +
+ ", startOffset=" + startOffset +
+ ", endOffset=" + endOffset +
+ ", objects=" + objects +
+ '}';
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) {
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/RangeMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/RangeMetadata.java
index 6fc0e7125b..47d10f0967 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/stream/RangeMetadata.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/RangeMetadata.java
@@ -21,18 +21,36 @@
import org.apache.kafka.common.metadata.RangeRecord;
import org.apache.kafka.server.common.ApiMessageAndVersion;
+/**
+ * RangeMetadata is the metadata of a range of the stream.
+ *
+ * The range represents a continuous sequence of data [startOffset, endOffset) in the stream.
+ */
public class RangeMetadata implements Comparable {
+
+ /**
+ * The id of the stream that the range belongs to.
+ */
private long streamId;
+ /**
+ * The epoch of the stream when the range is created.
+ */
private long epoch;
+ /**
+ * The index of the range in the stream.
+ */
private int rangeIndex;
/**
- * Inclusive
+ * Range start offset. (Inclusive)
*/
private long startOffset;
/**
- * Exclusive
+ * Range end offset. (Exclusive)
*/
private long endOffset;
+ /**
+ * The broker id of the broker that owns the range.
+ */
private int brokerId;
public RangeMetadata(long streamId, long epoch, int rangeIndex, long startOffset, long endOffset, int brokerId) {
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3ObjectMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3ObjectMetadata.java
index 1a595b3c9a..0fe45c173a 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3ObjectMetadata.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3ObjectMetadata.java
@@ -18,6 +18,8 @@
package org.apache.kafka.metadata.stream;
+import java.util.Objects;
+
public class S3ObjectMetadata {
private final long objectId;
private long objectSize;
@@ -48,4 +50,21 @@ public S3ObjectType getType() {
public String key() {
return ObjectUtils.genKey(0, "todocluster", objectId);
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ S3ObjectMetadata that = (S3ObjectMetadata) o;
+ return objectId == that.objectId && objectSize == that.objectSize && type == that.type;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(objectId, objectSize, type);
+ }
}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java
index 640ca43e24..87b0e71cdc 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3StreamObject.java
@@ -27,15 +27,15 @@ public class S3StreamObject {
private final long objectSize;
- private final S3ObjectStreamIndex streamIndex;
+ private final StreamOffsetRange streamIndex;
public S3StreamObject(long objectId, long objectSize, long streamId, long startOffset, long endOffset) {
this.objectId = objectId;
this.objectSize = objectSize;
- this.streamIndex = new S3ObjectStreamIndex(streamId, startOffset, endOffset);
+ this.streamIndex = new StreamOffsetRange(streamId, startOffset, endOffset);
}
- public S3ObjectStreamIndex streamIndex() {
+ public StreamOffsetRange streamIndex() {
return streamIndex;
}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java
index 845100ca30..69af4385ab 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/S3WALObject.java
@@ -37,11 +37,11 @@ public class S3WALObject implements Comparable {
private final long objectId;
private final int brokerId;
- private final Map> streamsIndex;
+ private final Map> streamsIndex;
private final S3ObjectType objectType = S3ObjectType.UNKNOWN;
- public S3WALObject(long objectId, int brokerId, final Map> streamsIndex, long orderId) {
+ public S3WALObject(long objectId, int brokerId, final Map> streamsIndex, long orderId) {
this.orderId = orderId;
this.objectId = objectId;
this.brokerId = brokerId;
@@ -49,16 +49,16 @@ public S3WALObject(long objectId, int brokerId, final Map indexes = streamsIndex.get(streamId);
+ List indexes = streamsIndex.get(streamId);
if (indexes == null || indexes.isEmpty()) {
return false;
}
- S3ObjectStreamIndex firstIndex = indexes.get(0);
- S3ObjectStreamIndex lastIndex = indexes.get(indexes.size() - 1);
+ StreamOffsetRange firstIndex = indexes.get(0);
+ StreamOffsetRange lastIndex = indexes.get(indexes.size() - 1);
return startOffset >= firstIndex.getStartOffset() && startOffset <= lastIndex.getEndOffset();
}
- public Map> streamsIndex() {
+ public Map> streamsIndex() {
return streamsIndex;
}
@@ -69,14 +69,14 @@ public ApiMessageAndVersion toRecord() {
.setOrderId(orderId)
.setStreamsIndex(
streamsIndex.values().stream().flatMap(List::stream)
- .map(S3ObjectStreamIndex::toRecordStreamIndex)
+ .map(StreamOffsetRange::toRecordStreamIndex)
.collect(Collectors.toList())), (short) 0);
}
public static S3WALObject of(WALObjectRecord record) {
- Map> collect = record.streamsIndex().stream()
- .map(index -> new S3ObjectStreamIndex(index.streamId(), index.startOffset(), index.endOffset()))
- .collect(Collectors.groupingBy(S3ObjectStreamIndex::getStreamId));
+ Map> collect = record.streamsIndex().stream()
+ .map(index -> new StreamOffsetRange(index.streamId(), index.startOffset(), index.endOffset()))
+ .collect(Collectors.groupingBy(StreamOffsetRange::getStreamId));
S3WALObject s3WalObject = new S3WALObject(record.objectId(), record.brokerId(),
collect, record.orderId());
return s3WalObject;
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3ObjectStreamIndex.java b/metadata/src/main/java/org/apache/kafka/metadata/stream/StreamOffsetRange.java
similarity index 58%
rename from metadata/src/main/java/org/apache/kafka/metadata/stream/S3ObjectStreamIndex.java
rename to metadata/src/main/java/org/apache/kafka/metadata/stream/StreamOffsetRange.java
index 5c8fd88071..e405a9d732 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/stream/S3ObjectStreamIndex.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/stream/StreamOffsetRange.java
@@ -20,38 +20,41 @@
import org.apache.kafka.common.metadata.WALObjectRecord.StreamIndex;
/**
- * ObjectStreamIndex is the index of a stream range in a WAL object or STREAM object.
+ * StreamOffsetRange represents [startOffset, endOffset) in the stream.
*/
-public class S3ObjectStreamIndex implements Comparable {
+public class StreamOffsetRange implements Comparable {
- private final Long streamId;
+ public static final StreamOffsetRange INVALID = new StreamOffsetRange(S3StreamConstant.INVALID_STREAM_ID,
+ S3StreamConstant.INVALID_OFFSET, S3StreamConstant.INVALID_OFFSET);
- private final Long startOffset;
+ private final long streamId;
- private final Long endOffset;
+ private final long startOffset;
- public S3ObjectStreamIndex(Long streamId, Long startOffset, Long endOffset) {
+ private final long endOffset;
+
+ public StreamOffsetRange(long streamId, long startOffset, long endOffset) {
this.streamId = streamId;
this.startOffset = startOffset;
this.endOffset = endOffset;
}
- public Long getStreamId() {
+ public long getStreamId() {
return streamId;
}
- public Long getStartOffset() {
+ public long getStartOffset() {
return startOffset;
}
- public Long getEndOffset() {
+ public long getEndOffset() {
return endOffset;
}
@Override
- public int compareTo(S3ObjectStreamIndex o) {
- int res = this.streamId.compareTo(o.streamId);
- return res == 0 ? this.startOffset.compareTo(o.startOffset) : res;
+ public int compareTo(StreamOffsetRange o) {
+ int res = Long.compare(this.streamId, o.streamId);
+ return res == 0 ? Long.compare(this.startOffset, o.startOffset) : res;
}
public StreamIndex toRecordStreamIndex() {
@@ -61,7 +64,7 @@ public StreamIndex toRecordStreamIndex() {
.setEndOffset(endOffset);
}
- public static S3ObjectStreamIndex of(StreamIndex index) {
- return new S3ObjectStreamIndex(index.streamId(), index.startOffset(), index.endOffset());
+ public static StreamOffsetRange of(StreamIndex index) {
+ return new StreamOffsetRange(index.streamId(), index.startOffset(), index.endOffset());
}
}
diff --git a/metadata/src/test/java/org/apache/kafka/image/BrokerS3WALMetadataImageTest.java b/metadata/src/test/java/org/apache/kafka/image/BrokerS3WALMetadataImageTest.java
index ce643ef872..c5a6be245e 100644
--- a/metadata/src/test/java/org/apache/kafka/image/BrokerS3WALMetadataImageTest.java
+++ b/metadata/src/test/java/org/apache/kafka/image/BrokerS3WALMetadataImageTest.java
@@ -28,7 +28,7 @@
import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.image.writer.RecordListWriter;
import org.apache.kafka.metadata.RecordTestUtils;
-import org.apache.kafka.metadata.stream.S3ObjectStreamIndex;
+import org.apache.kafka.metadata.stream.StreamOffsetRange;
import org.apache.kafka.metadata.stream.S3WALObject;
import org.apache.kafka.metadata.stream.SortedWALObjectsList;
import org.apache.kafka.server.common.ApiMessageAndVersion;
@@ -80,10 +80,10 @@ public void testS3WALObjects() {
// verify delta and check image's write
BrokerS3WALMetadataImage image1 = new BrokerS3WALMetadataImage(BROKER0, new SortedWALObjectsList(List.of(
new S3WALObject(0L, BROKER0, Map.of(
- STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 0L, 100L)),
- STREAM1, List.of(new S3ObjectStreamIndex(STREAM1, 0L, 200L))), 0L),
+ STREAM0, List.of(new StreamOffsetRange(STREAM0, 0L, 100L)),
+ STREAM1, List.of(new StreamOffsetRange(STREAM1, 0L, 200L))), 0L),
new S3WALObject(1L, BROKER0, Map.of(
- STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 101L, 200L))), 1L))));
+ STREAM0, List.of(new StreamOffsetRange(STREAM0, 101L, 200L))), 1L))));
assertEquals(image1, delta0.apply());
testToImageAndBack(image1);
@@ -96,7 +96,7 @@ public void testS3WALObjects() {
// verify delta and check image's write
BrokerS3WALMetadataImage image2 = new BrokerS3WALMetadataImage(BROKER0, new SortedWALObjectsList(List.of(
new S3WALObject(1L, BROKER0, Map.of(
- STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 101L, 200L))), 1L))));
+ STREAM0, List.of(new StreamOffsetRange(STREAM0, 101L, 200L))), 1L))));
assertEquals(image2, delta1.apply());
testToImageAndBack(image2);
}
diff --git a/metadata/src/test/java/org/apache/kafka/image/S3StreamsMetadataImageTest.java b/metadata/src/test/java/org/apache/kafka/image/S3StreamsMetadataImageTest.java
index 2cfae9d27d..ecbb1fec26 100644
--- a/metadata/src/test/java/org/apache/kafka/image/S3StreamsMetadataImageTest.java
+++ b/metadata/src/test/java/org/apache/kafka/image/S3StreamsMetadataImageTest.java
@@ -30,7 +30,7 @@
import org.apache.kafka.metadata.stream.InRangeObjects;
import org.apache.kafka.metadata.stream.RangeMetadata;
import org.apache.kafka.metadata.stream.S3ObjectMetadata;
-import org.apache.kafka.metadata.stream.S3ObjectStreamIndex;
+import org.apache.kafka.metadata.stream.StreamOffsetRange;
import org.apache.kafka.metadata.stream.S3StreamObject;
import org.apache.kafka.metadata.stream.S3WALObject;
import org.apache.kafka.metadata.stream.SortedWALObjectsList;
@@ -100,16 +100,16 @@ private void testToImageAndBack(S3StreamsMetadataImage image) {
@Test
public void testGetObjects() {
List broker0WalObjects = List.of(
- new S3WALObject(0, BROKER0, Map.of(STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 100L, 120L))), 0L),
- new S3WALObject(1, BROKER0, Map.of(STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 120L, 140L))), 1L),
- new S3WALObject(2, BROKER0, Map.of(STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 180L, 200L))), 2L),
+ new S3WALObject(0, BROKER0, Map.of(STREAM0, List.of(new StreamOffsetRange(STREAM0, 100L, 120L))), 0L),
+ new S3WALObject(1, BROKER0, Map.of(STREAM0, List.of(new StreamOffsetRange(STREAM0, 120L, 140L))), 1L),
+ new S3WALObject(2, BROKER0, Map.of(STREAM0, List.of(new StreamOffsetRange(STREAM0, 180L, 200L))), 2L),
new S3WALObject(3, BROKER0, Map.of(STREAM0, List.of(
- new S3ObjectStreamIndex(STREAM0, 400L, 420L), new S3ObjectStreamIndex(STREAM0, 500L, 520L))), 3L),
- new S3WALObject(4, BROKER0, Map.of(STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 520L, 600L))), 4L));
+ new StreamOffsetRange(STREAM0, 400L, 420L), new StreamOffsetRange(STREAM0, 500L, 520L))), 3L),
+ new S3WALObject(4, BROKER0, Map.of(STREAM0, List.of(new StreamOffsetRange(STREAM0, 520L, 600L))), 4L));
List broker1WalObjects = List.of(
- new S3WALObject(5, BROKER1, Map.of(STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 140L, 160L))), 0L),
- new S3WALObject(6, BROKER1, Map.of(STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 160L, 180L))), 1L),
- new S3WALObject(7, BROKER1, Map.of(STREAM0, List.of(new S3ObjectStreamIndex(STREAM0, 420L, 500L))), 2L));
+ new S3WALObject(5, BROKER1, Map.of(STREAM0, List.of(new StreamOffsetRange(STREAM0, 140L, 160L))), 0L),
+ new S3WALObject(6, BROKER1, Map.of(STREAM0, List.of(new StreamOffsetRange(STREAM0, 160L, 180L))), 1L),
+ new S3WALObject(7, BROKER1, Map.of(STREAM0, List.of(new StreamOffsetRange(STREAM0, 420L, 500L))), 2L));
BrokerS3WALMetadataImage broker0WALMetadataImage = new BrokerS3WALMetadataImage(BROKER0, new SortedWALObjectsList(broker0WalObjects));
BrokerS3WALMetadataImage broker1WALMetadataImage = new BrokerS3WALMetadataImage(BROKER1, new SortedWALObjectsList(broker1WalObjects));
Map ranges = Map.of(