Skip to content

Commit 03213a6

Browse files
committed
adjust the recovery way
1 parent bfb7908 commit 03213a6

File tree

2 files changed

+21
-110
lines changed

2 files changed

+21
-110
lines changed

fluss-server/src/main/java/org/apache/fluss/server/log/LogLoader.java

Lines changed: 12 additions & 106 deletions
Original file line numberDiff line numberDiff line change
@@ -19,10 +19,10 @@
1919

2020
import org.apache.fluss.config.ConfigOptions;
2121
import org.apache.fluss.config.Configuration;
22-
import org.apache.fluss.exception.InvalidOffsetException;
2322
import org.apache.fluss.exception.LogSegmentOffsetOverflowException;
2423
import org.apache.fluss.exception.LogStorageException;
2524
import org.apache.fluss.metadata.LogFormat;
25+
import org.apache.fluss.server.exception.CorruptIndexException;
2626
import org.apache.fluss.utils.FlussPaths;
2727
import org.apache.fluss.utils.types.Tuple2;
2828

@@ -33,12 +33,8 @@
3333
import java.io.IOException;
3434
import java.nio.file.Files;
3535
import java.nio.file.NoSuchFileException;
36-
import java.util.ArrayList;
3736
import java.util.Arrays;
3837
import java.util.Comparator;
39-
import java.util.Iterator;
40-
import java.util.List;
41-
import java.util.stream.Collectors;
4238

4339
/* This file is based on source code of Apache Kafka Project (https://kafka.apache.org/), licensed by the Apache
4440
* Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
@@ -172,61 +168,6 @@ private int recoverSegment(LogSegment segment) throws IOException {
172168
* overflow
173169
*/
174170
private Tuple2<Long, Long> recoverLog() throws IOException {
175-
if (!isCleanShutdown) {
176-
List<LogSegment> unflushed =
177-
logSegments.values(recoveryPointCheckpoint, Long.MAX_VALUE);
178-
int numUnflushed = unflushed.size();
179-
Iterator<LogSegment> unflushedIter = unflushed.iterator();
180-
boolean truncated = false;
181-
int numFlushed = 1;
182-
183-
long startTimeMs = System.currentTimeMillis();
184-
while (unflushedIter.hasNext() && !truncated) {
185-
LogSegment segment = unflushedIter.next();
186-
LOG.info(
187-
"Recovering unflushed segment {}. {}/{} recovered for bucket {}",
188-
segment.getBaseOffset(),
189-
numFlushed,
190-
numUnflushed,
191-
logSegments.getTableBucket());
192-
193-
int truncatedBytes = -1;
194-
try {
195-
truncatedBytes = recoverSegment(segment);
196-
} catch (Exception e) {
197-
if (e instanceof InvalidOffsetException) {
198-
long startOffset = segment.getBaseOffset();
199-
LOG.warn(
200-
"Found invalid offset during recovery for bucket {}. Deleting the corrupt segment "
201-
+ "and creating an empty one with starting offset {}",
202-
logSegments.getTableBucket(),
203-
startOffset);
204-
truncatedBytes = segment.truncateTo(startOffset);
205-
} else {
206-
throw e;
207-
}
208-
}
209-
210-
if (truncatedBytes > 0) {
211-
// we had an invalid message, delete all remaining log
212-
LOG.warn(
213-
"Corruption found in segment {} for bucket {}, truncating to offset {}",
214-
segment.getBaseOffset(),
215-
logSegments.getTableBucket(),
216-
segment.readNextOffset());
217-
removeAndDeleteSegments(unflushedIter);
218-
truncated = true;
219-
} else {
220-
numFlushed += 1;
221-
}
222-
}
223-
224-
LOG.info(
225-
"Recovery--11 for bucket {} completed in {} ms",
226-
logSegments.getTableBucket(),
227-
System.currentTimeMillis() - startTimeMs);
228-
}
229-
230171
if (logSegments.isEmpty()) {
231172
// TODO: use logStartOffset if issue https://github.com/apache/fluss/issues/744 ready
232173
logSegments.add(LogSegment.open(logTabletDir, 0L, conf, logFormat));
@@ -235,49 +176,6 @@ private Tuple2<Long, Long> recoverLog() throws IOException {
235176
return Tuple2.of(recoveryPointCheckpoint, logEndOffset);
236177
}
237178

238-
/**
239-
* This method deletes the given log segments and the associated writer snapshots.
240-
*
241-
* <p>This method does not need to convert IOException to {@link LogStorageException} because it
242-
* is either called before all logs are loaded or the immediate caller will catch and handle
243-
* IOException
244-
*
245-
* @param segmentsToDelete The log segments to schedule for deletion
246-
*/
247-
private void removeAndDeleteSegments(Iterator<LogSegment> segmentsToDelete) {
248-
if (segmentsToDelete.hasNext()) {
249-
List<LogSegment> toDelete = new ArrayList<>();
250-
segmentsToDelete.forEachRemaining(toDelete::add);
251-
252-
LOG.info(
253-
"Deleting segments for bucket {} as part of log recovery: {}",
254-
logSegments.getTableBucket(),
255-
toDelete.stream().map(LogSegment::toString).collect(Collectors.joining(",")));
256-
toDelete.forEach(segment -> logSegments.remove(segment.getBaseOffset()));
257-
258-
try {
259-
LocalLog.deleteSegmentFiles(
260-
toDelete, LocalLog.SegmentDeletionReason.LOG_TRUNCATION);
261-
} catch (IOException e) {
262-
LOG.error(
263-
"Failed to delete truncated segments {} for bucket {}",
264-
toDelete,
265-
logSegments.getTableBucket(),
266-
e);
267-
}
268-
269-
try {
270-
LogTablet.deleteWriterSnapshots(toDelete, writerStateManager);
271-
} catch (IOException e) {
272-
LOG.error(
273-
"Failed to delete truncated writer snapshots {} for bucket {}",
274-
toDelete,
275-
logSegments.getTableBucket(),
276-
e);
277-
}
278-
}
279-
}
280-
281179
/** Loads segments from disk into the provided segments. */
282180
private void loadSegmentFiles() throws IOException {
283181
long startTimeMs = System.currentTimeMillis();
@@ -305,17 +203,25 @@ private void loadSegmentFiles() throws IOException {
305203

306204
try {
307205
segment.sanityCheck(timeIndexFileNewlyCreated);
308-
} catch (IOException e) {
206+
} catch (Exception e) {
309207
if (e instanceof NoSuchFileException) {
310208
if (isCleanShutdown
311209
|| segment.getBaseOffset() < recoveryPointCheckpoint) {
312210
LOG.error(
313211
"Could not find offset index file corresponding to log file {} "
314212
+ "for bucket {}, recovering segment and rebuilding index files...",
315-
logSegments.getTableBucket(),
316-
segment.getFileLogRecords().file().getAbsoluteFile());
213+
segment.getFileLogRecords().file().getAbsoluteFile(),
214+
logSegments.getTableBucket());
317215
}
318216
recoverSegment(segment);
217+
} else if (e instanceof CorruptIndexException) {
218+
LOG.warn(
219+
"Found a corrupt index file corresponding to log file {} for bucket {}, recovering "
220+
+ "segment and rebuilding index files...",
221+
segment.getFileLogRecords().file().getAbsoluteFile(),
222+
logSegments.getTableBucket(),
223+
e);
224+
recoverSegment(segment);
319225
} else {
320226
throw e;
321227
}

fluss-server/src/main/java/org/apache/fluss/server/log/LogSegment.java

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -174,14 +174,14 @@ public void resizeIndexes(int size) throws IOException {
174174
}
175175

176176
public void sanityCheck(boolean timeIndexFileNewlyCreated) throws IOException {
177-
if (lazyOffsetIndex.file().exists()) {
177+
if (lazyOffsetIndex.file().exists() && lazyTimeIndex.file().exists()) {
178178
// Resize the time index file to 0 if it is newly created.
179179
if (timeIndexFileNewlyCreated) {
180180
timeIndex().resize(0);
181181
}
182-
// Sanity checks for time index and offset index are skipped because
183-
// we will recover the segments above the recovery point in recoverLog()
184-
// in any case so sanity checking them here is redundant.
182+
183+
lazyOffsetIndex.get().sanityCheck();
184+
lazyTimeIndex.get().sanityCheck();
185185
} else {
186186
throw new NoSuchFileException(
187187
"Offset index file "
@@ -310,6 +310,7 @@ public int recover() throws IOException {
310310
maxTimestampAndStartOffsetSoFar = TimestampOffset.UNKNOWN;
311311
try {
312312
for (LogRecordBatch batch : fileLogRecords.batches()) {
313+
long startTimestampMs = System.currentTimeMillis();
313314
batch.ensureValid();
314315
ensureOffsetInRange(batch.lastLogOffset());
315316

@@ -329,6 +330,10 @@ public int recover() throws IOException {
329330
// TODO Adding assign partition leader epoch follow KIP-101
330331

331332
validBytes += batch.sizeInBytes();
333+
LOG.info(
334+
"Recovered one batch of {} bytes costs {} ms",
335+
batch.sizeInBytes(),
336+
System.currentTimeMillis() - startTimestampMs);
332337
}
333338
} catch (CorruptRecordException | InvalidRecordException e) {
334339
LOG.warn(

0 commit comments

Comments
 (0)