divijvaidya commented on code in PR #18094: URL: https://github.com/apache/kafka/pull/18094#discussion_r1904114342
########## raft/src/main/java/org/apache/kafka/raft/TimingWheelExpirationService.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.raft; + +import org.apache.kafka.server.util.ShutdownableThread; +import org.apache.kafka.server.util.timer.Timer; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public class TimingWheelExpirationService implements ExpirationService { + + private static final long WORK_TIMEOUT_MS = 200L; + + private final ExpiredOperationReaper expirationReaper = new ExpiredOperationReaper(); Review Comment: We should probably lazy initialize this in the constructor. ########## raft/src/main/java/org/apache/kafka/raft/TimingWheelExpirationService.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.raft; + +import org.apache.kafka.server.util.ShutdownableThread; +import org.apache.kafka.server.util.timer.Timer; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public class TimingWheelExpirationService implements ExpirationService { + + private static final long WORK_TIMEOUT_MS = 200L; + + private final ExpiredOperationReaper expirationReaper = new ExpiredOperationReaper(); + private final Timer timer; + + public TimingWheelExpirationService(Timer timer) { + this.timer = timer; + expirationReaper.start(); + } + + @Override + public <T> CompletableFuture<T> failAfter(long timeoutMs) { + CompletableFuture<T> future = new CompletableFuture<>(); + return future.orTimeout(timeoutMs, TimeUnit.MILLISECONDS); + } + + public void shutdown() throws InterruptedException { + expirationReaper.shutdown(); + } + + class ExpiredOperationReaper extends ShutdownableThread { Review Comment: could be private ########## raft/src/main/java/org/apache/kafka/raft/SegmentPosition.java: ########## @@ -14,10 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package kafka.raft +package org.apache.kafka.raft; -import org.apache.kafka.raft.OffsetMetadata +public class SegmentPosition implements OffsetMetadata { Review Comment: This could be a `record` ``` public record SegmentPosition(long baseOffset, int relativePosition) implements OffsetMetadata { @Override public String toString() { return "(segmentBaseOffset=" + baseOffset + ",relativePositionInSegment=" + relativePosition + ")"; } } ``` ########## raft/src/main/java/org/apache/kafka/raft/TimingWheelExpirationService.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.raft; + +import org.apache.kafka.server.util.ShutdownableThread; +import org.apache.kafka.server.util.timer.Timer; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public class TimingWheelExpirationService implements ExpirationService { + + private static final long WORK_TIMEOUT_MS = 200L; + + private final ExpiredOperationReaper expirationReaper = new ExpiredOperationReaper(); + private final Timer timer; + + public TimingWheelExpirationService(Timer timer) { + this.timer = timer; + expirationReaper.start(); + } + + @Override + public <T> CompletableFuture<T> failAfter(long timeoutMs) { + CompletableFuture<T> future = new CompletableFuture<>(); + return future.orTimeout(timeoutMs, TimeUnit.MILLISECONDS); Review Comment: This is a behaviour change from scala code. Here, the timeout is executed by internal ForkJoinPool in Java. whereas earlier, it was executed by the thread executing the `TimerTaskCompletableFuture` i.e. `SystemTimer` thread. May I suggest to retain the original behaviour in this refactoring PR and make Java specific improvements separately in a different PR? ########## raft/src/main/java/org/apache/kafka/raft/TimingWheelExpirationService.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.raft; + +import org.apache.kafka.server.util.ShutdownableThread; +import org.apache.kafka.server.util.timer.Timer; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public class TimingWheelExpirationService implements ExpirationService { Review Comment: Is there a reason we got rid of `TimerTaskCompletableFuture` ? I think it was a nice pattern since it provided ability to have control when timeout occurred e.g. we could chose to log or perform custom actions. ``` private static class TimerTaskCompletableFuture<T> extends TimerTask ``` ########## raft/src/main/java/org/apache/kafka/raft/TimingWheelExpirationService.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.raft; + +import org.apache.kafka.server.util.ShutdownableThread; +import org.apache.kafka.server.util.timer.Timer; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public class TimingWheelExpirationService implements ExpirationService { + + private static final long WORK_TIMEOUT_MS = 200L; + + private final ExpiredOperationReaper expirationReaper = new ExpiredOperationReaper(); + private final Timer timer; + + public TimingWheelExpirationService(Timer timer) { + this.timer = timer; Review Comment: This timer is not being used in this code since we are doing nothing apart from advancing timer's clock. Ideally, the timer should control the task (future here) and control it's completion. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
