apoorvmittal10 commented on code in PR #16456:
URL: https://github.com/apache/kafka/pull/16456#discussion_r1665661239


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -3956,11 +3961,482 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  /**
+   * Handle a shareFetch request
+   */
   def handleShareFetchRequest(request: RequestChannel.Request): Unit = {
     val shareFetchRequest = request.body[ShareFetchRequest]
-    // TODO: Implement the ShareFetchRequest handling
-    requestHelper.sendMaybeThrottle(request, 
shareFetchRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
-    CompletableFuture.completedFuture[Unit](())
+
+    if (!config.isNewGroupCoordinatorEnabled) {
+      // The API is not supported by the "old" group coordinator (the 
default). If the
+      // new one is not enabled, we fail directly here.
+      requestHelper.sendMaybeThrottle(request, 
shareFetchRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
+      CompletableFuture.completedFuture[Unit](())
+      return
+    } else if (!config.isShareGroupEnabled) {
+      // The API is not supported when the "share" rebalance protocol has not 
been set explicitly
+      requestHelper.sendMaybeThrottle(request, 
shareFetchRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
+      CompletableFuture.completedFuture[Unit](())
+      return
+    }
+    val topicNames = metadataCache.topicIdsToNames()
+    val sharePartitionManager : SharePartitionManager = 
sharePartitionManagerOption match {
+      case Some(manager) => manager
+      case None => throw new IllegalStateException("ShareFetchRequest received 
but SharePartitionManager is not initialized")
+    }
+
+    val groupId = shareFetchRequest.data.groupId
+    val clientId = request.header.clientId
+    val memberId = shareFetchRequest.data().memberId()
+    val shareSessionEpoch = shareFetchRequest.data().shareSessionEpoch()
+
+    val shareFetchData = shareFetchRequest.shareFetchData(topicNames)
+    val forgottenTopics = shareFetchRequest.forgottenTopics(topicNames)
+    var cachedTopicPartitions : util.List[TopicIdPartition] = null
+
+    if (shareSessionEpoch == ShareFetchMetadata.FINAL_EPOCH) {
+      try {
+        cachedTopicPartitions = 
sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, 
Uuid.fromString(memberId))
+      } catch {
+        // Exception handling is needed when this value is being utilized on 
receiving FINAL_EPOCH.
+        case _: ShareSessionNotFoundException => cachedTopicPartitions = null
+      }
+    }
+
+    def isAcknowledgeDataPresentInFetchRequest() : Boolean = {
+      var isAcknowledgeDataPresent = false
+      shareFetchRequest.data().topics().forEach ( topic => {
+        breakable{
+          topic.partitions().forEach ( partition => {
+            if (partition.acknowledgementBatches() != null && 
!partition.acknowledgementBatches().isEmpty) {
+              isAcknowledgeDataPresent = true
+              break()
+            } else {
+              isAcknowledgeDataPresent = false
+            }
+          })
+        }
+      })
+      isAcknowledgeDataPresent
+    }
+
+    val isAcknowledgeDataPresent = isAcknowledgeDataPresentInFetchRequest()
+    var shareFetchResponse : ShareFetchResponse = null
+    // Variable to store any error thrown while the handling piggybacked 
acknowledgements
+    var acknowledgeError : Errors = Errors.NONE
+    // Variable to store the topic partition wise result of piggybacked 
acknowledgements
+    var acknowledgeResult = mutable.Map[TopicIdPartition, 
ShareAcknowledgeResponseData.PartitionData]()
+
+    def isInvalidShareFetchRequest() : Boolean = {
+      // The Initial Share Fetch Request should not Acknowledge any data
+      if (shareSessionEpoch == ShareFetchMetadata.INITIAL_EPOCH && 
isAcknowledgeDataPresent) {
+        return true
+      }
+      false
+    }
+
+    val newReqMetadata : ShareFetchMetadata = new 
ShareFetchMetadata(Uuid.fromString(memberId), shareSessionEpoch)
+    var shareFetchContext : ShareFetchContext = null
+    try {
+      // Creating the shareFetchContext for Share Session Handling
+      shareFetchContext = sharePartitionManager.newContext(groupId, 
shareFetchData, forgottenTopics, newReqMetadata)
+    } catch {
+      case e: Exception => shareFetchResponse = 
shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, e) 
match {
+        case response: ShareFetchResponse => response
+        case _ => null
+      }
+    }
+    // This check is done to make sure that there was no Share Session related 
error while creating shareFetchContext
+    if(shareFetchResponse == null) {
+      // check if the Request is Invalid
+      if(isInvalidShareFetchRequest()) {
+        shareFetchResponse = 
shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, 
Errors.INVALID_REQUEST.exception) match {
+          case response: ShareFetchResponse => response
+          case _ => null
+        }
+      } else {
+        val erroneousAndValidPartitionData : ErroneousAndValidPartitionData = 
shareFetchContext.getErroneousAndValidTopicIdPartitions
+        val topicIdPartitionSeq : mutable.Set[TopicIdPartition] = mutable.Set()
+        erroneousAndValidPartitionData.erroneous.forEach {
+          case(tp, _) => if (!topicIdPartitionSeq.contains(tp)) 
topicIdPartitionSeq += tp
+        }
+        erroneousAndValidPartitionData.validTopicIdPartitions.forEach {
+          case(tp, _) => if (!topicIdPartitionSeq.contains(tp)) 
topicIdPartitionSeq += tp
+        }
+        shareFetchData.forEach {
+          case(tp, _) => if (!topicIdPartitionSeq.contains(tp)) 
topicIdPartitionSeq += tp
+        }
+
+        val authorizedTopics = authHelper.filterByAuthorized(
+          request.context,
+          READ,
+          TOPIC,
+          topicIdPartitionSeq
+        )(_.topicPartition.topic)
+
+        // Handling the Acknowledgements from the ShareFetchRequest
+        // If this check is true, then we are sure that this is not an Initial 
ShareFetch Request, otherwise the request would have been invalid
+        if(isAcknowledgeDataPresent) {
+          if (!authHelper.authorize(request.context, READ, GROUP, groupId)) {
+            acknowledgeError = Errors.GROUP_AUTHORIZATION_FAILED
+          } else {
+            acknowledgeResult = handleAcknowledgements(request, topicNames, 
sharePartitionManager, authorizedTopics, groupId, memberId, true)

Review Comment:
   I do not see any futures handled here hence this maked the processing 
synchronous, irrespective if other API calls to sharepartitionmanager are 
async. Moreover I am failed to understand the behaviour of 
`handleAcknowledgements` method i.e. I do see `shareAcknowledgeResult` is 
returned right away but there might be a delay in getting reposne from 
`sharePartitionManager.acknowledge` method hence how that's handled?



-- 
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

Reply via email to