AndrewJSchofield commented on code in PR #16263: URL: https://github.com/apache/kafka/pull/16263#discussion_r1634861690
########## server/src/main/java/org/apache/kafka/server/share/ShareSession.java: ########## @@ -0,0 +1,176 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.requests.ShareFetchRequest; +import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; + +public class ShareSession { + + // Helper enum to return the possible type of modified list of TopicIdPartitions in cache + public enum ModifiedTopicIdPartitionType { + ADDED, + UPDATED, + REMOVED + } + + private final ShareSessionKey key; + private final ImplicitLinkedHashCollection<CachedSharePartition> partitionMap; + private final long creationMs; + + private long lastUsedMs; + // visible for testing + public int epoch; + // This is used by the ShareSessionCache to store the last known size of this session. + // If this is -1, the Session is not in the cache. + private int cachedSize = -1; + + /** + * The share session. + * Each share session is protected by its own lock, which must be taken before mutable + * fields are read or modified. This includes modification of the share session partition map. + * + * @param key The share session key to identify the share session uniquely. + * @param partitionMap The CachedPartitionMap. + * @param creationMs The time in milliseconds when this share session was created. + * @param lastUsedMs The last used time in milliseconds. This should only be updated by + * ShareSessionCache#touch. + * @param epoch The share session sequence number. + */ + public ShareSession(ShareSessionKey key, ImplicitLinkedHashCollection<CachedSharePartition> partitionMap, + long creationMs, long lastUsedMs, int epoch) { + this.key = key; + this.partitionMap = partitionMap; + this.creationMs = creationMs; + this.lastUsedMs = lastUsedMs; + this.epoch = epoch; + } + + public ShareSessionKey key() { + synchronized (this) { + return key; + } + } + + public int cachedSize() { + synchronized (this) { + return cachedSize; + } + } + + public void cachedSize(int size) { + synchronized (this) { + cachedSize = size; + } + } + + public long lastUsedMs() { + synchronized (this) { + return lastUsedMs; + } + } + + public void lastUsedMs(long ts) { + synchronized (this) { + lastUsedMs = ts; + } + } + + public ImplicitLinkedHashCollection<CachedSharePartition> partitionMap() { + synchronized (this) { + return partitionMap; + } + } + + // Visible for testing + public int epoch() { + synchronized (this) { + return epoch; + } + } + + public int size() { + synchronized (this) { + return partitionMap.size(); + } + } + + public Boolean isEmpty() { + synchronized (this) { + return partitionMap.isEmpty(); + } + } + + public LastUsedKey lastUsedKey() { + synchronized (this) { + return new LastUsedKey(key, lastUsedMs); + } + } + + // Visible for testing + public long creationMs() { + return creationMs; + } + + // Update the cached partition data based on the request. + public Map<ModifiedTopicIdPartitionType, List<TopicIdPartition>> update(Map<TopicIdPartition, + ShareFetchRequest.SharePartitionData> shareFetchData, + List<TopicIdPartition> toForget) { Review Comment: nit: indentation. ########## core/src/main/java/kafka/server/ShareSessionContext.java: ########## @@ -0,0 +1,243 @@ +/* + * 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 kafka.server; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import kafka.server.SharePartitionManager.ErroneousAndValidPartitionData; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ShareFetchMetadata; +import org.apache.kafka.common.requests.ShareFetchRequest; +import org.apache.kafka.common.requests.ShareFetchRequest.SharePartitionData; +import org.apache.kafka.common.requests.ShareFetchResponse; +import org.apache.kafka.server.share.CachedSharePartition; +import org.apache.kafka.server.share.ShareSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +/** + * The context for a share session fetch request. + */ +public class ShareSessionContext extends ShareFetchContext { + + private final ShareFetchMetadata reqMetadata; + private Map<TopicIdPartition, SharePartitionData> shareFetchData; + private final boolean isSubsequent; + private ShareSession session; + + private final Logger log = LoggerFactory.getLogger(ShareSessionContext.class); + + /** + * The share fetch context for the first request that starts a share session. + * + * @param reqMetadata The request metadata. + * @param shareFetchData The share partition data from the share fetch request. + */ + public ShareSessionContext(ShareFetchMetadata reqMetadata, + Map<TopicIdPartition, ShareFetchRequest.SharePartitionData> shareFetchData) { + this.reqMetadata = reqMetadata; + this.shareFetchData = shareFetchData; + this.isSubsequent = false; + } + + /** + * The share fetch context for a subsequent request that utilizes an existing share session. + * + * @param reqMetadata The request metadata. + * @param session The subsequent fetch request session. + */ + public ShareSessionContext(ShareFetchMetadata reqMetadata, ShareSession session) { + this.reqMetadata = reqMetadata; + this.session = session; + this.isSubsequent = true; + } + + public Map<TopicIdPartition, ShareFetchRequest.SharePartitionData> shareFetchData() { + return shareFetchData; + } + + public boolean isSubsequent() { + return isSubsequent; + } + + public ShareSession session() { + return session; + } + + @Override + ShareFetchResponse throttleResponse(int throttleTimeMs) { + if (!isSubsequent) { + return new ShareFetchResponse(ShareFetchResponse.toMessage(Errors.NONE, throttleTimeMs, + Collections.emptyIterator(), Collections.emptyList())); + } else { + int expectedEpoch = ShareFetchMetadata.nextEpoch(reqMetadata.epoch()); + int sessionEpoch; + synchronized (session) { + sessionEpoch = session.epoch; + } + if (sessionEpoch != expectedEpoch) { + log.debug("Subsequent share session {} expected epoch {}, but got {}. " + + "Possible duplicate request.", session.key(), expectedEpoch, sessionEpoch); + return new ShareFetchResponse(ShareFetchResponse.toMessage(Errors.INVALID_SHARE_SESSION_EPOCH, + throttleTimeMs, Collections.emptyIterator(), Collections.emptyList())); + } else + return new ShareFetchResponse(ShareFetchResponse.toMessage(Errors.NONE, throttleTimeMs, + Collections.emptyIterator(), Collections.emptyList())); + } + } + + // Iterator that goes over the given partition map and selects partitions that need to be included in the response. + // If updateShareContextAndRemoveUnselected is set to true, the share context will be updated for the selected + // partitions and also remove unselected ones as they are encountered. + private class PartitionIterator implements + Iterator<Entry<TopicIdPartition, PartitionData>> { + private final Iterator<Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData>> iterator; + private final boolean updateShareContextAndRemoveUnselected; + private Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> nextElement; + + + public PartitionIterator(Iterator<Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData>> iterator, boolean updateShareContextAndRemoveUnselected) { + this.iterator = iterator; + this.updateShareContextAndRemoveUnselected = updateShareContextAndRemoveUnselected; + } + + @Override + public boolean hasNext() { + while ((nextElement == null) && iterator.hasNext()) { + Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> element = iterator.next(); + TopicIdPartition topicPart = element.getKey(); + ShareFetchResponseData.PartitionData respData = element.getValue(); + CachedSharePartition cachedPart = session.partitionMap().find(new CachedSharePartition(topicPart)); + boolean mustRespond = cachedPart.maybeUpdateResponseData(respData, updateShareContextAndRemoveUnselected); + if (mustRespond) { + nextElement = element; + if (updateShareContextAndRemoveUnselected && ShareFetchResponse.recordsSize(respData) > 0) { + // Session.partitionMap is of type ImplicitLinkedHashCollection<> which tracks the order of insertion of elements. + // Since, we are updating an element in this case, we need to perform a remove and then a mustAdd to maintain the correct order + session.partitionMap().remove(cachedPart); + session.partitionMap().mustAdd(cachedPart); + } + } else { + if (updateShareContextAndRemoveUnselected) { + iterator.remove(); + } + } + } + return nextElement != null; + } + + @Override + public Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> next() { + if (!hasNext()) throw new NoSuchElementException(); + Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> element = nextElement; + nextElement = null; + return element; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + + @Override + int responseSize(LinkedHashMap<TopicIdPartition, PartitionData> updates, + short version) { + if (!isSubsequent) + return ShareFetchResponse.sizeOf(version, updates.entrySet().iterator()); + else { + synchronized (session) { + int expectedEpoch = ShareFetchMetadata.nextEpoch(reqMetadata.epoch()); + if (session.epoch != expectedEpoch) { + return ShareFetchResponse.sizeOf(version, Collections.emptyIterator()); + } else { + // Pass the partition iterator which updates neither the share fetch context nor the partition map. + return ShareFetchResponse.sizeOf(version, new PartitionIterator(updates.entrySet().iterator(), false)); + } + } + } + } + + @Override + ShareFetchResponse updateAndGenerateResponseData(String groupId, Uuid memberId, + LinkedHashMap<TopicIdPartition, ShareFetchResponseData.PartitionData> updates) { + if (!isSubsequent) { + return new ShareFetchResponse(ShareFetchResponse.toMessage( + Errors.NONE, 0, updates.entrySet().iterator(), Collections.emptyList())); + } else { + int expectedEpoch = ShareFetchMetadata.nextEpoch(reqMetadata.epoch()); + int sessionEpoch; + synchronized (session) { + sessionEpoch = session.epoch; + } + if (session.epoch != expectedEpoch) { + log.info("Subsequent share session {} expected epoch {}, but got {}. Possible duplicate request.", Review Comment: I suggest log.info is not ideal here. Maybe log.debug. ########## server/src/main/java/org/apache/kafka/server/share/ShareSession.java: ########## @@ -0,0 +1,176 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.requests.ShareFetchRequest; +import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; + +public class ShareSession { + + // Helper enum to return the possible type of modified list of TopicIdPartitions in cache + public enum ModifiedTopicIdPartitionType { + ADDED, + UPDATED, + REMOVED + } + + private final ShareSessionKey key; + private final ImplicitLinkedHashCollection<CachedSharePartition> partitionMap; + private final long creationMs; + + private long lastUsedMs; + // visible for testing + public int epoch; + // This is used by the ShareSessionCache to store the last known size of this session. + // If this is -1, the Session is not in the cache. + private int cachedSize = -1; + + /** + * The share session. + * Each share session is protected by its own lock, which must be taken before mutable + * fields are read or modified. This includes modification of the share session partition map. + * + * @param key The share session key to identify the share session uniquely. + * @param partitionMap The CachedPartitionMap. + * @param creationMs The time in milliseconds when this share session was created. + * @param lastUsedMs The last used time in milliseconds. This should only be updated by + * ShareSessionCache#touch. + * @param epoch The share session sequence number. + */ + public ShareSession(ShareSessionKey key, ImplicitLinkedHashCollection<CachedSharePartition> partitionMap, + long creationMs, long lastUsedMs, int epoch) { + this.key = key; + this.partitionMap = partitionMap; + this.creationMs = creationMs; + this.lastUsedMs = lastUsedMs; + this.epoch = epoch; + } + + public ShareSessionKey key() { + synchronized (this) { + return key; + } + } + + public int cachedSize() { + synchronized (this) { + return cachedSize; + } + } + + public void cachedSize(int size) { + synchronized (this) { + cachedSize = size; + } + } + + public long lastUsedMs() { + synchronized (this) { + return lastUsedMs; + } + } + + public void lastUsedMs(long ts) { + synchronized (this) { + lastUsedMs = ts; + } + } + + public ImplicitLinkedHashCollection<CachedSharePartition> partitionMap() { + synchronized (this) { + return partitionMap; + } + } + + // Visible for testing + public int epoch() { + synchronized (this) { + return epoch; + } + } + + public int size() { + synchronized (this) { + return partitionMap.size(); + } + } + + public Boolean isEmpty() { + synchronized (this) { + return partitionMap.isEmpty(); + } + } + + public LastUsedKey lastUsedKey() { + synchronized (this) { + return new LastUsedKey(key, lastUsedMs); + } + } + + // Visible for testing + public long creationMs() { + return creationMs; + } + + // Update the cached partition data based on the request. + public Map<ModifiedTopicIdPartitionType, List<TopicIdPartition>> update(Map<TopicIdPartition, + ShareFetchRequest.SharePartitionData> shareFetchData, + List<TopicIdPartition> toForget) { + List<TopicIdPartition> added = new ArrayList<>(); + List<TopicIdPartition> updated = new ArrayList<>(); + List<TopicIdPartition> removed = new ArrayList<>(); + synchronized (this) { + shareFetchData.forEach((topicIdPartition, sharePartitionData) -> { + CachedSharePartition cachedSharePartitionKey = new CachedSharePartition(topicIdPartition, sharePartitionData, true); + CachedSharePartition cachedPart = partitionMap.find(cachedSharePartitionKey); + if (cachedPart == null) { + partitionMap.mustAdd(cachedSharePartitionKey); + added.add(topicIdPartition); + } else { + cachedPart.updateRequestParams(sharePartitionData); + updated.add(topicIdPartition); + } + }); + toForget.forEach(topicIdPartition -> { + if (partitionMap.remove(new CachedSharePartition(topicIdPartition))) + removed.add(topicIdPartition); + }); + } + Map<ModifiedTopicIdPartitionType, List<TopicIdPartition>> result = new HashMap<>(); + result.put(ModifiedTopicIdPartitionType.ADDED, added); + result.put(ModifiedTopicIdPartitionType.UPDATED, updated); + result.put(ModifiedTopicIdPartitionType.REMOVED, removed); + return result; + } + + public String toString() { + return "ShareSession(" + + " key=" + key + Review Comment: nit: The formatting is cleaner without the space before "key=". ########## server/src/main/java/org/apache/kafka/server/share/ShareSession.java: ########## @@ -0,0 +1,176 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.requests.ShareFetchRequest; +import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; + +public class ShareSession { + + // Helper enum to return the possible type of modified list of TopicIdPartitions in cache + public enum ModifiedTopicIdPartitionType { + ADDED, + UPDATED, + REMOVED + } + + private final ShareSessionKey key; + private final ImplicitLinkedHashCollection<CachedSharePartition> partitionMap; + private final long creationMs; + + private long lastUsedMs; + // visible for testing + public int epoch; + // This is used by the ShareSessionCache to store the last known size of this session. + // If this is -1, the Session is not in the cache. + private int cachedSize = -1; + + /** + * The share session. + * Each share session is protected by its own lock, which must be taken before mutable + * fields are read or modified. This includes modification of the share session partition map. + * + * @param key The share session key to identify the share session uniquely. + * @param partitionMap The CachedPartitionMap. + * @param creationMs The time in milliseconds when this share session was created. + * @param lastUsedMs The last used time in milliseconds. This should only be updated by + * ShareSessionCache#touch. + * @param epoch The share session sequence number. + */ + public ShareSession(ShareSessionKey key, ImplicitLinkedHashCollection<CachedSharePartition> partitionMap, + long creationMs, long lastUsedMs, int epoch) { + this.key = key; + this.partitionMap = partitionMap; + this.creationMs = creationMs; + this.lastUsedMs = lastUsedMs; + this.epoch = epoch; + } + + public ShareSessionKey key() { + synchronized (this) { Review Comment: Because key is final, you shouldn't need to synchronize here. ########## core/src/main/java/kafka/server/ShareSessionContext.java: ########## @@ -0,0 +1,243 @@ +/* + * 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 kafka.server; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import kafka.server.SharePartitionManager.ErroneousAndValidPartitionData; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ShareFetchMetadata; +import org.apache.kafka.common.requests.ShareFetchRequest; +import org.apache.kafka.common.requests.ShareFetchRequest.SharePartitionData; +import org.apache.kafka.common.requests.ShareFetchResponse; +import org.apache.kafka.server.share.CachedSharePartition; +import org.apache.kafka.server.share.ShareSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +/** + * The context for a share session fetch request. + */ +public class ShareSessionContext extends ShareFetchContext { + + private final ShareFetchMetadata reqMetadata; + private Map<TopicIdPartition, SharePartitionData> shareFetchData; + private final boolean isSubsequent; + private ShareSession session; + + private final Logger log = LoggerFactory.getLogger(ShareSessionContext.class); + + /** + * The share fetch context for the first request that starts a share session. + * + * @param reqMetadata The request metadata. + * @param shareFetchData The share partition data from the share fetch request. + */ + public ShareSessionContext(ShareFetchMetadata reqMetadata, + Map<TopicIdPartition, ShareFetchRequest.SharePartitionData> shareFetchData) { + this.reqMetadata = reqMetadata; + this.shareFetchData = shareFetchData; + this.isSubsequent = false; + } + + /** + * The share fetch context for a subsequent request that utilizes an existing share session. + * + * @param reqMetadata The request metadata. + * @param session The subsequent fetch request session. + */ + public ShareSessionContext(ShareFetchMetadata reqMetadata, ShareSession session) { + this.reqMetadata = reqMetadata; + this.session = session; + this.isSubsequent = true; + } + + public Map<TopicIdPartition, ShareFetchRequest.SharePartitionData> shareFetchData() { + return shareFetchData; + } + + public boolean isSubsequent() { + return isSubsequent; + } + + public ShareSession session() { + return session; + } + + @Override + ShareFetchResponse throttleResponse(int throttleTimeMs) { + if (!isSubsequent) { + return new ShareFetchResponse(ShareFetchResponse.toMessage(Errors.NONE, throttleTimeMs, + Collections.emptyIterator(), Collections.emptyList())); + } else { + int expectedEpoch = ShareFetchMetadata.nextEpoch(reqMetadata.epoch()); + int sessionEpoch; + synchronized (session) { + sessionEpoch = session.epoch; + } + if (sessionEpoch != expectedEpoch) { + log.debug("Subsequent share session {} expected epoch {}, but got {}. " + + "Possible duplicate request.", session.key(), expectedEpoch, sessionEpoch); + return new ShareFetchResponse(ShareFetchResponse.toMessage(Errors.INVALID_SHARE_SESSION_EPOCH, + throttleTimeMs, Collections.emptyIterator(), Collections.emptyList())); + } else + return new ShareFetchResponse(ShareFetchResponse.toMessage(Errors.NONE, throttleTimeMs, + Collections.emptyIterator(), Collections.emptyList())); + } + } + + // Iterator that goes over the given partition map and selects partitions that need to be included in the response. + // If updateShareContextAndRemoveUnselected is set to true, the share context will be updated for the selected + // partitions and also remove unselected ones as they are encountered. + private class PartitionIterator implements + Iterator<Entry<TopicIdPartition, PartitionData>> { + private final Iterator<Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData>> iterator; + private final boolean updateShareContextAndRemoveUnselected; + private Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> nextElement; + + + public PartitionIterator(Iterator<Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData>> iterator, boolean updateShareContextAndRemoveUnselected) { + this.iterator = iterator; + this.updateShareContextAndRemoveUnselected = updateShareContextAndRemoveUnselected; + } + + @Override + public boolean hasNext() { + while ((nextElement == null) && iterator.hasNext()) { + Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> element = iterator.next(); + TopicIdPartition topicPart = element.getKey(); + ShareFetchResponseData.PartitionData respData = element.getValue(); + CachedSharePartition cachedPart = session.partitionMap().find(new CachedSharePartition(topicPart)); + boolean mustRespond = cachedPart.maybeUpdateResponseData(respData, updateShareContextAndRemoveUnselected); + if (mustRespond) { + nextElement = element; + if (updateShareContextAndRemoveUnselected && ShareFetchResponse.recordsSize(respData) > 0) { + // Session.partitionMap is of type ImplicitLinkedHashCollection<> which tracks the order of insertion of elements. + // Since, we are updating an element in this case, we need to perform a remove and then a mustAdd to maintain the correct order + session.partitionMap().remove(cachedPart); + session.partitionMap().mustAdd(cachedPart); + } + } else { + if (updateShareContextAndRemoveUnselected) { + iterator.remove(); + } + } + } + return nextElement != null; + } + + @Override + public Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> next() { + if (!hasNext()) throw new NoSuchElementException(); + Map.Entry<TopicIdPartition, ShareFetchResponseData.PartitionData> element = nextElement; + nextElement = null; + return element; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + + @Override + int responseSize(LinkedHashMap<TopicIdPartition, PartitionData> updates, + short version) { + if (!isSubsequent) + return ShareFetchResponse.sizeOf(version, updates.entrySet().iterator()); + else { + synchronized (session) { + int expectedEpoch = ShareFetchMetadata.nextEpoch(reqMetadata.epoch()); + if (session.epoch != expectedEpoch) { + return ShareFetchResponse.sizeOf(version, Collections.emptyIterator()); + } else { + // Pass the partition iterator which updates neither the share fetch context nor the partition map. + return ShareFetchResponse.sizeOf(version, new PartitionIterator(updates.entrySet().iterator(), false)); + } + } + } + } + + @Override + ShareFetchResponse updateAndGenerateResponseData(String groupId, Uuid memberId, + LinkedHashMap<TopicIdPartition, ShareFetchResponseData.PartitionData> updates) { + if (!isSubsequent) { + return new ShareFetchResponse(ShareFetchResponse.toMessage( + Errors.NONE, 0, updates.entrySet().iterator(), Collections.emptyList())); + } else { + int expectedEpoch = ShareFetchMetadata.nextEpoch(reqMetadata.epoch()); + int sessionEpoch; + synchronized (session) { + sessionEpoch = session.epoch; + } + if (session.epoch != expectedEpoch) { Review Comment: You have just captured session.epoch in the synchronized block and then this line makes an unsynchronized access. -- 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