@ -65,17 +65,14 @@ public class RecordCollectorTest {
@@ -65,17 +65,14 @@ public class RecordCollectorTest {
) ;
private final Cluster cluster = new Cluster ( "cluster" , Collections . singletonList ( Node . noNode ( ) ) , infos ,
Collections . < String > emptySet ( ) , Collections . < String > emptySet ( ) ) ;
Collections . emptySet ( ) , Collections . emptySet ( ) ) ;
private final ByteArraySerializer byteArraySerializer = new ByteArraySerializer ( ) ;
private final StringSerializer stringSerializer = new StringSerializer ( ) ;
private final StreamPartitioner < String , Object > streamPartitioner = new StreamPartitioner < String , Object > ( ) {
@Override
public Integer partition ( final String topic , final String key , final Object value , final int numPartitions ) {
private final StreamPartitioner < String , Object > streamPartitioner = ( topic , key , value , numPartitions ) - > {
return Integer . parseInt ( key ) % numPartitions ;
}
} ;
@Test
@ -362,4 +359,55 @@ public class RecordCollectorTest {
@@ -362,4 +359,55 @@ public class RecordCollectorTest {
} ) ;
collector . send ( "topic1" , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
}
@Test
public void testRecordHeaderPassThroughSerializer ( ) {
final CustomStringSerializer keySerializer = new CustomStringSerializer ( ) ;
final CustomStringSerializer valueSerializer = new CustomStringSerializer ( ) ;
keySerializer . configure ( Collections . EMPTY_MAP , true ) ;
final RecordCollectorImpl collector = new RecordCollectorImpl (
"test" ,
logContext ,
new DefaultProductionExceptionHandler ( ) ,
new Metrics ( ) . sensor ( "skipped-records" )
) ;
final MockProducer < byte [ ] , byte [ ] > mockProducer = new MockProducer < > ( cluster , true , new DefaultPartitioner ( ) ,
byteArraySerializer , byteArraySerializer ) ;
collector . init ( mockProducer ) ;
collector . send ( "topic1" , "3" , "0" , new RecordHeaders ( ) , null , keySerializer , valueSerializer , streamPartitioner ) ;
final List < ProducerRecord < byte [ ] , byte [ ] > > recordHistory = mockProducer . history ( ) ;
for ( final ProducerRecord < byte [ ] , byte [ ] > sentRecord : recordHistory ) {
final Headers headers = sentRecord . headers ( ) ;
assertEquals ( 2 , headers . toArray ( ) . length ) ;
assertEquals ( new RecordHeader ( "key" , "key" . getBytes ( ) ) , headers . lastHeader ( "key" ) ) ;
assertEquals ( new RecordHeader ( "value" , "value" . getBytes ( ) ) , headers . lastHeader ( "value" ) ) ;
}
}
private static class CustomStringSerializer extends StringSerializer {
private boolean isKey ;
private CustomStringSerializer ( ) {
}
@Override
public void configure ( final Map < String , ? > configs , final boolean isKey ) {
this . isKey = isKey ;
super . configure ( configs , isKey ) ;
}
@Override
public byte [ ] serialize ( final String topic , final Headers headers , final String data ) {
if ( isKey ) {
headers . add ( new RecordHeader ( "key" , "key" . getBytes ( ) ) ) ;
} else {
headers . add ( new RecordHeader ( "value" , "value" . getBytes ( ) ) ) ;
}
return serialize ( topic , data ) ;
}
}
}