1616
1717package com .tuplejump .kafka .connect .cassandra
1818
19- import java .util .{Collection => JCollection , Map => JMap , Date => JDate }
19+ import java .util .{Collection => JCollection , Date => JDate , Map => JMap }
2020
21- import org .apache .kafka .connect .connector .Task
22-
23- import scala .collection .JavaConverters ._
21+ import com .tuplejump .kafka .connect .cassandra .Configuration .Query
2422import org .apache .kafka .clients .consumer .OffsetAndMetadata
2523import 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
2925import 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 ._
3031
3132class CassandraSinkTask extends SinkTask with TaskLifecycle {
33+
3234 import CassandraSinkTask ._
3335
36+ private val FieldMappingParam = " field.mapping"
37+
3438 def taskClass : Class [_ <: Task ] = classOf [CassandraSinkTask ]
3539
3640 override def put (records : JCollection [SinkRecord ]): Unit =
3741 records.asScala.foreach { record =>
3842 configuration.find(record.topic) match {
3943 case Some (topic) =>
40- val query = convert(record, topic)
44+ val query : Query = convert(record, topic, configuration.config.get( FieldMappingParam ) )
4145 session.execute(query)
4246 case other =>
4347 throw new ConnectException (" Failed to get cassandra session." )
@@ -51,33 +55,107 @@ class CassandraSinkTask extends SinkTask with TaskLifecycle {
5155
5256/** INTERNAL API. */
5357private [kafka] object CassandraSinkTask {
58+
5459 import Configuration ._
5560
61+ import scala .util .parsing .json ._
62+
5663 /* TODO: Use keySchema, partition and kafkaOffset
5764 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 ]) = {
5979 val valueSchema = record.valueSchema
60- val columnNames = valueSchema.fields.asScala.map(_.name).toSet
61- val columnValues = valueSchema.`type`() match {
80+ valueSchema.`type`() match {
6281 case STRUCT =>
6382 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
6586 case other =>
6687 throw new DataException (
6788 s " Unable to create insert statement with unsupported value schema type $other. " )
6889 }
69- s " INSERT INTO ${sink.namespace}( ${columnNames.mkString(" ," )}) VALUES( $columnValues) "
7090 }
7191
7292 /* 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 {
7595 case x if x.`type`() == Schema .STRING_SCHEMA .`type`() =>
76- s " ' ${result.get(col ).toString}' "
96+ s " ' ${result.get(field ).toString}' "
7797 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 ])
7999 s " $time"
80100 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
82128 }
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+
83161}
0 commit comments