-
Notifications
You must be signed in to change notification settings - Fork 878
/
Copy pathProducer.cs
928 lines (814 loc) · 36.6 KB
/
Producer.cs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
// Copyright 2016-2018 Confluent Inc.
//
// Licensed 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.
//
// Refer to LICENSE for more information.
using System;
using System.Collections.Generic;
using System.Linq;
using System.Runtime.InteropServices;
using System.Threading;
using System.Threading.Tasks;
using Confluent.Kafka.Impl;
using Confluent.Kafka.Internal;
namespace Confluent.Kafka
{
/// <summary>
/// A high level producer with serialization capability.
/// </summary>
internal class Producer<TKey, TValue> : IProducer<TKey, TValue>, IClient
{
internal class Config
{
public IEnumerable<KeyValuePair<string, string>> config;
public Action<Error> errorHandler;
public Action<LogMessage> logHandler;
public Action<string> statisticsHandler;
}
private ISerializer<TKey> keySerializer;
private ISerializer<TValue> valueSerializer;
private IAsyncSerializer<TKey> asyncKeySerializer;
private IAsyncSerializer<TValue> asyncValueSerializer;
private static readonly Dictionary<Type, object> defaultSerializers = new Dictionary<Type, object>
{
{ typeof(Null), Serializers.Null },
{ typeof(int), Serializers.Int32 },
{ typeof(long), Serializers.Int64 },
{ typeof(string), Serializers.Utf8 },
{ typeof(float), Serializers.Single },
{ typeof(double), Serializers.Double },
{ typeof(byte[]), Serializers.ByteArray }
};
private int cancellationDelayMaxMs;
private bool disposeHasBeenCalled = false;
private object disposeHasBeenCalledLockObj = new object();
private bool manualPoll = false;
private bool enableDeliveryReports = true;
private bool enableDeliveryReportKey = true;
private bool enableDeliveryReportValue = true;
private bool enableDeliveryReportTimestamp = true;
private bool enableDeliveryReportHeaders = true;
private bool enableDeliveryReportPersistedStatus = true;
private SafeKafkaHandle ownedKafkaHandle;
private Handle borrowedHandle;
private SafeKafkaHandle KafkaHandle
=> ownedKafkaHandle != null
? ownedKafkaHandle
: borrowedHandle.LibrdkafkaHandle;
private Task callbackTask;
private CancellationTokenSource callbackCts;
private int eventsServedCount = 0;
private object pollSyncObj = new object();
private Task StartPollTask(CancellationToken ct)
=> Task.Factory.StartNew(() =>
{
try
{
while (true)
{
ct.ThrowIfCancellationRequested();
int eventsServedCount_ = ownedKafkaHandle.Poll((IntPtr)cancellationDelayMaxMs);
// note: lock {} is equivalent to Monitor.Enter then Monitor.Exit
if (eventsServedCount_ > 0)
{
lock (pollSyncObj)
{
this.eventsServedCount += eventsServedCount_;
Monitor.Pulse(pollSyncObj);
}
}
}
}
catch (OperationCanceledException) {}
}, ct, TaskCreationOptions.LongRunning, TaskScheduler.Default);
private Action<Error> errorHandler;
private Librdkafka.ErrorDelegate errorCallbackDelegate;
private void ErrorCallback(IntPtr rk, ErrorCode err, string reason, IntPtr opaque)
{
// Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios).
if (ownedKafkaHandle.IsClosed) { return; }
errorHandler?.Invoke(KafkaHandle.CreatePossiblyFatalError(err, reason));
}
private Action<string> statisticsHandler;
private Librdkafka.StatsDelegate statisticsCallbackDelegate;
private int StatisticsCallback(IntPtr rk, IntPtr json, UIntPtr json_len, IntPtr opaque)
{
// Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios).
if (ownedKafkaHandle.IsClosed) { return 0; }
statisticsHandler?.Invoke(Util.Marshal.PtrToStringUTF8(json));
return 0; // instruct librdkafka to immediately free the json ptr.
}
private Action<LogMessage> logHandler;
private object loggerLockObj = new object();
private Librdkafka.LogDelegate logCallbackDelegate;
private void LogCallback(IntPtr rk, SyslogLevel level, string fac, string buf)
{
// Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios).
// Note: kafkaHandle can be null if the callback is during construction (in that case, we want the delegate to run).
if (ownedKafkaHandle != null && ownedKafkaHandle.IsClosed) { return; }
logHandler?.Invoke(new LogMessage(Util.Marshal.PtrToStringUTF8(Librdkafka.name(rk)), level, fac, buf));
}
private Librdkafka.DeliveryReportDelegate DeliveryReportCallback;
private void DeliveryReportCallbackImpl(IntPtr rk, IntPtr rkmessage, IntPtr opaque)
{
// Ensure registered handlers are never called as a side-effect of Dispose/Finalize (prevents deadlocks in common scenarios).
if (ownedKafkaHandle.IsClosed) { return; }
var msg = Util.Marshal.PtrToStructure<rd_kafka_message>(rkmessage);
// the msg._private property has dual purpose. Here, it is an opaque pointer set
// by Topic.Produce to be an IDeliveryHandler. When Consuming, it's for internal
// use (hence the name).
if (msg._private == IntPtr.Zero)
{
// Note: this can occur if the ProduceAsync overload that accepts a DeliveryHandler
// was used and the delivery handler was set to null.
return;
}
var gch = GCHandle.FromIntPtr(msg._private);
var deliveryHandler = (IDeliveryHandler) gch.Target;
gch.Free();
Headers headers = null;
if (this.enableDeliveryReportHeaders)
{
headers = new Headers();
Librdkafka.message_headers(rkmessage, out IntPtr hdrsPtr);
if (hdrsPtr != IntPtr.Zero)
{
for (var i=0; ; ++i)
{
var err = Librdkafka.header_get_all(hdrsPtr, (IntPtr)i, out IntPtr namep, out IntPtr valuep, out IntPtr sizep);
if (err != ErrorCode.NoError)
{
break;
}
var headerName = Util.Marshal.PtrToStringUTF8(namep);
byte[] headerValue = null;
if (valuep != IntPtr.Zero)
{
headerValue = new byte[(int)sizep];
Marshal.Copy(valuep, headerValue, 0, (int)sizep);
}
headers.Add(headerName, headerValue);
}
}
}
IntPtr timestampType = (IntPtr)TimestampType.NotAvailable;
long timestamp = 0;
if (enableDeliveryReportTimestamp)
{
timestamp = Librdkafka.message_timestamp(rkmessage, out timestampType);
}
PersistenceStatus messageStatus = PersistenceStatus.PossiblyPersisted;
if (enableDeliveryReportPersistedStatus)
{
messageStatus = Librdkafka.message_status(rkmessage);
}
deliveryHandler.HandleDeliveryReport(
new DeliveryReport<Null, Null>
{
// Topic is not set here in order to avoid the marshalling cost.
// Instead, the delivery handler is expected to cache the topic string.
Partition = msg.partition,
Offset = msg.offset,
Error = KafkaHandle.CreatePossiblyFatalError(msg.err, null),
Status = messageStatus,
Message = new Message<Null, Null> { Timestamp = new Timestamp(timestamp, (TimestampType)timestampType), Headers = headers }
}
);
}
private void ProduceImpl(
string topic,
byte[] val, int valOffset, int valLength,
byte[] key, int keyOffset, int keyLength,
Timestamp timestamp,
Partition partition,
IEnumerable<IHeader> headers,
IDeliveryHandler deliveryHandler)
{
if (timestamp.Type != TimestampType.CreateTime)
{
if (timestamp != Timestamp.Default)
{
throw new ArgumentException("Timestamp must be either Timestamp.Default, or Timestamp.CreateTime.");
}
}
ErrorCode err;
if (this.enableDeliveryReports && deliveryHandler != null)
{
// Passes the TaskCompletionSource to the delivery report callback via the msg_opaque pointer
// Note: There is a level of indirection between the GCHandle and
// physical memory address. GCHandle.ToIntPtr doesn't get the
// physical address, it gets an id that refers to the object via
// a handle-table.
var gch = GCHandle.Alloc(deliveryHandler);
var ptr = GCHandle.ToIntPtr(gch);
err = KafkaHandle.Produce(
topic,
val, valOffset, valLength,
key, keyOffset, keyLength,
partition.Value,
timestamp.UnixTimestampMs,
headers,
ptr);
if (err != ErrorCode.NoError)
{
// note: freed in the delivery handler callback otherwise.
gch.Free();
}
}
else
{
err = KafkaHandle.Produce(
topic,
val, valOffset, valLength,
key, keyOffset, keyLength,
partition.Value,
timestamp.UnixTimestampMs,
headers,
IntPtr.Zero);
}
if (err != ErrorCode.NoError)
{
throw new KafkaException(KafkaHandle.CreatePossiblyFatalError(err, null));
}
}
/// <summary>
/// Refer to <see cref="Confluent.Kafka.IProducer{TKey,TValue}.Poll(TimeSpan)" />
/// </summary>
public int Poll(TimeSpan timeout)
{
if (manualPoll)
{
return this.KafkaHandle.Poll((IntPtr)timeout.TotalMillisecondsAsInt());
}
lock (pollSyncObj)
{
if (eventsServedCount == 0)
{
Monitor.Wait(pollSyncObj, timeout);
}
var result = eventsServedCount;
eventsServedCount = 0;
return result;
}
}
/// <summary>
/// Refer to <see cref="Confluent.Kafka.IProducer{TKey,TValue}.Flush(TimeSpan)" />
/// </summary>
public int Flush(TimeSpan timeout)
=> KafkaHandle.Flush(timeout.TotalMillisecondsAsInt());
/// <summary>
/// Refer to <see cref="Confluent.Kafka.IProducer{TKey,TValue}.Flush(CancellationToken)" />
/// </summary>
public void Flush(CancellationToken cancellationToken)
{
while (true)
{
int result = KafkaHandle.Flush(100);
if (result == 0)
{
return;
}
if (cancellationToken.IsCancellationRequested)
{
// TODO: include flush number in exception.
throw new OperationCanceledException();
}
}
}
/// <summary>
/// Releases all resources used by this <see cref="Producer{TKey,TValue}" />.
/// </summary>
public void Dispose()
{
Dispose(true);
GC.SuppressFinalize(this);
}
/// <summary>
/// Releases the unmanaged resources used by the
/// <see cref="Producer{TKey,TValue}" />
/// and optionally disposes the managed resources.
/// </summary>
/// <param name="disposing">
/// true to release both managed and unmanaged resources;
/// false to release only unmanaged resources.
/// </param>
protected virtual void Dispose(bool disposing)
{
// Calling Dispose a second or subsequent time should be a no-op.
lock (disposeHasBeenCalledLockObj)
{
if (disposeHasBeenCalled) { return; }
disposeHasBeenCalled = true;
}
// do nothing if we borrowed a handle.
if (ownedKafkaHandle == null) { return; }
if (disposing)
{
if (!this.manualPoll)
{
callbackCts.Cancel();
try
{
// Note: It's necessary to wait on callbackTask before disposing kafkaHandle
// since the poll loop makes use of this.
callbackTask.Wait();
}
catch (AggregateException e)
{
if (e.InnerException.GetType() != typeof(TaskCanceledException))
{
throw e.InnerException;
}
}
finally
{
callbackCts.Dispose();
}
}
// calls to rd_kafka_destroy may result in callbacks
// as a side-effect. however the callbacks this class
// registers with librdkafka ensure that any registered
// events are not called if kafkaHandle has been closed.
// this avoids deadlocks in common scenarios.
ownedKafkaHandle.Dispose();
}
}
/// <summary>
/// <see cref="IClient.Name" />
/// </summary>
public string Name
=> KafkaHandle.Name;
/// <summary>
/// <see cref="IClient.AddBrokers(string)" />
/// </summary>
public int AddBrokers(string brokers)
=> KafkaHandle.AddBrokers(brokers);
/// <summary>
/// <see cref="IClient.Handle" />
/// </summary>
public Handle Handle
{
get
{
if (this.ownedKafkaHandle != null)
{
return new Handle { Owner = this, LibrdkafkaHandle = ownedKafkaHandle };
}
return borrowedHandle;
}
}
private void InitializeSerializers(
ISerializer<TKey> keySerializer,
ISerializer<TValue> valueSerializer,
IAsyncSerializer<TKey> asyncKeySerializer,
IAsyncSerializer<TValue> asyncValueSerializer)
{
// setup key serializer.
if (keySerializer == null && asyncKeySerializer == null)
{
if (!defaultSerializers.TryGetValue(typeof(TKey), out object serializer))
{
throw new ArgumentNullException(
$"Key serializer not specified and there is no default serializer defined for type {typeof(TKey).Name}.");
}
this.keySerializer = (ISerializer<TKey>)serializer;
}
else if (keySerializer == null && asyncKeySerializer != null)
{
this.asyncKeySerializer = asyncKeySerializer;
}
else if (keySerializer != null && asyncKeySerializer == null)
{
this.keySerializer = keySerializer;
}
else
{
throw new InvalidOperationException("FATAL: Both async and sync key serializers were set.");
}
// setup value serializer.
if (valueSerializer == null && asyncValueSerializer == null)
{
if (!defaultSerializers.TryGetValue(typeof(TValue), out object serializer))
{
throw new ArgumentNullException(
$"Value serializer not specified and there is no default serializer defined for type {typeof(TValue).Name}.");
}
this.valueSerializer = (ISerializer<TValue>)serializer;
}
else if (valueSerializer == null && asyncValueSerializer != null)
{
this.asyncValueSerializer = asyncValueSerializer;
}
else if (valueSerializer != null && asyncValueSerializer == null)
{
this.valueSerializer = valueSerializer;
}
else
{
throw new InvalidOperationException("FATAL: Both async and sync value serializers were set.");
}
}
internal Producer(DependentProducerBuilder<TKey, TValue> builder)
{
this.borrowedHandle = builder.Handle;
if (!borrowedHandle.Owner.GetType().Name.Contains("Producer")) // much simpler than checking actual types.
{
throw new Exception("A Producer instance may only be constructed using the handle of another Producer instance.");
}
InitializeSerializers(
builder.KeySerializer, builder.ValueSerializer,
builder.AsyncKeySerializer, builder.AsyncValueSerializer);
}
internal Producer(ProducerBuilder<TKey, TValue> builder)
{
var baseConfig = builder.ConstructBaseConfig(this);
// TODO: Make Tasks auto complete when EnableDeliveryReportsPropertyName is set to false.
// TODO: Hijack the "delivery.report.only.error" configuration parameter and add functionality to enforce that Tasks
// that never complete are never created when this is set to true.
this.statisticsHandler = baseConfig.statisticsHandler;
this.logHandler = baseConfig.logHandler;
this.errorHandler = baseConfig.errorHandler;
var config = Confluent.Kafka.Config.ExtractCancellationDelayMaxMs(baseConfig.config, out this.cancellationDelayMaxMs);
this.DeliveryReportCallback = DeliveryReportCallbackImpl;
Librdkafka.Initialize(null);
var modifiedConfig = config
.Where(prop =>
prop.Key != ConfigPropertyNames.Producer.EnableBackgroundPoll &&
prop.Key != ConfigPropertyNames.Producer.EnableDeliveryReports &&
prop.Key != ConfigPropertyNames.Producer.DeliveryReportFields);
if (modifiedConfig.Where(obj => obj.Key == "delivery.report.only.error").Count() > 0)
{
// A managed object is kept alive over the duration of the produce request. If there is no
// delivery report generated, there will be a memory leak. We could possibly support this
// property by keeping track of delivery reports in managed code, but this seems like
// more trouble than it's worth.
throw new ArgumentException("The 'delivery.report.only.error' property is not supported by this client");
}
var enableBackgroundPollObj = config.FirstOrDefault(prop => prop.Key == ConfigPropertyNames.Producer.EnableBackgroundPoll).Value;
if (enableBackgroundPollObj != null)
{
this.manualPoll = !bool.Parse(enableBackgroundPollObj);
}
var enableDeliveryReportsObj = config.FirstOrDefault(prop => prop.Key == ConfigPropertyNames.Producer.EnableDeliveryReports).Value;
if (enableDeliveryReportsObj != null)
{
this.enableDeliveryReports = bool.Parse(enableDeliveryReportsObj);
}
var deliveryReportEnabledFieldsObj = config.FirstOrDefault(prop => prop.Key == ConfigPropertyNames.Producer.DeliveryReportFields).Value;
if (deliveryReportEnabledFieldsObj != null)
{
var fields = deliveryReportEnabledFieldsObj.Replace(" ", "");
if (fields != "all")
{
this.enableDeliveryReportKey = false;
this.enableDeliveryReportValue = false;
this.enableDeliveryReportHeaders = false;
this.enableDeliveryReportTimestamp = false;
this.enableDeliveryReportPersistedStatus = false;
if (fields != "none")
{
var parts = fields.Split(',');
foreach (var part in parts)
{
switch (part)
{
case "key": this.enableDeliveryReportKey = true; break;
case "value": this.enableDeliveryReportValue = true; break;
case "timestamp": this.enableDeliveryReportTimestamp = true; break;
case "headers": this.enableDeliveryReportHeaders = true; break;
case "status": this.enableDeliveryReportPersistedStatus = true; break;
default: throw new ArgumentException(
$"Unknown delivery report field name '{part}' in config value '{ConfigPropertyNames.Producer.DeliveryReportFields}'.");
}
}
}
}
}
var configHandle = SafeConfigHandle.Create();
modifiedConfig.ToList().ForEach((kvp) => {
if (kvp.Value == null) throw new ArgumentNullException($"'{kvp.Key}' configuration parameter must not be null.");
configHandle.Set(kvp.Key, kvp.Value);
});
IntPtr configPtr = configHandle.DangerousGetHandle();
if (enableDeliveryReports)
{
Librdkafka.conf_set_dr_msg_cb(configPtr, DeliveryReportCallback);
}
// Explicitly keep references to delegates so they are not reclaimed by the GC.
errorCallbackDelegate = ErrorCallback;
logCallbackDelegate = LogCallback;
statisticsCallbackDelegate = StatisticsCallback;
if (errorHandler != null)
{
Librdkafka.conf_set_error_cb(configPtr, errorCallbackDelegate);
}
if (logHandler != null)
{
Librdkafka.conf_set_log_cb(configPtr, logCallbackDelegate);
}
if (statisticsHandler != null)
{
Librdkafka.conf_set_stats_cb(configPtr, statisticsCallbackDelegate);
}
this.ownedKafkaHandle = SafeKafkaHandle.Create(RdKafkaType.Producer, configPtr, this);
configHandle.SetHandleAsInvalid(); // config object is no longer useable.
if (!manualPoll)
{
callbackCts = new CancellationTokenSource();
callbackTask = StartPollTask(callbackCts.Token);
}
InitializeSerializers(
builder.KeySerializer, builder.ValueSerializer,
builder.AsyncKeySerializer, builder.AsyncValueSerializer);
}
/// <summary>
/// Refer to <see cref="Confluent.Kafka.IProducer{TKey,TValue}.ProduceAsync(TopicPartition, Message{TKey, TValue})" />
/// </summary>
public async Task<DeliveryResult<TKey, TValue>> ProduceAsync(
TopicPartition topicPartition,
Message<TKey, TValue> message)
{
byte[] keyBytes;
try
{
keyBytes = (keySerializer != null)
? keySerializer.Serialize(message.Key, new SerializationContext(MessageComponentType.Key, topicPartition.Topic))
: await asyncKeySerializer.SerializeAsync(message.Key, new SerializationContext(MessageComponentType.Key, topicPartition.Topic));
}
catch (Exception ex)
{
throw new ProduceException<TKey, TValue>(
new Error(ErrorCode.Local_KeySerialization),
new DeliveryResult<TKey, TValue>
{
Message = message,
TopicPartitionOffset = new TopicPartitionOffset(topicPartition, Offset.Unset)
},
ex);
}
byte[] valBytes;
try
{
valBytes = (valueSerializer != null)
? valueSerializer.Serialize(message.Value, new SerializationContext(MessageComponentType.Value, topicPartition.Topic))
: await asyncValueSerializer.SerializeAsync(message.Value, new SerializationContext(MessageComponentType.Value, topicPartition.Topic));
}
catch (Exception ex)
{
throw new ProduceException<TKey, TValue>(
new Error(ErrorCode.Local_ValueSerialization),
new DeliveryResult<TKey, TValue>
{
Message = message,
TopicPartitionOffset = new TopicPartitionOffset(topicPartition, Offset.Unset)
},
ex);
}
try
{
if (enableDeliveryReports)
{
var handler = new TypedTaskDeliveryHandlerShim<TKey, TValue>(
topicPartition.Topic,
enableDeliveryReportKey ? message.Key : default(TKey),
enableDeliveryReportValue ? message.Value : default(TValue));
ProduceImpl(
topicPartition.Topic,
valBytes, 0, valBytes == null ? 0 : valBytes.Length,
keyBytes, 0, keyBytes == null ? 0 : keyBytes.Length,
message.Timestamp, topicPartition.Partition, message.Headers,
handler);
return await handler.Task;
}
else
{
ProduceImpl(
topicPartition.Topic,
valBytes, 0, valBytes == null ? 0 : valBytes.Length,
keyBytes, 0, keyBytes == null ? 0 : keyBytes.Length,
message.Timestamp, topicPartition.Partition, message.Headers,
null);
var result = new DeliveryResult<TKey, TValue>
{
TopicPartitionOffset = new TopicPartitionOffset(topicPartition, Offset.Unset),
Message = message
};
return result;
}
}
catch (KafkaException ex)
{
throw new ProduceException<TKey, TValue>(
ex.Error,
new DeliveryResult<TKey, TValue>
{
Message = message,
TopicPartitionOffset = new TopicPartitionOffset(topicPartition, Offset.Unset)
});
}
}
/// <summary>
/// Refer to <see cref="Confluent.Kafka.IProducer{TKey,TValue}.ProduceAsync(string, Message{TKey, TValue})" />
/// </summary>
public Task<DeliveryResult<TKey, TValue>> ProduceAsync(
string topic,
Message<TKey, TValue> message
)
=> ProduceAsync(new TopicPartition(topic, Partition.Any), message);
/// <summary>
/// Refer to <see cref="Confluent.Kafka.IProducer{TKey,TValue}.Produce(string, Message{TKey, TValue}, Action{DeliveryReport{TKey, TValue}})" />
/// </summary>
public void Produce(
string topic,
Message<TKey, TValue> message,
Action<DeliveryReport<TKey, TValue>> deliveryHandler = null
)
=> Produce(new TopicPartition(topic, Partition.Any), message, deliveryHandler);
/// <summary>
/// Refer to <see cref="Confluent.Kafka.IProducer{TKey,TValue}.Produce(TopicPartition, Message{TKey, TValue}, Action{DeliveryReport{TKey, TValue}})" />
/// </summary>
public void Produce(
TopicPartition topicPartition,
Message<TKey, TValue> message,
Action<DeliveryReport<TKey, TValue>> deliveryHandler = null)
{
if (deliveryHandler != null && !enableDeliveryReports)
{
throw new InvalidOperationException("A delivery handler was specified, but delivery reports are disabled.");
}
byte[] keyBytes;
try
{
keyBytes = (keySerializer != null)
? keySerializer.Serialize(message.Key, new SerializationContext(MessageComponentType.Key, topicPartition.Topic))
: throw new InvalidOperationException("Produce called with an IAsyncSerializer key serializer configured but an ISerializer is required.");
}
catch (Exception ex)
{
throw new ProduceException<TKey, TValue>(
new Error(ErrorCode.Local_KeySerialization, ex.ToString()),
new DeliveryResult<TKey, TValue>
{
Message = message,
TopicPartitionOffset = new TopicPartitionOffset(topicPartition, Offset.Unset),
}
);
}
byte[] valBytes;
try
{
valBytes = (valueSerializer != null)
? valueSerializer.Serialize(message.Value, new SerializationContext(MessageComponentType.Value, topicPartition.Topic))
: throw new InvalidOperationException("Produce called with an IAsyncSerializer value serializer configured but an ISerializer is required.");
}
catch (Exception ex)
{
throw new ProduceException<TKey, TValue>(
new Error(ErrorCode.Local_ValueSerialization, ex.ToString()),
new DeliveryResult<TKey, TValue>
{
Message = message,
TopicPartitionOffset = new TopicPartitionOffset(topicPartition, Offset.Unset),
}
);
}
try
{
ProduceImpl(
topicPartition.Topic,
valBytes, 0, valBytes == null ? 0 : valBytes.Length,
keyBytes, 0, keyBytes == null ? 0 : keyBytes.Length,
message.Timestamp, topicPartition.Partition,
message.Headers,
new TypedDeliveryHandlerShim_Action<TKey, TValue>(
topicPartition.Topic,
enableDeliveryReportKey ? message.Key : default(TKey),
enableDeliveryReportValue ? message.Value : default(TValue),
deliveryHandler));
}
catch (KafkaException ex)
{
throw new ProduceException<TKey, TValue>(
ex.Error,
new DeliveryReport<TKey, TValue>
{
Message = message,
TopicPartitionOffset = new TopicPartitionOffset(topicPartition, Offset.Unset)
});
}
}
private class TypedTaskDeliveryHandlerShim<K, V> : TaskCompletionSource<DeliveryResult<K, V>>, IDeliveryHandler
{
public TypedTaskDeliveryHandlerShim(string topic, K key, V val)
#if !NET45
: base(TaskCreationOptions.RunContinuationsAsynchronously)
#endif
{
Topic = topic;
Key = key;
Value = val;
}
public string Topic;
public K Key;
public V Value;
public void HandleDeliveryReport(DeliveryReport<Null, Null> deliveryReport)
{
if (deliveryReport == null)
{
#if NET45
System.Threading.Tasks.Task.Run(() => TrySetResult(null));
#else
TrySetResult(null);
#endif
return;
}
var dr = new DeliveryResult<K, V>
{
TopicPartitionOffset = deliveryReport.TopicPartitionOffset,
Status = deliveryReport.Status,
Message = new Message<K, V>
{
Key = Key,
Value = Value,
Timestamp = deliveryReport.Message.Timestamp,
Headers = deliveryReport.Message.Headers
}
};
// topic is cached in this object, not set in the deliveryReport to avoid the
// cost of marshalling it.
dr.Topic = Topic;
#if NET45
if (deliveryReport.Error.IsError)
{
System.Threading.Tasks.Task.Run(() => SetException(new ProduceException<K, V>(deliveryReport.Error, dr)));
}
else
{
System.Threading.Tasks.Task.Run(() => TrySetResult(dr));
}
#else
if (deliveryReport.Error.IsError)
{
TrySetException(new ProduceException<K, V>(deliveryReport.Error, dr));
}
else
{
TrySetResult(dr);
}
#endif
}
}
private class TypedDeliveryHandlerShim_Action<K, V> : IDeliveryHandler
{
public TypedDeliveryHandlerShim_Action(string topic, K key, V val, Action<DeliveryReport<K, V>> handler)
{
Topic = topic;
Key = key;
Value = val;
Handler = handler;
}
public string Topic;
public K Key;
public V Value;
public Action<DeliveryReport<K, V>> Handler;
public void HandleDeliveryReport(DeliveryReport<Null, Null> deliveryReport)
{
if (deliveryReport == null)
{
return;
}
var dr = new DeliveryReport<K, V>
{
TopicPartitionOffsetError = deliveryReport.TopicPartitionOffsetError,
Status = deliveryReport.Status,
Message = new Message<K, V>
{
Key = Key,
Value = Value,
Timestamp = deliveryReport.Message == null
? new Timestamp(0, TimestampType.NotAvailable)
: deliveryReport.Message.Timestamp,
Headers = deliveryReport.Message?.Headers
}
};
// topic is cached in this object, not set in the deliveryReport to avoid the
// cost of marshalling it.
dr.Topic = Topic;
if (Handler != null)
{
Handler(dr);
}
}
}
}
}