MINOR: Various cleanups in storage (#15711)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Mickael Maison 2024-04-15 13:30:41 +02:00 committed by GitHub
parent 49e7c795dc
commit 3617dda9a5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 20 additions and 54 deletions

View File

@ -72,15 +72,11 @@ public class ProducerManager implements Closeable {
}
try {
Callback callback = new Callback() {
@Override
public void onCompletion(RecordMetadata metadata,
Exception exception) {
if (exception != null) {
future.completeExceptionally(exception);
} else {
future.complete(metadata);
}
Callback callback = (metadata, exception) -> {
if (exception != null) {
future.completeExceptionally(exception);
} else {
future.complete(metadata);
}
};
producer.send(new ProducerRecord<>(rlmmConfig.remoteLogMetadataTopicName(), metadataPartitionNum, null,

View File

@ -54,7 +54,7 @@ public class CleanShutdownFileHandler {
public int version;
public Long brokerEpoch;
public Content() {};
public Content() {}
public Content(int version, Long brokerEpoch) {
this.version = version;
@ -86,7 +86,6 @@ public class CleanShutdownFileHandler {
}
}
@SuppressWarnings("unchecked")
public OptionalLong read() {
try {
String text = Utils.readFileAsString(cleanShutdownFile.toPath().toString());

View File

@ -52,7 +52,7 @@ public class InMemoryLeaderEpochCheckpoint implements LeaderEpochCheckpoint {
public ByteBuffer readAsByteBuffer() throws IOException {
ByteArrayOutputStream stream = new ByteArrayOutputStream();
try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(stream, StandardCharsets.UTF_8));) {
try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(stream, StandardCharsets.UTF_8))) {
CheckpointFile.CheckpointWriteBuffer<EpochEntry> writeBuffer = new CheckpointFile.CheckpointWriteBuffer<>(writer, 0, LeaderEpochCheckpointFile.FORMATTER);
writeBuffer.write(epochs);
writer.flush();

View File

@ -27,7 +27,4 @@ public class IndexOffsetOverflowException extends KafkaException {
super(message);
}
public IndexOffsetOverflowException(String message, Throwable cause) {
super(message, cause);
}
}

View File

@ -19,5 +19,5 @@ package org.apache.kafka.storage.internals.log;
public enum LeaderHwChange {
INCREASED,
SAME,
NONE;
NONE
}

View File

@ -77,7 +77,7 @@ public class LogAppendInfo {
int validBytes,
long lastOffsetOfFirstBatch) {
this(firstOffset, lastOffset, lastLeaderEpoch, maxTimestamp, shallowOffsetOfMaxTimestamp, logAppendTime, logStartOffset,
recordValidationStats, sourceCompression, validBytes, lastOffsetOfFirstBatch, Collections.<RecordError>emptyList(),
recordValidationStats, sourceCompression, validBytes, lastOffsetOfFirstBatch, Collections.emptyList(),
LeaderHwChange.NONE);
}

View File

@ -226,7 +226,6 @@ public class LogConfig extends AbstractConfig {
@SuppressWarnings("deprecation")
private static final String MESSAGE_FORMAT_VERSION_DOC = TopicConfig.MESSAGE_FORMAT_VERSION_DOC;
@SuppressWarnings("deprecation")
private static final LogConfigDef CONFIG = new LogConfigDef();
static {
CONFIG.
@ -390,7 +389,6 @@ public class LogConfig extends AbstractConfig {
//In the transition period before messageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
// we are using its value if messageTimestampBeforeMaxMs default value hasn't changed.
@SuppressWarnings("deprecation")
private long getMessageTimestampBeforeMaxMs() {
final Long messageTimestampBeforeMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG);
if (!messageTimestampBeforeMaxMs.equals(Long.MAX_VALUE)) {
@ -402,7 +400,6 @@ public class LogConfig extends AbstractConfig {
//In the transition period before messageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
// we are using its value if messageTimestampAfterMaxMs default value hasn't changed.
@SuppressWarnings("deprecation")
private long getMessageTimestampAfterMaxMs() {
final Long messageTimestampAfterMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG);
if (!messageTimestampAfterMaxMs.equals(Long.MAX_VALUE)) {
@ -412,7 +409,6 @@ public class LogConfig extends AbstractConfig {
}
}
@SuppressWarnings("deprecation")
public RecordVersion recordVersion() {
return messageFormatVersion.highestSupportedRecordVersion();
}

View File

@ -449,7 +449,7 @@ public class RemoteIndexCache implements Closeable {
}
}
public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) {
lock.readLock().lock();
try {
return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
@ -561,7 +561,7 @@ public class RemoteIndexCache implements Closeable {
}
}
public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) {
entryLock.readLock().lock();
try {
if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");

View File

@ -32,8 +32,6 @@ import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.collection.JavaConverters;
import scala.collection.Seq;
@ -46,7 +44,6 @@ import java.util.concurrent.ExecutionException;
@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
public class TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest {
private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.class);
private static final int SEG_SIZE = 1024 * 1024;

View File

@ -75,7 +75,7 @@ public class TopicBasedRemoteLogMetadataManagerRestartTest {
}
}
private void stopTopicBasedRemoteLogMetadataManagerHarness() throws IOException {
private void stopTopicBasedRemoteLogMetadataManagerHarness() {
remoteLogMetadataManagerHarness.closeRemoteLogMetadataManager();
}

View File

@ -123,7 +123,7 @@ public final class LocalTieredStorageCondition {
* @param conjuct Another condition which truth is required for the resulting new condition to be true.
* @return A new condition which is true iff this condition and {@code conjuct} are both true.
*/
public final LocalTieredStorageCondition and(final LocalTieredStorageCondition conjuct) {
public LocalTieredStorageCondition and(final LocalTieredStorageCondition conjuct) {
//
// To keep things simple, only authorize to append to the condition chain of elementary (not composed)
// conditions. It also allows to protect from cycles.

View File

@ -51,10 +51,7 @@ public final class LocalTieredStorageEvent implements Comparable<LocalTieredStor
private final EventType type;
private final RemoteLogSegmentId segmentId;
private final int timestamp;
private final Optional<RemoteLogSegmentFileset> fileset;
private final Optional<RemoteLogSegmentMetadata> metadata;
private final int startPosition;
private final int endPosition;
private final Optional<Exception> exception;
/**
@ -124,10 +121,7 @@ public final class LocalTieredStorageEvent implements Comparable<LocalTieredStor
this.type = builder.eventType;
this.segmentId = builder.segmentId;
this.timestamp = builder.timestamp;
this.fileset = ofNullable(builder.fileset);
this.metadata = ofNullable(builder.metadata);
this.startPosition = builder.startPosition;
this.endPosition = builder.endPosition;
this.exception = ofNullable(builder.exception);
}
@ -137,10 +131,10 @@ public final class LocalTieredStorageEvent implements Comparable<LocalTieredStor
}
public static class Builder {
private int brokerId;
private EventType eventType;
private RemoteLogSegmentId segmentId;
private int timestamp;
private final int brokerId;
private final EventType eventType;
private final RemoteLogSegmentId segmentId;
private final int timestamp;
private RemoteLogSegmentFileset fileset;
private RemoteLogSegmentMetadata metadata;
private int startPosition;

View File

@ -387,7 +387,7 @@ public final class LocalTieredStorageTest {
getClass().getSimpleName(), testName, DATE_TIME_FORMATTER.format(LocalDateTime.now()));
}
public final class Verifier {
public static final class Verifier {
private final LocalTieredStorage remoteStorage;
private final TopicIdPartition topicIdPartition;

View File

@ -70,12 +70,12 @@ public final class TieredStorageTestBuilder {
private final int defaultProducedBatchSize = 1;
private final long defaultEarliestLocalOffsetExpectedInLogDirectory = 0;
private final Map<TopicPartition, List<DeletableSpec>> deletables = new HashMap<>();
private final List<TieredStorageTestAction> actions = new ArrayList<>();
private Map<TopicPartition, ProducableSpec> producables = new HashMap<>();
private Map<TopicPartition, List<OffloadableSpec>> offloadables = new HashMap<>();
private Map<TopicPartition, ConsumableSpec> consumables = new HashMap<>();
private Map<TopicPartition, FetchableSpec> fetchables = new HashMap<>();
private Map<TopicPartition, List<DeletableSpec>> deletables = new HashMap<>();
private List<TieredStorageTestAction> actions = new ArrayList<>();
public TieredStorageTestBuilder() {
}

View File

@ -26,11 +26,6 @@ public final class ExpandPartitionCountSpec {
private final int partitionCount;
private final Map<Integer, List<Integer>> assignment;
public ExpandPartitionCountSpec(String topicName,
int partitionCount) {
this(topicName, partitionCount, null);
}
public ExpandPartitionCountSpec(String topicName,
int partitionCount,
Map<Integer, List<Integer>> assignment) {

View File

@ -50,14 +50,6 @@ public final class LocalTieredStorageOutput<K, V> implements LocalTieredStorageT
return row(file, offset, record, " ");
}
private String row(String file, Object offset) {
return row(file, offset, "");
}
private String row(String file) {
return row(file, "", "");
}
private String row() {
return row("", "", "");
}