ijuma commented on code in PR #13046: URL: https://github.com/apache/kafka/pull/13046#discussion_r1066641727
########## storage/src/main/java/org/apache/kafka/server/log/internals/CheckpointFileWithFailureHandler.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.server.log.internals; + +import org.apache.kafka.common.errors.KafkaStorageException; +import org.apache.kafka.server.common.CheckpointFile; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +public class CheckpointFileWithFailureHandler<T> { + + public final File file; + public final int version; Review Comment: `version` was private in the `Scala` implementation, why is it public here? ########## storage/src/main/java/org/apache/kafka/server/log/internals/EpochEntry.java: ########## @@ -0,0 +1,51 @@ +/* + * 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.server.log.internals; + +// Mapping of epoch to the first offset of the subsequent epoch +public class EpochEntry { + public final int epoch; + public final long startOffset; + + public EpochEntry(int epoch, long startOffset) { + this.epoch = epoch; + this.startOffset = startOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + EpochEntry that = (EpochEntry) o; + return epoch == that.epoch && startOffset == that.startOffset; + } + + @Override + public int hashCode() { + int result = epoch; + result = 31 * result + (int) (startOffset ^ (startOffset >>> 32)); Review Comment: Use `Long.hashCode`. ########## server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java: ########## @@ -72,7 +71,7 @@ public CheckpointFile(File file, tempPath = Paths.get(absolutePath.toString() + ".tmp"); } - public void write(Collection<T> entries) throws IOException { + public void write(List<T> entries) throws IOException { Review Comment: Why did we change this? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org