/
FileLogInputStream.java
259 lines (217 loc) · 9.21 KB
/
FileLogInputStream.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.record.AbstractLegacyRecordBatch.LegacyFileChannelRecordBatch;
import org.apache.kafka.common.record.DefaultRecordBatch.DefaultFileChannelRecordBatch;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.CloseableIterator;
import org.apache.kafka.common.utils.Utils;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.Iterator;
import java.util.Objects;
import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
import static org.apache.kafka.common.record.Records.HEADER_SIZE_UP_TO_MAGIC;
import static org.apache.kafka.common.record.Records.MAGIC_OFFSET;
import static org.apache.kafka.common.record.Records.OFFSET_OFFSET;
import static org.apache.kafka.common.record.Records.SIZE_OFFSET;
/**
* A log input stream which is backed by a {@link FileChannel}.
*/
public class FileLogInputStream implements LogInputStream<FileLogInputStream.FileChannelRecordBatch> {
private int position;
private final int end;
private final FileRecords fileRecords;
private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(HEADER_SIZE_UP_TO_MAGIC);
/**
* Create a new log input stream over the FileChannel
* @param records Underlying FileRecords instance
* @param start Position in the file channel to start from
* @param end Position in the file channel not to read past
*/
FileLogInputStream(FileRecords records,
int start,
int end) {
this.fileRecords = records;
this.position = start;
this.end = end;
}
@Override
public FileChannelRecordBatch nextBatch() throws IOException {
FileChannel channel = fileRecords.channel();
if (position >= end - HEADER_SIZE_UP_TO_MAGIC)
return null;
logHeaderBuffer.rewind();
Utils.readFullyOrFail(channel, logHeaderBuffer, position, "log header");
logHeaderBuffer.rewind();
long offset = logHeaderBuffer.getLong(OFFSET_OFFSET);
int size = logHeaderBuffer.getInt(SIZE_OFFSET);
// V0 has the smallest overhead, stricter checking is done later
if (size < LegacyRecord.RECORD_OVERHEAD_V0)
throw new CorruptRecordException(String.format("Found record size %d smaller than minimum record " +
"overhead (%d) in file %s.", size, LegacyRecord.RECORD_OVERHEAD_V0, fileRecords.file()));
if (position > end - LOG_OVERHEAD - size)
return null;
byte magic = logHeaderBuffer.get(MAGIC_OFFSET);
final FileChannelRecordBatch batch;
if (magic < RecordBatch.MAGIC_VALUE_V2)
batch = new LegacyFileChannelRecordBatch(offset, magic, fileRecords, position, size);
else
batch = new DefaultFileChannelRecordBatch(offset, magic, fileRecords, position, size);
position += batch.sizeInBytes();
return batch;
}
/**
* Log entry backed by an underlying FileChannel. This allows iteration over the record batches
* without needing to read the record data into memory until it is needed. The downside
* is that entries will generally no longer be readable when the underlying channel is closed.
*/
public abstract static class FileChannelRecordBatch extends AbstractRecordBatch {
protected final long offset;
protected final byte magic;
protected final FileRecords fileRecords;
protected final int position;
protected final int batchSize;
private RecordBatch fullBatch;
private RecordBatch batchHeader;
FileChannelRecordBatch(long offset,
byte magic,
FileRecords fileRecords,
int position,
int batchSize) {
this.offset = offset;
this.magic = magic;
this.fileRecords = fileRecords;
this.position = position;
this.batchSize = batchSize;
}
@Override
public CompressionType compressionType() {
return loadBatchHeader().compressionType();
}
@Override
public TimestampType timestampType() {
return loadBatchHeader().timestampType();
}
@Override
public long checksum() {
return loadBatchHeader().checksum();
}
@Override
public long maxTimestamp() {
return loadBatchHeader().maxTimestamp();
}
public int position() {
return position;
}
@Override
public byte magic() {
return magic;
}
@Override
public Iterator<Record> iterator() {
return loadFullBatch().iterator();
}
@Override
public CloseableIterator<Record> streamingIterator(BufferSupplier bufferSupplier) {
return loadFullBatch().streamingIterator(bufferSupplier);
}
@Override
public boolean isValid() {
return loadFullBatch().isValid();
}
@Override
public void ensureValid() {
loadFullBatch().ensureValid();
}
@Override
public int sizeInBytes() {
return LOG_OVERHEAD + batchSize;
}
@Override
public void writeTo(ByteBuffer buffer) {
FileChannel channel = fileRecords.channel();
try {
int limit = buffer.limit();
buffer.limit(buffer.position() + sizeInBytes());
Utils.readFully(channel, buffer, position);
buffer.limit(limit);
} catch (IOException e) {
throw new KafkaException("Failed to read record batch at position " + position + " from " + fileRecords, e);
}
}
protected abstract RecordBatch toMemoryRecordBatch(ByteBuffer buffer);
protected abstract int headerSize();
protected RecordBatch loadFullBatch() {
if (fullBatch == null) {
batchHeader = null;
fullBatch = loadBatchWithSize(sizeInBytes(), "full record batch");
}
return fullBatch;
}
protected RecordBatch loadBatchHeader() {
if (fullBatch != null)
return fullBatch;
if (batchHeader == null)
batchHeader = loadBatchWithSize(headerSize(), "record batch header");
return batchHeader;
}
private RecordBatch loadBatchWithSize(int size, String description) {
FileChannel channel = fileRecords.channel();
try {
ByteBuffer buffer = ByteBuffer.allocate(size);
Utils.readFullyOrFail(channel, buffer, position, description);
buffer.rewind();
return toMemoryRecordBatch(buffer);
} catch (IOException e) {
throw new KafkaException("Failed to load record batch at position " + position + " from " + fileRecords, e);
}
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
FileChannelRecordBatch that = (FileChannelRecordBatch) o;
FileChannel channel = fileRecords == null ? null : fileRecords.channel();
FileChannel thatChannel = that.fileRecords == null ? null : that.fileRecords.channel();
return offset == that.offset &&
position == that.position &&
batchSize == that.batchSize &&
Objects.equals(channel, thatChannel);
}
@Override
public int hashCode() {
FileChannel channel = fileRecords == null ? null : fileRecords.channel();
int result = Long.hashCode(offset);
result = 31 * result + (channel != null ? channel.hashCode() : 0);
result = 31 * result + position;
result = 31 * result + batchSize;
return result;
}
@Override
public String toString() {
return "FileChannelRecordBatch(magic: " + magic +
", offset: " + offset +
", size: " + batchSize + ")";
}
}
}