bIter = b.iterator();
160 | while (aIter.hasNext() && bIter.hasNext()) {
161 | if (!headerEqualityByValue(aIter.next(), bIter.next()))
162 | return false;
163 | }
164 | return !aIter.hasNext() && !bIter.hasNext();
165 | }
166 |
167 | private boolean headerEqualityByValue(Header a, Header b) {
168 | // This is an alternative implementation of ConnectHeader::equals that use proper Value equality by value
169 | // (even if they are byte arrays)
170 | if (a == b) {
171 | return true;
172 | }
173 | if (!Objects.equals(a.key(), b.key())) {
174 | return false;
175 | }
176 | try {
177 | // This particular case is not handled by ConnectHeader::equals
178 | byte[] aBytes = a.value();
179 | byte[] bBytes = b.value();
180 | return Arrays.equals(aBytes, bBytes);
181 | } catch (ClassCastException e) {
182 | return a.value() == b.value();
183 | }
184 | }
185 |
186 |
187 | }
188 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/record/RecordSerde.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.record;
2 |
3 | import org.apache.kafka.common.header.Header;
4 | import org.apache.kafka.common.header.internals.RecordHeaders;
5 | import org.apache.kafka.common.record.TimestampType;
6 |
7 | import java.io.*;
8 | import java.nio.charset.StandardCharsets;
9 |
10 | /**
11 | * Record Format:
12 | * offset: int64
13 | * timestampType: int32 -2 if timestamp is null
14 | * [timestamp: int64] if timestampType != NO_TIMESTAMP_TYPE && timestamp != null
15 | * keyLength: int32
16 | * [key: byte[keyLength]] if keyLength >= 0
17 | * valueLength: int32
18 | * [value: byte[valueLength]] if valueLength >= 0
19 | * headerCount: int32
20 | * headers: Header[headerCount]
21 | *
22 | * Header Format:
23 | * headerKeyLength: int32
24 | * headerKey: byte[headerKeyLength]
25 | * headerValueLength: int32
26 | * [headerValue: byte[headerValueLength]] if headerValueLength >= 0
27 | */
28 | public class RecordSerde {
29 | public static Record read(String topic, int partition, InputStream inputStream) throws IOException {
30 | DataInputStream dataStream = new DataInputStream(inputStream);
31 | long offset = dataStream.readLong();
32 | int timestampTypeInt = dataStream.readInt();
33 | TimestampType timestampType;
34 | Long timestamp;
35 | // See comment in `write()`
36 | if (timestampTypeInt == -2) {
37 | timestampType = TimestampType.CREATE_TIME;
38 | timestamp=null;
39 | } else {
40 | switch (timestampTypeInt) {
41 | case -1:
42 | timestampType = TimestampType.NO_TIMESTAMP_TYPE;
43 | break;
44 | case 0:
45 | timestampType = TimestampType.CREATE_TIME;
46 | break;
47 | case 1:
48 | timestampType = TimestampType.LOG_APPEND_TIME;
49 | break;
50 | default:
51 | throw new RuntimeException("Unexpected TimestampType. Expected -1,0 or 1. Got " + timestampTypeInt);
52 | }
53 | if (timestampType != TimestampType.NO_TIMESTAMP_TYPE) {
54 | timestamp = dataStream.readLong();
55 | } else {
56 | timestamp = null;
57 | }
58 | }
59 | int keyLength = dataStream.readInt();
60 | byte[] key = null;
61 | if (keyLength >= 0) {
62 | key = new byte[keyLength];
63 | int readBytes = dataStream.read(key);
64 | if (readBytes != keyLength) {
65 | throw new IOException(String.format("Expected to read %d bytes, got %d", keyLength, readBytes));
66 | }
67 | }
68 |
69 | int valueLength = dataStream.readInt();
70 | byte[] value = null;
71 | if (valueLength >= 0) {
72 | value = new byte[valueLength];
73 | int readBytes = dataStream.read(value);
74 | if (readBytes != valueLength) {
75 | throw new IOException(String.format("Expected to read %d bytes, got %d", valueLength, readBytes));
76 | }
77 | }
78 | int headerCount = dataStream.readInt();
79 | RecordHeaders headers = new RecordHeaders();
80 | for (int i = 0; i < headerCount; i++) {
81 | // Key
82 | int headerKeyLength = dataStream.readInt();
83 | if (headerKeyLength < 0) {
84 | throw new RuntimeException("Invalid negative header key size " + headerKeyLength);
85 | }
86 | byte[] headerKeyBytes = new byte[headerKeyLength];
87 | int readBytes = dataStream.read(headerKeyBytes);
88 | if (readBytes != headerKeyLength) {
89 | throw new IOException(String.format("Expected to read %d bytes, got %d", headerKeyLength, readBytes));
90 | }
91 | String headerKey = new String(headerKeyBytes, StandardCharsets.UTF_8);
92 | // Value
93 | int headerValueLength = dataStream.readInt();
94 | byte[] headerValue = null;
95 | if (headerValueLength >= 0) {
96 | headerValue = new byte[headerValueLength];
97 | int hvReadBytes = dataStream.read(headerValue);
98 | if (hvReadBytes != headerValueLength) {
99 | throw new IOException(String.format("Expected to read %d bytes, got %d", headerValueLength, hvReadBytes));
100 | }
101 | }
102 | headers.add(headerKey, headerValue);
103 | }
104 |
105 | return new Record(topic, partition, key, value, offset, timestamp, timestampType, headers);
106 | }
107 |
108 | public static void write(OutputStream outputStream, Record record) throws IOException {
109 | DataOutputStream dataStream = new DataOutputStream(outputStream);
110 | dataStream.writeLong(record.kafkaOffset());
111 | // There is a special case where the timestamp type eqauls `CREATE_TIME` but is actually `null`.
112 | // This should not happen normally and I see it as a bug in the Client implementation of pykafka
113 | // But as Kafka accepts that value, so should Kafka Backup. Thus, this dirty workaround: we write the
114 | // timestamp type `-2` if the type is CREATE_TIME but the timestamp itself is null. Otherwise we would have
115 | // needed to change the byte format and for now I think this is the better solution.
116 | if (record.timestampType() == TimestampType.CREATE_TIME && record.timestamp() == null) {
117 | dataStream.writeInt(-2);
118 | } else {
119 | dataStream.writeInt(record.timestampType().id);
120 | if (record.timestampType() != TimestampType.NO_TIMESTAMP_TYPE) {
121 | dataStream.writeLong(record.timestamp());
122 | }
123 | }
124 | if (record.key() != null) {
125 | dataStream.writeInt(record.key().length);
126 | dataStream.write(record.key());
127 | } else {
128 | dataStream.writeInt(-1);
129 | }
130 | if (record.value() != null) {
131 | dataStream.writeInt(record.value().length);
132 | dataStream.write(record.value());
133 | } else {
134 | dataStream.writeInt(-1);
135 | }
136 | Header[] headers = record.headers().toArray();
137 | dataStream.writeInt(headers.length);
138 | for (Header header : record.headers()) {
139 | byte[] headerKeyBytes = header.key().getBytes(StandardCharsets.UTF_8);
140 | dataStream.writeInt(headerKeyBytes.length);
141 | dataStream.write(headerKeyBytes);
142 | if (header.value() != null) {
143 | dataStream.writeInt(header.value().length);
144 | dataStream.write(header.value());
145 | } else {
146 | dataStream.writeInt(-1);
147 | }
148 | }
149 | }
150 | }
151 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/segment/SegmentIndex.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.segment;
2 |
3 | import java.io.EOFException;
4 | import java.io.FileInputStream;
5 | import java.io.FileOutputStream;
6 | import java.io.IOException;
7 | import java.nio.file.Files;
8 | import java.nio.file.Path;
9 | import java.util.ArrayList;
10 | import java.util.List;
11 | import java.util.Optional;
12 |
13 | public class SegmentIndex {
14 | private static final byte V1_MAGIC_BYTE = 0x01;
15 | private Path indexFile;
16 | private List index = new ArrayList<>();
17 | private long lastValidRecordOffset = -1;
18 | private long lastValidIndexPosition = 1; // mind the magic byte!
19 | private FileOutputStream fileOutputStream;
20 | private FileInputStream fileInputStream;
21 |
22 | public SegmentIndex(Path indexFile) throws IOException, IndexException {
23 | this.indexFile = indexFile;
24 | initFile();
25 | while (true) {
26 | try {
27 | SegmentIndexEntry segmentIndexEntry = SegmentIndexEntry.fromStream(fileInputStream);
28 | if (segmentIndexEntry.getOffset() <= lastValidRecordOffset) {
29 | throw new IndexException("Offsets must be always increasing! There is something terribly wrong in your index!");
30 | }
31 | index.add(segmentIndexEntry);
32 | lastValidRecordOffset = segmentIndexEntry.getOffset();
33 | lastValidIndexPosition = fileInputStream.getChannel().position();
34 | } catch (EOFException e) {
35 | // reached End of File
36 | break;
37 | }
38 | }
39 | }
40 |
41 | private void initFile() throws IOException, IndexException {
42 | if (!Files.isRegularFile(indexFile)) {
43 | Files.createFile(indexFile);
44 | fileOutputStream = new FileOutputStream(indexFile.toFile());
45 | fileOutputStream.write(V1_MAGIC_BYTE);
46 | } else {
47 | fileOutputStream = new FileOutputStream(indexFile.toFile(), true);
48 | }
49 | this.fileInputStream = new FileInputStream(indexFile.toFile());
50 | byte[] v1Validation = new byte[1];
51 | if (fileInputStream.read(v1Validation) != 1 || v1Validation[0] != V1_MAGIC_BYTE) {
52 | throw new IndexException("Cannot validate Magic Byte in the beginning of the index " + indexFile);
53 | }
54 | }
55 |
56 | void addEntry(SegmentIndexEntry segmentIndexEntry) throws IOException, IndexException {
57 | if (segmentIndexEntry.getOffset() <= lastValidRecordOffset) {
58 | throw new IndexException("Offsets must be always increasing! There is something terribly wrong in your index!");
59 | }
60 | fileOutputStream.getChannel().position(lastValidIndexPosition);
61 | segmentIndexEntry.writeToStream(fileOutputStream);
62 | lastValidIndexPosition = fileOutputStream.getChannel().position();
63 | lastValidRecordOffset = segmentIndexEntry.getOffset();
64 | index.add(segmentIndexEntry);
65 | }
66 |
67 | Optional lastIndexEntry() {
68 | if (!index.isEmpty()) {
69 | return Optional.of(index.get(index.size() - 1));
70 | } else {
71 | return Optional.empty();
72 | }
73 | }
74 |
75 | long lastValidStartPosition() {
76 | if (!index.isEmpty()) {
77 | return index.get(index.size() - 1).recordFilePosition();
78 | } else {
79 | return 0L;
80 | }
81 |
82 | }
83 |
84 | Optional getByPosition(int position) {
85 | if (position >= index.size()) {
86 | return Optional.empty();
87 | } else {
88 | return Optional.of(index.get(position));
89 | }
90 | }
91 |
92 | Optional findByOffset(long offset) {
93 | for (SegmentIndexEntry current : index) {
94 | if (current.getOffset() == offset) {
95 | return Optional.of(current.recordFilePosition());
96 | }
97 | }
98 | return Optional.empty();
99 | }
100 |
101 | Optional findEarliestWithHigherOrEqualOffset(long offset) {
102 | for (SegmentIndexEntry current : index) {
103 | if (current.getOffset() >= offset) {
104 | return Optional.of(current.recordFilePosition());
105 | }
106 | }
107 | return Optional.empty();
108 | }
109 |
110 | int size() {
111 | return index.size();
112 | }
113 |
114 | public List index() {
115 | return index;
116 | }
117 |
118 | void flush() throws IOException {
119 | fileOutputStream.flush();
120 | }
121 |
122 | void close() throws IOException {
123 | fileInputStream.close();
124 | fileOutputStream.close();
125 | }
126 |
127 | public static class IndexException extends Exception {
128 | IndexException(String message) {
129 | super(message);
130 | }
131 | }
132 |
133 | }
134 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/segment/SegmentIndexEntry.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.segment;
2 |
3 | import java.io.*;
4 | import java.util.Objects;
5 |
6 | /**
7 | * Format:
8 | * offset: int64
9 | * recordFilePosition: int64
10 | * recordLength: int64
11 | */
12 | public class SegmentIndexEntry {
13 | private final long offset;
14 | private final long recordFilePosition;
15 | private final long recordByteLength;
16 |
17 | SegmentIndexEntry(long offset, long recordFilePosition, long recordByteLength) {
18 | this.offset = offset;
19 | this.recordFilePosition = recordFilePosition;
20 | this.recordByteLength = recordByteLength;
21 | }
22 |
23 | static SegmentIndexEntry fromStream(InputStream byteStream) throws IOException {
24 | DataInputStream stream = new DataInputStream(byteStream);
25 | long offset = stream.readLong();
26 | long recordFileOffset = stream.readLong();
27 | long recordByteLength = stream.readLong();
28 | return new SegmentIndexEntry(offset, recordFileOffset, recordByteLength);
29 | }
30 |
31 | public long getOffset() {
32 | return offset;
33 | }
34 |
35 | public long recordFilePosition() {
36 | return recordFilePosition;
37 | }
38 |
39 | public long recordByteLength() {
40 | return recordByteLength;
41 | }
42 |
43 | void writeToStream(OutputStream byteStream) throws IOException {
44 | DataOutputStream stream = new DataOutputStream(byteStream);
45 | stream.writeLong(offset);
46 | stream.writeLong(recordFilePosition);
47 | stream.writeLong(recordByteLength);
48 | }
49 |
50 | @Override
51 | public int hashCode() {
52 | return Objects.hash(offset, recordFilePosition, recordByteLength);
53 | }
54 |
55 | @Override
56 | public boolean equals(Object o) {
57 | if (this == o)
58 | return true;
59 | if (o == null || getClass() != o.getClass())
60 | return false;
61 |
62 | SegmentIndexEntry that = (SegmentIndexEntry) o;
63 |
64 | return Objects.equals(getOffset(), that.getOffset())
65 | && Objects.equals(recordFilePosition(), that.recordFilePosition())
66 | && Objects.equals(recordByteLength(), that.recordByteLength());
67 | }
68 |
69 | @Override
70 | public String toString() {
71 | return String.format("SegmentIndexEntry{offset: %d, recordFilePosition: %d, recordByteLength: %d}",
72 | offset, recordFilePosition, recordByteLength);
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/segment/SegmentIndexRestore.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.segment;
2 |
3 | import de.azapps.kafkabackup.common.record.Record;
4 |
5 | import java.io.EOFException;
6 | import java.io.IOException;
7 | import java.nio.file.Files;
8 | import java.nio.file.Path;
9 |
10 | public class SegmentIndexRestore {
11 | private final SegmentIndex segmentIndex;
12 | private final UnverifiedSegmentReader reader;
13 |
14 | public SegmentIndexRestore(Path segmentFile) throws IOException, RestoreException, SegmentIndex.IndexException {
15 | int partition = SegmentUtils.getPartitionFromSegment(segmentFile);
16 | long startOffset = SegmentUtils.getStartOffsetFromSegment(segmentFile);
17 | Path parent = segmentFile.toAbsolutePath().getParent();
18 | if (parent == null) {
19 | throw new RestoreException("Segment file " + segmentFile + " does not exist");
20 | }
21 | Path indexFile = SegmentUtils.indexFile(parent, partition, startOffset);
22 |
23 | if (!Files.isRegularFile(segmentFile)) {
24 | throw new RestoreException("Segment file " + segmentFile + " does not exist");
25 | }
26 | if (Files.isRegularFile(indexFile)) {
27 | throw new RestoreException("Index file " + indexFile + " must not exist");
28 | }
29 | segmentIndex = new SegmentIndex(indexFile);
30 | reader = new UnverifiedSegmentReader(segmentFile);
31 | }
32 |
33 | public void restore() throws IOException, SegmentIndex.IndexException {
34 | long lastPosition = 1; // mind the magic byte!
35 | while (true) {
36 | try {
37 | Record record = reader.read();
38 | long currentPosition = reader.position();
39 | SegmentIndexEntry indexEntry = new SegmentIndexEntry(record.kafkaOffset(), lastPosition, currentPosition - lastPosition);
40 | segmentIndex.addEntry(indexEntry);
41 | lastPosition = currentPosition;
42 | } catch (EOFException e) {
43 | break;
44 | }
45 | }
46 | segmentIndex.flush();
47 | segmentIndex.close();
48 | }
49 |
50 | public static class RestoreException extends Exception {
51 | RestoreException(String message) {
52 | super(message);
53 | }
54 | }
55 |
56 |
57 | }
58 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/segment/SegmentReader.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.segment;
2 |
3 | import de.azapps.kafkabackup.common.record.Record;
4 | import de.azapps.kafkabackup.common.record.RecordSerde;
5 |
6 | import java.io.EOFException;
7 | import java.io.FileInputStream;
8 | import java.io.IOException;
9 | import java.nio.channels.FileChannel;
10 | import java.nio.file.Files;
11 | import java.nio.file.Path;
12 | import java.util.ArrayList;
13 | import java.util.List;
14 | import java.util.Optional;
15 |
16 | public class SegmentReader {
17 | private final String topic;
18 | private final int partition;
19 | private final String filePrefix;
20 | private final SegmentIndex segmentIndex;
21 | private final FileInputStream recordInputStream;
22 | private final long lastValidStartPosition;
23 |
24 | public SegmentReader(String topic, int partition, Path topicDir, long startOffset) throws IOException, SegmentIndex.IndexException {
25 | this(topic, partition, topicDir, SegmentUtils.filePrefix(partition, startOffset));
26 | }
27 |
28 | public SegmentReader(String topic, int partition, Path topicDir, String filePrefix) throws IOException, SegmentIndex.IndexException {
29 | this.topic = topic;
30 | this.partition = partition;
31 | this.filePrefix = filePrefix;
32 |
33 | Path indexFile = SegmentUtils.indexFile(topicDir, filePrefix);
34 | Path recordFile = SegmentUtils.recordsFile(topicDir, filePrefix);
35 | if (!Files.isRegularFile(indexFile)) {
36 | throw new RuntimeException("Index for Segment not found: " + indexFile.toString());
37 | }
38 | if (!Files.isRegularFile(recordFile)) {
39 | throw new RuntimeException("Segment not found: " + recordFile.toString());
40 | }
41 | segmentIndex = new SegmentIndex(indexFile);
42 | recordInputStream = new FileInputStream(recordFile.toFile());
43 | SegmentUtils.ensureValidSegment(recordInputStream);
44 | lastValidStartPosition = segmentIndex.lastValidStartPosition();
45 | }
46 |
47 | public void seek(long offset) throws IOException {
48 | Optional optionalPosition = segmentIndex.findEarliestWithHigherOrEqualOffset(offset);
49 | if (optionalPosition.isPresent()) {
50 | recordInputStream.getChannel().position(optionalPosition.get());
51 | } else {
52 | // If we couldn't find such a record, skip to EOF. This will make sure that hasMoreData() returns false.
53 | FileChannel fileChannel = recordInputStream.getChannel();
54 | fileChannel.position(fileChannel.size());
55 | }
56 | }
57 |
58 | public boolean hasMoreData() throws IOException {
59 | return recordInputStream.getChannel().position() <= lastValidStartPosition;
60 | }
61 |
62 | public Record read() throws IOException {
63 | if (!hasMoreData()) {
64 | throw new EOFException("Already read the last valid record in topic " + topic + ", segment " + filePrefix);
65 | }
66 | return RecordSerde.read(topic, partition, recordInputStream);
67 | }
68 |
69 | public List readN(int n) throws IOException {
70 | List records = new ArrayList<>(n);
71 | while (hasMoreData() && records.size() < n) {
72 | Record record = read();
73 | records.add(record);
74 | }
75 | return records;
76 | }
77 |
78 | public List readFully() throws IOException {
79 | List records = new ArrayList<>(segmentIndex.size());
80 | while (hasMoreData()) {
81 | Record record = read();
82 | records.add(record);
83 | }
84 | return records;
85 | }
86 |
87 | public void close() throws IOException {
88 | recordInputStream.close();
89 | segmentIndex.close();
90 | }
91 | }
92 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/segment/SegmentUtils.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.segment;
2 |
3 | import java.io.FileInputStream;
4 | import java.io.IOException;
5 | import java.nio.file.Path;
6 | import java.nio.file.Paths;
7 | import java.util.regex.Matcher;
8 | import java.util.regex.Pattern;
9 |
10 | public class SegmentUtils {
11 |
12 | static final byte V1_MAGIC_BYTE = 0x01;
13 | private static final Pattern SEGMENT_PATTERN = Pattern.compile("^segment_partition_([0-9]+)_from_offset_([0-9]+)_records$");
14 |
15 | public static String filePrefix(int partition, long startOffset) {
16 | return String.format("segment_partition_%03d_from_offset_%010d", partition, startOffset);
17 | }
18 |
19 | static void ensureValidSegment(FileInputStream inputStream) throws IOException {
20 | inputStream.getChannel().position(0);
21 | byte[] v1Validation = new byte[1];
22 | if (inputStream.read(v1Validation) != 1 || v1Validation[0] != SegmentUtils.V1_MAGIC_BYTE) {
23 | throw new IOException("Cannot validate Magic Byte in the beginning of the Segment");
24 | }
25 | }
26 |
27 | public static Path indexFile(Path topicDir, int partition, long startOffset) {
28 | return indexFile(topicDir, filePrefix(partition, startOffset));
29 | }
30 |
31 | static Path indexFile(Path topicDir, String filePrefix) {
32 | return Paths.get(topicDir.toString(), filePrefix + "_index");
33 | }
34 |
35 | public static Path recordsFile(Path topicDir, int partition, long startOffset) {
36 | return recordsFile(topicDir, filePrefix(partition, startOffset));
37 | }
38 |
39 | static Path recordsFile(Path topicDir, String filePrefix) {
40 | return Paths.get(topicDir.toString(), filePrefix + "_records");
41 | }
42 |
43 | public static boolean isSegment(Path file) {
44 | Path fpath = file.getFileName();
45 | if (fpath == null) {
46 | return false;
47 | }
48 | Matcher m = SEGMENT_PATTERN.matcher(fpath.toString());
49 | return m.find();
50 | }
51 |
52 | public static int getPartitionFromSegment(Path file) {
53 | Path fpath = file.getFileName();
54 | if (fpath == null) {
55 | throw new RuntimeException("File " + file + " is not a Segment");
56 | }
57 | Matcher m = SEGMENT_PATTERN.matcher(fpath.toString());
58 | if (m.find()) {
59 | String partitionStr = m.group(1);
60 | return Integer.parseInt(partitionStr);
61 | } else {
62 | throw new RuntimeException("File " + file + " is not a Segment");
63 | }
64 | }
65 |
66 | public static long getStartOffsetFromSegment(Path file) {
67 | Path fpath = file.getFileName();
68 | if (fpath == null) {
69 | throw new RuntimeException("File " + file + " is not a Segment");
70 | }
71 | Matcher m = SEGMENT_PATTERN.matcher(fpath.toString());
72 | if (m.find()) {
73 | String offsetStr = m.group(2);
74 | return Long.parseLong(offsetStr);
75 | } else {
76 | throw new RuntimeException("File " + file + " is not a Segment");
77 | }
78 | }
79 |
80 | }
81 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/segment/SegmentWriter.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.segment;
2 |
3 | import de.azapps.kafkabackup.common.record.Record;
4 | import de.azapps.kafkabackup.common.record.RecordSerde;
5 |
6 | import java.io.FileInputStream;
7 | import java.io.FileOutputStream;
8 | import java.io.IOException;
9 | import java.nio.file.Files;
10 | import java.nio.file.Path;
11 | import java.util.Optional;
12 |
13 | public class SegmentWriter {
14 | private final String topic;
15 | private final int partition;
16 | private final long startOffset;
17 | private final SegmentIndex segmentIndex;
18 | private final FileOutputStream recordOutputStream;
19 |
20 | public SegmentWriter(String topic, int partition, long startOffset, Path topicDir) throws IOException, SegmentIndex.IndexException {
21 | this.topic = topic;
22 | this.partition = partition;
23 | this.startOffset = startOffset;
24 |
25 | Path indexFile = SegmentUtils.indexFile(topicDir, partition, startOffset);
26 | segmentIndex = new SegmentIndex(indexFile);
27 |
28 | Path recordFile = SegmentUtils.recordsFile(topicDir, partition, startOffset);
29 | if (!Files.isRegularFile(recordFile)) {
30 | Files.createFile(recordFile);
31 | recordOutputStream = new FileOutputStream(recordFile.toFile());
32 | recordOutputStream.write(SegmentUtils.V1_MAGIC_BYTE);
33 | } else {
34 | // Validate Magic Byte
35 | FileInputStream inputStream = new FileInputStream(recordFile.toFile());
36 | SegmentUtils.ensureValidSegment(inputStream);
37 | inputStream.close();
38 |
39 | // move to last committed position of the file
40 | recordOutputStream = new FileOutputStream(recordFile.toFile(), true);
41 | Optional optionalPreviousIndexEntry = segmentIndex.lastIndexEntry();
42 | if (optionalPreviousIndexEntry.isPresent()) {
43 | SegmentIndexEntry previousSegmentIndexEntry = optionalPreviousIndexEntry.get();
44 | long position = previousSegmentIndexEntry.recordFilePosition() + previousSegmentIndexEntry.recordByteLength();
45 | recordOutputStream.getChannel().position(position);
46 | } else {
47 | recordOutputStream.getChannel().position(1);
48 | }
49 | }
50 | }
51 |
52 | public long lastWrittenOffset() {
53 | return segmentIndex.lastIndexEntry().map(SegmentIndexEntry::getOffset).orElse(-1L);
54 | }
55 |
56 | public void append(Record record) throws IOException, SegmentIndex.IndexException, SegmentException {
57 | if (!record.topic().equals(topic)) {
58 | throw new SegmentException("Trying to append to wrong topic!\n" +
59 | "Expected topic: " + topic + " given topic: " + record.topic());
60 | }
61 | if (record.kafkaPartition() != partition) {
62 | throw new SegmentException("Trying to append to wrong partition!\n" +
63 | "Expected partition: " + partition + " given partition: " + partition);
64 | }
65 | if (record.kafkaOffset() < startOffset) {
66 | throw new SegmentException("Try to append a record with an offset smaller than the start offset. Something is very wrong. \n" +
67 | "Topic: " + record.topic() + "Partition: " + record.kafkaPartition() + " StartOffset: " + startOffset + " RecordOffset: " + record.kafkaOffset() + "\n" +
68 | "You probably forgot to delete a previous Backup\n");
69 | }
70 | if (record.kafkaOffset() <= lastWrittenOffset()) {
71 | // We are handling the offsets ourselves. This should never happen!
72 | throw new SegmentException("Trying to override a written record. There is something terribly wrong in your setup! Please check whether you are trying to override an existing backup" +
73 | "Topic: " + record.topic() + "Partition: " + record.kafkaPartition() + " lastWrittenOffset: " + lastWrittenOffset() + " RecordOffset: " + record.kafkaOffset());
74 | }
75 | long startPosition = recordOutputStream.getChannel().position();
76 | RecordSerde.write(recordOutputStream, record);
77 | long recordByteLength = recordOutputStream.getChannel().position() - startPosition;
78 | SegmentIndexEntry segmentIndexEntry = new SegmentIndexEntry(record.kafkaOffset(), startPosition, recordByteLength);
79 | segmentIndex.addEntry(segmentIndexEntry);
80 | }
81 |
82 | public String filePrefix() {
83 | return SegmentUtils.filePrefix(partition, startOffset);
84 | }
85 |
86 | public long size() throws IOException {
87 | return recordOutputStream.getChannel().size();
88 | }
89 |
90 | public void flush() throws IOException {
91 | recordOutputStream.flush();
92 | segmentIndex.flush();
93 | }
94 |
95 | public void close() throws IOException {
96 | recordOutputStream.close();
97 | segmentIndex.close();
98 | }
99 |
100 | public static class SegmentException extends Exception {
101 | SegmentException(String message) {
102 | super(message);
103 | }
104 | }
105 | }
106 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/common/segment/UnverifiedSegmentReader.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.common.segment;
2 |
3 | import de.azapps.kafkabackup.common.record.Record;
4 | import de.azapps.kafkabackup.common.record.RecordSerde;
5 |
6 | import java.io.EOFException;
7 | import java.io.FileInputStream;
8 | import java.io.IOException;
9 | import java.nio.file.Path;
10 | import java.util.ArrayList;
11 | import java.util.List;
12 |
13 | public class UnverifiedSegmentReader {
14 | private String topic;
15 | private int partition;
16 | private FileInputStream recordInputStream;
17 |
18 | public UnverifiedSegmentReader(Path recordFile) throws IOException {
19 | this(recordFile, "topic", 0);
20 | }
21 |
22 | public UnverifiedSegmentReader(Path recordFile, String topic, int partition) throws IOException {
23 | recordInputStream = new FileInputStream(recordFile.toFile());
24 | this.topic = topic;
25 | this.partition = partition;
26 | SegmentUtils.ensureValidSegment(recordInputStream);
27 | }
28 |
29 | public Record read() throws IOException {
30 | return RecordSerde.read(topic, partition, recordInputStream);
31 | }
32 |
33 | public List readN(int n) throws IOException {
34 | List records = new ArrayList<>(n);
35 | while (records.size() <= n) {
36 | try {
37 | Record record = read();
38 | records.add(record);
39 | } catch (EOFException e) {
40 | break;
41 | }
42 | }
43 | return records;
44 | }
45 |
46 | public List readFully() throws IOException {
47 | List records = new ArrayList<>();
48 | while (true) {
49 | try {
50 | Record record = read();
51 | records.add(record);
52 | } catch (EOFException e) {
53 | break;
54 | }
55 | }
56 | return records;
57 | }
58 |
59 | public long position() throws IOException {
60 | return recordInputStream.getChannel().position();
61 | }
62 |
63 | public void close() throws IOException {
64 | recordInputStream.close();
65 | }
66 | }
67 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/sink/BackupSinkConfig.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.sink;
2 |
3 | import org.apache.kafka.common.config.AbstractConfig;
4 | import org.apache.kafka.common.config.ConfigDef;
5 |
6 | import java.util.HashMap;
7 | import java.util.Map;
8 |
9 | class BackupSinkConfig extends AbstractConfig {
10 | static final String CLUSTER_PREFIX = "cluster.";
11 | static final String CLUSTER_BOOTSTRAP_SERVERS = CLUSTER_PREFIX + "bootstrap.servers";
12 | static final String ADMIN_CLIENT_PREFIX = "admin.";
13 | static final String TARGET_DIR_CONFIG = "target.dir";
14 | static final String MAX_SEGMENT_SIZE = "max.segment.size.bytes";
15 | static final String SNAPSHOT = "snapshot";
16 |
17 | static final ConfigDef CONFIG_DEF = new ConfigDef()
18 | .define(TARGET_DIR_CONFIG, ConfigDef.Type.STRING,
19 | ConfigDef.Importance.HIGH, "TargetDir")
20 | .define(MAX_SEGMENT_SIZE, ConfigDef.Type.INT, 1024 ^ 3, // 1 GiB
21 | ConfigDef.Importance.LOW, "Maximum segment size")
22 | .define(SNAPSHOT, ConfigDef.Type.BOOLEAN, false,
23 | ConfigDef.Importance.LOW, "Creates a snapshot. Terminates connector when end of all partitions has been reached.");
24 |
25 | BackupSinkConfig(Map, ?> props) {
26 | super(CONFIG_DEF, props, true);
27 | if (!props.containsKey(TARGET_DIR_CONFIG)) {
28 | throw new RuntimeException("Missing Configuration Variable: " + TARGET_DIR_CONFIG);
29 | }
30 | if (!props.containsKey(MAX_SEGMENT_SIZE)) {
31 | throw new RuntimeException("Missing Configuration Variable: " + MAX_SEGMENT_SIZE);
32 | }
33 | }
34 |
35 | Map adminConfig() {
36 | Map props = new HashMap<>();
37 | props.putAll(originalsWithPrefix(CLUSTER_PREFIX));
38 | props.putAll(originalsWithPrefix(ADMIN_CLIENT_PREFIX));
39 | return props;
40 | }
41 |
42 | String targetDir() {
43 | return getString(TARGET_DIR_CONFIG);
44 | }
45 |
46 | Integer maxSegmentSizeBytes() {
47 | return getInt(MAX_SEGMENT_SIZE);
48 | }
49 |
50 | Boolean snapShotMode() { return getBoolean(SNAPSHOT); }
51 |
52 | Map consumerConfig() {
53 | return new HashMap<>(originalsWithPrefix(CLUSTER_PREFIX));
54 | }
55 |
56 | }
57 |
--------------------------------------------------------------------------------
/src/main/java/de/azapps/kafkabackup/sink/BackupSinkConnector.java:
--------------------------------------------------------------------------------
1 | package de.azapps.kafkabackup.sink;
2 |
3 | import org.apache.kafka.common.config.ConfigDef;
4 | import org.apache.kafka.connect.connector.Task;
5 | import org.apache.kafka.connect.errors.ConnectException;
6 | import org.apache.kafka.connect.sink.SinkConnector;
7 |
8 | import java.util.ArrayList;
9 | import java.util.List;
10 | import java.util.Map;
11 |
12 | public class BackupSinkConnector extends SinkConnector {
13 | private Map config;
14 |
15 | @Override
16 | public void start(Map props) {
17 | config = props;
18 | }
19 |
20 | @Override
21 | public Class extends Task> taskClass() {
22 | return BackupSinkTask.class;
23 | }
24 |
25 | @Override
26 | public List