chirag-wadhwa5 commented on code in PR #16456:
URL: https://github.com/apache/kafka/pull/16456#discussion_r1663976290


##########
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:
   Thanks for the review. `handleAcknowledgements` would internally call 
`acknowledge` in `sharePartitionManager`, which is an asyn function returning a 
future. According to the code that we already have in kip-932 branch, 
`handleAcknowledgements` method returns the value by waiting for the future to 
execute completely. So, this piece is synchronous. We can maybe return a future 
from `handleAcknowledgements` and wait for its completion here here, but I'm 
not sure how would that help us in any way.



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