Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,8 @@ package object cassandra {
def convert(schema: Schema, result: Struct, col: String): AnyRef =
schema.field(col).schema match {
case x if x.`type`() == Schema.STRING_SCHEMA.`type`() =>
s"'${result.get(col).toString}'"
val fieldValue = result.get(col)
if(fieldValue != null) s"'${fieldValue.toString}'" else "null"
case x if x.name() == Timestamp.LOGICAL_NAME =>
val time = Timestamp.fromLogical(x, result.get(col).asInstanceOf[JDate])
s"$time"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,31 @@ class SchemaSpec extends AbstractFlatSpec {
query.cql should be("INSERT INTO keyspacex.tablex(available,name,age) VALUES(false,'user',15)")
}

it should "convert a struct schema with multiple fields including nullable string field" in {
val topic = "test_kfk"
val sc = sinkConfig(topic, "keyspacex", "tablex", List("available", "name", "type", "age"))

val nullableStringSchema = SchemaBuilder.string().optional().build()

val schema = SchemaBuilder.struct.name("record").version(1)
.field("available", Schema.BOOLEAN_SCHEMA)
.field("name", Schema.STRING_SCHEMA)
.field("type", nullableStringSchema)
.field("age", Schema.INT32_SCHEMA).build

val value = new Struct(schema).put("name", "user").put("available", false).put("age", 15).put("type", null)
val record = new SinkRecord("test_kfk", 1, SchemaBuilder.struct.build, "key", schema, value, 0)

schema.asColumnNames should be (sc.schema.columnNames)

sc.schema.route.topic should be (record.topic)
sc.schema is record should be (true)

sc.query.cql should be ("INSERT INTO keyspacex.tablex(available,name,type,age) VALUES(?,?,?,?)")
val query = record.as(sc.schema.namespace)
query.cql should be("INSERT INTO keyspacex.tablex(available,name,type,age) VALUES(false,'user',null,15)")
}

it should "convert cassandra column defs to a source schema" in {
val colDef = Map(
"id" -> DataType.cint(),
Expand Down