mirror of https://github.com/apache/kafka.git
MINOR: Fix usage of none in javadoc (#15674)
- Use `Empty` instead of 'none' when referring to `Optional` values. - `Headers.lastHeader` returns `null` when no header is found. - Fix minor spelling mistakes. Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
4b2278ff9c
commit
8e61f04228
|
@ -672,7 +672,7 @@ public class SubscriptionState {
|
|||
* Unset the preferred read replica. This causes the fetcher to go back to the leader for fetches.
|
||||
*
|
||||
* @param tp The topic partition
|
||||
* @return the removed preferred read replica if set, None otherwise.
|
||||
* @return the removed preferred read replica if set, Empty otherwise.
|
||||
*/
|
||||
public synchronized Optional<Integer> clearPreferredReadReplica(TopicPartition tp) {
|
||||
final TopicPartitionState topicPartitionState = assignedStateOrNull(tp);
|
||||
|
|
|
@ -50,7 +50,7 @@ public interface Headers extends Iterable<Header> {
|
|||
* Returns just one (the very last) header for the given key, if present.
|
||||
*
|
||||
* @param key to get the last header for.
|
||||
* @return this last header matching the given key, returns none if not present.
|
||||
* @return this last header matching the given key, returns null if not present.
|
||||
*/
|
||||
Header lastHeader(String key);
|
||||
|
||||
|
|
|
@ -484,9 +484,9 @@ public class RemoteLogManager implements Closeable {
|
|||
* <p>
|
||||
* This method returns an option of TimestampOffset. The returned value is determined using the following ordered list of rules:
|
||||
* <p>
|
||||
* - If there are no messages in the remote storage, return None
|
||||
* - If all the messages in the remote storage have smaller offsets, return None
|
||||
* - If all the messages in the remote storage have smaller timestamps, return None
|
||||
* - If there are no messages in the remote storage, return Empty
|
||||
* - If all the messages in the remote storage have smaller offsets, return Empty
|
||||
* - If all the messages in the remote storage have smaller timestamps, return Empty
|
||||
* - Otherwise, return an option of TimestampOffset. The offset is the offset of the first message whose timestamp
|
||||
* is greater than or equals to the target timestamp and whose offset is greater than or equals to the startingOffset.
|
||||
*
|
||||
|
@ -494,7 +494,7 @@ public class RemoteLogManager implements Closeable {
|
|||
* @param timestamp The timestamp to search for.
|
||||
* @param startingOffset The starting offset to search.
|
||||
* @param leaderEpochCache LeaderEpochFileCache of the topic partition.
|
||||
* @return the timestamp and offset of the first message that meets the requirements. None will be returned if there
|
||||
* @return the timestamp and offset of the first message that meets the requirements. Empty will be returned if there
|
||||
* is no such message.
|
||||
*/
|
||||
public Optional<FileRecords.TimestampAndOffset> findOffsetByTimestamp(TopicPartition tp,
|
||||
|
|
|
@ -497,7 +497,7 @@ public interface FieldType {
|
|||
}
|
||||
|
||||
/**
|
||||
* Gets the fixed length of the field, or None if the field is variable-length.
|
||||
* Gets the fixed length of the field, or Empty if the field is variable-length.
|
||||
*/
|
||||
default Optional<Integer> fixedLength() {
|
||||
return Optional.empty();
|
||||
|
|
|
@ -31,7 +31,7 @@ import java.util.Objects;
|
|||
* A single supported version V is represented as "V".
|
||||
* A bounded range from A to B is represented as "A-B".
|
||||
* All versions greater than A is represented as "A+".
|
||||
* The NONE range is represented as an the string "none".
|
||||
* The NONE range is represented as the string "none".
|
||||
*/
|
||||
public final class Versions {
|
||||
private final short lowest;
|
||||
|
|
|
@ -1132,12 +1132,12 @@ public class ClassicGroup implements Group {
|
|||
|
||||
/**
|
||||
* Collects the set of topics that the members are subscribed to when the Protocol Type is equal
|
||||
* to 'consumer'. None is returned if
|
||||
* to 'consumer'. Empty is returned if
|
||||
* - the protocol type is not equal to 'consumer';
|
||||
* - the protocol is not defined yet; or
|
||||
* - the protocol metadata does not comply with the schema.
|
||||
*
|
||||
* @return the subscribed topics or None based on the condition above.
|
||||
* @return the subscribed topics or Empty based on the condition above.
|
||||
*/
|
||||
public Optional<Set<String>> computeSubscribedTopics() {
|
||||
if (!protocolType.isPresent()) {
|
||||
|
|
|
@ -1201,7 +1201,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
|
|||
private final AtomicLong lastHighWatermark = new AtomicLong(NO_OFFSET);
|
||||
|
||||
/**
|
||||
* @return The last high watermark received or NO_OFFSET is none is pending.
|
||||
* @return The last high watermark received or NO_OFFSET if none is pending.
|
||||
*/
|
||||
public long lastHighWatermark() {
|
||||
return lastHighWatermark.get();
|
||||
|
|
|
@ -33,7 +33,7 @@ public final class Json {
|
|||
private static final ObjectMapper MAPPER = new ObjectMapper();
|
||||
|
||||
/**
|
||||
* Parse a JSON string into a JsonValue if possible. `None` is returned if `input` is not valid JSON.
|
||||
* Parse a JSON string into a JsonValue if possible. `Empty` is returned if `input` is not valid JSON.
|
||||
*/
|
||||
public static Optional<JsonValue> parseFull(String input) {
|
||||
try {
|
||||
|
@ -52,7 +52,7 @@ public final class Json {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a JSON byte array into a JsonValue if possible. `None` is returned if `input` is not valid JSON.
|
||||
* Parse a JSON byte array into a JsonValue if possible. `Empty` is returned if `input` is not valid JSON.
|
||||
*/
|
||||
public static Optional<JsonValue> parseBytes(byte[] input) throws IOException {
|
||||
try {
|
||||
|
|
|
@ -55,7 +55,7 @@ public interface JsonValue {
|
|||
}
|
||||
|
||||
/**
|
||||
* If this is a JSON object, return a JsonObject wrapped by a `Some`. Otherwise, return None.
|
||||
* If this is a JSON object, return a JsonObject wrapped by an `Optional`. Otherwise, return Empty.
|
||||
*/
|
||||
default Optional<JsonObject> asJsonObjectOptional() {
|
||||
if (this instanceof JsonObject) {
|
||||
|
@ -77,7 +77,7 @@ public interface JsonValue {
|
|||
}
|
||||
|
||||
/**
|
||||
* If this is a JSON array, return a JsonArray wrapped by a `Some`. Otherwise, return None.
|
||||
* If this is a JSON array, return a JsonArray wrapped by an `Optional`. Otherwise, return Empty.
|
||||
*/
|
||||
default Optional<JsonArray> asJsonArrayOptional() {
|
||||
if (this instanceof JsonArray) {
|
||||
|
|
|
@ -726,21 +726,21 @@ public class LogSegment implements Closeable {
|
|||
*
|
||||
* This method returns an option of TimestampOffset. The returned value is determined using the following ordered list of rules:
|
||||
*
|
||||
* - If all the messages in the segment have smaller offsets, return None
|
||||
* - If all the messages in the segment have smaller timestamps, return None
|
||||
* - If all the messages in the segment have smaller offsets, return Empty
|
||||
* - If all the messages in the segment have smaller timestamps, return Empty
|
||||
* - If all the messages in the segment have larger timestamps, or no message in the segment has a timestamp
|
||||
* the returned the offset will be max(the base offset of the segment, startingOffset) and the timestamp will be Message.NoTimestamp.
|
||||
* - Otherwise, return an option of TimestampOffset. The offset is the offset of the first message whose timestamp
|
||||
* is greater than or equals to the target timestamp and whose offset is greater than or equals to the startingOffset.
|
||||
*
|
||||
* This method only returns None when 1) all messages' offset < startOffing or 2) the log is not empty but we did not
|
||||
* This method only returns Empty when 1) all messages' offset < startOffing or 2) the log is not empty, but we did not
|
||||
* see any message when scanning the log from the indexed position. The latter could happen if the log is truncated
|
||||
* after we get the indexed position but before we scan the log from there. In this case we simply return None and the
|
||||
* after we get the indexed position but before we scan the log from there. In this case we simply return Empty and the
|
||||
* caller will need to check on the truncated log and maybe retry or even do the search on another log segment.
|
||||
*
|
||||
* @param timestampMs The timestamp to search for.
|
||||
* @param startingOffset The starting offset to search.
|
||||
* @return the timestamp and offset of the first message that meets the requirements. None will be returned if there is no such message.
|
||||
* @return the timestamp and offset of the first message that meets the requirements. Empty will be returned if there is no such message.
|
||||
*/
|
||||
public Optional<FileRecords.TimestampAndOffset> findOffsetByTimestamp(long timestampMs, long startingOffset) throws IOException {
|
||||
// Get the index entry with a timestamp less than or equal to the target timestamp
|
||||
|
@ -822,7 +822,7 @@ public class LogSegment implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* The largest timestamp this segment contains, if maxTimestampSoFar >= 0, otherwise None.
|
||||
* The largest timestamp this segment contains, if maxTimestampSoFar >= 0, otherwise Empty.
|
||||
*/
|
||||
public OptionalLong largestRecordTimestamp() throws IOException {
|
||||
long maxTimestampSoFar = maxTimestampSoFar();
|
||||
|
|
|
@ -162,7 +162,7 @@ public class LogSegments {
|
|||
*
|
||||
* @param offset the segment to be retrieved
|
||||
*
|
||||
* @return the segment if it exists, otherwise None.
|
||||
* @return the segment if it exists, otherwise Empty.
|
||||
*/
|
||||
public Optional<LogSegment> get(long offset) {
|
||||
return Optional.ofNullable(segments.get(offset));
|
||||
|
|
Loading…
Reference in New Issue