chia7712 commented on code in PR #20246: URL: https://github.com/apache/kafka/pull/20246#discussion_r2494588852
########## clients/src/main/java/org/apache/kafka/clients/consumer/ShareAcquireMode.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.clients.consumer; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.Arrays; +import java.util.Locale; +import java.util.stream.Collectors; + +public enum ShareAcquireMode { + BATCH_OPTIMIZED("batch_optimized", (byte) 0), + RECORD_LIMIT("record_limit", (byte) 1); + + public final String name; + + public final byte id; Review Comment: Since a method called `id()` already exists, is the `public` modifier necessary here? ########## clients/src/main/java/org/apache/kafka/clients/consumer/ShareAcquireMode.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.clients.consumer; Review Comment: Is the `org.apache.kafka.clients.consumer.internals` better package? ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1762,10 +1779,57 @@ private List<AcquiredRecords> createBatches( .setLastOffset(lastAcquiredOffset) .setDeliveryCount((short) 1)); - result.forEach(acquiredRecords -> { - // Schedule acquisition lock timeout for the batch. - AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(memberId, acquiredRecords.firstOffset(), acquiredRecords.lastOffset()); - // Add the new batch to the in-flight records along with the acquisition lock timeout task for the batch. + if (isRecordLimitMode) { + // In record_limit mode, there will always be only one single batch in the result. + AcquiredRecords acquiredRecords = result.get(0); + // When the count of acquired records exceeds the max fetch limit, only initialize and schedule acquisition lock for + // acquired records up to the max fetch boundary and remaining offsets should still in available state. + // i.e. acquired records are 10-19 (10 records) and max fetch records is 5, then only 10-14 should be acquired + // and offset 15-19 should still in available state. + if (acquiredRecords.lastOffset() - acquiredRecords.firstOffset() + 1 > maxFetchRecords) { + InFlightBatch inFlightBatch = new InFlightBatch( + timer, + time, + memberId, + acquiredRecords.firstOffset(), + acquiredRecords.lastOffset(), + RecordState.ACQUIRED, + 1, + null, + timeoutHandler, + sharePartitionMetrics); + int delayMs = recordLockDurationMsOrDefault(groupConfigManager, groupId, defaultRecordLockDurationMs); + long lastOffset = acquiredRecords.firstOffset() + maxFetchRecords - 1; + acquiredRecords.setLastOffset(lastOffset); + inFlightBatch.maybeInitializeOffsetStateUpdate(lastOffset, delayMs); + updateFindNextFetchOffset(true); + + cachedState.put(acquiredRecords.firstOffset(), inFlightBatch); + sharePartitionMetrics.recordInFlightBatchMessageCount( + acquiredRecords.lastOffset() - acquiredRecords.firstOffset() + 1); Review Comment: Should we use `lastOffset` rather than `acquiredRecords.lastOffset()` to update the metrics? otherwise, it may cause duplicate count? ########## clients/src/main/java/org/apache/kafka/clients/consumer/ShareAcquireMode.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.clients.consumer; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.Arrays; +import java.util.Locale; +import java.util.stream.Collectors; + +public enum ShareAcquireMode { + BATCH_OPTIMIZED("batch_optimized", (byte) 0), + RECORD_LIMIT("record_limit", (byte) 1); + + public final String name; + + public final byte id; + + ShareAcquireMode(final String name, final byte id) { + this.name = name; + this.id = id; + } + + /** + * Case-insensitive acquire mode lookup by string name. + */ + public static ShareAcquireMode of(final String name) { + try { + return ShareAcquireMode.valueOf(name.toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Invalid value `" + name + "` for configuration " + + name + ". The value must either be 'batch_optimized' or 'record_limit'."); + } + } + + public byte id() { + return id; + } + + public static ShareAcquireMode forId(byte id) { + switch (id) { + case 0: + return BATCH_OPTIMIZED; + case 1: + return RECORD_LIMIT; + default: + throw new IllegalArgumentException("Unknown share acquire mode id: " + id); + } + } + + @Override + public String toString() { + return "ShareAcquireMode(" + name + " (" + id + "))"; Review Comment: Should this return the `name` instead? That would be consistent with other enum classes used for configuration -- 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]
