48
48
import org .apache .kafka .common .errors .RecordTooLargeException ;
49
49
import org .apache .kafka .common .errors .TimeoutException ;
50
50
import org .apache .kafka .common .errors .UnknownTopicOrPartitionException ;
51
+ import org .apache .kafka .common .header .Headers ;
51
52
import org .apache .kafka .common .header .internals .RecordHeader ;
53
+ import org .apache .kafka .common .header .internals .RecordHeaders ;
52
54
import org .apache .kafka .common .internals .ClusterResourceListeners ;
53
55
import org .apache .kafka .common .message .AddOffsetsToTxnResponseData ;
54
56
import org .apache .kafka .common .message .ApiVersionsResponseData ;
@@ -1084,13 +1086,14 @@ public void testTopicExpiryInMetadata() throws InterruptedException {
1084
1086
1085
1087
@ SuppressWarnings ("unchecked" )
1086
1088
@ Test
1087
- public void testHeaders () {
1089
+ public void testHeadersSuccess () {
1088
1090
doTestHeaders (Serializer .class );
1089
1091
}
1090
1092
1091
1093
private <T extends Serializer <String >> void doTestHeaders (Class <T > serializerClassToMock ) {
1092
1094
Map <String , Object > configs = new HashMap <>();
1093
1095
configs .put (ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , "localhost:9999" );
1096
+ configs .put (ProducerConfig .INTERCEPTOR_CLASSES_CONFIG , ProducerInterceptorForHeaders .class .getName ());
1094
1097
Serializer <String > keySerializer = mock (serializerClassToMock );
1095
1098
Serializer <String > valueSerializer = mock (serializerClassToMock );
1096
1099
@@ -1119,7 +1122,9 @@ private <T extends Serializer<String>> void doTestHeaders(Class<T> serializerCla
1119
1122
producer .send (record , null );
1120
1123
1121
1124
//ensure headers are closed and cannot be mutated post send
1122
- assertThrows (IllegalStateException .class , () -> record .headers ().add (new RecordHeader ("test" , "test" .getBytes ())));
1125
+ RecordHeaders recordHeaders = (RecordHeaders ) record .headers ();
1126
+ assertTrue (recordHeaders .isReadOnly ());
1127
+ assertThrows (IllegalStateException .class , () -> recordHeaders .add (new RecordHeader ("test" , "test" .getBytes ())));
1123
1128
1124
1129
//ensure existing headers are not changed, and last header for key is still original value
1125
1130
assertArrayEquals (record .headers ().lastHeader ("test" ).value (), "header2" .getBytes ());
@@ -1130,6 +1135,28 @@ private <T extends Serializer<String>> void doTestHeaders(Class<T> serializerCla
1130
1135
producer .close (Duration .ofMillis (0 ));
1131
1136
}
1132
1137
1138
+ @ Test
1139
+ public void testHeadersFailure () {
1140
+ Properties props = new Properties ();
1141
+ props .put (ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , "localhost:9999" );
1142
+ props .put (ProducerConfig .MAX_BLOCK_MS_CONFIG , 5 );
1143
+ props .put (ProducerConfig .INTERCEPTOR_CLASSES_CONFIG , ProducerInterceptorForHeaders .class .getName ());
1144
+ Serializer <String > keySerializer = mock (StringSerializer .class );
1145
+ Serializer <String > valueSerializer = mock (StringSerializer .class );
1146
+
1147
+ KafkaProducer <String , String > producer = new KafkaProducer <>(props , keySerializer , valueSerializer );
1148
+ ProducerRecord <String , String > record = new ProducerRecord <>("topic" , "key" , "value" );
1149
+ Future <RecordMetadata > future = producer .send (record , (recordMetadata , exception ) -> { });
1150
+ try {
1151
+ TestUtils .assertFutureThrows (TimeoutException .class , future );
1152
+ //ensure headers are writable if send failure
1153
+ RecordHeaders recordHeaders = (RecordHeaders ) record .headers ();
1154
+ assertFalse (recordHeaders .isReadOnly ());
1155
+ } finally {
1156
+ producer .close (Duration .ofMillis (0 ));
1157
+ }
1158
+ }
1159
+
1133
1160
@ Test
1134
1161
public void closeShouldBeIdempotent () {
1135
1162
Properties producerProps = new Properties ();
@@ -2500,6 +2527,29 @@ public void configure(Map<String, ?> configs) {
2500
2527
}
2501
2528
}
2502
2529
2530
+ public static class ProducerInterceptorForHeaders implements ProducerInterceptor <byte [], byte []> {
2531
+
2532
+ @ Override
2533
+ public ProducerRecord <byte [], byte []> onSend (ProducerRecord <byte [], byte []> record ) {
2534
+ return record ;
2535
+ }
2536
+
2537
+ @ Override
2538
+ public void onAcknowledgement (RecordMetadata metadata , Exception exception , Headers headers ) {
2539
+ RecordHeaders recordHeaders = (RecordHeaders ) headers ;
2540
+ // Ensure that the headers are read-only, no matter send success or send failure
2541
+ assertTrue (recordHeaders .isReadOnly ());
2542
+ }
2543
+
2544
+ @ Override
2545
+ public void close () {
2546
+ }
2547
+
2548
+ @ Override
2549
+ public void configure (Map <String , ?> configs ) {
2550
+ }
2551
+ }
2552
+
2503
2553
public static class ProducerInterceptorForClientId implements ProducerInterceptor <byte [], byte []> {
2504
2554
2505
2555
@ Override
0 commit comments