apoorvmittal10 commented on code in PR #21262: URL: https://github.com/apache/kafka/pull/21262#discussion_r2672966231
########## server/src/main/java/org/apache/kafka/server/share/fetch/acquire/AcquireStrategy.java: ########## @@ -0,0 +1,77 @@ +/* + * 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.share.fetch.acquire; Review Comment: I think the package should be fetch itself. Same for other classes in the PR. ```suggestion package org.apache.kafka.server.share.fetch; ``` ########## server/src/main/java/org/apache/kafka/server/share/fetch/acquire/BatchOptimizedStrategy.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.share.fetch.acquire; + +import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.server.share.fetch.AcquisitionLockTimerTask; +import org.apache.kafka.server.share.fetch.InFlightBatch; + +import java.util.ArrayList; +import java.util.List; + +/** + * Strategy for batch-optimized acquisition mode. + * <p> + * This mode optimizes for throughput by: + * <ul> + * <li>Splitting large fetches into multiple batches based on configured batch size</li> + * <li>Acquiring complete batches without per-record limits</li> + * <li>All records in the range are acquired and scheduled with acquisition locks</li> + * </ul> + */ +public class BatchOptimizedStrategy implements AcquireStrategy { + + public static final BatchOptimizedStrategy INSTANCE = new BatchOptimizedStrategy(); + + private BatchOptimizedStrategy() { + // Singleton class. + } + + @Override + public boolean requiresSubsetMatch(InFlightBatch inFlightBatch, int maxRecordsToAcquire, int acquiredCount) { + // batch_optimized mode doesn't force subset matching based on record count alone. + // Subset matching is determined by other factors (fullMatch, offsetState, throttling). + return false; + } Review Comment: If this is not needed in BatchOptimized mode then this API is not needed in interface. Remove the method from interface. ########## server/src/main/java/org/apache/kafka/server/share/fetch/acquire/AcquireStrategy.java: ########## @@ -0,0 +1,77 @@ +/* + * 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.share.fetch.acquire; + +import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords; +import org.apache.kafka.server.share.fetch.InFlightBatch; + +import java.util.List; + +/** + * Strategy interface for acquiring records in different modes. + * <p> + * Two modes are supported: + * <ul> + * <li>{@code BATCH_OPTIMIZED} - Acquires records in batches optimized for throughput, + * potentially splitting large fetches into multiple batches based on batch size.</li> + * <li>{@code RECORD_LIMIT} - Acquires up to the requested number of records, + * creating a single batch only.</li> + * </ul> + */ +public interface AcquireStrategy { Review Comment: Just on a high level, not totally aligned with the APIs exposed here in the method. I was thinking that you just need one method `acquire` which shall process as per the Strategy. -- 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]
