26
26
27
27
import com .google .cloud .bigquery .Field ;
28
28
import com .google .cloud .bigquery .LegacySQLTypeName ;
29
+ import java .lang .invoke .MethodHandles ;
30
+ import java .lang .invoke .MethodType ;
29
31
import java .util .HashMap ;
30
32
import java .util .Map ;
31
33
import org .apache .kafka .connect .sink .SinkRecord ;
34
+ import org .slf4j .Logger ;
35
+ import org .slf4j .LoggerFactory ;
32
36
33
37
/**
34
38
* Helper class to construct schema and record for Kafka Data Field.
35
39
*/
36
40
public class KafkaDataBuilder {
37
41
42
+ private static final Logger logger = LoggerFactory .getLogger (KafkaDataBuilder .class );
43
+
38
44
public static final String KAFKA_DATA_TOPIC_FIELD_NAME = "topic" ;
39
45
public static final String KAFKA_DATA_PARTITION_FIELD_NAME = "partition" ;
40
46
public static final String KAFKA_DATA_OFFSET_FIELD_NAME = "offset" ;
41
47
public static final String KAFKA_DATA_INSERT_TIME_FIELD_NAME = "insertTime" ;
42
48
49
+ // This is a marker variable for methods necessary to keep original sink record metadata.
50
+ // These methods in SinkRecord class are available only since Kafka Connect API version 3.6.
51
+ private static final boolean KAFKA_CONNECT_API_POST_3_6 ;
52
+
53
+ static {
54
+ boolean kafkaConnectApiPost36 ;
55
+ try {
56
+ MethodHandles .lookup ().findVirtual (
57
+ SinkRecord .class ,
58
+ "originalTopic" ,
59
+ MethodType .methodType (String .class )
60
+ );
61
+ MethodHandles .lookup ().findVirtual (
62
+ SinkRecord .class ,
63
+ "originalKafkaPartition" ,
64
+ MethodType .methodType (Integer .class )
65
+ );
66
+ MethodHandles .lookup ().findVirtual (
67
+ SinkRecord .class ,
68
+ "originalKafkaOffset" ,
69
+ MethodType .methodType (long .class )
70
+ );
71
+ kafkaConnectApiPost36 = true ;
72
+ } catch (NoSuchMethodException | IllegalAccessException e ) {
73
+ logger .warn ("This connector cannot retain original topic/partition/offset fields in SinkRecord. "
74
+ + "If these fields are mutated in upstream SMTs, they will be lost. "
75
+ + "Upgrade to Kafka Connect 3.6 to provision reliable metadata into resulting table." , e );
76
+ kafkaConnectApiPost36 = false ;
77
+ }
78
+ KAFKA_CONNECT_API_POST_3_6 = kafkaConnectApiPost36 ;
79
+ }
80
+
43
81
/**
44
82
* Construct schema for Kafka Data Field
45
83
*
@@ -60,6 +98,30 @@ public static Field buildKafkaDataField(String kafkaDataFieldName) {
60
98
.setMode (com .google .cloud .bigquery .Field .Mode .NULLABLE ).build ();
61
99
}
62
100
101
+ private static String tryGetOriginalTopic (SinkRecord kafkaConnectRecord ) {
102
+ if (KAFKA_CONNECT_API_POST_3_6 ) {
103
+ return kafkaConnectRecord .originalTopic ();
104
+ } else {
105
+ return kafkaConnectRecord .topic ();
106
+ }
107
+ }
108
+
109
+ private static Integer tryGetOriginalKafkaPartition (SinkRecord kafkaConnectRecord ) {
110
+ if (KAFKA_CONNECT_API_POST_3_6 ) {
111
+ return kafkaConnectRecord .originalKafkaPartition ();
112
+ } else {
113
+ return kafkaConnectRecord .kafkaPartition ();
114
+ }
115
+ }
116
+
117
+ private static long tryGetOriginalKafkaOffset (SinkRecord kafkaConnectRecord ) {
118
+ if (KAFKA_CONNECT_API_POST_3_6 ) {
119
+ return kafkaConnectRecord .originalKafkaOffset ();
120
+ } else {
121
+ return kafkaConnectRecord .kafkaOffset ();
122
+ }
123
+ }
124
+
63
125
/**
64
126
* Construct a map of Kafka Data record
65
127
*
@@ -68,9 +130,9 @@ public static Field buildKafkaDataField(String kafkaDataFieldName) {
68
130
*/
69
131
public static Map <String , Object > buildKafkaDataRecord (SinkRecord kafkaConnectRecord ) {
70
132
HashMap <String , Object > kafkaData = new HashMap <>();
71
- kafkaData .put (KAFKA_DATA_TOPIC_FIELD_NAME , kafkaConnectRecord . topic ( ));
72
- kafkaData .put (KAFKA_DATA_PARTITION_FIELD_NAME , kafkaConnectRecord . kafkaPartition ( ));
73
- kafkaData .put (KAFKA_DATA_OFFSET_FIELD_NAME , kafkaConnectRecord . kafkaOffset ( ));
133
+ kafkaData .put (KAFKA_DATA_TOPIC_FIELD_NAME , tryGetOriginalTopic ( kafkaConnectRecord ));
134
+ kafkaData .put (KAFKA_DATA_PARTITION_FIELD_NAME , tryGetOriginalKafkaPartition ( kafkaConnectRecord ));
135
+ kafkaData .put (KAFKA_DATA_OFFSET_FIELD_NAME , tryGetOriginalKafkaOffset ( kafkaConnectRecord ));
74
136
kafkaData .put (KAFKA_DATA_INSERT_TIME_FIELD_NAME , System .currentTimeMillis () / 1000.0 );
75
137
return kafkaData ;
76
138
}
0 commit comments