private List <DeleteRecordsReport> extractDeleteRecordsReports(IntPtr resultPtr) => SafeKafkaHandle.GetTopicPartitionOffsetErrorList(resultPtr) .Select(a => new DeleteRecordsReport { Topic = a.Topic, Partition = a.Partition, Offset = a.Offset, Error = a.Error }) .ToList();
private void CommitCallback( IntPtr rk, ErrorCode err, IntPtr offsets, IntPtr opaque) { // Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios). if (kafkaHandle.IsClosed) { return; } offsetsCommittedHandler?.Invoke(new CommittedOffsets( SafeKafkaHandle.GetTopicPartitionOffsetErrorList(offsets), kafkaHandle.CreatePossiblyFatalError(err, null) )); }
private void RebalanceCallback( IntPtr rk, ErrorCode err, IntPtr partitionsPtr, IntPtr opaque) { try { // Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios). if (kafkaHandle.IsClosed) { // The RebalanceCallback should never be invoked as a side effect of Dispose. // If for some reason flow of execution gets here, something is badly wrong. // (and we have a closed librdkafka handle that is expecting an assign call...) throw new Exception("Unexpected rebalance callback on disposed kafkaHandle"); } if (kafkaHandle.RebalanceProtocol == "COOPERATIVE" && this.revokedOrLostHandlerIsFunc) { throw new InvalidOperationException("Neither revoked nor lost partition handlers may return an updated assignment when a COOPERATIVE assignor is in use"); } var partitions = SafeKafkaHandle.GetTopicPartitionOffsetErrorList(partitionsPtr).Select(p => p.TopicPartition).ToList(); if (err == ErrorCode.Local_AssignPartitions) { if (partitionsAssignedHandler == null) { if (kafkaHandle.RebalanceProtocol == "COOPERATIVE") { IncrementalAssign(partitions.Select(p => new TopicPartitionOffset(p, Offset.Unset))); } else { Assign(partitions.Select(p => new TopicPartitionOffset(p, Offset.Unset))); } return; } lock (assignCallCountLockObj) { assignCallCount = 0; } var assignTo = partitionsAssignedHandler(partitions); lock (assignCallCountLockObj) { if (assignCallCount > 0) { throw new InvalidOperationException("(Incremental)Assign/Unassign must not be called in the partitions assigned handler"); } } if (kafkaHandle.RebalanceProtocol == "COOPERATIVE") { if (assignTo.Count() != partitions.Count()) { throw new InvalidOperationException("The partitions assigned handler must not return a different set of topic partitions than it was provided"); } var sortedPartitions = partitions.OrderBy(p => p).ToList(); var sortedAssignTo = assignTo.OrderBy(p => p.TopicPartition); var partitionsIter = sortedPartitions.GetEnumerator(); foreach (var p in sortedAssignTo) { partitionsIter.MoveNext(); if (p.TopicPartition != partitionsIter.Current) { throw new InvalidOperationException("The partitions assigned handler must not return a different set of topic partitions than it was provided"); } } IncrementalAssign(sortedAssignTo); } else { Assign(assignTo); } return; } if (err == ErrorCode.Local_RevokePartitions) { if (partitionsRevokedHandler == null && (!kafkaHandle.AssignmentLost || partitionsLostHandler == null)) { if (kafkaHandle.RebalanceProtocol == "COOPERATIVE") { IncrementalUnassign(partitions); } else { Unassign(); } return; } var assignmentWithPositions = new List <TopicPartitionOffset>(); foreach (var tp in partitions) { try { assignmentWithPositions.Add(new TopicPartitionOffset(tp, Position(tp))); } catch { assignmentWithPositions.Add(new TopicPartitionOffset(tp, Offset.Unset)); } } lock (assignCallCountLockObj) { assignCallCount = 0; } var assignTo = kafkaHandle.AssignmentLost ? (partitionsLostHandler != null ? partitionsLostHandler(assignmentWithPositions) : partitionsRevokedHandler(assignmentWithPositions)) : partitionsRevokedHandler(assignmentWithPositions); lock (assignCallCountLockObj) { if (assignCallCount > 0) { throw new InvalidOperationException("Assign/Unassign must not be called in the partitions revoked handler"); } } if (kafkaHandle.RebalanceProtocol == "COOPERATIVE") { // assignTo is always empty, not used in the COOPERATIVE case. IncrementalUnassign(partitions); } else { Unassign(); } return; } throw new KafkaException(kafkaHandle.CreatePossiblyFatalError(err, null)); } catch (Exception e) { Unassign(); handlerException = e; } }
private void RebalanceCallback( IntPtr rk, ErrorCode err, IntPtr partitions, IntPtr opaque) { try { var partitionAssignment = SafeKafkaHandle.GetTopicPartitionOffsetErrorList(partitions).Select(p => p.TopicPartition).ToList(); // Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios). if (kafkaHandle.IsClosed) { // The RebalanceCallback should never be invoked as a side effect of Dispose. // If for some reason flow of execution gets here, something is badly wrong. // (and we have a closed librdkafka handle that is expecting an assign call...) throw new Exception("Unexpected rebalance callback on disposed kafkaHandle"); } if (err == ErrorCode.Local_AssignPartitions) { if (partitionsAssignedHandler == null) { Assign(partitionAssignment.Select(p => new TopicPartitionOffset(p, Offset.Unset))); return; } lock (assignCallCountLockObj) { assignCallCount = 0; } var assignTo = partitionsAssignedHandler(partitionAssignment); lock (assignCallCountLockObj) { if (assignCallCount > 0) { throw new InvalidOperationException("Assign/Unassign must not be called in the partitions assigned handler."); } } Assign(assignTo); return; } if (err == ErrorCode.Local_RevokePartitions) { if (partitionsRevokedHandler == null) { Unassign(); return; } var assignmentWithPositions = new List <TopicPartitionOffset>(); foreach (var tp in partitionAssignment) { try { assignmentWithPositions.Add(new TopicPartitionOffset(tp, Position(tp))); } catch { assignmentWithPositions.Add(new TopicPartitionOffset(tp, Offset.Unset)); } } lock (assignCallCountLockObj) { assignCallCount = 0; } var assignTo = partitionsRevokedHandler(assignmentWithPositions); lock (assignCallCountLockObj) { if (assignCallCount > 0) { throw new InvalidOperationException("Assign/Unassign must not be called in the partitions revoked handler."); } } // This distinction is important because calling Assign whilst the consumer is being // closed (which will generally trigger this callback) is disallowed. if (assignTo.Count() > 0) { Assign(assignTo); } else { Unassign(); } return; } throw new KafkaException(kafkaHandle.CreatePossiblyFatalError(err, null)); } catch (Exception e) { handlerException = e; } }
private void RebalanceCallback( IntPtr rk, ErrorCode err, IntPtr partitions, IntPtr opaque) { var partitionList = SafeKafkaHandle.GetTopicPartitionOffsetErrorList(partitions).Select(p => p.TopicPartition).ToList(); // Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios). if (kafkaHandle.IsClosed) { // The RebalanceCallback should never be invoked as a side effect of Dispose. // If for some reason flow of execution gets here, something is badly wrong. // (and we have a closed librdkafka handle that is expecting an assign call...) throw new Exception("unexpected rebalance callback on disposed kafkaHandle"); } // Note: The contract with librdkafka requires the application to acknowledge rebalances by calling Assign. // To make the API less error prone, this is done automatically by the C# binding if required - the number // of times assign is called by user code is tracked, and if this is zero, then assign is called automatically. if (err == ErrorCode.Local_AssignPartitions) { var handler = OnPartitionsAssigned; if (handler != null && handler.GetInvocationList().Length > 0) { assignCallCount = 0; handler(this, partitionList); if (assignCallCount == 1) { return; } if (assignCallCount > 1) { throw new InvalidOperationException( $"Assign/Unassign was called {assignCallCount} times after OnPartitionsAssigned was raised. It must be called at most once."); } } Assign(partitionList.Select(p => new TopicPartitionOffset(p, Offset.Invalid))); } else if (err == ErrorCode.Local_RevokePartitions) { var handler = OnPartitionsRevoked; if (handler != null && handler.GetInvocationList().Length > 0) { assignCallCount = 0; handler(this, partitionList); if (assignCallCount == 1) { return; } if (assignCallCount > 1) { throw new InvalidOperationException( $"Assign/Unassign was called {assignCallCount} times after OnPartitionsAssigned was raised. It must be called at most once."); } } Unassign(); } else { throw new KafkaException(kafkaHandle.CreatePossiblyFatalError(err, null)); } }