1 /***********************************************************************
2  * Copyright (c) 2013-2024 Commonwealth Computer Research, Inc.
3  * All rights reserved. This program and the accompanying materials
4  * are made available under the terms of the Apache License, Version 2.0
5  * which accompanies this distribution and is available at
6  * http://www.opensource.org/licenses/apache2.0.php.
7  ***********************************************************************/
8 
9 package org.locationtech.geomesa.kafka.confluent
10 
11 import com.typesafe.scalalogging.LazyLogging
12 import io.confluent.kafka.schemaregistry.client.{CachedSchemaRegistryClient, SchemaRegistryClient}
13 import io.confluent.kafka.serializers.{KafkaAvroDeserializer, KafkaAvroSerializer}
14 import org.apache.avro.Schema.{Field, Type}
15 import org.apache.avro.generic.{GenericData, GenericRecord}
16 import org.apache.avro.{JsonProperties, Schema}
17 import org.geotools.api.feature.simple.{SimpleFeature, SimpleFeatureType}
18 import org.locationtech.geomesa.features.SerializationOption.SerializationOption
19 import org.locationtech.geomesa.features.{ScalaSimpleFeature, SimpleFeatureSerializer}
20 import org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper
21 import org.locationtech.geomesa.kafka.data.KafkaDataStore
22 import org.locationtech.geomesa.security.SecurityUtils
23 import org.locationtech.geomesa.utils.text.{DateParsing, WKBUtils, WKTUtils}
24 import org.locationtech.jts.geom.Geometry
25 
26 import java.io.{InputStream, OutputStream}
27 import java.net.URL
28 import java.nio.ByteBuffer
29 import java.time.format.DateTimeFormatter
30 import java.util.Date
31 import java.util.concurrent.atomic.AtomicBoolean
32 import scala.collection.JavaConverters._
33 import scala.util.Try
34 import scala.util.control.NonFatal
35 
36 class ConfluentFeatureSerializer(
37     sft: SimpleFeatureType,
38     schemaRegistryClient: SchemaRegistryClient,
39     schemaOverride: Option[Schema] = None,
40     val options: Set[SerializationOption] = Set.empty
41   ) extends SimpleFeatureSerializer with LazyLogging {
42 
43   private val schema = schemaOverride.getOrElse {
44     val schemaId =
45       Option(sft.getUserData.get(ConfluentMetadata.SchemaIdKey))
46           .map(_.toString.toInt)
47           .getOrElse {
48             throw new IllegalStateException(s"Cannot create ConfluentFeatureSerializer because SimpleFeatureType " +
49                 s"'${sft.getTypeName}' does not have schema id at key '${ConfluentMetadata.SchemaIdKey}'")
50           }
51     schemaRegistryClient.getById(schemaId)
52   }
53 
54   private val schemaValidationCheck = new AtomicBoolean(false)
55 
56   private val serializers = new ThreadLocal[ConfluentFeatureMapper]() {
57     override def initialValue(): ConfluentFeatureMapper = {
58       val mapper = new ConfluentFeatureMapper(sft, schema, schemaRegistryClient)
59       if (schemaValidationCheck.compareAndSet(false, true)) {
60         val violations = mapper.checkSchemaViolations()
61         if (violations.nonEmpty) {
62           logger.warn(
63             "The following required schema fields are not mapped to any feature type attributes, " +
64                 s"and may cause errors during serialization: ${violations.mkString(", ")}")
65         }
66       }
67       mapper
68     }
69   }
70 
71   override def deserialize(id: String, bytes: Array[Byte]): SimpleFeature =
72     serializers.get.read(id, bytes)
73 
74   override def deserialize(bytes: Array[Byte]): SimpleFeature = deserialize("", bytes)
75 
76   override def deserialize(bytes: Array[Byte], offset: Int, length: Int): SimpleFeature =
77     deserialize("", bytes, offset, length)
78 
79   override def deserialize(id: String, bytes: Array[Byte], offset: Int, length: Int): SimpleFeature = {
80     val buf = if (offset == 0 && length == bytes.length) { bytes } else {
81       val buf = Array.ofDim[Byte](length)
82       System.arraycopy(bytes, offset, buf, 0, length)
83       buf
84     }
85     deserialize(id, buf)
86   }
87 
88   override def serialize(feature: SimpleFeature): Array[Byte] = serializers.get.write(feature)
89 
90   override def serialize(feature: SimpleFeature, out: OutputStream): Unit = out.write(serialize(feature))
91 
92   // implement the following if we need them
93 
94   override def deserialize(in: InputStream): SimpleFeature = throw new NotImplementedError()
95 
96   override def deserialize(id: String, in: InputStream): SimpleFeature =
97     throw new NotImplementedError()
98 }
99 
100 object ConfluentFeatureSerializer {
101 
102   import SchemaParser.{GeoMesaAvroDateFormat, GeoMesaAvroVisibilityField}
103 
104   def builder(sft: SimpleFeatureType, schemaRegistryUrl: URL, schemaOverride: Option[Schema] = None): Builder =
105     new Builder(sft, schemaRegistryUrl, schemaOverride)
106 
107   class Builder private[ConfluentFeatureSerializer](
108     sft: SimpleFeatureType,
109     schemaRegistryUrl: URL,
110     schemaOverride: Option[Schema] = None
111   ) extends SimpleFeatureSerializer.Builder[Builder] {
112     override def build(): ConfluentFeatureSerializer = {
113       val client = new CachedSchemaRegistryClient(schemaRegistryUrl.toExternalForm, 100)
114       new ConfluentFeatureSerializer(sft, client, schemaOverride, options.toSet)
115     }
116   }
117 
118   /**
119    * Mapping between Avro schema and SimpleFeatureType
120    *
121    * @param sftIndex index of the field in the sft
122    * @param schemaIndex index of the field in the avro schema
123    * @param default default value defined in the avro schema
124    * @param conversion convert from an avro value to a simple feature type attribute, and vice-versa
125    */
126   private case class FieldMapping(
127       sftIndex: Int,
128       schemaIndex: Int,
129       default: Option[AnyRef],
130       conversion: Option[FieldConverter]
131     )
132 
133   /**
134    * Converts between serialized Avro records and simple features
135    *
136    * @param sft simple feature type
137    * @param schema avro schema
138    * @param registry schema registry client
139    */
140   private class ConfluentFeatureMapper(sft: SimpleFeatureType, schema: Schema, registry: SchemaRegistryClient) {
141 
142     private val topic = KafkaDataStore.topic(sft)
143     private val kafkaSerializer = new KafkaAvroSerializer(registry)
144     private val kafkaDeserializer = new KafkaAvroDeserializer(registry)
145 
146     // feature type field index, schema field index and default value, any conversions necessary
147     private val fieldMappings = sft.getAttributeDescriptors.asScala.map { d =>
148       val field = schema.getField(d.getLocalName)
149 
150       val conversion =
151         if (classOf[Geometry].isAssignableFrom(d.getType.getBinding)) {
152           lazy val union = field.schema.getTypes.asScala.map(_.getType).filter(_ != Schema.Type.NULL).toSet
153           field.schema.getType match {
154             case Schema.Type.STRING => Some(WktConverter)
155             case Schema.Type.BYTES  => Some(WkbConverter)
156             case Schema.Type.UNION if union == Set(Schema.Type.STRING) => Some(WktConverter)
157             case Schema.Type.UNION if union == Set(Schema.Type.BYTES)  => Some(WkbConverter)
158             case _ => throw new IllegalStateException(s"Found a geometry field with an invalid schema: $field")
159           }
160         } else if (classOf[Date].isAssignableFrom(d.getType.getBinding)) {
161           d.getUserData.get(GeoMesaAvroDateFormat.KEY) match {
162             case GeoMesaAvroDateFormat.ISO_DATE     => Some(IsoDateConverter)
163             case GeoMesaAvroDateFormat.ISO_DATETIME => Some(IsoDateTimeConverter)
164             case GeoMesaAvroDateFormat.EPOCH_MILLIS => Some(EpochMillisConverter)
165             case null /* avro logical date type */  => Some(EpochMillisConverter)
166             case _ =>
167               throw new IllegalStateException(s"Found a date field with no format defined:" +
168                 s" ${d.getLocalName} ${d.getUserData.asScala.mkString(", ")}")
169           }
170         } else {
171           None
172         }
173 
174       FieldMapping(sft.indexOf(d.getLocalName), field.pos(), defaultValue(field), conversion)
175     }
176 
177     // visibility field index in the avro schema
178     private val visibilityField = schema.getFields.asScala.collectFirst {
179       case f if Option(f.getProp(GeoMesaAvroVisibilityField.KEY)).exists(_.toBoolean) => f.pos()
180     }
181 
182     // avro fields with default values that aren't part of the feature type
183     private val defaultFields = schema.getFields.asScala.flatMap(f => defaultValue(f).map(v => f.pos() -> v)).filter {
184       case (pos, _) => !fieldMappings.exists(_.schemaIndex == pos) && !visibilityField.contains(pos)
185     }
186 
187     /**
188      * Checks for required fields in the avro schema that are not part of the feature type
189      * (i.e. will never be written)
190      *
191      * @return list of fields that will cause schema validation errors during serialization
192      */
193     def checkSchemaViolations(): Seq[String] = {
194       val mappedPositions = fieldMappings.map(_.schemaIndex) ++ visibilityField.toSeq
195       schema.getFields.asScala.collect {
196         case f if requiredField(f) && !mappedPositions.contains(f.pos()) => f.name()
197       }.toSeq
198     }
199 
200     /**
201      * Serialize a feature as Avro
202      *
203      * @param feature feature to serialize
204      * @return
205      */
206     def write(feature: SimpleFeature): Array[Byte] = {
207       val record = new GenericData.Record(schema)
208       defaultFields.foreach { case (i, v) => record.put(i, v) }
209       visibilityField.foreach { pos => record.put(pos, SecurityUtils.getVisibility(feature)) }
210       fieldMappings.foreach { m =>
211         try {
212           feature.getAttribute(m.sftIndex) match {
213             case null => m.default.foreach(d => record.put(m.schemaIndex, d))
214             case v => record.put(m.schemaIndex, m.conversion.fold(v)(_.featureToRecord(v)))
215           }
216         } catch {
217           case NonFatal(e) =>
218             val d = sft.getDescriptor(m.sftIndex)
219             val v = Try(feature.getAttribute(m.sftIndex))
220             val s = schema.getField(d.getLocalName).schema()
221             throw new RuntimeException(
222               s"Cannot serialize field '${d.getLocalName}' with try-value '$v' into schema '$s':", e)
223         }
224       }
225 
226       kafkaSerializer.serialize(topic, record)
227     }
228 
229     /**
230      * Deserialize an Avro record into a feature
231      *
232      * @param id feature id
233      * @param bytes serialized avro bytes
234      * @return
235      */
236     def read(id: String, bytes: Array[Byte]): SimpleFeature = {
237       val record = kafkaDeserializer.deserialize(topic, bytes).asInstanceOf[GenericRecord]
238       val attributes = fieldMappings.map { m =>
239         try {
240           val v = record.get(m.schemaIndex)
241           m.conversion match {
242             case None => v
243             case Some(c) => c.recordToFeature(v)
244           }
245         } catch {
246           case NonFatal(e) =>
247             val d = sft.getDescriptor(m.sftIndex)
248             throw new RuntimeException(
249               s"Cannot deserialize field '${d.getLocalName}' into a '${d.getType.getBinding.getName}':", e)
250         }
251       }
252 
253       val feature = ScalaSimpleFeature.create(sft, id, attributes.toSeq: _*)
254 
255       // set the feature visibility if it exists
256       visibilityField.foreach { field =>
257         val vis = record.get(field)
258         if (vis != null) {
259           SecurityUtils.setFeatureVisibility(feature, vis.toString)
260         }
261       }
262 
263       feature
264     }
265 
266     // filter out JNull - bug in kafka avro deserialization https://issues.apache.org/jira/browse/AVRO-1954
267     private def defaultValue(f: Field): Option[AnyRef] =
268       Option(f.defaultVal()).filterNot(_.isInstanceOf[JsonProperties.Null])
269 
270     private def requiredField(f: Field): Boolean = {
271       defaultValue(f).isEmpty && {
272         f.schema().getType match {
273           case Type.NULL => false
274           case Type.UNION => !f.schema().getTypes.contains(Type.NULL)
275           case _ => true
276         }
277       }
278     }
279   }
280 
281   /**
282    * Converts between avro and feature attribute values
283    */
284   private sealed trait FieldConverter {
285     def recordToFeature(value: AnyRef): AnyRef
286     def featureToRecord(value: AnyRef): AnyRef
287   }
288 
289   /**
290    * Converts WKT text fields
291    */
292   private case object WktConverter extends FieldConverter {
293     override def recordToFeature(value: AnyRef): AnyRef = {
294       // note: value is an org.apache.avro.util.Utf8
295       if (value == null) { null } else { WKTUtils.read(value.toString) }
296     }
297 
298     override def featureToRecord(value: AnyRef): AnyRef =
299       if (value == null) { null } else { WKTUtils.write(value.asInstanceOf[Geometry]) }
300   }
301 
302   /**
303    * Converts WKB bytes fields
304    */
305   private case object WkbConverter extends FieldConverter {
306     override def recordToFeature(value: AnyRef): AnyRef =
307       if (value == null) { null } else { WKBUtils.read(unwrap(value.asInstanceOf[ByteBuffer])) }
308 
309     override def featureToRecord(value: AnyRef): AnyRef =
310       if (value == null) { null } else { ByteBuffer.wrap(WKBUtils.write(value.asInstanceOf[Geometry])) }
311 
312     private def unwrap(buf: ByteBuffer): Array[Byte] = {
313       if (buf.hasArray && buf.arrayOffset() == 0 && buf.limit() == buf.array().length) {
314         buf.array()
315       } else {
316         val array = Array.ofDim[Byte](buf.limit())
317         buf.get(array)
318         array
319       }
320     }
321   }
322 
323   /**
324    * Converts ISO_DATE formatted string fields
325    */
326   private case object IsoDateConverter extends FieldConverter {
327     override def recordToFeature(value: AnyRef): AnyRef = {
328       if (value == null) { null } else {
329         // note: value is an org.apache.avro.util.Utf8
330         DateParsing.parseDate(value.toString, DateTimeFormatter.ISO_DATE)
331       }
332     }
333 
334     override def featureToRecord(value: AnyRef): AnyRef = {
335       if (value == null) { null } else {
336         DateParsing.formatDate(value.asInstanceOf[Date], DateTimeFormatter.ISO_DATE)
337       }
338     }
339   }
340 
341   /**
342    * Converts ISO_DATE_TIME formatted string fields
343    */
344   private case object IsoDateTimeConverter extends FieldConverter {
345     override def recordToFeature(value: AnyRef): AnyRef = {
346       if (value == null) { null } else {
347         // note: value is an org.apache.avro.util.Utf8
348         DateParsing.parseDate(value.toString, DateTimeFormatter.ISO_DATE_TIME)
349       }
350     }
351 
352     override def featureToRecord(value: AnyRef): AnyRef = {
353       if (value == null) { null } else {
354         DateParsing.formatDate(value.asInstanceOf[Date], DateTimeFormatter.ISO_DATE_TIME)
355       }
356     }
357   }
358 
359   /**
360    * Converts milliseconds since epoch long fields
361    */
362   private case object EpochMillisConverter extends FieldConverter {
363     override def recordToFeature(value: AnyRef): AnyRef =
364       if (value == null) { null } else { new Date(value.asInstanceOf[java.lang.Long]) }
365 
366     override def featureToRecord(value: AnyRef): AnyRef =
367       if (value == null) { null } else { Long.box(value.asInstanceOf[Date].getTime) }
368   }
369 
370 }
Line Stmt Id Pos Tree Symbol Tests Code
43 8 2055 - 2504 Apply scala.Option.getOrElse ConfluentFeatureSerializer.this.schemaOverride.getOrElse[org.apache.avro.Schema]({ val schemaId: Int = scala.Option.apply[Object](ConfluentFeatureSerializer.this.sft.getUserData().get(ConfluentMetadata.SchemaIdKey)).map[Int](((x$1: Object) => scala.Predef.augmentString(x$1.toString()).toInt)).getOrElse[Int](throw new java.lang.IllegalStateException(scala.StringContext.apply("Cannot create ConfluentFeatureSerializer because SimpleFeatureType ").s().+(scala.StringContext.apply("\'", "\' does not have schema id at key \'", "\'").s(ConfluentFeatureSerializer.this.sft.getTypeName(), ConfluentMetadata.SchemaIdKey)))); ConfluentFeatureSerializer.this.schemaRegistryClient.getById(schemaId) })
45 1 2134 - 2163 Select org.locationtech.geomesa.kafka.confluent.ConfluentMetadata.SchemaIdKey ConfluentMetadata.SchemaIdKey
45 2 2114 - 2164 Apply java.util.Map.get ConfluentFeatureSerializer.this.sft.getUserData().get(ConfluentMetadata.SchemaIdKey)
46 3 2181 - 2191 Apply java.lang.Object.toString x$1.toString()
46 4 2181 - 2197 Select scala.collection.immutable.StringLike.toInt scala.Predef.augmentString(x$1.toString()).toInt
47 6 2107 - 2457 Apply scala.Option.getOrElse scala.Option.apply[Object](ConfluentFeatureSerializer.this.sft.getUserData().get(ConfluentMetadata.SchemaIdKey)).map[Int](((x$1: Object) => scala.Predef.augmentString(x$1.toString()).toInt)).getOrElse[Int](throw new java.lang.IllegalStateException(scala.StringContext.apply("Cannot create ConfluentFeatureSerializer because SimpleFeatureType ").s().+(scala.StringContext.apply("\'", "\' does not have schema id at key \'", "\'").s(ConfluentFeatureSerializer.this.sft.getTypeName(), ConfluentMetadata.SchemaIdKey))))
48 5 2234 - 2445 Throw <nosymbol> throw new java.lang.IllegalStateException(scala.StringContext.apply("Cannot create ConfluentFeatureSerializer because SimpleFeatureType ").s().+(scala.StringContext.apply("\'", "\' does not have schema id at key \'", "\'").s(ConfluentFeatureSerializer.this.sft.getTypeName(), ConfluentMetadata.SchemaIdKey)))
51 7 2462 - 2500 Apply io.confluent.kafka.schemaregistry.client.SchemaRegistryClient.getById ConfluentFeatureSerializer.this.schemaRegistryClient.getById(schemaId)
54 9 2544 - 2568 Apply java.util.concurrent.atomic.AtomicBoolean.<init> new java.util.concurrent.atomic.AtomicBoolean(false)
56 23 2598 - 2601 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.$anon.<init> new $anon()
58 10 2748 - 2751 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.sft ConfluentFeatureSerializer.this.sft
58 11 2753 - 2759 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.schema ConfluentFeatureSerializer.this.schema
58 12 2761 - 2781 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.schemaRegistryClient ConfluentFeatureSerializer.this.schemaRegistryClient
58 13 2721 - 2782 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.<init> new org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper(ConfluentFeatureSerializer.this.sft, ConfluentFeatureSerializer.this.schema, ConfluentFeatureSerializer.this.schemaRegistryClient)
59 14 2793 - 2841 Apply java.util.concurrent.atomic.AtomicBoolean.compareAndSet ConfluentFeatureSerializer.this.schemaValidationCheck.compareAndSet(false, true)
59 20 2843 - 3169 Block <nosymbol> { val violations: Seq[String] = mapper.checkSchemaViolations(); if (violations.nonEmpty) (if (ConfluentFeatureSerializer.this.logger.underlying.isWarnEnabled()) ConfluentFeatureSerializer.this.logger.underlying.warn("The following required schema fields are not mapped to any feature type attributes, ".+(scala.StringContext.apply("and may cause errors during serialization: ", "").s(violations.mkString(", ")))) else (): Unit) else () }
59 21 2789 - 2789 Literal <nosymbol> ()
59 22 2789 - 2789 Block <nosymbol> ()
60 15 2870 - 2900 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.checkSchemaViolations mapper.checkSchemaViolations()
61 16 2913 - 2932 Select scala.collection.TraversableOnce.nonEmpty violations.nonEmpty
61 18 2909 - 2909 Literal <nosymbol> ()
61 19 2909 - 2909 Block <nosymbol> ()
62 17 2946 - 3151 Typed <nosymbol> (if (ConfluentFeatureSerializer.this.logger.underlying.isWarnEnabled()) ConfluentFeatureSerializer.this.logger.underlying.warn("The following required schema fields are not mapped to any feature type attributes, ".+(scala.StringContext.apply("and may cause errors during serialization: ", "").s(violations.mkString(", ")))) else (): Unit)
72 24 3274 - 3305 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.read ConfluentFeatureSerializer.this.serializers.get().read(id, bytes)
74 25 3371 - 3393 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.deserialize ConfluentFeatureSerializer.this.deserialize("", bytes)
77 26 3489 - 3527 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.deserialize ConfluentFeatureSerializer.this.deserialize("", bytes, offset, length)
80 27 3661 - 3662 Literal <nosymbol> 0
80 28 3676 - 3688 Select scala.Array.length bytes.length
80 29 3666 - 3688 Apply scala.Int.== length.==(bytes.length)
80 30 3651 - 3688 Apply scala.Boolean.&& offset.==(0).&&(length.==(bytes.length))
80 31 3692 - 3697 Ident org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.bytes bytes
80 34 3705 - 3818 Block <nosymbol> { val buf: Array[Byte] = scala.Array.ofDim[Byte](length)((ClassTag.Byte: scala.reflect.ClassTag[Byte])); java.lang.System.arraycopy(bytes, offset, buf, 0, length); buf }
81 32 3723 - 3748 ApplyToImplicitArgs scala.Array.ofDim scala.Array.ofDim[Byte](length)((ClassTag.Byte: scala.reflect.ClassTag[Byte]))
82 33 3755 - 3802 Apply java.lang.System.arraycopy java.lang.System.arraycopy(bytes, offset, buf, 0, length)
85 35 3823 - 3843 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.deserialize ConfluentFeatureSerializer.this.deserialize(id, buf)
88 36 3913 - 3943 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.write ConfluentFeatureSerializer.this.serializers.get().write(feature)
90 37 4031 - 4049 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.serialize ConfluentFeatureSerializer.this.serialize(feature)
90 38 4021 - 4050 Apply java.io.OutputStream.write out.write(ConfluentFeatureSerializer.this.serialize(feature))
94 39 4159 - 4190 Throw <nosymbol> throw new scala.NotImplementedError()
97 40 4269 - 4300 Throw <nosymbol> throw new scala.NotImplementedError()
105 41 4532 - 4583 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.Builder.<init> new ConfluentFeatureSerializer.this.Builder(sft, schemaRegistryUrl, schemaOverride)
113 42 4898 - 4930 Apply java.net.URL.toExternalForm Builder.this.schemaRegistryUrl.toExternalForm()
113 43 4932 - 4935 Literal <nosymbol> 100
113 44 4867 - 4936 Apply io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient.<init> new io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient(Builder.this.schemaRegistryUrl.toExternalForm(), 100)
114 45 4974 - 4977 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.Builder.sft Builder.this.sft
114 46 4987 - 5001 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.Builder.schemaOverride Builder.this.schemaOverride
114 47 5003 - 5016 TypeApply scala.collection.TraversableOnce.toSet Builder.this.options.toSet[org.locationtech.geomesa.features.SerializationOption.SerializationOption]
114 48 4943 - 5017 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.<init> new ConfluentFeatureSerializer(Builder.this.sft, client, Builder.this.schemaOverride, Builder.this.options.toSet[org.locationtech.geomesa.features.SerializationOption.SerializationOption])
142 49 5888 - 5891 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.sft ConfluentFeatureMapper.this.sft
142 50 5867 - 5892 Apply org.locationtech.geomesa.kafka.data.KafkaDataStore.topic org.locationtech.geomesa.kafka.data.KafkaDataStore.topic(ConfluentFeatureMapper.this.sft)
143 51 5951 - 5959 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.registry ConfluentFeatureMapper.this.registry
143 52 5927 - 5960 Apply io.confluent.kafka.serializers.KafkaAvroSerializer.<init> new io.confluent.kafka.serializers.KafkaAvroSerializer(ConfluentFeatureMapper.this.registry)
144 53 6023 - 6031 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.registry ConfluentFeatureMapper.this.registry
144 54 5997 - 6032 Apply io.confluent.kafka.serializers.KafkaAvroDeserializer.<init> new io.confluent.kafka.serializers.KafkaAvroDeserializer(ConfluentFeatureMapper.this.registry)
147 55 6163 - 6190 Apply org.geotools.api.feature.simple.SimpleFeatureType.getAttributeDescriptors ConfluentFeatureMapper.this.sft.getAttributeDescriptors()
147 109 6203 - 6203 TypeApply scala.collection.mutable.Buffer.canBuildFrom mutable.this.Buffer.canBuildFrom[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping]
147 110 6163 - 7740 ApplyToImplicitArgs scala.collection.TraversableLike.map scala.collection.JavaConverters.asScalaBufferConverter[org.geotools.api.feature.type.AttributeDescriptor](ConfluentFeatureMapper.this.sft.getAttributeDescriptors()).asScala.map[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping, scala.collection.mutable.Buffer[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping]](((d: org.geotools.api.feature.type.AttributeDescriptor) => { val field: org.apache.avro.Schema.Field = ConfluentFeatureMapper.this.schema.getField(d.getLocalName()); val conversion: Option[Product with Serializable with org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter] = if (classOf[org.locationtech.jts.geom.Geometry].isAssignableFrom(d.getType().getBinding())) { <stable> <accessor> lazy val union: scala.collection.immutable.Set[org.apache.avro.Schema.Type] = scala.collection.JavaConverters.asScalaBufferConverter[org.apache.avro.Schema](field.schema().getTypes()).asScala.map[org.apache.avro.Schema.Type, scala.collection.mutable.Buffer[org.apache.avro.Schema.Type]](((x$2: org.apache.avro.Schema) => x$2.getType()))(mutable.this.Buffer.canBuildFrom[org.apache.avro.Schema.Type]).filter(((x$3: org.apache.avro.Schema.Type) => x$3.!=(NULL))).toSet[org.apache.avro.Schema.Type]; field.schema().getType() match { case STRING => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter) case BYTES => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter) case UNION if union.==(scala.Predef.Set.apply[org.apache.avro.Schema.Type](STRING)) => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter) case UNION if union.==(scala.Predef.Set.apply[org.apache.avro.Schema.Type](BYTES)) => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter) case _ => throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a geometry field with an invalid schema: ", "").s(field)) } } else if (classOf[java.util.Date].isAssignableFrom(d.getType().getBinding())) d.getUserData().get(SchemaParser.GeoMesaAvroDateFormat.KEY) match { case SchemaParser.GeoMesaAvroDateFormat.ISO_DATE => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateConverter.type](ConfluentFeatureSerializer.this.IsoDateConverter) case SchemaParser.GeoMesaAvroDateFormat.ISO_DATETIME => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateTimeConverter.type](ConfluentFeatureSerializer.this.IsoDateTimeConverter) case SchemaParser.GeoMesaAvroDateFormat.EPOCH_MILLIS => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter) case null => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter) case _ => throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a date field with no format defined:").s().+(scala.StringContext.apply(" ", " ", "").s(d.getLocalName(), scala.collection.JavaConverters.mapAsScalaMapConverter[Object, Object](d.getUserData()).asScala.mkString(", ")))) } else scala.None; ConfluentFeatureSerializer.this.FieldMapping.apply(ConfluentFeatureMapper.this.sft.indexOf(d.getLocalName()), field.pos(), ConfluentFeatureMapper.this.defaultValue(field), conversion) }))(mutable.this.Buffer.canBuildFrom[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping])
148 56 6244 - 6258 Apply org.geotools.api.feature.type.AttributeDescriptor.getLocalName d.getLocalName()
148 57 6228 - 6259 Apply org.apache.avro.Schema.getField ConfluentFeatureMapper.this.schema.getField(d.getLocalName())
151 58 6296 - 6313 Literal <nosymbol> classOf[org.locationtech.jts.geom.Geometry]
151 59 6331 - 6351 Apply org.geotools.api.feature.type.PropertyType.getBinding d.getType().getBinding()
151 60 6296 - 6352 Apply java.lang.Class.isAssignableFrom classOf[org.locationtech.jts.geom.Geometry].isAssignableFrom(d.getType().getBinding())
151 80 6354 - 6938 Block <nosymbol> { <stable> <accessor> lazy val union: scala.collection.immutable.Set[org.apache.avro.Schema.Type] = scala.collection.JavaConverters.asScalaBufferConverter[org.apache.avro.Schema](field.schema().getTypes()).asScala.map[org.apache.avro.Schema.Type, scala.collection.mutable.Buffer[org.apache.avro.Schema.Type]](((x$2: org.apache.avro.Schema) => x$2.getType()))(mutable.this.Buffer.canBuildFrom[org.apache.avro.Schema.Type]).filter(((x$3: org.apache.avro.Schema.Type) => x$3.!=(NULL))).toSet[org.apache.avro.Schema.Type]; field.schema().getType() match { case STRING => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter) case BYTES => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter) case UNION if union.==(scala.Predef.Set.apply[org.apache.avro.Schema.Type](STRING)) => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter) case UNION if union.==(scala.Predef.Set.apply[org.apache.avro.Schema.Type](BYTES)) => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter) case _ => throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a geometry field with an invalid schema: ", "").s(field)) } }
153 61 6474 - 6494 Apply org.apache.avro.Schema.getType field.schema().getType()
154 62 6547 - 6559 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter ConfluentFeatureSerializer.this.WktConverter
154 63 6542 - 6560 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter)
154 64 6542 - 6560 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter)
155 65 6605 - 6617 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter ConfluentFeatureSerializer.this.WkbConverter
155 66 6600 - 6618 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter)
155 67 6600 - 6618 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter)
156 68 6666 - 6689 Apply scala.collection.generic.GenericCompanion.apply scala.Predef.Set.apply[org.apache.avro.Schema.Type](STRING)
156 69 6657 - 6689 Apply java.lang.Object.== union.==(scala.Predef.Set.apply[org.apache.avro.Schema.Type](STRING))
156 70 6698 - 6710 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter ConfluentFeatureSerializer.this.WktConverter
156 71 6693 - 6711 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter)
156 72 6693 - 6711 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WktConverter.type](ConfluentFeatureSerializer.this.WktConverter)
157 73 6759 - 6781 Apply scala.collection.generic.GenericCompanion.apply scala.Predef.Set.apply[org.apache.avro.Schema.Type](BYTES)
157 74 6750 - 6781 Apply java.lang.Object.== union.==(scala.Predef.Set.apply[org.apache.avro.Schema.Type](BYTES))
157 75 6791 - 6803 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter ConfluentFeatureSerializer.this.WkbConverter
157 76 6786 - 6804 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter)
157 77 6786 - 6804 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.type](ConfluentFeatureSerializer.this.WkbConverter)
158 78 6827 - 6916 Throw <nosymbol> throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a geometry field with an invalid schema: ", "").s(field))
158 79 6827 - 6916 Block <nosymbol> throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a geometry field with an invalid schema: ", "").s(field))
160 81 6948 - 6961 Literal <nosymbol> classOf[java.util.Date]
160 82 6979 - 6999 Apply org.geotools.api.feature.type.PropertyType.getBinding d.getType().getBinding()
160 83 6948 - 7000 Apply java.lang.Class.isAssignableFrom classOf[java.util.Date].isAssignableFrom(d.getType().getBinding())
160 103 6944 - 7639 If <nosymbol> if (classOf[java.util.Date].isAssignableFrom(d.getType().getBinding())) d.getUserData().get(SchemaParser.GeoMesaAvroDateFormat.KEY) match { case SchemaParser.GeoMesaAvroDateFormat.ISO_DATE => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateConverter.type](ConfluentFeatureSerializer.this.IsoDateConverter) case SchemaParser.GeoMesaAvroDateFormat.ISO_DATETIME => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateTimeConverter.type](ConfluentFeatureSerializer.this.IsoDateTimeConverter) case SchemaParser.GeoMesaAvroDateFormat.EPOCH_MILLIS => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter) case null => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter) case _ => throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a date field with no format defined:").s().+(scala.StringContext.apply(" ", " ", "").s(d.getLocalName(), scala.collection.JavaConverters.mapAsScalaMapConverter[Object, Object](d.getUserData()).asScala.mkString(", ")))) } else scala.None
161 84 7032 - 7057 Select org.locationtech.geomesa.kafka.confluent.SchemaParser.GeoMesaAvroDateFormat.KEY SchemaParser.GeoMesaAvroDateFormat.KEY
161 85 7014 - 7058 Apply java.util.Map.get d.getUserData().get(SchemaParser.GeoMesaAvroDateFormat.KEY)
161 100 7014 - 7597 Match <nosymbol> d.getUserData().get(SchemaParser.GeoMesaAvroDateFormat.KEY) match { case SchemaParser.GeoMesaAvroDateFormat.ISO_DATE => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateConverter.type](ConfluentFeatureSerializer.this.IsoDateConverter) case SchemaParser.GeoMesaAvroDateFormat.ISO_DATETIME => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateTimeConverter.type](ConfluentFeatureSerializer.this.IsoDateTimeConverter) case SchemaParser.GeoMesaAvroDateFormat.EPOCH_MILLIS => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter) case null => scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter) case _ => throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a date field with no format defined:").s().+(scala.StringContext.apply(" ", " ", "").s(d.getLocalName(), scala.collection.JavaConverters.mapAsScalaMapConverter[Object, Object](d.getUserData()).asScala.mkString(", ")))) }
162 86 7127 - 7143 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateConverter ConfluentFeatureSerializer.this.IsoDateConverter
162 87 7122 - 7144 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateConverter.type](ConfluentFeatureSerializer.this.IsoDateConverter)
162 88 7122 - 7144 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateConverter.type](ConfluentFeatureSerializer.this.IsoDateConverter)
163 89 7205 - 7225 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateTimeConverter ConfluentFeatureSerializer.this.IsoDateTimeConverter
163 90 7200 - 7226 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateTimeConverter.type](ConfluentFeatureSerializer.this.IsoDateTimeConverter)
163 91 7200 - 7226 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.IsoDateTimeConverter.type](ConfluentFeatureSerializer.this.IsoDateTimeConverter)
164 92 7287 - 7307 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter ConfluentFeatureSerializer.this.EpochMillisConverter
164 93 7282 - 7308 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter)
164 94 7282 - 7308 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter)
165 95 7369 - 7389 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter ConfluentFeatureSerializer.this.EpochMillisConverter
165 96 7364 - 7390 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter)
165 97 7364 - 7390 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.EpochMillisConverter.type](ConfluentFeatureSerializer.this.EpochMillisConverter)
167 98 7427 - 7585 Throw <nosymbol> throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a date field with no format defined:").s().+(scala.StringContext.apply(" ", " ", "").s(d.getLocalName(), scala.collection.JavaConverters.mapAsScalaMapConverter[Object, Object](d.getUserData()).asScala.mkString(", "))))
167 99 7427 - 7585 Block <nosymbol> throw new java.lang.IllegalStateException(scala.StringContext.apply("Found a date field with no format defined:").s().+(scala.StringContext.apply(" ", " ", "").s(d.getLocalName(), scala.collection.JavaConverters.mapAsScalaMapConverter[Object, Object](d.getUserData()).asScala.mkString(", "))))
171 101 7625 - 7629 Select scala.None scala.None
171 102 7625 - 7629 Block scala.None scala.None
174 104 7672 - 7686 Apply org.geotools.api.feature.type.AttributeDescriptor.getLocalName d.getLocalName()
174 105 7660 - 7687 Apply org.geotools.api.feature.simple.SimpleFeatureType.indexOf ConfluentFeatureMapper.this.sft.indexOf(d.getLocalName())
174 106 7689 - 7700 Apply org.apache.avro.Schema.Field.pos field.pos()
174 107 7702 - 7721 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.defaultValue ConfluentFeatureMapper.this.defaultValue(field)
174 108 7647 - 7734 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.apply ConfluentFeatureSerializer.this.FieldMapping.apply(ConfluentFeatureMapper.this.sft.indexOf(d.getLocalName()), field.pos(), ConfluentFeatureMapper.this.defaultValue(field), conversion)
178 111 7825 - 7841 Apply org.apache.avro.Schema.getFields ConfluentFeatureMapper.this.schema.getFields()
178 118 7863 - 7863 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.$anonfun.<init> new $anonfun()
178 119 7825 - 7967 Apply scala.collection.TraversableOnce.collectFirst scala.collection.JavaConverters.asScalaBufferConverter[org.apache.avro.Schema.Field](ConfluentFeatureMapper.this.schema.getFields()).asScala.collectFirst[Int](({ @SerialVersionUID(value = 0) final <synthetic> class $anonfun extends scala.runtime.AbstractPartialFunction[org.apache.avro.Schema.Field,Int] with Serializable { def <init>(): <$anon: org.apache.avro.Schema.Field => Int> = { $anonfun.super.<init>(); () }; final override def applyOrElse[A1 <: org.apache.avro.Schema.Field, B1 >: Int](x1: A1, default: A1 => B1): B1 = ((x1.asInstanceOf[org.apache.avro.Schema.Field]: org.apache.avro.Schema.Field): org.apache.avro.Schema.Field @unchecked) match { case (f @ _) if scala.Option.apply[String](f.getProp(SchemaParser.GeoMesaAvroVisibilityField.KEY)).exists(((x$4: String) => scala.Predef.augmentString(x$4).toBoolean)) => f.pos() case (defaultCase$ @ _) => default.apply(x1) }; final def isDefinedAt(x1: org.apache.avro.Schema.Field): Boolean = ((x1.asInstanceOf[org.apache.avro.Schema.Field]: org.apache.avro.Schema.Field): org.apache.avro.Schema.Field @unchecked) match { case (f @ _) if scala.Option.apply[String](f.getProp(SchemaParser.GeoMesaAvroVisibilityField.KEY)).exists(((x$4: String) => scala.Predef.augmentString(x$4).toBoolean)) => true case (defaultCase$ @ _) => false } }; new $anonfun() }: PartialFunction[org.apache.avro.Schema.Field,Int]))
179 112 7898 - 7928 Select org.locationtech.geomesa.kafka.confluent.SchemaParser.GeoMesaAvroVisibilityField.KEY SchemaParser.GeoMesaAvroVisibilityField.KEY
179 113 7888 - 7929 Apply org.apache.avro.JsonProperties.getProp f.getProp(SchemaParser.GeoMesaAvroVisibilityField.KEY)
179 114 7938 - 7949 Select scala.collection.immutable.StringLike.toBoolean scala.Predef.augmentString(x$4).toBoolean
179 115 7881 - 7950 Apply scala.Option.exists scala.Option.apply[String](f.getProp(SchemaParser.GeoMesaAvroVisibilityField.KEY)).exists(((x$4: String) => scala.Predef.augmentString(x$4).toBoolean))
179 116 7954 - 7961 Apply org.apache.avro.Schema.Field.pos f.pos()
179 117 7954 - 7961 Block org.apache.avro.Schema.Field.pos f.pos()
183 120 8077 - 8093 Apply org.apache.avro.Schema.getFields ConfluentFeatureMapper.this.schema.getFields()
183 121 8140 - 8152 Apply scala.Predef.ArrowAssoc.-> scala.Predef.ArrowAssoc[Int](f.pos()).->[AnyRef](v)
183 122 8115 - 8153 Apply scala.Option.map ConfluentFeatureMapper.this.defaultValue(f).map[(Int, AnyRef)](((v: AnyRef) => scala.Predef.ArrowAssoc[Int](f.pos()).->[AnyRef](v)))
183 123 8115 - 8153 ApplyImplicitView scala.Option.option2Iterable scala.this.Option.option2Iterable[(Int, AnyRef)](ConfluentFeatureMapper.this.defaultValue(f).map[(Int, AnyRef)](((v: AnyRef) => scala.Predef.ArrowAssoc[Int](f.pos()).->[AnyRef](v))))
183 124 8109 - 8109 TypeApply scala.collection.mutable.Buffer.canBuildFrom mutable.this.Buffer.canBuildFrom[(Int, AnyRef)]
183 129 8077 - 8270 Apply scala.collection.TraversableLike.filter scala.collection.JavaConverters.asScalaBufferConverter[org.apache.avro.Schema.Field](ConfluentFeatureMapper.this.schema.getFields()).asScala.flatMap[(Int, AnyRef), scala.collection.mutable.Buffer[(Int, AnyRef)]](((f: org.apache.avro.Schema.Field) => scala.this.Option.option2Iterable[(Int, AnyRef)](ConfluentFeatureMapper.this.defaultValue(f).map[(Int, AnyRef)](((v: AnyRef) => scala.Predef.ArrowAssoc[Int](f.pos()).->[AnyRef](v))))))(mutable.this.Buffer.canBuildFrom[(Int, AnyRef)]).filter(((x0$1: (Int, AnyRef)) => x0$1 match { case (_1: Int, _2: AnyRef)(Int, AnyRef)((pos @ _), _) => ConfluentFeatureMapper.this.fieldMappings.exists(((x$5: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping) => x$5.schemaIndex.==(pos))).unary_!.&&(ConfluentFeatureMapper.this.visibilityField.contains[Int](pos).unary_!) }))
184 125 8209 - 8229 Apply scala.Int.== x$5.schemaIndex.==(pos)
184 126 8234 - 8264 Select scala.Boolean.unary_! ConfluentFeatureMapper.this.visibilityField.contains[Int](pos).unary_!
184 127 8187 - 8264 Apply scala.Boolean.&& ConfluentFeatureMapper.this.fieldMappings.exists(((x$5: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping) => x$5.schemaIndex.==(pos))).unary_!.&&(ConfluentFeatureMapper.this.visibilityField.contains[Int](pos).unary_!)
184 128 8187 - 8264 Block scala.Boolean.&& ConfluentFeatureMapper.this.fieldMappings.exists(((x$5: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping) => x$5.schemaIndex.==(pos))).unary_!.&&(ConfluentFeatureMapper.this.visibilityField.contains[Int](pos).unary_!)
194 130 8609 - 8622 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.schemaIndex x$6.schemaIndex
194 131 8608 - 8608 TypeApply scala.collection.mutable.Buffer.canBuildFrom mutable.this.Buffer.canBuildFrom[Int]
194 132 8627 - 8642 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.visibilityField ConfluentFeatureMapper.this.visibilityField
194 133 8627 - 8648 Select scala.collection.TraversableOnce.toSeq scala.this.Option.option2Iterable[Int](ConfluentFeatureMapper.this.visibilityField).toSeq
194 134 8591 - 8648 Apply scala.collection.mutable.BufferLike.++ ConfluentFeatureMapper.this.fieldMappings.map[Int, scala.collection.mutable.Buffer[Int]](((x$6: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping) => x$6.schemaIndex))(mutable.this.Buffer.canBuildFrom[Int]).++(scala.this.Option.option2Iterable[Int](ConfluentFeatureMapper.this.visibilityField).toSeq)
195 135 8655 - 8671 Apply org.apache.avro.Schema.getFields ConfluentFeatureMapper.this.schema.getFields()
195 141 8688 - 8688 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.$anonfun.<init> new $anonfun()
195 142 8688 - 8688 TypeApply scala.collection.mutable.Buffer.canBuildFrom mutable.this.Buffer.canBuildFrom[String]
196 136 8754 - 8761 Apply org.apache.avro.Schema.Field.pos f.pos()
196 137 8728 - 8762 Select scala.Boolean.unary_! mappedPositions.contains[Int](f.pos()).unary_!
196 138 8708 - 8762 Apply scala.Boolean.&& ConfluentFeatureMapper.this.requiredField(f).&&(mappedPositions.contains[Int](f.pos()).unary_!)
196 139 8766 - 8774 Apply org.apache.avro.Schema.Field.name f.name()
196 140 8766 - 8774 Block org.apache.avro.Schema.Field.name f.name()
197 143 8655 - 8788 Select scala.collection.SeqLike.toSeq scala.collection.JavaConverters.asScalaBufferConverter[org.apache.avro.Schema.Field](ConfluentFeatureMapper.this.schema.getFields()).asScala.collect[String, scala.collection.mutable.Buffer[String]](({ @SerialVersionUID(value = 0) final <synthetic> class $anonfun extends scala.runtime.AbstractPartialFunction[org.apache.avro.Schema.Field,String] with Serializable { def <init>(): <$anon: org.apache.avro.Schema.Field => String> = { $anonfun.super.<init>(); () }; final override def applyOrElse[A1 <: org.apache.avro.Schema.Field, B1 >: String](x1: A1, default: A1 => B1): B1 = ((x1.asInstanceOf[org.apache.avro.Schema.Field]: org.apache.avro.Schema.Field): org.apache.avro.Schema.Field @unchecked) match { case (f @ _) if ConfluentFeatureMapper.this.requiredField(f).&&(mappedPositions.contains[Int](f.pos()).unary_!) => f.name() case (defaultCase$ @ _) => default.apply(x1) }; final def isDefinedAt(x1: org.apache.avro.Schema.Field): Boolean = ((x1.asInstanceOf[org.apache.avro.Schema.Field]: org.apache.avro.Schema.Field): org.apache.avro.Schema.Field @unchecked) match { case (f @ _) if ConfluentFeatureMapper.this.requiredField(f).&&(mappedPositions.contains[Int](f.pos()).unary_!) => true case (defaultCase$ @ _) => false } }; new $anonfun() }: PartialFunction[org.apache.avro.Schema.Field,String]))(mutable.this.Buffer.canBuildFrom[String]).toSeq
207 144 9009 - 9015 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.schema ConfluentFeatureMapper.this.schema
207 145 8986 - 9016 Apply org.apache.avro.generic.GenericData.Record.<init> new org.apache.avro.generic.GenericData.Record(ConfluentFeatureMapper.this.schema)
208 146 9062 - 9078 Apply org.apache.avro.generic.GenericData.Record.put record.put(i, v)
208 147 9062 - 9078 Block org.apache.avro.generic.GenericData.Record.put record.put(i, v)
208 148 9023 - 9080 Apply scala.collection.IterableLike.foreach ConfluentFeatureMapper.this.defaultFields.foreach[Unit](((x0$1: (Int, AnyRef)) => x0$1 match { case (_1: Int, _2: AnyRef)(Int, AnyRef)((i @ _), (v @ _)) => record.put(i, v) }))
209 149 9136 - 9172 Apply org.locationtech.geomesa.security.SecurityUtils.getVisibility org.locationtech.geomesa.security.SecurityUtils.getVisibility(feature)
209 150 9120 - 9173 Apply org.apache.avro.generic.GenericData.Record.put record.put(pos, org.locationtech.geomesa.security.SecurityUtils.getVisibility(feature))
209 151 9087 - 9175 Apply scala.Option.foreach ConfluentFeatureMapper.this.visibilityField.foreach[Unit](((pos: Int) => record.put(pos, org.locationtech.geomesa.security.SecurityUtils.getVisibility(feature))))
210 172 9182 - 9834 Apply scala.collection.IterableLike.foreach ConfluentFeatureMapper.this.fieldMappings.foreach[Unit](((m: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping) => try { feature.getAttribute(m.sftIndex) match { case null => m.default.foreach[Unit](((d: AnyRef) => record.put(m.schemaIndex, d))) case (v @ _) => record.put(m.schemaIndex, m.conversion.fold[Object](v)(((x$7: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter) => x$7.featureToRecord(v)))) } } catch { case scala.util.control.NonFatal.unapply(<unapply-selector>) <unapply> ((e @ _)) => { val d: org.geotools.api.feature.type.AttributeDescriptor = ConfluentFeatureMapper.this.sft.getDescriptor(m.sftIndex); val v: scala.util.Try[Object] = scala.util.Try.apply[Object](feature.getAttribute(m.sftIndex)); val s: org.apache.avro.Schema = ConfluentFeatureMapper.this.schema.getField(d.getLocalName()).schema(); throw new scala.`package`.RuntimeException(scala.StringContext.apply("Cannot serialize field \'", "\' with try-value \'", "\' into schema \'", "\':").s(d.getLocalName(), v, s), e) } }))
212 152 9256 - 9266 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.sftIndex m.sftIndex
212 153 9235 - 9267 Apply org.geotools.api.feature.simple.SimpleFeature.getAttribute feature.getAttribute(m.sftIndex)
212 163 9235 - 9457 Match <nosymbol> feature.getAttribute(m.sftIndex) match { case null => m.default.foreach[Unit](((d: AnyRef) => record.put(m.schemaIndex, d))) case (v @ _) => record.put(m.schemaIndex, m.conversion.fold[Object](v)(((x$7: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter) => x$7.featureToRecord(v)))) }
213 154 9335 - 9348 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.schemaIndex m.schemaIndex
213 155 9324 - 9352 Apply org.apache.avro.generic.GenericData.Record.put record.put(m.schemaIndex, d)
213 156 9301 - 9353 Apply scala.Option.foreach m.default.foreach[Unit](((d: AnyRef) => record.put(m.schemaIndex, d)))
213 157 9301 - 9353 Block scala.Option.foreach m.default.foreach[Unit](((d: AnyRef) => record.put(m.schemaIndex, d)))
214 158 9387 - 9400 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.schemaIndex m.schemaIndex
214 159 9423 - 9443 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter.featureToRecord x$7.featureToRecord(v)
214 160 9402 - 9444 Apply scala.Option.fold m.conversion.fold[Object](v)(((x$7: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter) => x$7.featureToRecord(v)))
214 161 9376 - 9445 Apply org.apache.avro.generic.GenericData.Record.put record.put(m.schemaIndex, m.conversion.fold[Object](v)(((x$7: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter) => x$7.featureToRecord(v))))
214 162 9376 - 9445 Block org.apache.avro.generic.GenericData.Record.put record.put(m.schemaIndex, m.conversion.fold[Object](v)(((x$7: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter) => x$7.featureToRecord(v))))
217 171 9503 - 9816 Block <nosymbol> { val d: org.geotools.api.feature.type.AttributeDescriptor = ConfluentFeatureMapper.this.sft.getDescriptor(m.sftIndex); val v: scala.util.Try[Object] = scala.util.Try.apply[Object](feature.getAttribute(m.sftIndex)); val s: org.apache.avro.Schema = ConfluentFeatureMapper.this.schema.getField(d.getLocalName()).schema(); throw new scala.`package`.RuntimeException(scala.StringContext.apply("Cannot serialize field \'", "\' with try-value \'", "\' into schema \'", "\':").s(d.getLocalName(), v, s), e) }
218 164 9544 - 9554 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.sftIndex m.sftIndex
218 165 9526 - 9555 Apply org.geotools.api.feature.simple.SimpleFeatureType.getDescriptor ConfluentFeatureMapper.this.sft.getDescriptor(m.sftIndex)
219 166 9601 - 9611 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.sftIndex m.sftIndex
219 167 9580 - 9612 Apply org.geotools.api.feature.simple.SimpleFeature.getAttribute feature.getAttribute(m.sftIndex)
219 168 9576 - 9613 Apply scala.util.Try.apply scala.util.Try.apply[Object](feature.getAttribute(m.sftIndex))
220 169 9634 - 9674 Apply org.apache.avro.Schema.Field.schema ConfluentFeatureMapper.this.schema.getField(d.getLocalName()).schema()
221 170 9687 - 9816 Throw <nosymbol> throw new scala.`package`.RuntimeException(scala.StringContext.apply("Cannot serialize field \'", "\' with try-value \'", "\' into schema \'", "\':").s(d.getLocalName(), v, s), e)
226 173 9868 - 9873 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.topic ConfluentFeatureMapper.this.topic
226 174 9842 - 9882 Apply io.confluent.kafka.serializers.KafkaAvroSerializer.serialize ConfluentFeatureMapper.this.kafkaSerializer.serialize(ConfluentFeatureMapper.this.topic, record)
237 175 10160 - 10165 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.topic ConfluentFeatureMapper.this.topic
237 176 10130 - 10201 TypeApply scala.Any.asInstanceOf ConfluentFeatureMapper.this.kafkaDeserializer.deserialize(ConfluentFeatureMapper.this.topic, bytes).asInstanceOf[org.apache.avro.generic.GenericRecord]
238 188 10243 - 10243 TypeApply scala.collection.mutable.Buffer.canBuildFrom mutable.this.Buffer.canBuildFrom[Object]
238 189 10225 - 10690 ApplyToImplicitArgs scala.collection.TraversableLike.map ConfluentFeatureMapper.this.fieldMappings.map[Object, scala.collection.mutable.Buffer[Object]](((m: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping) => try { val v: Object = record.get(m.schemaIndex); m.conversion match { case scala.None => v case (value: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter)Some[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter]((c @ _)) => c.recordToFeature(v) } } catch { case scala.util.control.NonFatal.unapply(<unapply-selector>) <unapply> ((e @ _)) => { val d: org.geotools.api.feature.type.AttributeDescriptor = ConfluentFeatureMapper.this.sft.getDescriptor(m.sftIndex); throw new scala.`package`.RuntimeException(scala.StringContext.apply("Cannot deserialize field \'", "\' into a \'", "\':").s(d.getLocalName(), d.getType().getBinding().getName()), e) } }))(mutable.this.Buffer.canBuildFrom[Object])
239 183 10274 - 10426 Block <nosymbol> { val v: Object = record.get(m.schemaIndex); m.conversion match { case scala.None => v case (value: org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter)Some[org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter]((c @ _)) => c.recordToFeature(v) } }
240 177 10293 - 10306 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.schemaIndex m.schemaIndex
240 178 10282 - 10307 Apply org.apache.avro.generic.IndexedRecord.get record.get(m.schemaIndex)
241 179 10318 - 10330 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.conversion m.conversion
242 180 10364 - 10365 Ident org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.v v
243 181 10394 - 10414 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter.recordToFeature c.recordToFeature(v)
243 182 10394 - 10414 Block org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldConverter.recordToFeature c.recordToFeature(v)
246 187 10472 - 10672 Block <nosymbol> { val d: org.geotools.api.feature.type.AttributeDescriptor = ConfluentFeatureMapper.this.sft.getDescriptor(m.sftIndex); throw new scala.`package`.RuntimeException(scala.StringContext.apply("Cannot deserialize field \'", "\' into a \'", "\':").s(d.getLocalName(), d.getType().getBinding().getName()), e) }
247 184 10513 - 10523 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.FieldMapping.sftIndex m.sftIndex
247 185 10495 - 10524 Apply org.geotools.api.feature.simple.SimpleFeatureType.getDescriptor ConfluentFeatureMapper.this.sft.getDescriptor(m.sftIndex)
248 186 10537 - 10672 Throw <nosymbol> throw new scala.`package`.RuntimeException(scala.StringContext.apply("Cannot deserialize field \'", "\' into a \'", "\':").s(d.getLocalName(), d.getType().getBinding().getName()), e)
253 190 10738 - 10741 Select org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.ConfluentFeatureMapper.sft ConfluentFeatureMapper.this.sft
253 191 10747 - 10763 Select scala.collection.SeqLike.toSeq attributes.toSeq
253 192 10712 - 10768 Apply org.locationtech.geomesa.features.ScalaSimpleFeature.create org.locationtech.geomesa.features.ScalaSimpleFeature.create(ConfluentFeatureMapper.this.sft, id, (attributes.toSeq: _*))
256 200 10825 - 11008 Apply scala.Option.foreach ConfluentFeatureMapper.this.visibilityField.foreach[Any](((field: Int) => { val vis: Object = record.get(field); if (vis.!=(null)) org.locationtech.geomesa.security.SecurityUtils.setFeatureVisibility(feature, vis.toString()) else () }))
257 193 10878 - 10895 Apply org.apache.avro.generic.IndexedRecord.get record.get(field)
258 194 10908 - 10919 Apply java.lang.Object.!= vis.!=(null)
258 198 10904 - 10904 Literal <nosymbol> ()
258 199 10904 - 10904 Block <nosymbol> ()
259 195 10977 - 10989 Apply java.lang.Object.toString vis.toString()
259 196 10933 - 10990 Apply org.locationtech.geomesa.security.SecurityUtils.setFeatureVisibility org.locationtech.geomesa.security.SecurityUtils.setFeatureVisibility(feature, vis.toString())
259 197 10933 - 10990 Block org.locationtech.geomesa.security.SecurityUtils.setFeatureVisibility org.locationtech.geomesa.security.SecurityUtils.setFeatureVisibility(feature, vis.toString())
268 201 11209 - 11223 Apply org.apache.avro.Schema.Field.defaultVal f.defaultVal()
268 202 11235 - 11270 TypeApply scala.Any.isInstanceOf x$8.isInstanceOf[org.apache.avro.JsonProperties.Null]
268 203 11202 - 11271 Apply scala.Option.filterNot scala.Option.apply[Object](f.defaultVal()).filterNot(((x$8: Object) => x$8.isInstanceOf[org.apache.avro.JsonProperties.Null]))
271 212 11332 - 11542 Apply scala.Boolean.&& ConfluentFeatureMapper.this.defaultValue(f).isEmpty.&&(f.schema().getType() match { case NULL => false case UNION => f.schema().getTypes().contains(NULL).unary_! case _ => true })
272 204 11369 - 11387 Apply org.apache.avro.Schema.getType f.schema().getType()
273 205 11424 - 11429 Literal <nosymbol> false
273 206 11424 - 11429 Block <nosymbol> false
274 207 11489 - 11498 Literal <nosymbol> NULL
274 208 11459 - 11499 Select scala.Boolean.unary_! f.schema().getTypes().contains(NULL).unary_!
274 209 11459 - 11499 Block scala.Boolean.unary_! f.schema().getTypes().contains(NULL).unary_!
275 210 11520 - 11524 Literal <nosymbol> true
275 211 11520 - 11524 Block <nosymbol> true
295 213 11986 - 11999 Apply java.lang.Object.== value.==(null)
295 214 12003 - 12007 Literal <nosymbol> null
295 215 12003 - 12007 Block <nosymbol> null
295 216 12031 - 12045 Apply java.lang.Object.toString value.toString()
295 217 12017 - 12046 Apply org.locationtech.geomesa.utils.text.WKTUtils.read org.locationtech.geomesa.utils.text.WKTUtils.read(value.toString())
295 218 12017 - 12046 Block org.locationtech.geomesa.utils.text.WKTUtils.read org.locationtech.geomesa.utils.text.WKTUtils.read(value.toString())
299 219 12124 - 12137 Apply java.lang.Object.== value.==(null)
299 220 12141 - 12145 Literal <nosymbol> null
299 221 12141 - 12145 Block <nosymbol> null
299 222 12170 - 12198 TypeApply scala.Any.asInstanceOf value.asInstanceOf[org.locationtech.jts.geom.Geometry]
299 223 12155 - 12199 Apply org.locationtech.geomesa.utils.text.WKTUtils.write org.locationtech.geomesa.utils.text.WKTUtils.write(value.asInstanceOf[org.locationtech.jts.geom.Geometry])
299 224 12155 - 12199 Block org.locationtech.geomesa.utils.text.WKTUtils.write org.locationtech.geomesa.utils.text.WKTUtils.write(value.asInstanceOf[org.locationtech.jts.geom.Geometry])
307 225 12378 - 12391 Apply java.lang.Object.== value.==(null)
307 226 12395 - 12399 Literal <nosymbol> null
307 227 12395 - 12399 Block <nosymbol> null
307 228 12430 - 12460 TypeApply scala.Any.asInstanceOf value.asInstanceOf[java.nio.ByteBuffer]
307 229 12423 - 12461 Apply org.locationtech.geomesa.kafka.confluent.ConfluentFeatureSerializer.WkbConverter.unwrap WkbConverter.this.unwrap(value.asInstanceOf[java.nio.ByteBuffer])
307 230 12409 - 12462 Apply org.locationtech.geomesa.utils.text.WKBUtils.read org.locationtech.geomesa.utils.text.WKBUtils.read(WkbConverter.this.unwrap(value.asInstanceOf[java.nio.ByteBuffer]))
307 231 12409 - 12462 Block org.locationtech.geomesa.utils.text.WKBUtils.read org.locationtech.geomesa.utils.text.WKBUtils.read(WkbConverter.this.unwrap(value.asInstanceOf[java.nio.ByteBuffer]))
310 232 12534 - 12547 Apply java.lang.Object.== value.==(null)
310 233 12551 - 12555 Literal <nosymbol> null
310 234 12551 - 12555 Block <nosymbol> null
310 235 12596 - 12624 TypeApply scala.Any.asInstanceOf value.asInstanceOf[org.locationtech.jts.geom.Geometry]
310 236 12581 - 12625 Apply org.locationtech.geomesa.utils.text.WKBUtils.write org.locationtech.geomesa.utils.text.WKBUtils.write(value.asInstanceOf[org.locationtech.jts.geom.Geometry])
310 237 12565 - 12626 Apply java.nio.ByteBuffer.wrap java.nio.ByteBuffer.wrap(org.locationtech.geomesa.utils.text.WKBUtils.write(value.asInstanceOf[org.locationtech.jts.geom.Geometry]))
310 238 12565 - 12626 Block java.nio.ByteBuffer.wrap java.nio.ByteBuffer.wrap(org.locationtech.geomesa.utils.text.WKBUtils.write(value.asInstanceOf[org.locationtech.jts.geom.Geometry]))
313 239 12713 - 12735 Apply scala.Int.== buf.arrayOffset().==(0)
313 240 12754 - 12772 Select scala.Array.length buf.array().length
313 241 12739 - 12772 Apply scala.Int.== buf.limit().==(buf.array().length)
313 242 12697 - 12772 Apply scala.Boolean.&& buf.hasArray().&&(buf.arrayOffset().==(0)).&&(buf.limit().==(buf.array().length))
314 243 12784 - 12795 Apply java.nio.ByteBuffer.array buf.array()
314 244 12784 - 12795 Block java.nio.ByteBuffer.array buf.array()
315 248 12809 - 12906 Block <nosymbol> { val array: Array[Byte] = scala.Array.ofDim[Byte](buf.limit())((ClassTag.Byte: scala.reflect.ClassTag[Byte])); buf.get(array); array }
316 245 12849 - 12860 Apply java.nio.Buffer.limit buf.limit()
316 246 12831 - 12861 ApplyToImplicitArgs scala.Array.ofDim scala.Array.ofDim[Byte](buf.limit())((ClassTag.Byte: scala.reflect.ClassTag[Byte]))
317 247 12870 - 12884 Apply java.nio.ByteBuffer.get buf.get(array)
328 249 13111 - 13124 Apply java.lang.Object.== value.==(null)
328 250 13128 - 13132 Literal <nosymbol> null
328 251 13128 - 13132 Block <nosymbol> null
330 252 13227 - 13241 Apply java.lang.Object.toString value.toString()
330 253 13243 - 13269 Select java.time.format.DateTimeFormatter.ISO_DATE java.time.format.DateTimeFormatter.ISO_DATE
330 254 13205 - 13270 Apply org.locationtech.geomesa.utils.text.DateParsing.parseDate org.locationtech.geomesa.utils.text.DateParsing.parseDate(value.toString(), java.time.format.DateTimeFormatter.ISO_DATE)
330 255 13205 - 13270 Block org.locationtech.geomesa.utils.text.DateParsing.parseDate org.locationtech.geomesa.utils.text.DateParsing.parseDate(value.toString(), java.time.format.DateTimeFormatter.ISO_DATE)
335 256 13356 - 13369 Apply java.lang.Object.== value.==(null)
335 257 13373 - 13377 Literal <nosymbol> null
335 258 13373 - 13377 Block <nosymbol> null
336 259 13418 - 13442 TypeApply scala.Any.asInstanceOf value.asInstanceOf[java.util.Date]
336 260 13444 - 13470 Select java.time.format.DateTimeFormatter.ISO_DATE java.time.format.DateTimeFormatter.ISO_DATE
336 261 13395 - 13471 Apply org.locationtech.geomesa.utils.text.DateParsing.formatDate org.locationtech.geomesa.utils.text.DateParsing.formatDate(value.asInstanceOf[java.util.Date], java.time.format.DateTimeFormatter.ISO_DATE)
336 262 13395 - 13471 Block org.locationtech.geomesa.utils.text.DateParsing.formatDate org.locationtech.geomesa.utils.text.DateParsing.formatDate(value.asInstanceOf[java.util.Date], java.time.format.DateTimeFormatter.ISO_DATE)
346 263 13693 - 13706 Apply java.lang.Object.== value.==(null)
346 264 13710 - 13714 Literal <nosymbol> null
346 265 13710 - 13714 Block <nosymbol> null
348 266 13809 - 13823 Apply java.lang.Object.toString value.toString()
348 267 13825 - 13856 Select java.time.format.DateTimeFormatter.ISO_DATE_TIME java.time.format.DateTimeFormatter.ISO_DATE_TIME
348 268 13787 - 13857 Apply org.locationtech.geomesa.utils.text.DateParsing.parseDate org.locationtech.geomesa.utils.text.DateParsing.parseDate(value.toString(), java.time.format.DateTimeFormatter.ISO_DATE_TIME)
348 269 13787 - 13857 Block org.locationtech.geomesa.utils.text.DateParsing.parseDate org.locationtech.geomesa.utils.text.DateParsing.parseDate(value.toString(), java.time.format.DateTimeFormatter.ISO_DATE_TIME)
353 270 13943 - 13956 Apply java.lang.Object.== value.==(null)
353 271 13960 - 13964 Literal <nosymbol> null
353 272 13960 - 13964 Block <nosymbol> null
354 273 14005 - 14029 TypeApply scala.Any.asInstanceOf value.asInstanceOf[java.util.Date]
354 274 14031 - 14062 Select java.time.format.DateTimeFormatter.ISO_DATE_TIME java.time.format.DateTimeFormatter.ISO_DATE_TIME
354 275 13982 - 14063 Apply org.locationtech.geomesa.utils.text.DateParsing.formatDate org.locationtech.geomesa.utils.text.DateParsing.formatDate(value.asInstanceOf[java.util.Date], java.time.format.DateTimeFormatter.ISO_DATE_TIME)
354 276 13982 - 14063 Block org.locationtech.geomesa.utils.text.DateParsing.formatDate org.locationtech.geomesa.utils.text.DateParsing.formatDate(value.asInstanceOf[java.util.Date], java.time.format.DateTimeFormatter.ISO_DATE_TIME)
364 277 14282 - 14295 Apply java.lang.Object.== value.==(null)
364 278 14299 - 14303 Literal <nosymbol> null
364 279 14299 - 14303 Block <nosymbol> null
364 280 14322 - 14356 TypeApply scala.Any.asInstanceOf value.asInstanceOf[Long]
364 281 14322 - 14356 ApplyImplicitView scala.Predef.Long2long scala.Predef.Long2long(value.asInstanceOf[Long])
364 282 14313 - 14357 Apply java.util.Date.<init> new java.util.Date(scala.Predef.Long2long(value.asInstanceOf[Long]))
364 283 14313 - 14357 Block java.util.Date.<init> new java.util.Date(scala.Predef.Long2long(value.asInstanceOf[Long]))
367 284 14429 - 14442 Apply java.lang.Object.== value.==(null)
367 285 14446 - 14450 Literal <nosymbol> null
367 286 14446 - 14450 Block <nosymbol> null
367 287 14469 - 14501 Apply java.util.Date.getTime value.asInstanceOf[java.util.Date].getTime()
367 288 14460 - 14502 Apply scala.Long.box scala.Long.box(value.asInstanceOf[java.util.Date].getTime())
367 289 14460 - 14502 Block scala.Long.box scala.Long.box(value.asInstanceOf[java.util.Date].getTime())