MINOR Replaced File with Path in LogSegmentData. (#10424)

Replaced File with Path in LogSegment Data.

This is a followup of #10173

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
This commit is contained in:
Satish Duggana 2021-03-31 23:13:10 +05:30 committed by GitHub
parent b6a787a072
commit 3829df103e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 73 additions and 73 deletions

View File

@ -18,8 +18,8 @@ package org.apache.kafka.server.log.remote.storage;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.io.File;
import java.nio.ByteBuffer;
import java.nio.file.Path;
import java.util.Objects;
/**
@ -30,28 +30,27 @@ import java.util.Objects;
@InterfaceStability.Evolving
public class LogSegmentData {
private final File logSegment;
private final File offsetIndex;
private final File timeIndex;
private final File txnIndex;
private final File producerSnapshotIndex;
private final Path logSegment;
private final Path offsetIndex;
private final Path timeIndex;
private final Path txnIndex;
private final Path producerSnapshotIndex;
private final ByteBuffer leaderEpochIndex;
/**
* Creates a LogSegmentData instance with data and indexes.
*
* @param logSegment actual log segment file
* @param logSegment actual log segment file
* @param offsetIndex offset index file
* @param timeIndex time index file
* @param txnIndex transaction index file
* @param producerSnapshotIndex producer snapshot until this segment
* @param leaderEpochIndex leader-epoch-index until this segment
*/
public LogSegmentData(File logSegment,
File offsetIndex,
File timeIndex,
File txnIndex,
File producerSnapshotIndex,
public LogSegmentData(Path logSegment,
Path offsetIndex,
Path timeIndex,
Path txnIndex,
Path producerSnapshotIndex,
ByteBuffer leaderEpochIndex) {
this.logSegment = Objects.requireNonNull(logSegment, "logSegment can not be null");
this.offsetIndex = Objects.requireNonNull(offsetIndex, "offsetIndex can not be null");
@ -64,35 +63,35 @@ public class LogSegmentData {
/**
* @return Log segment file of this segment.
*/
public File logSegment() {
public Path logSegment() {
return logSegment;
}
/**
* @return Offset index file.
*/
public File offsetIndex() {
public Path offsetIndex() {
return offsetIndex;
}
/**
* @return Time index file of this segment.
*/
public File timeIndex() {
public Path timeIndex() {
return timeIndex;
}
/**
* @return Transaction index file of this segment.
*/
public File txnIndex() {
public Path txnIndex() {
return txnIndex;
}
/**
* @return Producer snapshot file until this segment.
*/
public File producerSnapshotIndex() {
public Path producerSnapshotIndex() {
return producerSnapshotIndex;
}
@ -112,12 +111,12 @@ public class LogSegmentData {
return false;
}
LogSegmentData that = (LogSegmentData) o;
return Objects.equals(logSegment, that.logSegment) && Objects
.equals(offsetIndex, that.offsetIndex) && Objects
.equals(timeIndex, that.timeIndex) && Objects
.equals(txnIndex, that.txnIndex) && Objects
.equals(producerSnapshotIndex, that.producerSnapshotIndex) && Objects
.equals(leaderEpochIndex, that.leaderEpochIndex);
return Objects.equals(logSegment, that.logSegment) &&
Objects.equals(offsetIndex, that.offsetIndex) &&
Objects.equals(timeIndex, that.timeIndex) &&
Objects.equals(txnIndex, that.txnIndex) &&
Objects.equals(producerSnapshotIndex, that.producerSnapshotIndex) &&
Objects.equals(leaderEpochIndex, that.leaderEpochIndex);
}
@Override

View File

@ -56,14 +56,14 @@ public class RemoteLogSegmentMetadata {
private final int brokerId;
/**
* Maximum timestamp in the segment
* Maximum timestamp in milli seconds in the segment
*/
private final long maxTimestamp;
private final long maxTimestampMs;
/**
* Epoch time at which the respective {@link #state} is set.
* Epoch time in milli seconds at which the respective {@link #state} is set.
*/
private final long eventTimestamp;
private final long eventTimestampMs;
/**
* LeaderEpoch vs offset for messages within this segment.
@ -82,16 +82,16 @@ public class RemoteLogSegmentMetadata {
/**
* Creates an instance with the given metadata of remote log segment.
*
* <p>
* {@code segmentLeaderEpochs} can not be empty. If all the records in this segment belong to the same leader epoch
* then it should have an entry with epoch mapping to start-offset of this segment.
*
* @param remoteLogSegmentId Universally unique remote log segment id.
* @param startOffset Start offset of this segment (inclusive).
* @param endOffset End offset of this segment (inclusive).
* @param maxTimestamp Maximum timestamp in this segment.
* @param maxTimestampMs Maximum timestamp in milli seconds in this segment.
* @param brokerId Broker id from which this event is generated.
* @param eventTimestamp Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage.
* @param eventTimestampMs Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage.
* @param segmentSizeInBytes Size of this segment in bytes.
* @param state State of the respective segment of remoteLogSegmentId.
* @param segmentLeaderEpochs leader epochs occurred within this segment.
@ -99,9 +99,9 @@ public class RemoteLogSegmentMetadata {
private RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId,
long startOffset,
long endOffset,
long maxTimestamp,
long maxTimestampMs,
int brokerId,
long eventTimestamp,
long eventTimestampMs,
int segmentSizeInBytes,
RemoteLogSegmentState state,
Map<Integer, Long> segmentLeaderEpochs) {
@ -111,8 +111,8 @@ public class RemoteLogSegmentMetadata {
this.startOffset = startOffset;
this.endOffset = endOffset;
this.brokerId = brokerId;
this.maxTimestamp = maxTimestamp;
this.eventTimestamp = eventTimestamp;
this.maxTimestampMs = maxTimestampMs;
this.eventTimestampMs = eventTimestampMs;
this.segmentSizeInBytes = segmentSizeInBytes;
if (segmentLeaderEpochs == null || segmentLeaderEpochs.isEmpty()) {
@ -124,33 +124,33 @@ public class RemoteLogSegmentMetadata {
/**
* Creates an instance with the given metadata of remote log segment and its state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}.
*
* <p>
* {@code segmentLeaderEpochs} can not be empty. If all the records in this segment belong to the same leader epoch
* then it should have an entry with epoch mapping to start-offset of this segment.
*
* @param remoteLogSegmentId Universally unique remote log segment id.
* @param startOffset Start offset of this segment (inclusive).
* @param endOffset End offset of this segment (inclusive).
* @param maxTimestamp Maximum timestamp in this segment
* @param maxTimestampMs Maximum timestamp in this segment
* @param brokerId Broker id from which this event is generated.
* @param eventTimestamp Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage.
* @param eventTimestampMs Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage.
* @param segmentSizeInBytes Size of this segment in bytes.
* @param segmentLeaderEpochs leader epochs occurred within this segment
*/
public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId,
long startOffset,
long endOffset,
long maxTimestamp,
long maxTimestampMs,
int brokerId,
long eventTimestamp,
long eventTimestampMs,
int segmentSizeInBytes,
Map<Integer, Long> segmentLeaderEpochs) {
this(remoteLogSegmentId,
startOffset,
endOffset,
maxTimestamp,
maxTimestampMs,
brokerId,
eventTimestamp, segmentSizeInBytes,
eventTimestampMs, segmentSizeInBytes,
RemoteLogSegmentState.COPY_SEGMENT_STARTED,
segmentLeaderEpochs);
}
@ -178,10 +178,10 @@ public class RemoteLogSegmentMetadata {
}
/**
* @return Epoch time at which this event is occurred.
* @return Epoch time in milli seconds at which this event is occurred.
*/
public long eventTimestamp() {
return eventTimestamp;
public long eventTimestampMs() {
return eventTimestampMs;
}
/**
@ -192,10 +192,10 @@ public class RemoteLogSegmentMetadata {
}
/**
* @return Maximum timestamp of a record within this segment.
* @return Maximum timestamp in milli seconds of a record within this segment.
*/
public long maxTimestamp() {
return maxTimestamp;
public long maxTimestampMs() {
return maxTimestampMs;
}
/**
@ -232,13 +232,13 @@ public class RemoteLogSegmentMetadata {
* @param rlsmUpdate update to be applied.
* @return a new instance created by applying the given update on this instance.
*/
public RemoteLogSegmentMetadata createRemoteLogSegmentWithUpdates(RemoteLogSegmentMetadataUpdate rlsmUpdate) {
public RemoteLogSegmentMetadata createWithUpdates(RemoteLogSegmentMetadataUpdate rlsmUpdate) {
if (!remoteLogSegmentId.equals(rlsmUpdate.remoteLogSegmentId())) {
throw new IllegalArgumentException("Given rlsmUpdate does not have this instance's remoteLogSegmentId.");
}
return new RemoteLogSegmentMetadata(remoteLogSegmentId, startOffset,
endOffset, maxTimestamp, rlsmUpdate.brokerId(), rlsmUpdate.eventTimestamp(),
endOffset, maxTimestampMs, rlsmUpdate.brokerId(), rlsmUpdate.eventTimestampMs(),
segmentSizeInBytes, rlsmUpdate.state(), segmentLeaderEpochs);
}
@ -252,7 +252,7 @@ public class RemoteLogSegmentMetadata {
}
RemoteLogSegmentMetadata that = (RemoteLogSegmentMetadata) o;
return startOffset == that.startOffset && endOffset == that.endOffset && brokerId == that.brokerId
&& maxTimestamp == that.maxTimestamp && eventTimestamp == that.eventTimestamp
&& maxTimestampMs == that.maxTimestampMs && eventTimestampMs == that.eventTimestampMs
&& segmentSizeInBytes == that.segmentSizeInBytes
&& Objects.equals(remoteLogSegmentId, that.remoteLogSegmentId)
&& Objects.equals(segmentLeaderEpochs, that.segmentLeaderEpochs) && state == that.state;
@ -260,8 +260,8 @@ public class RemoteLogSegmentMetadata {
@Override
public int hashCode() {
return Objects.hash(remoteLogSegmentId, startOffset, endOffset, brokerId, maxTimestamp, eventTimestamp,
segmentLeaderEpochs, segmentSizeInBytes, state);
return Objects.hash(remoteLogSegmentId, startOffset, endOffset, brokerId, maxTimestampMs,
eventTimestampMs, segmentLeaderEpochs, segmentSizeInBytes, state);
}
@Override
@ -271,8 +271,8 @@ public class RemoteLogSegmentMetadata {
", startOffset=" + startOffset +
", endOffset=" + endOffset +
", brokerId=" + brokerId +
", maxTimestamp=" + maxTimestamp +
", eventTimestamp=" + eventTimestamp +
", maxTimestampMs=" + maxTimestampMs +
", eventTimestampMs=" + eventTimestampMs +
", segmentLeaderEpochs=" + segmentLeaderEpochs +
", segmentSizeInBytes=" + segmentSizeInBytes +
", state=" + state +

View File

@ -34,9 +34,9 @@ public class RemoteLogSegmentMetadataUpdate {
private final RemoteLogSegmentId remoteLogSegmentId;
/**
* Epoch time at which this event is generated.
* Epoch time in milli seconds at which this event is generated.
*/
private final long eventTimestamp;
private final long eventTimestampMs;
/**
* It indicates the state in which the action is executed on this segment.
@ -50,16 +50,16 @@ public class RemoteLogSegmentMetadataUpdate {
/**
* @param remoteLogSegmentId Universally unique remote log segment id.
* @param eventTimestamp Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage.
* @param eventTimestampMs Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage.
* @param state State of the remote log segment.
* @param brokerId Broker id from which this event is generated.
*/
public RemoteLogSegmentMetadataUpdate(RemoteLogSegmentId remoteLogSegmentId, long eventTimestamp,
public RemoteLogSegmentMetadataUpdate(RemoteLogSegmentId remoteLogSegmentId, long eventTimestampMs,
RemoteLogSegmentState state, int brokerId) {
this.remoteLogSegmentId = Objects.requireNonNull(remoteLogSegmentId, "remoteLogSegmentId can not be null");
this.state = Objects.requireNonNull(state, "state can not be null");
this.brokerId = brokerId;
this.eventTimestamp = eventTimestamp;
this.eventTimestampMs = eventTimestampMs;
}
/**
@ -70,10 +70,10 @@ public class RemoteLogSegmentMetadataUpdate {
}
/**
* @return Epoch time at which this event is generated.
* @return Epoch time in milli seconds at which this event is generated.
*/
public long eventTimestamp() {
return eventTimestamp;
public long eventTimestampMs() {
return eventTimestampMs;
}
/**
@ -99,20 +99,22 @@ public class RemoteLogSegmentMetadataUpdate {
return false;
}
RemoteLogSegmentMetadataUpdate that = (RemoteLogSegmentMetadataUpdate) o;
return eventTimestamp == that.eventTimestamp && Objects
.equals(remoteLogSegmentId, that.remoteLogSegmentId) && state == that.state && brokerId == that.brokerId;
return eventTimestampMs == that.eventTimestampMs &&
Objects.equals(remoteLogSegmentId, that.remoteLogSegmentId) &&
state == that.state &&
brokerId == that.brokerId;
}
@Override
public int hashCode() {
return Objects.hash(remoteLogSegmentId, eventTimestamp, state, brokerId);
return Objects.hash(remoteLogSegmentId, eventTimestampMs, state, brokerId);
}
@Override
public String toString() {
return "RemoteLogSegmentMetadataUpdate{" +
"remoteLogSegmentId=" + remoteLogSegmentId +
", eventTimestamp=" + eventTimestamp +
", eventTimestampMs=" + eventTimestampMs +
", state=" + state +
", brokerId=" + brokerId +
'}';

View File

@ -45,27 +45,27 @@ public interface RemoteStorageManager extends Configurable, Closeable {
/**
* Represents offset index.
*/
Offset,
OFFSET,
/**
* Represents timestamp index.
*/
Timestamp,
TIMESTAMP,
/**
* Represents producer snapshot index.
*/
ProducerSnapshot,
PRODUCER_SNAPSHOT,
/**
* Represents transaction index.
*/
Transaction,
TRANSACTION,
/**
* Represents leader epoch index.
*/
LeaderEpoch,
LEADER_EPOCH,
}
/**
@ -137,5 +137,4 @@ public interface RemoteStorageManager extends Configurable, Closeable {
* @throws RemoteResourceNotFoundException when there are no resources associated with the given remoteLogSegmentMetadata.
*/
void deleteLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException;
}