diff --git a/kafka-connect-hbase/src/main/scala/com/datamountaineer/streamreactor/connect/hbase/StructFieldsExtractorBytes.scala b/kafka-connect-hbase/src/main/scala/com/datamountaineer/streamreactor/connect/hbase/StructFieldsExtractorBytes.scala index 30dae7866..d44311dae 100755 --- a/kafka-connect-hbase/src/main/scala/com/datamountaineer/streamreactor/connect/hbase/StructFieldsExtractorBytes.scala +++ b/kafka-connect-hbase/src/main/scala/com/datamountaineer/streamreactor/connect/hbase/StructFieldsExtractorBytes.scala @@ -58,22 +58,22 @@ case class StructFieldsExtractorBytes(includeAllFields: Boolean, fieldsAliasMap: .map { value => Option(field.schema().name()).collect { case Decimal.LOGICAL_NAME => - value match { - case java.math.BigDecimal => value.fromBigDecimal() - case arr: Array[Byte] => Decimal.toLogical(field.schema, arr).fromBigDecimal() + value.asInstanceOf[Any] match { + case _:java.math.BigDecimal => value.fromBigDecimal() + case arr: Array[Byte] => Decimal.toLogical(field.schema, arr).asInstanceOf[Any].fromBigDecimal() case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Time.LOGICAL_NAME => value.asInstanceOf[Any] match { - case i: Int => StructFieldsExtractorBytes.TimeFormat.format(Time.toLogical(field.schema, i)).fromString() - case d@java.util.Date => StructFieldsExtractorBytes.TimeFormat.format(d).fromString() + case i: Int => StructFieldsExtractorBytes.TimeFormat.format(Time.toLogical(field.schema, i)).asInstanceOf[Any].fromString() + case d:java.util.Date => StructFieldsExtractorBytes.TimeFormat.format(d).asInstanceOf[Any].fromString() case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Timestamp.LOGICAL_NAME => value.asInstanceOf[Any] match { - case d@java.util.Date => StructFieldsExtractorBytes.DateFormat.format(d).fromString() - case l: Long => StructFieldsExtractorBytes.DateFormat.format(Timestamp.toLogical(field.schema, l)).fromString() + case d:java.util.Date => StructFieldsExtractorBytes.DateFormat.format(d).asInstanceOf[Any].fromString() + case l: Long => StructFieldsExtractorBytes.DateFormat.format(Timestamp.toLogical(field.schema, l)).asInstanceOf[Any].fromString() case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } }.getOrElse { diff --git a/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/StructFieldsExtractor.scala b/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/StructFieldsExtractor.scala index b322cb85a..b02996b8f 100755 --- a/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/StructFieldsExtractor.scala +++ b/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/StructFieldsExtractor.scala @@ -82,21 +82,21 @@ case class StructFieldsExtractor(includeAllFields: Boolean, schema.name() match { case Decimal.LOGICAL_NAME => - value match { - case java.math.BigDecimal => value + value.asInstanceOf[Any] match { + case _:java.math.BigDecimal => value case arr: Array[Byte] => Decimal.toLogical(schema, arr) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Time.LOGICAL_NAME => value.asInstanceOf[Any] match { case i: Int => StructFieldsExtractor.TimeFormat.format(Time.toLogical(schema, i)) - case d@java.util.Date => StructFieldsExtractor.TimeFormat.format(d) + case d:java.util.Date => StructFieldsExtractor.TimeFormat.format(d) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Timestamp.LOGICAL_NAME => value.asInstanceOf[Any] match { - case d@java.util.Date => StructFieldsExtractor.DateFormat.format(d) + case d:java.util.Date => StructFieldsExtractor.DateFormat.format(d) case l: Long => StructFieldsExtractor.DateFormat.format(Timestamp.toLogical(schema, l)) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } diff --git a/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/writers/TagsExtractor.scala b/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/writers/TagsExtractor.scala index d2306502f..072801d51 100644 --- a/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/writers/TagsExtractor.scala +++ b/kafka-connect-influxdb/src/main/scala/com/datamountaineer/streamreactor/connect/influx/writers/TagsExtractor.scala @@ -100,21 +100,21 @@ object TagsExtractor extends StrictLogging { Option(struct.get(field)).map { value => schema.name() match { case Decimal.LOGICAL_NAME => - value match { - case java.math.BigDecimal => value + value.asInstanceOf[Any] match { + case _:java.math.BigDecimal => value case arr: Array[Byte] => Decimal.toLogical(schema, arr) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Time.LOGICAL_NAME => value.asInstanceOf[Any] match { case i: Int => StructFieldsExtractor.TimeFormat.format(Time.toLogical(schema, i)) - case d@java.util.Date => StructFieldsExtractor.TimeFormat.format(d) + case d:java.util.Date => StructFieldsExtractor.TimeFormat.format(d) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Timestamp.LOGICAL_NAME => value.asInstanceOf[Any] match { - case d@java.util.Date => StructFieldsExtractor.DateFormat.format(d) + case d:java.util.Date => StructFieldsExtractor.DateFormat.format(d) case l: Long => StructFieldsExtractor.DateFormat.format(Timestamp.toLogical(schema, l)) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } diff --git a/kafka-connect-voltdb/src/main/scala/com/datamountaineer/streamreactor/connect/voltdb/StructFieldsExtractor.scala b/kafka-connect-voltdb/src/main/scala/com/datamountaineer/streamreactor/connect/voltdb/StructFieldsExtractor.scala index 00ddc6f4b..e8f138efe 100755 --- a/kafka-connect-voltdb/src/main/scala/com/datamountaineer/streamreactor/connect/voltdb/StructFieldsExtractor.scala +++ b/kafka-connect-voltdb/src/main/scala/com/datamountaineer/streamreactor/connect/voltdb/StructFieldsExtractor.scala @@ -59,21 +59,21 @@ case class StructFieldsExtractor(targetTable: String, //handle specific schema schema.name() match { case Decimal.LOGICAL_NAME => - value match { - case java.math.BigDecimal => value + value.asInstanceOf[Any] match { + case _:java.math.BigDecimal => value case arr: Array[Byte] => Decimal.toLogical(schema, arr) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Time.LOGICAL_NAME => value.asInstanceOf[Any] match { case i: Int => StructFieldsExtractor.TimeFormat.format(Time.toLogical(schema, i)) - case d@java.util.Date => StructFieldsExtractor.TimeFormat.format(d) + case d:java.util.Date => StructFieldsExtractor.TimeFormat.format(d) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") } case Timestamp.LOGICAL_NAME => value.asInstanceOf[Any] match { - case d@java.util.Date => StructFieldsExtractor.DateFormat.format(d) + case d:java.util.Date => StructFieldsExtractor.DateFormat.format(d) case l: Long => StructFieldsExtractor.DateFormat.format(Timestamp.toLogical(schema, l)) case _ => throw new IllegalArgumentException(s"${field.name()} is not handled for value:$value") }