16
16
17
17
package com .tuplejump .kafka .connect .cassandra
18
18
19
- import java .util .{Collection => JCollection , Map => JMap , Date => JDate }
19
+ import java .util .{Collection => JCollection , Date => JDate , Map => JMap }
20
20
21
- import org .apache .kafka .connect .connector .Task
22
-
23
- import scala .collection .JavaConverters ._
21
+ import com .tuplejump .kafka .connect .cassandra .Configuration .Query
24
22
import org .apache .kafka .clients .consumer .OffsetAndMetadata
25
23
import org .apache .kafka .common .TopicPartition
26
- import org .apache .kafka .connect .sink .{SinkRecord , SinkTask }
27
- import org .apache .kafka .connect .errors .{ConnectException , DataException }
28
- import org .apache .kafka .connect .data .{Schema , Struct , Timestamp }
24
+ import org .apache .kafka .connect .connector .Task
29
25
import org .apache .kafka .connect .data .Schema .Type ._
26
+ import org .apache .kafka .connect .data .{Field , Schema , Struct , Timestamp }
27
+ import org .apache .kafka .connect .errors .{ConnectException , DataException }
28
+ import org .apache .kafka .connect .sink .{SinkRecord , SinkTask }
29
+
30
+ import scala .collection .JavaConverters ._
30
31
31
32
class CassandraSinkTask extends SinkTask with TaskLifecycle {
33
+
32
34
import CassandraSinkTask ._
33
35
36
+ private val FieldMappingParam = " field.mapping"
37
+
34
38
def taskClass : Class [_ <: Task ] = classOf [CassandraSinkTask ]
35
39
36
40
override def put (records : JCollection [SinkRecord ]): Unit =
37
41
records.asScala.foreach { record =>
38
42
configuration.find(record.topic) match {
39
43
case Some (topic) =>
40
- val query = convert(record, topic)
44
+ val query : Query = convert(record, topic, configuration.config.get( FieldMappingParam ) )
41
45
session.execute(query)
42
46
case other =>
43
47
throw new ConnectException (" Failed to get cassandra session." )
@@ -51,33 +55,107 @@ class CassandraSinkTask extends SinkTask with TaskLifecycle {
51
55
52
56
/** INTERNAL API. */
53
57
private [kafka] object CassandraSinkTask {
58
+
54
59
import Configuration ._
55
60
61
+ import scala .util .parsing .json ._
62
+
56
63
/* TODO: Use keySchema, partition and kafkaOffset
57
64
TODO: Add which types are currently supported in README */
58
- def convert (record : SinkRecord , sink : SinkConfig ): Query = {
65
+ def convert (record : SinkRecord , sink : SinkConfig , fieldMappingProperty : Option [String ] = None ): Query = {
66
+ val colNamesVsValues : Map [String , String ] = {
67
+ fieldMappingProperty match {
68
+ case Some (fieldMappingString) => convertToCqlData(record, fieldMappingString)
69
+ case None => convertToCqlData(record)
70
+ }
71
+ }
72
+ colNamesVsValues.view.map(e => Vector (e._1, e._2)).transpose match {
73
+ case columnNames :: columnValues :: Nil =>
74
+ s " INSERT INTO ${sink.namespace}( ${columnNames.mkString(" ," )}) VALUES( ${columnValues.mkString(" ," )}) "
75
+ }
76
+ }
77
+
78
+ def convertToCqlData (record : SinkRecord ): (Map [String , String ]) = {
59
79
val valueSchema = record.valueSchema
60
- val columnNames = valueSchema.fields.asScala.map(_.name).toSet
61
- val columnValues = valueSchema.`type`() match {
80
+ valueSchema.`type`() match {
62
81
case STRUCT =>
63
82
val struct : Struct = record.value.asInstanceOf [Struct ]
64
- columnNames.map(schema(valueSchema, struct, _)).mkString(" ," )
83
+ valueSchema.fields.asScala.map { field =>
84
+ (field.name, schema(valueSchema, struct, field))
85
+ }.toMap
65
86
case other =>
66
87
throw new DataException (
67
88
s " Unable to create insert statement with unsupported value schema type $other. " )
68
89
}
69
- s " INSERT INTO ${sink.namespace}( ${columnNames.mkString(" ," )}) VALUES( $columnValues) "
70
90
}
71
91
72
92
/* TODO support all types. */
73
- def schema (valueSchema : Schema , result : Struct , col : String ): AnyRef =
74
- valueSchema. field(col) .schema match {
93
+ def schema (valueSchema : Schema , result : Struct , field : Field ): String = {
94
+ field.schema match {
75
95
case x if x.`type`() == Schema .STRING_SCHEMA .`type`() =>
76
- s " ' ${result.get(col ).toString}' "
96
+ s " ' ${result.get(field ).toString}' "
77
97
case x if x.name() == Timestamp .LOGICAL_NAME =>
78
- val time = Timestamp .fromLogical(x, result.get(col ).asInstanceOf [JDate ])
98
+ val time = Timestamp .fromLogical(x, result.get(field ).asInstanceOf [JDate ])
79
99
s " $time"
80
100
case y =>
81
- result.get(col)
101
+ String .valueOf(result.get(field))
102
+ }
103
+ }
104
+
105
+
106
+ // scalastyle:off
107
+ def convertToCqlData (record : SinkRecord , fieldMappingString : String ): Map [String , String ] = {
108
+ lazy val exception = new DataException (s " Invalid fieldMapping received - $fieldMappingString" )
109
+ val result = scala.collection.mutable.Map .empty[String , String ]
110
+ JSON .parseFull(fieldMappingString) match {
111
+ case Some (data) =>
112
+ data match {
113
+ case map : Map [_, _] =>
114
+ val valueSchema = record.valueSchema
115
+ valueSchema.`type`() match {
116
+ case STRUCT =>
117
+ val struct : Struct = record.value.asInstanceOf [Struct ]
118
+ populateResultsMap(result, struct, map.asInstanceOf [Map [String , Any ]])
119
+ case other =>
120
+ throw new DataException (
121
+ s " Unable to create insert statement with unsupported value schema type $other. " )
122
+ }
123
+ case other =>
124
+ throw exception
125
+ }
126
+ case None =>
127
+ throw exception
82
128
}
129
+ result.toMap
130
+ }
131
+
132
+ private def populateResultsMap (results : scala.collection.mutable.Map [String , String ], struct : Struct , fieldMapping : Map [String , Any ]): Unit = {
133
+ lazy val exception = new DataException (s " Mismatch between fieldMapping and Schema " )
134
+ struct.schema.fields.asScala.foreach { field =>
135
+ val fieldMappingValue = fieldMapping.get(field.name)
136
+ field.schema.`type`() match {
137
+ case STRUCT =>
138
+ fieldMappingValue match {
139
+ case Some (value) =>
140
+ value match {
141
+ case newMap : Map [_, _] => populateResultsMap(results, struct.get(field).asInstanceOf [Struct ], newMap.asInstanceOf [Map [String , Any ]])
142
+ case _ => throw exception
143
+ }
144
+ case None =>
145
+ }
146
+ case _ =>
147
+ fieldMappingValue match {
148
+ case Some (value) =>
149
+ value match {
150
+ case strValue : String => results.put(strValue, schema(field.schema, struct, field))
151
+ case _ => throw exception
152
+ }
153
+ case None =>
154
+ }
155
+ }
156
+ }
157
+ }
158
+
159
+ // scalastyle:on
160
+
83
161
}
0 commit comments