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.hbase.data
10 
11 import com.typesafe.scalalogging.{LazyLogging, StrictLogging}
12 import org.apache.hadoop.fs.Path
13 import org.apache.hadoop.hbase.TableName
14 import org.apache.hadoop.hbase.client._
15 import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange
16 import org.apache.hadoop.hbase.filter.{FilterList, KeyOnlyFilter, MultiRowRangeFilter, Filter => HFilter}
17 import org.apache.hadoop.hbase.io.compress.Compression
18 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
19 import org.apache.hadoop.hbase.regionserver.BloomType
20 import org.apache.hadoop.hbase.security.visibility.CellVisibility
21 import org.locationtech.geomesa.hbase.HBaseSystemProperties
22 import org.locationtech.geomesa.index.api.IndexAdapter.RequiredVisibilityWriter
23 import org.locationtech.geomesa.utils.concurrent.CachedThreadPool
24 import org.locationtech.geomesa.utils.io.{IsCloseable, IsFlushableImplicits}
25 
26 import java.nio.charset.StandardCharsets
27 import java.util.regex.Pattern
28 import java.util.{Collections, Locale, UUID}
29 import scala.util.Try
30 // noinspection ScalaDeprecation
31 import org.geotools.api.feature.simple.{SimpleFeature, SimpleFeatureType}
32 import org.locationtech.geomesa.hbase.HBaseSystemProperties.{CoprocessorPath, CoprocessorUrl, TableAvailabilityTimeout}
33 import org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures
34 import org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures
35 import org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures
36 import org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures
37 import org.locationtech.geomesa.hbase.aggregators.{HBaseArrowAggregator, HBaseBinAggregator, HBaseDensityAggregator, HBaseStatsAggregator}
38 import org.locationtech.geomesa.hbase.data.HBaseQueryPlan.{CoprocessorPlan, EmptyPlan, ScanPlan, TableScan}
39 import org.locationtech.geomesa.hbase.rpc.coprocessor.GeoMesaCoprocessor
40 import org.locationtech.geomesa.hbase.rpc.filter._
41 import org.locationtech.geomesa.hbase.utils.HBaseVersions
42 import org.locationtech.geomesa.index.api.IndexAdapter.BaseIndexWriter
43 import org.locationtech.geomesa.index.api.QueryPlan.{FeatureReducer, IndexResultsToFeatures}
44 import org.locationtech.geomesa.index.api.WritableFeature.FeatureWrapper
45 import org.locationtech.geomesa.index.api._
46 import org.locationtech.geomesa.index.conf.QueryHints
47 import org.locationtech.geomesa.index.filters.{S2Filter, S3Filter, Z2Filter, Z3Filter}
48 import org.locationtech.geomesa.index.index.id.IdIndex
49 import org.locationtech.geomesa.index.index.s2.{S2Index, S2IndexValues}
50 import org.locationtech.geomesa.index.index.s3.{S3Index, S3IndexValues}
51 import org.locationtech.geomesa.index.index.z2.{Z2Index, Z2IndexValues}
52 import org.locationtech.geomesa.index.index.z3.{Z3Index, Z3IndexValues}
53 import org.locationtech.geomesa.index.iterators.StatsScan
54 import org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer
55 import org.locationtech.geomesa.utils.index.ByteArrays
56 import org.locationtech.geomesa.utils.io.{CloseWithLogging, FlushWithLogging, WithClose}
57 import org.locationtech.geomesa.utils.text.StringSerialization
58 
59 import java.util.concurrent.TimeUnit
60 import scala.util.Random
61 import scala.util.control.NonFatal
62 
63 class HBaseIndexAdapter(ds: HBaseDataStore) extends IndexAdapter[HBaseDataStore] with StrictLogging {
64 
65   import HBaseIndexAdapter._
66   import org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType
67 
68   import scala.collection.JavaConverters._
69 
70   override def createTable(
71       index: GeoMesaFeatureIndex[_, _],
72       partition: Option[String],
73       splits: => Seq[Array[Byte]]): Unit = {
74     // write table name to metadata
75     val name = TableName.valueOf(index.configureTableName(partition, tableNameLimit))
76 
77     WithClose(ds.connection.getAdmin) { admin =>
78       if (!admin.tableExists(name)) {
79         logger.debug(s"Creating table $name")
80 
81         val conf = admin.getConfiguration
82 
83         val compression = index.sft.getCompression.map { alg =>
84           logger.debug(s"Setting compression '$alg' on table $name for feature ${index.sft.getTypeName}")
85           // note: all compression types in HBase are case-sensitive and lower-cased
86           Compression.getCompressionAlgorithmByName(alg.toLowerCase(Locale.US))
87         }
88 
89         val cols = groups.apply(index.sft).map(_._1)
90         val bloom = Some(BloomType.NONE)
91         val encoding = if (index.name == IdIndex.name) { None } else { Some(DataBlockEncoding.FAST_DIFF) }
92 
93         // noinspection ScalaDeprecation
94         val coprocessor = if (!ds.config.remoteFilter) { None } else {
95           def urlFromSysProp: Option[Path] = CoprocessorUrl.option.orElse(CoprocessorPath.option).map(new Path(_))
96           lazy val coprocessorUrl = ds.config.coprocessors.url.orElse(urlFromSysProp).orElse {
97             try {
98               // the jar should be under hbase.dynamic.jars.dir to enable filters, so look there
99               val dir = new Path(conf.get("hbase.dynamic.jars.dir"))
100               WithClose(dir.getFileSystem(conf)) { fs =>
101                 if (!fs.isDirectory(dir)) { None } else {
102                   fs.listStatus(dir).collectFirst {
103                     case s if distributedJarNamePattern.matcher(s.getPath.getName).matches() => s.getPath
104                   }
105                 }
106               }
107             } catch {
108               case NonFatal(e) => logger.warn("Error checking dynamic jar path:", e); None
109             }
110           }
111           // if the coprocessors are installed site-wide don't register them in the table descriptor.
112           // this key is CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY - but don't want to pull in
113           // a dependency on hbase-server just for this constant
114           val installed = Option(conf.get("hbase.coprocessor.user.region.classes"))
115           val names = installed.map(_.split(":").toSet).getOrElse(Set.empty[String])
116           if (names.contains(CoprocessorClass)) { None } else {
117             logger.debug(s"Using coprocessor path ${coprocessorUrl.orNull}")
118             // TODO: Warn if the path given is different from paths registered in other coprocessors
119             // if so, other tables would need updating
120             Some(CoprocessorClass -> coprocessorUrl)
121           }
122         }
123 
124         try {
125           HBaseVersions.createTableAsync(admin, name, cols, bloom, compression, encoding, None, coprocessor, splits)
126         } catch {
127           case _: org.apache.hadoop.hbase.TableExistsException => // ignore, another thread created it for us
128         }
129       }
130 
131       waitForTable(admin, name)
132     }
133   }
134 
135   override def renameTable(from: String, to: String): Unit = {
136     WithClose(ds.connection.getAdmin) { admin =>
137       val existing = TableName.valueOf(from)
138       val renamed = TableName.valueOf(to)
139       if (admin.tableExists(existing)) {
140         // renaming in hbase requires creating a snapshot and using that to create the new table
141         val snapshot = StringSerialization.alphaNumericSafeString(UUID.randomUUID().toString)
142         admin.disableTable(existing)
143         admin.snapshot(snapshot, existing)
144         admin.cloneSnapshot(snapshot, renamed)
145         admin.deleteSnapshot(snapshot)
146         admin.deleteTable(existing)
147         waitForTable(admin, renamed)
148       }
149     }
150   }
151 
152   override def deleteTables(tables: Seq[String]): Unit = {
153     WithClose(ds.connection.getAdmin) { admin =>
154       def deleteOne(name: String): Unit = {
155         val table = TableName.valueOf(name)
156         if (admin.tableExists(table)) {
157           HBaseVersions.disableTableAsync(admin, table)
158           val timeout = TableAvailabilityTimeout.toUnboundedDuration.filter(_.isFinite)
159           logger.debug(s"Waiting for table '$table' to be disabled with " +
160               s"${timeout.map(t => s"a timeout of $t").getOrElse("no timeout")}")
161           val stop = timeout.map(t => System.currentTimeMillis() + t.toMillis)
162           while (!admin.isTableDisabled(table) && stop.forall(_ > System.currentTimeMillis())) {
163             Thread.sleep(1000)
164           }
165           // no async operation, but timeout can be controlled through hbase-site.xml "hbase.client.sync.wait.timeout.msec"
166           admin.deleteTable(table)
167         }
168       }
169       tables.toList.map(t => CachedThreadPool.submit(() => deleteOne(t))).foreach(_.get)
170     }
171   }
172 
173   override def clearTables(tables: Seq[String], prefix: Option[Array[Byte]]): Unit = {
174     def clearOne(name: String): Unit = {
175       val tableName = TableName.valueOf(name)
176       WithClose(ds.connection.getTable(tableName)) { table =>
177         val scan = new Scan().setFilter(new KeyOnlyFilter)
178         prefix.foreach(scan.setRowPrefixFilter)
179         ds.applySecurity(scan)
180         val mutateParams = new BufferedMutatorParams(tableName)
181         WithClose(table.getScanner(scan), ds.connection.getBufferedMutator(mutateParams)) { case (scanner, mutator) =>
182           scanner.iterator.asScala.grouped(10000).foreach { result =>
183             // TODO GEOMESA-2546 set delete visibilities
184             val deletes = result.map(r => new Delete(r.getRow))
185             mutator.mutate(deletes.asJava)
186           }
187         }
188       }
189     }
190     tables.toList.map(t => CachedThreadPool.submit(() => clearOne(t))).foreach(_.get)
191   }
192 
193   override def createQueryPlan(strategy: QueryStrategy): HBaseQueryPlan = {
194 
195     import org.locationtech.geomesa.index.conf.QueryHints.RichHints
196 
197     val QueryStrategy(filter, byteRanges, _, _, ecql, hints, _) = strategy
198     val index = filter.index
199 
200     // index api defines empty start/end for open-ended range
201     // index api defines start row inclusive, end row exclusive
202     // both these conventions match the conventions for hbase scan objects
203     val ranges = byteRanges.map {
204       case BoundedByteRange(start, stop) => new RowRange(start, true, stop, false)
205       case SingleRowByteRange(row)       => new RowRange(row, true, ByteArrays.rowFollowingRow(row), false)
206     }
207     val small = byteRanges.headOption.exists(_.isInstanceOf[SingleRowByteRange])
208 
209     val tables = index.getTablesForQuery(filter.filter).map(TableName.valueOf)
210     val (colFamily, schema) = groups.group(index.sft, hints.getTransformDefinition, ecql)
211 
212     val transform: Option[(String, SimpleFeatureType)] = hints.getTransform
213 
214     // check for an empty query plan, if there are no tables or ranges to scan
215     def empty(reducer: Option[FeatureReducer]): Option[HBaseQueryPlan] =
216       if (tables.isEmpty || ranges.isEmpty) { Some(EmptyPlan(filter, reducer)) } else { None }
217 
218     if (!ds.config.remoteFilter) {
219       // everything is done client side
220       // note: we assume visibility filtering is still done server-side as it's part of core hbase
221       // note: we use the full filter here, since we can't use the z3 server-side filter
222       // for some attribute queries we wouldn't need the full filter...
223       val reducer = Some(new LocalTransformReducer(schema, filter.filter, None, transform, hints))
224       empty(reducer).getOrElse {
225         val scans = configureScans(tables, ranges, small, colFamily, Seq.empty, coprocessor = false)
226         val resultsToFeatures = new HBaseResultsToFeatures(index, schema)
227         val sort = hints.getSortFields
228         val max = hints.getMaxFeatures
229         val project = hints.getProjection
230         ScanPlan(filter, ranges, scans, resultsToFeatures, reducer, sort, max, project)
231       }
232     } else {
233       // TODO pull this out to be SPI loaded so that new indices can be added seamlessly
234       val indexFilter = strategy.index match {
235         case _: Z3Index =>
236           strategy.values.map { case v: Z3IndexValues =>
237             (Z3HBaseFilter.Priority, Z3HBaseFilter(Z3Filter(v), index.keySpace.sharding.length))
238           }
239 
240         case _: Z2Index =>
241           strategy.values.map { case v: Z2IndexValues =>
242             (Z2HBaseFilter.Priority, Z2HBaseFilter(Z2Filter(v), index.keySpace.sharding.length))
243           }
244 
245         case _: S2Index =>
246           strategy.values.map { case v: S2IndexValues =>
247             (S2HBaseFilter.Priority, S2HBaseFilter(S2Filter(v), index.keySpace.sharding.length))
248           }
249 
250         case _: S3Index =>
251           strategy.values.map { case v: S3IndexValues =>
252             (S3HBaseFilter.Priority, S3HBaseFilter(S3Filter(v), index.keySpace.sharding.length))
253           }
254         // TODO GEOMESA-1807 deal with non-points in a pushdown XZ filter
255 
256         case _ => None
257       }
258 
259       val max = hints.getMaxFeatures
260       val projection = hints.getProjection
261       lazy val returnSchema = transform.map(_._2).getOrElse(schema)
262       lazy val filters = {
263         val cqlFilter = if (ecql.isEmpty && transform.isEmpty && hints.getSampling.isEmpty) { Seq.empty } else {
264           Seq((CqlTransformFilter.Priority, CqlTransformFilter(schema, strategy.index, ecql, transform, hints)))
265         }
266         (cqlFilter ++ indexFilter).sortBy(_._1).map(_._2)
267       }
268       lazy val coprocessorOptions =
269         Map(GeoMesaCoprocessor.YieldOpt -> String.valueOf(ds.config.coprocessors.yieldPartialResults))
270       lazy val scans = configureScans(tables, ranges, small, colFamily, filters, coprocessor = false)
271       lazy val coprocessorScans =
272         configureScans(tables, ranges, small, colFamily, indexFilter.toSeq.map(_._2), coprocessor = true)
273       lazy val resultsToFeatures = new HBaseResultsToFeatures(index, returnSchema)
274       lazy val localReducer = Some(new LocalTransformReducer(returnSchema, None, None, None, hints))
275 
276       if (hints.isDensityQuery) {
277         empty(None).getOrElse {
278           if (ds.config.coprocessors.enabled.density) {
279             val options = HBaseDensityAggregator.configure(schema, index, ecql, hints) ++ coprocessorOptions
280             val results = new HBaseDensityResultsToFeatures()
281             CoprocessorPlan(filter, ranges, coprocessorScans, options, results, None, max, projection)
282           } else {
283             if (hints.isSkipReduce) {
284               // override the return sft to reflect what we're actually returning,
285               // since the density sft is only created in the local reduce step
286               hints.hints.put(QueryHints.Internal.RETURN_SFT, returnSchema)
287             }
288             ScanPlan(filter, ranges, scans, resultsToFeatures, localReducer, None, max, projection)
289           }
290         }
291       } else if (hints.isArrowQuery) {
292         val config = HBaseArrowAggregator.configure(schema, index, ds.stats, filter.filter, ecql, hints)
293         val reducer = Some(config.reduce)
294         empty(reducer).getOrElse {
295           if (ds.config.coprocessors.enabled.arrow) {
296             val options = config.config ++ coprocessorOptions
297             val results = new HBaseArrowResultsToFeatures()
298             CoprocessorPlan(filter, ranges, coprocessorScans, options, results, reducer, max, projection)
299           } else {
300             if (hints.isSkipReduce) {
301               // override the return sft to reflect what we're actually returning,
302               // since the arrow sft is only created in the local reduce step
303               hints.hints.put(QueryHints.Internal.RETURN_SFT, returnSchema)
304             }
305             ScanPlan(filter, ranges, scans, resultsToFeatures, localReducer, None, max, projection)
306           }
307         }
308       } else if (hints.isStatsQuery) {
309         val reducer = Some(StatsScan.StatsReducer(returnSchema, hints))
310         empty(reducer).getOrElse {
311           if (ds.config.coprocessors.enabled.stats) {
312             val options = HBaseStatsAggregator.configure(schema, index, ecql, hints) ++ coprocessorOptions
313             val results = new HBaseStatsResultsToFeatures()
314             CoprocessorPlan(filter, ranges, coprocessorScans, options, results, reducer, max, projection)
315           } else {
316             if (hints.isSkipReduce) {
317               // override the return sft to reflect what we're actually returning,
318               // since the stats sft is only created in the local reduce step
319               hints.hints.put(QueryHints.Internal.RETURN_SFT, returnSchema)
320             }
321             ScanPlan(filter, ranges, scans, resultsToFeatures, localReducer, None, max, projection)
322           }
323         }
324       } else if (hints.isBinQuery) {
325         empty(None).getOrElse {
326           if (ds.config.coprocessors.enabled.bin) {
327             val options = HBaseBinAggregator.configure(schema, index, ecql, hints) ++ coprocessorOptions
328             val results = new HBaseBinResultsToFeatures()
329             CoprocessorPlan(filter, ranges, coprocessorScans, options , results, None, max, projection)
330           } else {
331             if (hints.isSkipReduce) {
332               // override the return sft to reflect what we're actually returning,
333               // since the bin sft is only created in the local reduce step
334               hints.hints.put(QueryHints.Internal.RETURN_SFT, returnSchema)
335             }
336             ScanPlan(filter, ranges, scans, resultsToFeatures, localReducer, None, max, projection)
337           }
338         }
339       } else {
340         empty(None).getOrElse {
341           ScanPlan(filter, ranges, scans, resultsToFeatures, None, hints.getSortFields, max, projection)
342         }
343       }
344     }
345   }
346 
347   override def createWriter(
348       sft: SimpleFeatureType,
349       indices: Seq[GeoMesaFeatureIndex[_, _]],
350       partition: Option[String],
351       atomic: Boolean): HBaseIndexWriter = {
352     require(!atomic, "HBase data store does not currently support atomic writes")
353     val wrapper = WritableFeature.wrapper(sft, groups)
354     if (sft.isVisibilityRequired) {
355       new HBaseIndexWriter(ds, indices, wrapper, partition) with RequiredVisibilityWriter
356     } else {
357       new HBaseIndexWriter(ds, indices, wrapper, partition)
358     }
359   }
360 
361   /**
362    * Configure the hbase scan
363    *
364    * @param tables tables being scanned, used for region location information
365    * @param ranges ranges to scan, non-empty. needs to be mutable as we will sort it in place
366    * @param small whether 'small' ranges (i.e. gets)
367    * @param colFamily col family to scan
368    * @param filters scan filters
369    * @param coprocessor is this a coprocessor scan or not
370    * @return
371    */
372   protected def configureScans(
373       tables: Seq[TableName],
374       ranges: Seq[RowRange],
375       small: Boolean,
376       colFamily: Array[Byte],
377       filters: Seq[HFilter],
378       coprocessor: Boolean): Seq[TableScan] = {
379     val cacheBlocks = HBaseSystemProperties.ScannerBlockCaching.toBoolean.get // has a default value so .get is safe
380     val cacheSize = HBaseSystemProperties.ScannerCaching.toInt
381 
382     logger.debug(s"HBase client scanner: block caching: $cacheBlocks, caching: $cacheSize")
383 
384     if (small && !coprocessor) {
385       val filter = filters match {
386         case Nil    => None
387         case Seq(f) => Some(f)
388         case f      => Some(new FilterList(f: _*))
389       }
390       // note: we have to copy the ranges for each table scan
391       tables.map { table =>
392         val scans = ranges.map { r =>
393           val scan = new Scan(r.getStartRow, r.getStopRow)
394           scan.addFamily(colFamily).setCacheBlocks(cacheBlocks).setSmall(true)
395           filter.foreach(scan.setFilter)
396           cacheSize.foreach(scan.setCaching)
397           ds.applySecurity(scan)
398           scan
399         }
400         TableScan(table, scans)
401       }
402     } else {
403       // split and group ranges by region server
404       // note: we have to copy the ranges for each table scan anyway
405       val rangesPerTable: Seq[(TableName, collection.Map[String, java.util.List[RowRange]])] =
406         tables.map(t => t -> groupRangesByRegion(t, ranges))
407 
408       def createGroup(group: java.util.List[RowRange]): Scan = {
409         val scan = new Scan(group.get(0).getStartRow, group.get(group.size() - 1).getStopRow)
410         val mrrf = if (group.size() < 2) { filters } else {
411           // TODO GEOMESA-1806
412           // currently, the MultiRowRangeFilter constructor will call sortAndMerge a second time
413           // this is unnecessary as we have already sorted and merged
414           // note: mrrf first priority
415           filters.+:(new MultiRowRangeFilter(group))
416         }
417         scan.setFilter(if (mrrf.lengthCompare(1) > 0) { new FilterList(mrrf: _*) } else { mrrf.headOption.orNull })
418         scan.addFamily(colFamily).setCacheBlocks(cacheBlocks)
419         cacheSize.foreach(scan.setCaching)
420 
421         // apply visibilities
422         ds.applySecurity(scan)
423 
424         scan
425       }
426 
427       rangesPerTable.map { case (table, rangesPerRegion) =>
428         val maxRangesPerGroup = {
429           def calcMax(maxPerGroup: Int, threads: Int): Int = {
430             val totalRanges = rangesPerRegion.values.map(_.size).sum
431             math.min(maxPerGroup, math.max(1, math.ceil(totalRanges.toDouble / threads).toInt))
432           }
433           if (coprocessor) {
434             calcMax(ds.config.coprocessors.maxRangesPerExtendedScan, ds.config.coprocessors.threads)
435           } else {
436             calcMax(ds.config.queries.maxRangesPerExtendedScan, ds.config.queries.threads)
437           }
438         }
439 
440         val groupedScans = Seq.newBuilder[Scan]
441 
442         rangesPerRegion.foreach { case (_, list) =>
443           // our ranges are non-overlapping, so just sort them but don't bother merging them
444           Collections.sort(list)
445 
446           var i = 0
447           while (i < list.size()) {
448             val groupSize = math.min(maxRangesPerGroup, list.size() - i)
449             groupedScans += createGroup(list.subList(i, i + groupSize))
450             i += groupSize
451           }
452         }
453 
454         // shuffle the ranges, otherwise our threads will tend to all hit the same region server at once
455         TableScan(table, Random.shuffle(groupedScans.result))
456       }
457     }
458   }
459 
460   /**
461    * Split and group ranges by region server
462    *
463    * @param table table being scanned
464    * @param ranges ranges to group
465    * @return
466    */
467   private def groupRangesByRegion(
468       table: TableName,
469       ranges: Seq[RowRange]): scala.collection.Map[String, java.util.List[RowRange]] = {
470     val rangesPerRegion = scala.collection.mutable.Map.empty[String, java.util.List[RowRange]]
471     WithClose(ds.connection.getRegionLocator(table)) { locator =>
472       ranges.foreach(groupRange(locator, _, rangesPerRegion))
473     }
474     rangesPerRegion
475   }
476 
477   /**
478    * Group the range based on the region server hosting it. Splits ranges as needed if they span
479    * more than one region
480    *
481    * @param locator region locator
482    * @param range range to group
483    * @param result collected results
484    */
485   @scala.annotation.tailrec
486   private def groupRange(
487       locator: RegionLocator,
488       range: RowRange,
489       result: scala.collection.mutable.Map[String, java.util.List[RowRange]]): Unit = {
490     var encodedName: String = null
491     var split: Array[Byte] = null
492     try {
493       val regionInfo = locator.getRegionLocation(range.getStartRow).getRegionInfo
494       encodedName = regionInfo.getEncodedName
495       val regionEndKey = regionInfo.getEndKey // note: this is exclusive
496       if (regionEndKey.nonEmpty &&
497           (range.getStopRow.isEmpty || ByteArrays.ByteOrdering.compare(regionEndKey, range.getStopRow) <= 0)) {
498         if (ByteArrays.ByteOrdering.compare(range.getStartRow, regionEndKey) < 0) {
499           split = regionEndKey
500         } else {
501           logger.warn(s"HBase region location does not correspond to requested range:\n" +
502               s"  requested row: ${ByteArrays.toHex(range.getStartRow)}\n" +
503               s"  region: $encodedName ${ByteArrays.toHex(regionInfo.getStartKey)} :: ${ByteArrays.toHex(regionEndKey)}")
504         }
505       }
506     } catch {
507       case NonFatal(e) => logger.warn(s"Error checking range location for '$range''", e)
508     }
509     val buffer = result.getOrElseUpdate(encodedName, new java.util.ArrayList())
510     if (split == null) {
511       buffer.add(range)
512     } else {
513       // split the range based on the current region
514       buffer.add(new RowRange(range.getStartRow, true, split, false))
515       groupRange(locator, new RowRange(split, true, range.getStopRow, false), result)
516     }
517   }
518 }
519 
520 object HBaseIndexAdapter extends LazyLogging {
521 
522   private val distributedJarNamePattern = Pattern.compile("^geomesa-hbase-distributed-runtime.*\\.jar$")
523 
524   // these are in the geomesa-hbase-server module, so not accessible directly
525   val CoprocessorClass = "org.locationtech.geomesa.hbase.server.coprocessor.GeoMesaCoprocessor"
526   val AggregatorPackage = "org.locationtech.geomesa.hbase.server.common"
527 
528   val durability: Durability = HBaseSystemProperties.WalDurability.option match {
529     case Some(value) =>
530       Durability.values.find(_.toString.equalsIgnoreCase(value)).getOrElse {
531         logger.error(s"Invalid HBase WAL durability setting: $value. Falling back to default durability")
532         Durability.USE_DEFAULT
533       }
534     case None => Durability.USE_DEFAULT
535   }
536 
537   /**
538    * Waits for a table to come online after being created
539    *
540    * @param admin hbase admin
541    * @param table table name
542    */
543   def waitForTable(admin: Admin, table: TableName): Unit = {
544     if (!admin.isTableAvailable(table)) {
545       val timeout = TableAvailabilityTimeout.toUnboundedDuration.filter(_.isFinite)
546       logger.debug(s"Waiting for table '$table' to become available with " +
547           s"${timeout.map(t => s"a timeout of $t").getOrElse("no timeout")}")
548       val stop = timeout.map(t => System.currentTimeMillis() + t.toMillis)
549       while (!admin.isTableAvailable(table) && stop.forall(_ > System.currentTimeMillis())) {
550         Thread.sleep(1000)
551       }
552     }
553   }
554 
555   /**
556     * Deserializes row bytes into simple features
557     *
558     * @param _index index
559     * @param _sft sft
560     */
561   class HBaseResultsToFeatures(_index: GeoMesaFeatureIndex[_, _], _sft: SimpleFeatureType) extends
562     IndexResultsToFeatures[Result](_index, _sft) {
563 
564     def this() = this(null, null) // no-arg constructor required for serialization
565 
566     override def apply(result: Result): SimpleFeature = {
567       val cell = result.rawCells()(0)
568       val id = index.getIdFromRow(cell.getRowArray, cell.getRowOffset, cell.getRowLength, null)
569       serializer.deserialize(id, cell.getValueArray, cell.getValueOffset, cell.getValueLength)
570     }
571   }
572 
573   /**
574     * Writer for hbase
575     *
576     * @param ds datastore
577     * @param indices indices to write to
578     * @param partition partition to write to
579     */
580   class HBaseIndexWriter(
581       ds: HBaseDataStore,
582       indices: Seq[GeoMesaFeatureIndex[_, _]],
583       wrapper: FeatureWrapper[WritableFeature],
584       partition: Option[String]
585     ) extends BaseIndexWriter(indices, wrapper) {
586 
587     import org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType
588 
589     private val batchSize = HBaseSystemProperties.WriteBatchSize.toLong
590     private val flushTimeout = HBaseSystemProperties.WriteFlushTimeout.toLong
591     private val deleteVis = HBaseSystemProperties.DeleteVis.option.map(new CellVisibility(_))
592 
593     private val pools = {
594       // mimic config from default hbase connection
595       val maxThreads = math.max(1, ds.connection.getConfiguration.getInt("hbase.htable.threads.max", Int.MaxValue))
596       Array.fill(indices.length)(new CachedThreadPool(maxThreads))
597     }
598 
599     private val mutators = indices.toArray.map { index =>
600       // should always be writing to a single table here
601       val table = index.getTableName(partition)
602       val params = new BufferedMutatorParams(TableName.valueOf(table))
603       batchSize.foreach(params.writeBufferSize)
604       flushTimeout.foreach(params.setWriteBufferPeriodicFlushTimeoutMs)
605 
606       // We have to pass a pool explicitly and close it after manually,
607       // cause of HBase issue where pools got leaked and never closed
608       // (in case of long running Spark jobs 24+ hours the workers go out of memory without custom pool)
609       params.pool(pools(indices.indexOf(index)))
610       ds.connection.getBufferedMutator(params)
611     }
612 
613     private val expiration = indices.headOption.flatMap(_.sft.getFeatureExpiration).orNull
614 
615     private var i = 0
616 
617     override protected def append(feature: WritableFeature, values: Array[RowKeyValue[_]]): Unit = {
618       val ttl = if (expiration != null) {
619         val t = expiration.expires(feature.feature) - System.currentTimeMillis
620         if (t > 0) {
621           t
622         }
623         else {
624           logger.warn("Feature is already past its TTL; not added to database")
625           return
626         }
627       } else {
628         0L
629       }
630 
631       i = 0
632       while (i < values.length) {
633         val mutator = mutators(i)
634         values(i) match {
635           case kv: SingleRowKeyValue[_] =>
636             kv.values.foreach { value =>
637               val put = new Put(kv.row)
638               put.addImmutable(value.cf, value.cq, value.value)
639               if (!value.vis.isEmpty) {
640                 put.setCellVisibility(new CellVisibility(new String(value.vis, StandardCharsets.UTF_8)))
641               }
642               put.setDurability(durability)
643               if (ttl > 0) put.setTTL(ttl)
644               mutator.mutate(put)
645             }
646 
647           case mkv: MultiRowKeyValue[_] =>
648             mkv.rows.foreach { row =>
649               mkv.values.foreach { value =>
650                 val put = new Put(row)
651                 put.addImmutable(value.cf, value.cq, value.value)
652                 if (!value.vis.isEmpty) {
653                   put.setCellVisibility(new CellVisibility(new String(value.vis, StandardCharsets.UTF_8)))
654                 }
655                 put.setDurability(durability)
656                 if (ttl > 0) put.setTTL(ttl)
657                 mutator.mutate(put)
658               }
659             }
660         }
661         i += 1
662       }
663     }
664 
665     override protected def update(
666         feature: WritableFeature,
667         values: Array[RowKeyValue[_]],
668         previous: WritableFeature,
669         previousValues: Array[RowKeyValue[_]]): Unit = {
670       delete(previous, previousValues)
671       // for updates, ensure that our timestamps don't clobber each other
672       flush()
673       Thread.sleep(1)
674       append(feature, values)
675     }
676 
677     override protected def delete(feature: WritableFeature, values: Array[RowKeyValue[_]]): Unit = {
678       i = 0
679       while (i < values.length) {
680         val mutator = mutators(i)
681         values(i) match {
682           case kv: SingleRowKeyValue[_] =>
683             kv.values.foreach { value =>
684               val del = new Delete(kv.row)
685               del.addFamily(value.cf) // note: passing in the column qualifier seems to keep deletes from working
686               if (!value.vis.isEmpty) {
687                 del.setCellVisibility(new CellVisibility(new String(value.vis, StandardCharsets.UTF_8)))
688               } else {
689                 deleteVis.foreach(del.setCellVisibility)
690               }
691               mutator.mutate(del)
692             }
693 
694           case mkv: MultiRowKeyValue[_] =>
695             mkv.rows.foreach { row =>
696               mkv.values.foreach { value =>
697                 val del = new Delete(row)
698                 del.addFamily(value.cf) // note: passing in the column qualifier seems to keep deletes from working
699                 if (!value.vis.isEmpty) {
700                   del.setCellVisibility(new CellVisibility(new String(value.vis, StandardCharsets.UTF_8)))
701                 } else {
702                   deleteVis.foreach(del.setCellVisibility)
703                 }
704                 mutator.mutate(del)
705               }
706             }
707         }
708         i += 1
709       }
710     }
711 
712     override def flush(): Unit = FlushWithLogging.raise(mutators)(BufferedMutatorIsFlushable.arrayIsFlushable)
713 
714     override def close(): Unit = {
715       try { CloseWithLogging.raise(mutators) } finally {
716         pools.foreach(CloseWithLogging(_)(IsCloseable.executorServiceIsCloseable))
717       }
718       if (!pools.foldLeft(true) { case (terminated, pool) => terminated && pool.awaitTermination(60, TimeUnit.SECONDS) }) {
719         logger.warn("Failed to terminate thread pool after 60 seconds")
720       }
721     }
722   }
723 
724   object BufferedMutatorIsFlushable extends IsFlushableImplicits[BufferedMutator] {
725     override protected def flush(f: BufferedMutator): Try[Unit] = Try(f.flush())
726   }
727 
728 }
Line Stmt Id Pos Tree Symbol Tests Code
75 89951 4306 - 4320 Select org.locationtech.geomesa.index.api.IndexAdapter.tableNameLimit HBaseIndexAdapter.this.tableNameLimit
75 89953 4252 - 4322 Apply org.apache.hadoop.hbase.TableName.valueOf org.apache.hadoop.hbase.TableName.valueOf(index.configureTableName(partition, HBaseIndexAdapter.this.tableNameLimit))
75 89952 4270 - 4321 Apply org.locationtech.geomesa.index.api.GeoMesaFeatureIndex.configureTableName index.configureTableName(partition, HBaseIndexAdapter.this.tableNameLimit)
77 89954 4338 - 4360 Apply org.apache.hadoop.hbase.client.Connection.getAdmin HBaseIndexAdapter.this.ds.connection.getAdmin()
77 90003 4328 - 7147 ApplyToImplicitArgs org.locationtech.geomesa.utils.io.WithClose.apply org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.hbase.client.Admin, Unit](HBaseIndexAdapter.this.ds.connection.getAdmin())(((admin: org.apache.hadoop.hbase.client.Admin) => { if (admin.tableExists(name).unary_!) { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Creating table {}", (name: AnyRef)) else (): Unit); val conf: org.apache.hadoop.conf.Configuration = admin.getConfiguration(); val compression: Option[org.apache.hadoop.hbase.io.compress.Compression.Algorithm] = org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType(index.sft).getCompression.map[org.apache.hadoop.hbase.io.compress.Compression.Algorithm](((alg: String) => { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Setting compression \'{}\' on table {} for feature {}", (alg: AnyRef), (name: AnyRef), (index.sft.getTypeName(): AnyRef)) else (): Unit); org.apache.hadoop.hbase.io.compress.Compression.getCompressionAlgorithmByName(alg.toLowerCase(java.util.Locale.US)) })); val cols: Seq[Array[Byte]] = HBaseIndexAdapter.this.groups.apply(index.sft).map[Array[Byte], Seq[Array[Byte]]](((x$1: (Array[Byte], org.geotools.api.feature.simple.SimpleFeatureType)) => x$1._1))(collection.this.Seq.canBuildFrom[Array[Byte]]); val bloom: Some[org.apache.hadoop.hbase.regionserver.BloomType] = scala.Some.apply[org.apache.hadoop.hbase.regionserver.BloomType](NONE); val encoding: Option[org.apache.hadoop.hbase.io.encoding.DataBlockEncoding] = if (index.name.==(org.locationtech.geomesa.index.index.id.IdIndex.name)) scala.None else scala.Some.apply[org.apache.hadoop.hbase.io.encoding.DataBlockEncoding](FAST_DIFF); val coprocessor: Option[(String, Option[org.apache.hadoop.fs.Path])] = if (HBaseIndexAdapter.this.ds.config.remoteFilter.unary_!) scala.None else { def urlFromSysProp: Option[org.apache.hadoop.fs.Path] = org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorUrl.option.orElse[String](org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorPath.option).map[org.apache.hadoop.fs.Path](((x$2: String) => new org.apache.hadoop.fs.Path(x$2))); <stable> <accessor> lazy val coprocessorUrl: Option[org.apache.hadoop.fs.Path] = HBaseIndexAdapter.this.ds.config.coprocessors.url.orElse[org.apache.hadoop.fs.Path](urlFromSysProp).orElse[org.apache.hadoop.fs.Path](try { val dir: org.apache.hadoop.fs.Path = new org.apache.hadoop.fs.Path(conf.get("hbase.dynamic.jars.dir")); org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.fs.FileSystem, Option[org.apache.hadoop.fs.Path]](dir.getFileSystem(conf))(((fs: org.apache.hadoop.fs.FileSystem) => if (fs.isDirectory(dir).unary_!) scala.None else scala.Predef.refArrayOps[org.apache.hadoop.fs.FileStatus](fs.listStatus(dir)).collectFirst[org.apache.hadoop.fs.Path](({ @SerialVersionUID(value = 0) final <synthetic> class $anonfun extends scala.runtime.AbstractPartialFunction[org.apache.hadoop.fs.FileStatus,org.apache.hadoop.fs.Path] with Serializable { def <init>(): <$anon: org.apache.hadoop.fs.FileStatus => org.apache.hadoop.fs.Path> = { $anonfun.super.<init>(); () }; final override def applyOrElse[A1 <: org.apache.hadoop.fs.FileStatus, B1 >: org.apache.hadoop.fs.Path](x1: A1, default: A1 => B1): B1 = ((x1.asInstanceOf[org.apache.hadoop.fs.FileStatus]: org.apache.hadoop.fs.FileStatus): org.apache.hadoop.fs.FileStatus @unchecked) match { case (s @ _) if HBaseIndexAdapter.distributedJarNamePattern.matcher(s.getPath().getName()).matches() => s.getPath() case (defaultCase$ @ _) => default.apply(x1) }; final def isDefinedAt(x1: org.apache.hadoop.fs.FileStatus): Boolean = ((x1.asInstanceOf[org.apache.hadoop.fs.FileStatus]: org.apache.hadoop.fs.FileStatus): org.apache.hadoop.fs.FileStatus @unchecked) match { case (s @ _) if HBaseIndexAdapter.distributedJarNamePattern.matcher(s.getPath().getName()).matches() => true case (defaultCase$ @ _) => false } }; new $anonfun() }: PartialFunction[org.apache.hadoop.fs.FileStatus,org.apache.hadoop.fs.Path]))))(io.this.IsCloseable.closeableIsCloseable) } catch { case scala.util.control.NonFatal.unapply(<unapply-selector>) <unapply> ((e @ _)) => { (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn("Error checking dynamic jar path:", e) else (): Unit); scala.None } }); val installed: Option[String] = scala.Option.apply[String](conf.get("hbase.coprocessor.user.region.classes")); val names: scala.collection.immutable.Set[String] = installed.map[scala.collection.immutable.Set[String]](((x$3: String) => scala.Predef.refArrayOps[String](x$3.split(":")).toSet[String])).getOrElse[scala.collection.immutable.Set[String]](scala.Predef.Set.empty[String]); if (names.contains(HBaseIndexAdapter.CoprocessorClass)) scala.None else { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Using coprocessor path {}", coprocessorUrl.orNull[Any](scala.Predef.$conforms[Null]).asInstanceOf[AnyRef]) else (): Unit); scala.Some.apply[(String, Option[org.apache.hadoop.fs.Path])](scala.Predef.ArrowAssoc[String](HBaseIndexAdapter.CoprocessorClass).->[Option[org.apache.hadoop.fs.Path]](coprocessorUrl)) } }; try { org.locationtech.geomesa.hbase.utils.HBaseVersions.createTableAsync(admin, name, cols, bloom, compression, encoding, scala.None, coprocessor, splits) } catch { case (_: org.apache.hadoop.hbase.TableExistsException) => () } } else (); HBaseIndexAdapter.waitForTable(admin, name) }))(io.this.IsCloseable.closeableIsCloseable)
77 90002 4362 - 4362 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.closeableIsCloseable io.this.IsCloseable.closeableIsCloseable
78 89955 4383 - 4407 Select scala.Boolean.unary_! admin.tableExists(name).unary_!
78 89999 4379 - 4379 Literal <nosymbol> ()
78 89998 4409 - 7108 Block <nosymbol> { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Creating table {}", (name: AnyRef)) else (): Unit); val conf: org.apache.hadoop.conf.Configuration = admin.getConfiguration(); val compression: Option[org.apache.hadoop.hbase.io.compress.Compression.Algorithm] = org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType(index.sft).getCompression.map[org.apache.hadoop.hbase.io.compress.Compression.Algorithm](((alg: String) => { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Setting compression \'{}\' on table {} for feature {}", (alg: AnyRef), (name: AnyRef), (index.sft.getTypeName(): AnyRef)) else (): Unit); org.apache.hadoop.hbase.io.compress.Compression.getCompressionAlgorithmByName(alg.toLowerCase(java.util.Locale.US)) })); val cols: Seq[Array[Byte]] = HBaseIndexAdapter.this.groups.apply(index.sft).map[Array[Byte], Seq[Array[Byte]]](((x$1: (Array[Byte], org.geotools.api.feature.simple.SimpleFeatureType)) => x$1._1))(collection.this.Seq.canBuildFrom[Array[Byte]]); val bloom: Some[org.apache.hadoop.hbase.regionserver.BloomType] = scala.Some.apply[org.apache.hadoop.hbase.regionserver.BloomType](NONE); val encoding: Option[org.apache.hadoop.hbase.io.encoding.DataBlockEncoding] = if (index.name.==(org.locationtech.geomesa.index.index.id.IdIndex.name)) scala.None else scala.Some.apply[org.apache.hadoop.hbase.io.encoding.DataBlockEncoding](FAST_DIFF); val coprocessor: Option[(String, Option[org.apache.hadoop.fs.Path])] = if (HBaseIndexAdapter.this.ds.config.remoteFilter.unary_!) scala.None else { def urlFromSysProp: Option[org.apache.hadoop.fs.Path] = org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorUrl.option.orElse[String](org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorPath.option).map[org.apache.hadoop.fs.Path](((x$2: String) => new org.apache.hadoop.fs.Path(x$2))); <stable> <accessor> lazy val coprocessorUrl: Option[org.apache.hadoop.fs.Path] = HBaseIndexAdapter.this.ds.config.coprocessors.url.orElse[org.apache.hadoop.fs.Path](urlFromSysProp).orElse[org.apache.hadoop.fs.Path](try { val dir: org.apache.hadoop.fs.Path = new org.apache.hadoop.fs.Path(conf.get("hbase.dynamic.jars.dir")); org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.fs.FileSystem, Option[org.apache.hadoop.fs.Path]](dir.getFileSystem(conf))(((fs: org.apache.hadoop.fs.FileSystem) => if (fs.isDirectory(dir).unary_!) scala.None else scala.Predef.refArrayOps[org.apache.hadoop.fs.FileStatus](fs.listStatus(dir)).collectFirst[org.apache.hadoop.fs.Path](({ @SerialVersionUID(value = 0) final <synthetic> class $anonfun extends scala.runtime.AbstractPartialFunction[org.apache.hadoop.fs.FileStatus,org.apache.hadoop.fs.Path] with Serializable { def <init>(): <$anon: org.apache.hadoop.fs.FileStatus => org.apache.hadoop.fs.Path> = { $anonfun.super.<init>(); () }; final override def applyOrElse[A1 <: org.apache.hadoop.fs.FileStatus, B1 >: org.apache.hadoop.fs.Path](x1: A1, default: A1 => B1): B1 = ((x1.asInstanceOf[org.apache.hadoop.fs.FileStatus]: org.apache.hadoop.fs.FileStatus): org.apache.hadoop.fs.FileStatus @unchecked) match { case (s @ _) if HBaseIndexAdapter.distributedJarNamePattern.matcher(s.getPath().getName()).matches() => s.getPath() case (defaultCase$ @ _) => default.apply(x1) }; final def isDefinedAt(x1: org.apache.hadoop.fs.FileStatus): Boolean = ((x1.asInstanceOf[org.apache.hadoop.fs.FileStatus]: org.apache.hadoop.fs.FileStatus): org.apache.hadoop.fs.FileStatus @unchecked) match { case (s @ _) if HBaseIndexAdapter.distributedJarNamePattern.matcher(s.getPath().getName()).matches() => true case (defaultCase$ @ _) => false } }; new $anonfun() }: PartialFunction[org.apache.hadoop.fs.FileStatus,org.apache.hadoop.fs.Path]))))(io.this.IsCloseable.closeableIsCloseable) } catch { case scala.util.control.NonFatal.unapply(<unapply-selector>) <unapply> ((e @ _)) => { (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn("Error checking dynamic jar path:", e) else (): Unit); scala.None } }); val installed: Option[String] = scala.Option.apply[String](conf.get("hbase.coprocessor.user.region.classes")); val names: scala.collection.immutable.Set[String] = installed.map[scala.collection.immutable.Set[String]](((x$3: String) => scala.Predef.refArrayOps[String](x$3.split(":")).toSet[String])).getOrElse[scala.collection.immutable.Set[String]](scala.Predef.Set.empty[String]); if (names.contains(HBaseIndexAdapter.CoprocessorClass)) scala.None else { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Using coprocessor path {}", coprocessorUrl.orNull[Any](scala.Predef.$conforms[Null]).asInstanceOf[AnyRef]) else (): Unit); scala.Some.apply[(String, Option[org.apache.hadoop.fs.Path])](scala.Predef.ArrowAssoc[String](HBaseIndexAdapter.CoprocessorClass).->[Option[org.apache.hadoop.fs.Path]](coprocessorUrl)) } }; try { org.locationtech.geomesa.hbase.utils.HBaseVersions.createTableAsync(admin, name, cols, bloom, compression, encoding, scala.None, coprocessor, splits) } catch { case (_: org.apache.hadoop.hbase.TableExistsException) => () } }
78 90000 4379 - 4379 Block <nosymbol> ()
81 89956 4477 - 4499 Apply org.apache.hadoop.hbase.client.Admin.getConfiguration admin.getConfiguration()
83 89957 4527 - 4536 Select org.locationtech.geomesa.index.api.GeoMesaFeatureIndex.sft index.sft
83 89961 4527 - 4845 Apply scala.Option.map org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType(index.sft).getCompression.map[org.apache.hadoop.hbase.io.compress.Compression.Algorithm](((alg: String) => { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Setting compression \'{}\' on table {} for feature {}", (alg: AnyRef), (name: AnyRef), (index.sft.getTypeName(): AnyRef)) else (): Unit); org.apache.hadoop.hbase.io.compress.Compression.getCompressionAlgorithmByName(alg.toLowerCase(java.util.Locale.US)) }))
86 89959 4808 - 4834 Apply java.lang.String.toLowerCase alg.toLowerCase(java.util.Locale.US)
86 89958 4824 - 4833 Select java.util.Locale.US java.util.Locale.US
86 89960 4766 - 4835 Apply org.apache.hadoop.hbase.io.compress.Compression.getCompressionAlgorithmByName org.apache.hadoop.hbase.io.compress.Compression.getCompressionAlgorithmByName(alg.toLowerCase(java.util.Locale.US))
89 89963 4894 - 4898 Select scala.Tuple2._1 x$1._1
89 89962 4879 - 4888 Select org.locationtech.geomesa.index.api.GeoMesaFeatureIndex.sft index.sft
89 89965 4866 - 4899 ApplyToImplicitArgs scala.collection.TraversableLike.map HBaseIndexAdapter.this.groups.apply(index.sft).map[Array[Byte], Seq[Array[Byte]]](((x$1: (Array[Byte], org.geotools.api.feature.simple.SimpleFeatureType)) => x$1._1))(collection.this.Seq.canBuildFrom[Array[Byte]])
89 89964 4893 - 4893 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[Array[Byte]]
90 89966 4920 - 4940 Apply scala.Some.apply scala.Some.apply[org.apache.hadoop.hbase.regionserver.BloomType](NONE)
91 89967 4982 - 4994 Select org.locationtech.geomesa.index.index.id.IdIndex.name org.locationtech.geomesa.index.index.id.IdIndex.name
91 89969 4998 - 5002 Select scala.None scala.None
91 89968 4968 - 4994 Apply java.lang.Object.== index.name.==(org.locationtech.geomesa.index.index.id.IdIndex.name)
91 89971 5012 - 5045 Apply scala.Some.apply scala.Some.apply[org.apache.hadoop.hbase.io.encoding.DataBlockEncoding](FAST_DIFF)
91 89970 4998 - 5002 Block scala.None scala.None
91 89972 5012 - 5045 Block scala.Some.apply scala.Some.apply[org.apache.hadoop.hbase.io.encoding.DataBlockEncoding](FAST_DIFF)
94 89973 5120 - 5143 Select scala.Boolean.unary_! HBaseIndexAdapter.this.ds.config.remoteFilter.unary_!
94 89975 5147 - 5151 Block scala.None scala.None
94 89974 5147 - 5151 Select scala.None scala.None
94 89992 5159 - 6830 Block <nosymbol> { def urlFromSysProp: Option[org.apache.hadoop.fs.Path] = org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorUrl.option.orElse[String](org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorPath.option).map[org.apache.hadoop.fs.Path](((x$2: String) => new org.apache.hadoop.fs.Path(x$2))); <stable> <accessor> lazy val coprocessorUrl: Option[org.apache.hadoop.fs.Path] = HBaseIndexAdapter.this.ds.config.coprocessors.url.orElse[org.apache.hadoop.fs.Path](urlFromSysProp).orElse[org.apache.hadoop.fs.Path](try { val dir: org.apache.hadoop.fs.Path = new org.apache.hadoop.fs.Path(conf.get("hbase.dynamic.jars.dir")); org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.fs.FileSystem, Option[org.apache.hadoop.fs.Path]](dir.getFileSystem(conf))(((fs: org.apache.hadoop.fs.FileSystem) => if (fs.isDirectory(dir).unary_!) scala.None else scala.Predef.refArrayOps[org.apache.hadoop.fs.FileStatus](fs.listStatus(dir)).collectFirst[org.apache.hadoop.fs.Path](({ @SerialVersionUID(value = 0) final <synthetic> class $anonfun extends scala.runtime.AbstractPartialFunction[org.apache.hadoop.fs.FileStatus,org.apache.hadoop.fs.Path] with Serializable { def <init>(): <$anon: org.apache.hadoop.fs.FileStatus => org.apache.hadoop.fs.Path> = { $anonfun.super.<init>(); () }; final override def applyOrElse[A1 <: org.apache.hadoop.fs.FileStatus, B1 >: org.apache.hadoop.fs.Path](x1: A1, default: A1 => B1): B1 = ((x1.asInstanceOf[org.apache.hadoop.fs.FileStatus]: org.apache.hadoop.fs.FileStatus): org.apache.hadoop.fs.FileStatus @unchecked) match { case (s @ _) if HBaseIndexAdapter.distributedJarNamePattern.matcher(s.getPath().getName()).matches() => s.getPath() case (defaultCase$ @ _) => default.apply(x1) }; final def isDefinedAt(x1: org.apache.hadoop.fs.FileStatus): Boolean = ((x1.asInstanceOf[org.apache.hadoop.fs.FileStatus]: org.apache.hadoop.fs.FileStatus): org.apache.hadoop.fs.FileStatus @unchecked) match { case (s @ _) if HBaseIndexAdapter.distributedJarNamePattern.matcher(s.getPath().getName()).matches() => true case (defaultCase$ @ _) => false } }; new $anonfun() }: PartialFunction[org.apache.hadoop.fs.FileStatus,org.apache.hadoop.fs.Path]))))(io.this.IsCloseable.closeableIsCloseable) } catch { case scala.util.control.NonFatal.unapply(<unapply-selector>) <unapply> ((e @ _)) => { (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn("Error checking dynamic jar path:", e) else (): Unit); scala.None } }); val installed: Option[String] = scala.Option.apply[String](conf.get("hbase.coprocessor.user.region.classes")); val names: scala.collection.immutable.Set[String] = installed.map[scala.collection.immutable.Set[String]](((x$3: String) => scala.Predef.refArrayOps[String](x$3.split(":")).toSet[String])).getOrElse[scala.collection.immutable.Set[String]](scala.Predef.Set.empty[String]); if (names.contains(HBaseIndexAdapter.CoprocessorClass)) scala.None else { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Using coprocessor path {}", coprocessorUrl.orNull[Any](scala.Predef.$conforms[Null]).asInstanceOf[AnyRef]) else (): Unit); scala.Some.apply[(String, Option[org.apache.hadoop.fs.Path])](scala.Predef.ArrowAssoc[String](HBaseIndexAdapter.CoprocessorClass).->[Option[org.apache.hadoop.fs.Path]](coprocessorUrl)) } }
95 89977 5263 - 5274 Apply org.apache.hadoop.fs.Path.<init> new org.apache.hadoop.fs.Path(x$2)
95 89976 5235 - 5257 Select org.locationtech.geomesa.utils.conf.GeoMesaSystemProperties.SystemProperty.option org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorPath.option
95 89978 5206 - 5275 Apply scala.Option.map org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorUrl.option.orElse[String](org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.CoprocessorPath.option).map[org.apache.hadoop.fs.Path](((x$2: String) => new org.apache.hadoop.fs.Path(x$2)))
114 89979 6323 - 6372 Apply org.apache.hadoop.conf.Configuration.get conf.get("hbase.coprocessor.user.region.classes")
114 89980 6316 - 6373 Apply scala.Option.apply scala.Option.apply[String](conf.get("hbase.coprocessor.user.region.classes"))
115 89981 6410 - 6422 Apply java.lang.String.split x$3.split(":")
115 89983 6440 - 6457 TypeApply scala.collection.generic.ImmutableSetFactory.empty scala.Predef.Set.empty[String]
115 89982 6410 - 6428 TypeApply scala.collection.TraversableOnce.toSet scala.Predef.refArrayOps[String](x$3.split(":")).toSet[String]
115 89984 6396 - 6458 Apply scala.Option.getOrElse installed.map[scala.collection.immutable.Set[String]](((x$3: String) => scala.Predef.refArrayOps[String](x$3.split(":")).toSet[String])).getOrElse[scala.collection.immutable.Set[String]](scala.Predef.Set.empty[String])
116 89985 6488 - 6504 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.CoprocessorClass HBaseIndexAdapter.CoprocessorClass
116 89987 6509 - 6513 Select scala.None scala.None
116 89986 6473 - 6505 Apply scala.collection.SetLike.contains names.contains(HBaseIndexAdapter.CoprocessorClass)
116 89988 6509 - 6513 Block scala.None scala.None
116 89991 6521 - 6820 Block <nosymbol> { (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug("Using coprocessor path {}", coprocessorUrl.orNull[Any](scala.Predef.$conforms[Null]).asInstanceOf[AnyRef]) else (): Unit); scala.Some.apply[(String, Option[org.apache.hadoop.fs.Path])](scala.Predef.ArrowAssoc[String](HBaseIndexAdapter.CoprocessorClass).->[Option[org.apache.hadoop.fs.Path]](coprocessorUrl)) }
120 89989 6773 - 6807 Apply scala.Predef.ArrowAssoc.-> scala.Predef.ArrowAssoc[String](HBaseIndexAdapter.CoprocessorClass).->[Option[org.apache.hadoop.fs.Path]](coprocessorUrl)
120 89990 6768 - 6808 Apply scala.Some.apply scala.Some.apply[(String, Option[org.apache.hadoop.fs.Path])](scala.Predef.ArrowAssoc[String](HBaseIndexAdapter.CoprocessorClass).->[Option[org.apache.hadoop.fs.Path]](coprocessorUrl))
125 89993 6936 - 6940 Select scala.None scala.None
125 89995 6856 - 6962 Block org.locationtech.geomesa.hbase.utils.HBaseVersions.createTableAsync org.locationtech.geomesa.hbase.utils.HBaseVersions.createTableAsync(admin, name, cols, bloom, compression, encoding, scala.None, coprocessor, splits)
125 89994 6856 - 6962 Apply org.locationtech.geomesa.hbase.utils.HBaseVersions.createTableAsync org.locationtech.geomesa.hbase.utils.HBaseVersions.createTableAsync(admin, name, cols, bloom, compression, encoding, scala.None, coprocessor, splits)
127 89997 7044 - 7046 Block <nosymbol> ()
127 89996 7044 - 7046 Literal <nosymbol> ()
131 90001 7116 - 7141 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.waitForTable HBaseIndexAdapter.waitForTable(admin, name)
136 90004 7230 - 7252 Apply org.apache.hadoop.hbase.client.Connection.getAdmin HBaseIndexAdapter.this.ds.connection.getAdmin()
136 90019 7254 - 7254 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.closeableIsCloseable io.this.IsCloseable.closeableIsCloseable
136 90020 7220 - 7836 ApplyToImplicitArgs org.locationtech.geomesa.utils.io.WithClose.apply org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.hbase.client.Admin, Unit](HBaseIndexAdapter.this.ds.connection.getAdmin())(((admin: org.apache.hadoop.hbase.client.Admin) => { val existing: org.apache.hadoop.hbase.TableName = org.apache.hadoop.hbase.TableName.valueOf(from); val renamed: org.apache.hadoop.hbase.TableName = org.apache.hadoop.hbase.TableName.valueOf(to); if (admin.tableExists(existing)) { val snapshot: String = org.locationtech.geomesa.utils.text.StringSerialization.alphaNumericSafeString(java.util.UUID.randomUUID().toString()); admin.disableTable(existing); admin.snapshot(snapshot, existing); admin.cloneSnapshot(snapshot, renamed); admin.deleteSnapshot(snapshot); admin.deleteTable(existing); HBaseIndexAdapter.waitForTable(admin, renamed) } else () }))(io.this.IsCloseable.closeableIsCloseable)
137 90005 7286 - 7309 Apply org.apache.hadoop.hbase.TableName.valueOf org.apache.hadoop.hbase.TableName.valueOf(from)
138 90006 7330 - 7351 Apply org.apache.hadoop.hbase.TableName.valueOf org.apache.hadoop.hbase.TableName.valueOf(to)
139 90007 7362 - 7389 Apply org.apache.hadoop.hbase.client.Admin.tableExists admin.tableExists(existing)
139 90017 7358 - 7358 Literal <nosymbol> ()
139 90016 7391 - 7830 Block <nosymbol> { val snapshot: String = org.locationtech.geomesa.utils.text.StringSerialization.alphaNumericSafeString(java.util.UUID.randomUUID().toString()); admin.disableTable(existing); admin.snapshot(snapshot, existing); admin.cloneSnapshot(snapshot, renamed); admin.deleteSnapshot(snapshot); admin.deleteTable(existing); HBaseIndexAdapter.waitForTable(admin, renamed) }
139 90018 7358 - 7358 Block <nosymbol> ()
141 90009 7513 - 7583 Apply org.locationtech.geomesa.utils.text.StringSerialization.alphaNumericSafeString org.locationtech.geomesa.utils.text.StringSerialization.alphaNumericSafeString(java.util.UUID.randomUUID().toString())
141 90008 7556 - 7582 Apply java.util.UUID.toString java.util.UUID.randomUUID().toString()
142 90010 7592 - 7620 Apply org.apache.hadoop.hbase.client.Admin.disableTable admin.disableTable(existing)
143 90011 7629 - 7663 Apply org.apache.hadoop.hbase.client.Admin.snapshot admin.snapshot(snapshot, existing)
144 90012 7672 - 7710 Apply org.apache.hadoop.hbase.client.Admin.cloneSnapshot admin.cloneSnapshot(snapshot, renamed)
145 90013 7719 - 7749 Apply org.apache.hadoop.hbase.client.Admin.deleteSnapshot admin.deleteSnapshot(snapshot)
146 90014 7758 - 7785 Apply org.apache.hadoop.hbase.client.Admin.deleteTable admin.deleteTable(existing)
147 90015 7794 - 7822 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.waitForTable HBaseIndexAdapter.waitForTable(admin, renamed)
153 90021 7915 - 7937 Apply org.apache.hadoop.hbase.client.Connection.getAdmin HBaseIndexAdapter.this.ds.connection.getAdmin()
153 90049 7905 - 8870 ApplyToImplicitArgs org.locationtech.geomesa.utils.io.WithClose.apply org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.hbase.client.Admin, Unit](HBaseIndexAdapter.this.ds.connection.getAdmin())(((admin: org.apache.hadoop.hbase.client.Admin) => { def deleteOne(name: String): Unit = { val table: org.apache.hadoop.hbase.TableName = org.apache.hadoop.hbase.TableName.valueOf(name); if (admin.tableExists(table)) { org.locationtech.geomesa.hbase.utils.HBaseVersions.disableTableAsync(admin, table); val timeout: Option[scala.concurrent.duration.Duration] = org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.TableAvailabilityTimeout.toUnboundedDuration.filter(((x$4: scala.concurrent.duration.Duration) => x$4.isFinite())); (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug(scala.StringContext.apply("Waiting for table \'", "\' to be disabled with ").s(table).+(scala.StringContext.apply("", "").s(timeout.map[String](((t: scala.concurrent.duration.Duration) => scala.StringContext.apply("a timeout of ", "").s(t))).getOrElse[String]("no timeout")))) else (): Unit); val stop: Option[Long] = timeout.map[Long](((t: scala.concurrent.duration.Duration) => java.lang.System.currentTimeMillis().+(t.toMillis))); while$1(){ if (admin.isTableDisabled(table).unary_!.&&(stop.forall(((x$5: Long) => x$5.>(java.lang.System.currentTimeMillis()))))) { java.lang.Thread.sleep(1000L); while$1() } else () }; admin.deleteTable(table) } else () }; tables.toList.map[java.util.concurrent.Future[_], List[java.util.concurrent.Future[_]]](((t: String) => org.locationtech.geomesa.utils.concurrent.CachedThreadPool.submit((() => deleteOne(t)))))(immutable.this.List.canBuildFrom[java.util.concurrent.Future[_]]).foreach[Any](((x$6: java.util.concurrent.Future[_]) => x$6.get())) }))(io.this.IsCloseable.closeableIsCloseable)
153 90048 7939 - 7939 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.closeableIsCloseable io.this.IsCloseable.closeableIsCloseable
155 90022 8014 - 8037 Apply org.apache.hadoop.hbase.TableName.valueOf org.apache.hadoop.hbase.TableName.valueOf(name)
156 90023 8050 - 8074 Apply org.apache.hadoop.hbase.client.Admin.tableExists admin.tableExists(table)
156 90041 8046 - 8046 Literal <nosymbol> ()
156 90040 8076 - 8767 Block <nosymbol> { org.locationtech.geomesa.hbase.utils.HBaseVersions.disableTableAsync(admin, table); val timeout: Option[scala.concurrent.duration.Duration] = org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.TableAvailabilityTimeout.toUnboundedDuration.filter(((x$4: scala.concurrent.duration.Duration) => x$4.isFinite())); (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug(scala.StringContext.apply("Waiting for table \'", "\' to be disabled with ").s(table).+(scala.StringContext.apply("", "").s(timeout.map[String](((t: scala.concurrent.duration.Duration) => scala.StringContext.apply("a timeout of ", "").s(t))).getOrElse[String]("no timeout")))) else (): Unit); val stop: Option[Long] = timeout.map[Long](((t: scala.concurrent.duration.Duration) => java.lang.System.currentTimeMillis().+(t.toMillis))); while$1(){ if (admin.isTableDisabled(table).unary_!.&&(stop.forall(((x$5: Long) => x$5.>(java.lang.System.currentTimeMillis()))))) { java.lang.Thread.sleep(1000L); while$1() } else () }; admin.deleteTable(table) }
156 90042 8046 - 8046 Block <nosymbol> ()
157 90024 8088 - 8133 Apply org.locationtech.geomesa.hbase.utils.HBaseVersions.disableTableAsync org.locationtech.geomesa.hbase.utils.HBaseVersions.disableTableAsync(admin, table)
158 90025 8210 - 8220 Apply scala.concurrent.duration.Duration.isFinite x$4.isFinite()
158 90026 8158 - 8221 Apply scala.Option.filter org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.TableAvailabilityTimeout.toUnboundedDuration.filter(((x$4: scala.concurrent.duration.Duration) => x$4.isFinite()))
161 90027 8447 - 8457 Select scala.concurrent.duration.Duration.toMillis t.toMillis
161 90029 8401 - 8458 Apply scala.Option.map timeout.map[Long](((t: scala.concurrent.duration.Duration) => java.lang.System.currentTimeMillis().+(t.toMillis)))
161 90028 8418 - 8457 Apply scala.Long.+ java.lang.System.currentTimeMillis().+(t.toMillis)
162 90031 8521 - 8551 Apply scala.Long.> x$5.>(java.lang.System.currentTimeMillis())
162 90030 8525 - 8551 Apply java.lang.System.currentTimeMillis java.lang.System.currentTimeMillis()
162 90033 8476 - 8552 Apply scala.Boolean.&& admin.isTableDisabled(table).unary_!.&&(stop.forall(((x$5: Long) => x$5.>(java.lang.System.currentTimeMillis()))))
162 90032 8509 - 8552 Apply scala.Option.forall stop.forall(((x$5: Long) => x$5.>(java.lang.System.currentTimeMillis())))
162 90037 8469 - 8469 Literal <nosymbol> ()
162 90036 8568 - 8586 Block <nosymbol> { java.lang.Thread.sleep(1000L); while$1() }
162 90038 8469 - 8469 Block <nosymbol> ()
163 90035 8580 - 8580 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.while$1 while$1()
163 90034 8568 - 8586 Apply java.lang.Thread.sleep java.lang.Thread.sleep(1000L)
166 90039 8733 - 8757 Apply org.apache.hadoop.hbase.client.Admin.deleteTable admin.deleteTable(table)
169 90043 8835 - 8847 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.deleteOne deleteOne(t)
169 90045 8799 - 8799 TypeApply scala.collection.immutable.List.canBuildFrom immutable.this.List.canBuildFrom[java.util.concurrent.Future[_]]
169 90044 8805 - 8848 Apply org.locationtech.geomesa.utils.concurrent.CachedThreadPool.submit org.locationtech.geomesa.utils.concurrent.CachedThreadPool.submit((() => deleteOne(t)))
169 90047 8782 - 8864 Apply scala.collection.immutable.List.foreach tables.toList.map[java.util.concurrent.Future[_], List[java.util.concurrent.Future[_]]](((t: String) => org.locationtech.geomesa.utils.concurrent.CachedThreadPool.submit((() => deleteOne(t)))))(immutable.this.List.canBuildFrom[java.util.concurrent.Future[_]]).foreach[Any](((x$6: java.util.concurrent.Future[_]) => x$6.get()))
169 90046 8858 - 8863 Apply java.util.concurrent.Future.get x$6.get()
175 90050 9026 - 9049 Apply org.apache.hadoop.hbase.TableName.valueOf org.apache.hadoop.hbase.TableName.valueOf(name)
176 90051 9066 - 9099 Apply org.apache.hadoop.hbase.client.Connection.getTable HBaseIndexAdapter.this.ds.connection.getTable(tableName)
176 90073 9101 - 9101 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.closeableIsCloseable io.this.IsCloseable.closeableIsCloseable
176 90074 9056 - 9696 ApplyToImplicitArgs org.locationtech.geomesa.utils.io.WithClose.apply org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.hbase.client.Table, Unit](HBaseIndexAdapter.this.ds.connection.getTable(tableName))(((table: org.apache.hadoop.hbase.client.Table) => { val scan: org.apache.hadoop.hbase.client.Scan = new org.apache.hadoop.hbase.client.Scan().setFilter(new org.apache.hadoop.hbase.filter.KeyOnlyFilter()); prefix.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Array[Byte]) => scan.setRowPrefixFilter(x$1)) }); HBaseIndexAdapter.this.ds.applySecurity(scan); val mutateParams: org.apache.hadoop.hbase.client.BufferedMutatorParams = new org.apache.hadoop.hbase.client.BufferedMutatorParams(tableName); org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.hbase.client.ResultScanner, org.apache.hadoop.hbase.client.BufferedMutator, Unit](table.getScanner(scan), HBaseIndexAdapter.this.ds.connection.getBufferedMutator(mutateParams))(((x0$1: org.apache.hadoop.hbase.client.ResultScanner, x1$1: org.apache.hadoop.hbase.client.BufferedMutator) => scala.Tuple2.apply[org.apache.hadoop.hbase.client.ResultScanner, org.apache.hadoop.hbase.client.BufferedMutator](x0$1, x1$1) match { case (_1: org.apache.hadoop.hbase.client.ResultScanner, _2: org.apache.hadoop.hbase.client.BufferedMutator)(org.apache.hadoop.hbase.client.ResultScanner, org.apache.hadoop.hbase.client.BufferedMutator)((scanner @ _), (mutator @ _)) => scala.collection.JavaConverters.asScalaIteratorConverter[org.apache.hadoop.hbase.client.Result](scanner.iterator()).asScala.grouped[org.apache.hadoop.hbase.client.Result](10000).foreach[Unit](((result: Seq[org.apache.hadoop.hbase.client.Result]) => { val deletes: Seq[org.apache.hadoop.hbase.client.Delete] = result.map[org.apache.hadoop.hbase.client.Delete, Seq[org.apache.hadoop.hbase.client.Delete]](((r: org.apache.hadoop.hbase.client.Result) => new org.apache.hadoop.hbase.client.Delete(r.getRow())))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Delete]); mutator.mutate(scala.collection.JavaConverters.seqAsJavaListConverter[org.apache.hadoop.hbase.client.Delete](deletes).asJava) })) }))(io.this.IsCloseable.closeableIsCloseable, io.this.IsCloseable.closeableIsCloseable) }))(io.this.IsCloseable.closeableIsCloseable)
177 90053 9131 - 9170 Apply org.apache.hadoop.hbase.client.Scan.setFilter new org.apache.hadoop.hbase.client.Scan().setFilter(new org.apache.hadoop.hbase.filter.KeyOnlyFilter())
177 90052 9152 - 9169 Apply org.apache.hadoop.hbase.filter.KeyOnlyFilter.<init> new org.apache.hadoop.hbase.filter.KeyOnlyFilter()
178 90055 9179 - 9218 Apply scala.Option.foreach prefix.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Array[Byte]) => scan.setRowPrefixFilter(x$1)) })
178 90054 9194 - 9217 Apply org.apache.hadoop.hbase.client.Scan.setRowPrefixFilter scan.setRowPrefixFilter(x$1)
179 90056 9227 - 9249 Apply org.locationtech.geomesa.hbase.data.HBaseDataStore.applySecurity HBaseIndexAdapter.this.ds.applySecurity(scan)
180 90057 9277 - 9313 Apply org.apache.hadoop.hbase.client.BufferedMutatorParams.<init> new org.apache.hadoop.hbase.client.BufferedMutatorParams(tableName)
181 90059 9356 - 9402 Apply org.apache.hadoop.hbase.client.Connection.getBufferedMutator HBaseIndexAdapter.this.ds.connection.getBufferedMutator(mutateParams)
181 90058 9332 - 9354 Apply org.apache.hadoop.hbase.client.Table.getScanner table.getScanner(scan)
181 90071 9404 - 9404 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.closeableIsCloseable io.this.IsCloseable.closeableIsCloseable
181 90070 9404 - 9404 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.closeableIsCloseable io.this.IsCloseable.closeableIsCloseable
181 90072 9322 - 9688 ApplyToImplicitArgs org.locationtech.geomesa.utils.io.WithClose.apply org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.hbase.client.ResultScanner, org.apache.hadoop.hbase.client.BufferedMutator, Unit](table.getScanner(scan), HBaseIndexAdapter.this.ds.connection.getBufferedMutator(mutateParams))(((x0$1: org.apache.hadoop.hbase.client.ResultScanner, x1$1: org.apache.hadoop.hbase.client.BufferedMutator) => scala.Tuple2.apply[org.apache.hadoop.hbase.client.ResultScanner, org.apache.hadoop.hbase.client.BufferedMutator](x0$1, x1$1) match { case (_1: org.apache.hadoop.hbase.client.ResultScanner, _2: org.apache.hadoop.hbase.client.BufferedMutator)(org.apache.hadoop.hbase.client.ResultScanner, org.apache.hadoop.hbase.client.BufferedMutator)((scanner @ _), (mutator @ _)) => scala.collection.JavaConverters.asScalaIteratorConverter[org.apache.hadoop.hbase.client.Result](scanner.iterator()).asScala.grouped[org.apache.hadoop.hbase.client.Result](10000).foreach[Unit](((result: Seq[org.apache.hadoop.hbase.client.Result]) => { val deletes: Seq[org.apache.hadoop.hbase.client.Delete] = result.map[org.apache.hadoop.hbase.client.Delete, Seq[org.apache.hadoop.hbase.client.Delete]](((r: org.apache.hadoop.hbase.client.Result) => new org.apache.hadoop.hbase.client.Delete(r.getRow())))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Delete]); mutator.mutate(scala.collection.JavaConverters.seqAsJavaListConverter[org.apache.hadoop.hbase.client.Delete](deletes).asJava) })) }))(io.this.IsCloseable.closeableIsCloseable, io.this.IsCloseable.closeableIsCloseable)
182 90061 9476 - 9481 Literal <nosymbol> 10000
182 90060 9443 - 9459 Apply org.apache.hadoop.hbase.client.ResultScanner.iterator scanner.iterator()
182 90069 9443 - 9678 Block scala.collection.Iterator.foreach scala.collection.JavaConverters.asScalaIteratorConverter[org.apache.hadoop.hbase.client.Result](scanner.iterator()).asScala.grouped[org.apache.hadoop.hbase.client.Result](10000).foreach[Unit](((result: Seq[org.apache.hadoop.hbase.client.Result]) => { val deletes: Seq[org.apache.hadoop.hbase.client.Delete] = result.map[org.apache.hadoop.hbase.client.Delete, Seq[org.apache.hadoop.hbase.client.Delete]](((r: org.apache.hadoop.hbase.client.Result) => new org.apache.hadoop.hbase.client.Delete(r.getRow())))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Delete]); mutator.mutate(scala.collection.JavaConverters.seqAsJavaListConverter[org.apache.hadoop.hbase.client.Delete](deletes).asJava) }))
182 90068 9443 - 9678 Apply scala.collection.Iterator.foreach scala.collection.JavaConverters.asScalaIteratorConverter[org.apache.hadoop.hbase.client.Result](scanner.iterator()).asScala.grouped[org.apache.hadoop.hbase.client.Result](10000).foreach[Unit](((result: Seq[org.apache.hadoop.hbase.client.Result]) => { val deletes: Seq[org.apache.hadoop.hbase.client.Delete] = result.map[org.apache.hadoop.hbase.client.Delete, Seq[org.apache.hadoop.hbase.client.Delete]](((r: org.apache.hadoop.hbase.client.Result) => new org.apache.hadoop.hbase.client.Delete(r.getRow())))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Delete]); mutator.mutate(scala.collection.JavaConverters.seqAsJavaListConverter[org.apache.hadoop.hbase.client.Delete](deletes).asJava) }))
184 90063 9602 - 9622 Apply org.apache.hadoop.hbase.client.Delete.<init> new org.apache.hadoop.hbase.client.Delete(r.getRow())
184 90062 9613 - 9621 Apply org.apache.hadoop.hbase.client.Result.getRow r.getRow()
184 90065 9586 - 9623 ApplyToImplicitArgs scala.collection.TraversableLike.map result.map[org.apache.hadoop.hbase.client.Delete, Seq[org.apache.hadoop.hbase.client.Delete]](((r: org.apache.hadoop.hbase.client.Result) => new org.apache.hadoop.hbase.client.Delete(r.getRow())))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Delete])
184 90064 9596 - 9596 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Delete]
185 90067 9636 - 9666 Apply org.apache.hadoop.hbase.client.BufferedMutator.mutate mutator.mutate(scala.collection.JavaConverters.seqAsJavaListConverter[org.apache.hadoop.hbase.client.Delete](deletes).asJava)
185 90066 9651 - 9665 Select scala.collection.convert.Decorators.AsJava.asJava scala.collection.JavaConverters.seqAsJavaListConverter[org.apache.hadoop.hbase.client.Delete](deletes).asJava
190 90075 9760 - 9771 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.clearOne clearOne(t)
190 90077 9724 - 9724 TypeApply scala.collection.immutable.List.canBuildFrom immutable.this.List.canBuildFrom[java.util.concurrent.Future[_]]
190 90076 9730 - 9772 Apply org.locationtech.geomesa.utils.concurrent.CachedThreadPool.submit org.locationtech.geomesa.utils.concurrent.CachedThreadPool.submit((() => clearOne(t)))
190 90079 9707 - 9788 Apply scala.collection.immutable.List.foreach tables.toList.map[java.util.concurrent.Future[_], List[java.util.concurrent.Future[_]]](((t: String) => org.locationtech.geomesa.utils.concurrent.CachedThreadPool.submit((() => clearOne(t)))))(immutable.this.List.canBuildFrom[java.util.concurrent.Future[_]]).foreach[Any](((x$7: java.util.concurrent.Future[_]) => x$7.get()))
190 90078 9782 - 9787 Apply java.util.concurrent.Future.get x$7.get()
197 90081 9970 - 9970 Select scala.Tuple4._2 x$8._2
197 90080 9962 - 9962 Select scala.Tuple4._1 x$8._1
197 90083 9994 - 9994 Select scala.Tuple4._4 x$8._4
197 90082 9988 - 9988 Select scala.Tuple4._3 x$8._3
198 90084 10031 - 10043 Select org.locationtech.geomesa.index.api.FilterStrategy.index filter.index
203 90093 10263 - 10476 ApplyToImplicitArgs scala.collection.TraversableLike.map byteRanges.map[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange, Seq[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]](((x0$1: org.locationtech.geomesa.index.api.ByteRange) => x0$1 match { case (lower: Array[Byte], upper: Array[Byte])org.locationtech.geomesa.index.api.BoundedByteRange((start @ _), (stop @ _)) => new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(start, true, stop, false) case (row: Array[Byte])org.locationtech.geomesa.index.api.SingleRowByteRange((row @ _)) => new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(row, true, org.locationtech.geomesa.utils.index.ByteArrays.rowFollowingRow(row), false) }))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])
203 90092 10278 - 10278 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]
204 90085 10324 - 10362 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange.<init> new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(start, true, stop, false)
204 90086 10324 - 10362 Block org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange.<init> new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(start, true, stop, false)
205 90087 10425 - 10429 Literal <nosymbol> true
205 90089 10464 - 10469 Literal <nosymbol> false
205 90088 10431 - 10462 Apply org.locationtech.geomesa.utils.index.ByteArrays.rowFollowingRow org.locationtech.geomesa.utils.index.ByteArrays.rowFollowingRow(row)
205 90091 10407 - 10470 Block org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange.<init> new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(row, true, org.locationtech.geomesa.utils.index.ByteArrays.rowFollowingRow(row), false)
205 90090 10407 - 10470 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange.<init> new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(row, true, org.locationtech.geomesa.utils.index.ByteArrays.rowFollowingRow(row), false)
207 90095 10493 - 10557 Apply scala.Option.exists byteRanges.headOption.exists(((x$9: org.locationtech.geomesa.index.api.ByteRange) => x$9.isInstanceOf[org.locationtech.geomesa.index.api.SingleRowByteRange]))
207 90094 10522 - 10556 TypeApply scala.Any.isInstanceOf x$9.isInstanceOf[org.locationtech.geomesa.index.api.SingleRowByteRange]
209 90097 10618 - 10618 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.TableName]
209 90096 10619 - 10636 Apply org.apache.hadoop.hbase.TableName.valueOf org.apache.hadoop.hbase.TableName.valueOf(x$1)
209 90098 10576 - 10637 ApplyToImplicitArgs scala.collection.TraversableLike.map index.getTablesForQuery(filter.filter).map[org.apache.hadoop.hbase.TableName, Seq[org.apache.hadoop.hbase.TableName]]({ ((x$1: String) => org.apache.hadoop.hbase.TableName.valueOf(x$1)) })(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.TableName])
210 90099 10647 - 10647 Select scala.Tuple2._1 x$10._1
210 90100 10658 - 10658 Select scala.Tuple2._2 x$10._2
212 90101 10786 - 10804 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.getTransform org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getTransform
216 90103 10968 - 11000 Apply scala.Boolean.|| tables.isEmpty.||(ranges.isEmpty)
216 90102 10986 - 11000 Select scala.collection.SeqLike.isEmpty ranges.isEmpty
216 90105 11004 - 11036 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.EmptyPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.EmptyPlan.apply(filter, reducer))
216 90104 11009 - 11035 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.EmptyPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.EmptyPlan.apply(filter, reducer)
216 90107 11046 - 11050 Select scala.None scala.None
216 90106 11004 - 11036 Block scala.Some.apply scala.Some.apply[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.EmptyPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.EmptyPlan.apply(filter, reducer))
216 90108 11046 - 11050 Block scala.None scala.None
218 90109 11062 - 11085 Select scala.Boolean.unary_! HBaseIndexAdapter.this.ds.config.remoteFilter.unary_!
218 90122 11087 - 11917 Block <nosymbol> { val reducer: Some[org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer] = scala.Some.apply[org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer](new org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer(schema, filter.filter, scala.None, transform, hints)); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan]({ val scans: Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan] = HBaseIndexAdapter.this.configureScans(tables, ranges, small, colFamily, scala.collection.Seq.empty[Nothing], false); val resultsToFeatures: org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseResultsToFeatures = new HBaseIndexAdapter.HBaseResultsToFeatures(index, schema); val sort: Option[Seq[(String, Boolean)]] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields; val max: Option[Int] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getMaxFeatures; val project: Option[org.locationtech.geomesa.index.utils.Reprojection.QueryReferenceSystems] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getProjection; org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, reducer, sort, max, project) }) }
223 90111 11414 - 11486 Apply org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer.<init> new org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer(schema, filter.filter, scala.None, transform, hints)
223 90110 11463 - 11467 Select scala.None scala.None
223 90112 11409 - 11487 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer](new org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer(schema, filter.filter, scala.None, transform, hints))
224 90121 11494 - 11911 Apply scala.Option.getOrElse empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan]({ val scans: Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan] = HBaseIndexAdapter.this.configureScans(tables, ranges, small, colFamily, scala.collection.Seq.empty[Nothing], false); val resultsToFeatures: org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseResultsToFeatures = new HBaseIndexAdapter.HBaseResultsToFeatures(index, schema); val sort: Option[Seq[(String, Boolean)]] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields; val max: Option[Int] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getMaxFeatures; val project: Option[org.locationtech.geomesa.index.utils.Reprojection.QueryReferenceSystems] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getProjection; org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, reducer, sort, max, project) })
225 90113 11590 - 11599 TypeApply scala.collection.generic.GenericCompanion.empty scala.collection.Seq.empty[Nothing]
225 90115 11541 - 11621 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.configureScans HBaseIndexAdapter.this.configureScans(tables, ranges, small, colFamily, scala.collection.Seq.empty[Nothing], false)
225 90114 11615 - 11620 Literal <nosymbol> false
226 90116 11654 - 11695 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseResultsToFeatures.<init> new HBaseIndexAdapter.HBaseResultsToFeatures(index, schema)
227 90117 11715 - 11734 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.getSortFields org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields
228 90118 11753 - 11773 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.getMaxFeatures org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getMaxFeatures
229 90119 11796 - 11815 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.getProjection org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getProjection
230 90120 11824 - 11903 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, reducer, sort, max, project)
232 90247 11923 - 17641 Block <nosymbol> { val indexFilter: Option[(Int, org.apache.hadoop.hbase.filter.FilterBase)] = strategy.index match { case (_: org.locationtech.geomesa.index.index.z3.Z3Index) => strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter)](((x0$2: _$2) => x0$2 match { case (v @ (_: org.locationtech.geomesa.index.index.z3.Z3IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z3Filter.apply(v), index.keySpace.sharding.length)) })) case (_: org.locationtech.geomesa.index.index.z2.Z2Index) => strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter)](((x0$3: _$2) => x0$3 match { case (v @ (_: org.locationtech.geomesa.index.index.z2.Z2IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z2Filter.apply(v), index.keySpace.sharding.length)) })) case (_: org.locationtech.geomesa.index.index.s2.S2Index) => strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter)](((x0$4: _$2) => x0$4 match { case (v @ (_: org.locationtech.geomesa.index.index.s2.S2IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.apply(org.locationtech.geomesa.index.filters.S2Filter.apply(v), index.keySpace.sharding.length)) })) case (_: org.locationtech.geomesa.index.index.s3.S3Index) => strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter)](((x0$5: _$2) => x0$5 match { case (v @ (_: org.locationtech.geomesa.index.index.s3.S3IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.apply(org.locationtech.geomesa.index.filters.S3Filter.apply(v), index.keySpace.sharding.length)) })) case _ => scala.None }; val max: Option[Int] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getMaxFeatures; val projection: Option[org.locationtech.geomesa.index.utils.Reprojection.QueryReferenceSystems] = org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getProjection; <stable> <accessor> lazy val returnSchema: org.geotools.api.feature.simple.SimpleFeatureType = transform.map[org.geotools.api.feature.simple.SimpleFeatureType](((x$11: (String, org.geotools.api.feature.simple.SimpleFeatureType)) => x$11._2)).getOrElse[org.geotools.api.feature.simple.SimpleFeatureType](schema); <stable> <accessor> lazy val filters: Seq[org.apache.hadoop.hbase.filter.FilterBase] = { val cqlFilter: Seq[(Int, org.locationtech.geomesa.hbase.rpc.filter.CqlTransformFilter)] = if (ecql.isEmpty.&&(transform.isEmpty).&&(org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSampling.isEmpty)) scala.collection.Seq.empty[Nothing] else scala.collection.Seq.apply[(Int, org.locationtech.geomesa.hbase.rpc.filter.CqlTransformFilter)](scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.CqlTransformFilter](org.locationtech.geomesa.hbase.rpc.filter.CqlTransformFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.CqlTransformFilter.apply(schema, strategy.index, ecql, transform, hints))); cqlFilter.++[(Int, org.apache.hadoop.hbase.filter.FilterBase), Seq[(Int, org.apache.hadoop.hbase.filter.FilterBase)]](scala.this.Option.option2Iterable[(Int, org.apache.hadoop.hbase.filter.FilterBase)](indexFilter))(collection.this.Seq.canBuildFrom[(Int, org.apache.hadoop.hbase.filter.FilterBase)]).sortBy[Int](((x$12: (Int, org.apache.hadoop.hbase.filter.FilterBase)) => x$12._1))(math.this.Ordering.Int).map[org.apache.hadoop.hbase.filter.FilterBase, Seq[org.apache.hadoop.hbase.filter.FilterBase]](((x$13: (Int, org.apache.hadoop.hbase.filter.FilterBase)) => x$13._2))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.FilterBase]) }; <stable> <accessor> lazy val coprocessorOptions: scala.collection.immutable.Map[String,String] = scala.Predef.Map.apply[String, String](scala.Predef.ArrowAssoc[String](org.locationtech.geomesa.hbase.rpc.coprocessor.GeoMesaCoprocessor.YieldOpt).->[String](java.lang.String.valueOf(HBaseIndexAdapter.this.ds.config.coprocessors.yieldPartialResults))); <stable> <accessor> lazy val scans: Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan] = HBaseIndexAdapter.this.configureScans(tables, ranges, small, colFamily, filters, false); <stable> <accessor> lazy val coprocessorScans: Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan] = HBaseIndexAdapter.this.configureScans(tables, ranges, small, colFamily, scala.this.Option.option2Iterable[(Int, org.apache.hadoop.hbase.filter.FilterBase)](indexFilter).toSeq.map[org.apache.hadoop.hbase.filter.FilterBase, Seq[org.apache.hadoop.hbase.filter.Filter]](((x$14: (Int, org.apache.hadoop.hbase.filter.FilterBase)) => x$14._2))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.FilterBase]), true); <stable> <accessor> lazy val resultsToFeatures: org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseResultsToFeatures = new HBaseIndexAdapter.HBaseResultsToFeatures(index, returnSchema); <stable> <accessor> lazy val localReducer: Some[org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer] = scala.Some.apply[org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer](new org.locationtech.geomesa.index.planning.LocalQueryRunner.LocalTransformReducer(returnSchema, scala.None, scala.None, scala.None, hints)); if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isDensityQuery) empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.density) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) else if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isArrowQuery) { val config: org.locationtech.geomesa.index.iterators.ArrowScan.ArrowScanConfig = org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.configure(schema, index, HBaseIndexAdapter.this.ds.stats, filter.filter, ecql, hints); val reducer: Some[org.locationtech.geomesa.index.api.QueryPlan.FeatureReducer] = scala.Some.apply[org.locationtech.geomesa.index.api.QueryPlan.FeatureReducer](config.reduce); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.arrow) { val options: scala.collection.immutable.Map[String,String] = config.config.++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) } else if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isStatsQuery) { val reducer: Some[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer] = scala.Some.apply[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer](org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer.apply(returnSchema, hints)); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.stats) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) } else if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isBinQuery) empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.bin) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) else empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, scala.None, org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields, max, projection)) }
234 90123 12038 - 12052 Select org.locationtech.geomesa.index.api.QueryStrategy.index strategy.index
236 90131 12098 - 12253 Block scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter)](((x0$2: _$2) => x0$2 match { case (v @ (_: org.locationtech.geomesa.index.index.z3.Z3IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z3Filter.apply(v), index.keySpace.sharding.length)) }))
236 90130 12098 - 12253 Apply scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter)](((x0$2: _$2) => x0$2 match { case (v @ (_: org.locationtech.geomesa.index.index.z3.Z3IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z3Filter.apply(v), index.keySpace.sharding.length)) }))
237 90125 12196 - 12207 Apply org.locationtech.geomesa.index.filters.Z3Filter.apply org.locationtech.geomesa.index.filters.Z3Filter.apply(v)
237 90124 12158 - 12180 Select org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.Priority org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.Priority
237 90127 12182 - 12240 Apply org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.apply org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z3Filter.apply(v), index.keySpace.sharding.length)
237 90126 12209 - 12239 Select org.locationtech.geomesa.index.api.ShardStrategy.length index.keySpace.sharding.length
237 90129 12157 - 12241 Block scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z3Filter.apply(v), index.keySpace.sharding.length))
237 90128 12157 - 12241 Apply scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z3HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z3Filter.apply(v), index.keySpace.sharding.length))
241 90139 12292 - 12447 Block scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter)](((x0$3: _$2) => x0$3 match { case (v @ (_: org.locationtech.geomesa.index.index.z2.Z2IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z2Filter.apply(v), index.keySpace.sharding.length)) }))
241 90138 12292 - 12447 Apply scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter)](((x0$3: _$2) => x0$3 match { case (v @ (_: org.locationtech.geomesa.index.index.z2.Z2IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z2Filter.apply(v), index.keySpace.sharding.length)) }))
242 90133 12390 - 12401 Apply org.locationtech.geomesa.index.filters.Z2Filter.apply org.locationtech.geomesa.index.filters.Z2Filter.apply(v)
242 90132 12352 - 12374 Select org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.Priority org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.Priority
242 90135 12376 - 12434 Apply org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.apply org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z2Filter.apply(v), index.keySpace.sharding.length)
242 90134 12403 - 12433 Select org.locationtech.geomesa.index.api.ShardStrategy.length index.keySpace.sharding.length
242 90137 12351 - 12435 Block scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z2Filter.apply(v), index.keySpace.sharding.length))
242 90136 12351 - 12435 Apply scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.Z2HBaseFilter.apply(org.locationtech.geomesa.index.filters.Z2Filter.apply(v), index.keySpace.sharding.length))
246 90147 12486 - 12641 Block scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter)](((x0$4: _$2) => x0$4 match { case (v @ (_: org.locationtech.geomesa.index.index.s2.S2IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.apply(org.locationtech.geomesa.index.filters.S2Filter.apply(v), index.keySpace.sharding.length)) }))
246 90146 12486 - 12641 Apply scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter)](((x0$4: _$2) => x0$4 match { case (v @ (_: org.locationtech.geomesa.index.index.s2.S2IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.apply(org.locationtech.geomesa.index.filters.S2Filter.apply(v), index.keySpace.sharding.length)) }))
247 90141 12584 - 12595 Apply org.locationtech.geomesa.index.filters.S2Filter.apply org.locationtech.geomesa.index.filters.S2Filter.apply(v)
247 90140 12546 - 12568 Select org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.Priority org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.Priority
247 90143 12570 - 12628 Apply org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.apply org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.apply(org.locationtech.geomesa.index.filters.S2Filter.apply(v), index.keySpace.sharding.length)
247 90142 12597 - 12627 Select org.locationtech.geomesa.index.api.ShardStrategy.length index.keySpace.sharding.length
247 90145 12545 - 12629 Block scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.apply(org.locationtech.geomesa.index.filters.S2Filter.apply(v), index.keySpace.sharding.length))
247 90144 12545 - 12629 Apply scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S2HBaseFilter.apply(org.locationtech.geomesa.index.filters.S2Filter.apply(v), index.keySpace.sharding.length))
251 90155 12680 - 12835 Block scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter)](((x0$5: _$2) => x0$5 match { case (v @ (_: org.locationtech.geomesa.index.index.s3.S3IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.apply(org.locationtech.geomesa.index.filters.S3Filter.apply(v), index.keySpace.sharding.length)) }))
251 90154 12680 - 12835 Apply scala.Option.map strategy.values.map[(Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter)](((x0$5: _$2) => x0$5 match { case (v @ (_: org.locationtech.geomesa.index.index.s3.S3IndexValues)) => scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.apply(org.locationtech.geomesa.index.filters.S3Filter.apply(v), index.keySpace.sharding.length)) }))
252 90149 12778 - 12789 Apply org.locationtech.geomesa.index.filters.S3Filter.apply org.locationtech.geomesa.index.filters.S3Filter.apply(v)
252 90148 12740 - 12762 Select org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.Priority org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.Priority
252 90151 12764 - 12822 Apply org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.apply org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.apply(org.locationtech.geomesa.index.filters.S3Filter.apply(v), index.keySpace.sharding.length)
252 90150 12791 - 12821 Select org.locationtech.geomesa.index.api.ShardStrategy.length index.keySpace.sharding.length
252 90153 12739 - 12823 Block scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.apply(org.locationtech.geomesa.index.filters.S3Filter.apply(v), index.keySpace.sharding.length))
252 90152 12739 - 12823 Apply scala.Tuple2.apply scala.Tuple2.apply[Int, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter](org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.Priority, org.locationtech.geomesa.hbase.rpc.filter.S3HBaseFilter.apply(org.locationtech.geomesa.index.filters.S3Filter.apply(v), index.keySpace.sharding.length))
256 90157 12929 - 12933 Block scala.None scala.None
256 90156 12929 - 12933 Select scala.None scala.None
259 90158 12959 - 12979 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.getMaxFeatures org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getMaxFeatures
260 90159 13003 - 13022 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.getProjection org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getProjection
276 90160 13996 - 14016 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isDensityQuery org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isDensityQuery
277 90161 14034 - 14038 Select scala.None scala.None
277 90177 14028 - 14813 Apply scala.Option.getOrElse empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.density) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) })
277 90178 14028 - 14813 Block scala.Option.getOrElse empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.density) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) })
278 90162 14066 - 14104 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.EnabledCoprocessors.density HBaseIndexAdapter.this.ds.config.coprocessors.enabled.density
278 90167 14106 - 14393 Block <nosymbol> { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) }
279 90163 14134 - 14216 Apply scala.collection.immutable.MapLike.++ org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions)
280 90164 14243 - 14278 Apply org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures.<init> new org.locationtech.geomesa.hbase.aggregators.HBaseDensityAggregator.HBaseDensityResultsToFeatures()
281 90165 14359 - 14363 Select scala.None scala.None
281 90166 14291 - 14381 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection)
282 90176 14399 - 14803 Block <nosymbol> { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }
283 90168 14417 - 14435 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isSkipReduce org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce
283 90173 14413 - 14413 Block <nosymbol> ()
283 90172 14413 - 14413 Literal <nosymbol> ()
286 90169 14632 - 14662 Select org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT
286 90171 14616 - 14677 Block java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
286 90170 14616 - 14677 Apply java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
288 90175 14704 - 14791 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection)
288 90174 14769 - 14773 Select scala.None scala.None
291 90179 14831 - 14849 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isArrowQuery org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isArrowQuery
291 90199 14851 - 15754 Block <nosymbol> { val config: org.locationtech.geomesa.index.iterators.ArrowScan.ArrowScanConfig = org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.configure(schema, index, HBaseIndexAdapter.this.ds.stats, filter.filter, ecql, hints); val reducer: Some[org.locationtech.geomesa.index.api.QueryPlan.FeatureReducer] = scala.Some.apply[org.locationtech.geomesa.index.api.QueryPlan.FeatureReducer](config.reduce); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.arrow) { val options: scala.collection.immutable.Map[String,String] = config.config.++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) }
291 90246 14827 - 17635 If <nosymbol> if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isArrowQuery) { val config: org.locationtech.geomesa.index.iterators.ArrowScan.ArrowScanConfig = org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.configure(schema, index, HBaseIndexAdapter.this.ds.stats, filter.filter, ecql, hints); val reducer: Some[org.locationtech.geomesa.index.api.QueryPlan.FeatureReducer] = scala.Some.apply[org.locationtech.geomesa.index.api.QueryPlan.FeatureReducer](config.reduce); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.arrow) { val options: scala.collection.immutable.Map[String,String] = config.config.++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) } else if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isStatsQuery) { val reducer: Some[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer] = scala.Some.apply[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer](org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer.apply(returnSchema, hints)); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.stats) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) } else if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isBinQuery) empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.bin) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) else empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, scala.None, org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields, max, projection))
292 90181 14874 - 14957 Apply org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.configure org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.configure(schema, index, HBaseIndexAdapter.this.ds.stats, filter.filter, ecql, hints)
292 90180 14920 - 14928 Select org.locationtech.geomesa.hbase.data.HBaseDataStore.stats HBaseIndexAdapter.this.ds.stats
293 90183 14980 - 14999 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.index.api.QueryPlan.FeatureReducer](config.reduce)
293 90182 14985 - 14998 Select org.locationtech.geomesa.index.iterators.ArrowScan.ArrowScanConfig.reduce config.reduce
294 90198 15008 - 15746 Apply scala.Option.getOrElse empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.arrow) { val options: scala.collection.immutable.Map[String,String] = config.config.++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) })
295 90184 15049 - 15085 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.EnabledCoprocessors.arrow HBaseIndexAdapter.this.ds.config.coprocessors.enabled.arrow
295 90188 15087 - 15328 Block <nosymbol> { val options: scala.collection.immutable.Map[String,String] = config.config.++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) }
296 90185 15115 - 15150 Apply scala.collection.immutable.MapLike.++ config.config.++[String](coprocessorOptions)
297 90186 15177 - 15210 Apply org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures.<init> new org.locationtech.geomesa.hbase.aggregators.HBaseArrowAggregator.HBaseArrowResultsToFeatures()
298 90187 15223 - 15316 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection)
299 90197 15334 - 15736 Block <nosymbol> { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }
300 90189 15352 - 15370 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isSkipReduce org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce
300 90193 15348 - 15348 Literal <nosymbol> ()
300 90194 15348 - 15348 Block <nosymbol> ()
303 90191 15549 - 15610 Apply java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
303 90190 15565 - 15595 Select org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT
303 90192 15549 - 15610 Block java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
305 90195 15702 - 15706 Select scala.None scala.None
305 90196 15637 - 15724 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection)
308 90200 15764 - 15782 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isStatsQuery org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isStatsQuery
308 90218 15784 - 16657 Block <nosymbol> { val reducer: Some[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer] = scala.Some.apply[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer](org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer.apply(returnSchema, hints)); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.stats) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) }
308 90245 15760 - 17635 If <nosymbol> if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isStatsQuery) { val reducer: Some[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer] = scala.Some.apply[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer](org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer.apply(returnSchema, hints)); empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.stats) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) } else if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isBinQuery) empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.bin) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) else empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, scala.None, org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields, max, projection))
309 90201 15813 - 15856 Apply org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer.apply org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer.apply(returnSchema, hints)
309 90202 15808 - 15857 Apply scala.Some.apply scala.Some.apply[org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer](org.locationtech.geomesa.index.iterators.StatsScan.StatsReducer.apply(returnSchema, hints))
310 90217 15866 - 16649 Apply scala.Option.getOrElse empty(reducer).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.stats) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) })
311 90203 15907 - 15943 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.EnabledCoprocessors.stats HBaseIndexAdapter.this.ds.config.coprocessors.enabled.stats
311 90207 15945 - 16231 Block <nosymbol> { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection) }
312 90204 15973 - 16053 Apply scala.collection.immutable.MapLike.++ org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions)
313 90205 16080 - 16113 Apply org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures.<init> new org.locationtech.geomesa.hbase.aggregators.HBaseStatsAggregator.HBaseStatsResultsToFeatures()
314 90206 16126 - 16219 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, reducer, max, projection)
315 90216 16237 - 16639 Block <nosymbol> { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }
316 90208 16255 - 16273 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isSkipReduce org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce
316 90213 16251 - 16251 Block <nosymbol> ()
316 90212 16251 - 16251 Literal <nosymbol> ()
319 90209 16468 - 16498 Select org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT
319 90211 16452 - 16513 Block java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
319 90210 16452 - 16513 Apply java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
321 90215 16540 - 16627 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection)
321 90214 16605 - 16609 Select scala.None scala.None
324 90219 16667 - 16683 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isBinQuery org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isBinQuery
324 90244 16663 - 17635 If <nosymbol> if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isBinQuery) empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.bin) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }) else empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, scala.None, org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields, max, projection))
325 90220 16701 - 16705 Select scala.None scala.None
325 90237 16695 - 17465 Block scala.Option.getOrElse empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.bin) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) })
325 90236 16695 - 17465 Apply scala.Option.getOrElse empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](if (HBaseIndexAdapter.this.ds.config.coprocessors.enabled.bin) { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) } else { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) })
326 90221 16733 - 16767 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.EnabledCoprocessors.bin HBaseIndexAdapter.this.ds.config.coprocessors.enabled.bin
326 90226 16769 - 17049 Block <nosymbol> { val options: scala.collection.immutable.Map[String,String] = org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions); val results: org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures = new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures(); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection) }
327 90222 16797 - 16875 Apply scala.collection.immutable.MapLike.++ org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.configure(schema, index, ecql, hints).++[String](coprocessorOptions)
328 90223 16902 - 16933 Apply org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures.<init> new org.locationtech.geomesa.hbase.aggregators.HBaseBinAggregator.HBaseBinResultsToFeatures()
329 90225 16946 - 17037 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.CoprocessorPlan.apply(filter, ranges, coprocessorScans, options, results, scala.None, max, projection)
329 90224 17015 - 17019 Select scala.None scala.None
330 90235 17055 - 17455 Block <nosymbol> { if (org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce) org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema) else (); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection) }
331 90227 17073 - 17091 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.isSkipReduce org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).isSkipReduce
331 90231 17069 - 17069 Literal <nosymbol> ()
331 90232 17069 - 17069 Block <nosymbol> ()
334 90229 17268 - 17329 Apply java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
334 90228 17284 - 17314 Select org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT
334 90230 17268 - 17329 Block java.awt.RenderingHints.put org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).hints.put(org.locationtech.geomesa.index.conf.QueryHints.Internal.RETURN_SFT, returnSchema)
336 90233 17421 - 17425 Select scala.None scala.None
336 90234 17356 - 17443 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, localReducer, scala.None, max, projection)
340 90238 17495 - 17499 Select scala.None scala.None
340 90243 17489 - 17627 Block scala.Option.getOrElse empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, scala.None, org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields, max, projection))
340 90242 17489 - 17627 Apply scala.Option.getOrElse empty(scala.None).getOrElse[org.locationtech.geomesa.hbase.data.HBaseQueryPlan](org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, scala.None, org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields, max, projection))
341 90239 17574 - 17578 Select scala.None scala.None
341 90241 17523 - 17617 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.ScanPlan.apply(filter, ranges, scans, resultsToFeatures, scala.None, org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields, max, projection)
341 90240 17580 - 17599 Select org.locationtech.geomesa.index.conf.QueryHints.RichHints.getSortFields org.locationtech.geomesa.index.conf.QueryHints.RichHints(hints).getSortFields
352 90249 17852 - 17911 Literal <nosymbol> "HBase data store does not currently support atomic writes"
352 90248 17843 - 17850 Select scala.Boolean.unary_! atomic.unary_!
352 90250 17835 - 17912 Apply scala.Predef.require scala.Predef.require(atomic.unary_!, "HBase data store does not currently support atomic writes")
353 90251 17960 - 17966 Select org.locationtech.geomesa.index.api.IndexAdapter.groups HBaseIndexAdapter.this.groups
353 90252 17931 - 17967 Apply org.locationtech.geomesa.index.api.WritableFeature.wrapper org.locationtech.geomesa.index.api.WritableFeature.wrapper(sft, HBaseIndexAdapter.this.groups)
354 90253 17976 - 18000 Select org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType.isVisibilityRequired org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType(sft).isVisibilityRequired
355 90255 18010 - 18093 Block <nosymbol> { final class $anon extends HBaseIndexAdapter.HBaseIndexWriter with org.locationtech.geomesa.index.api.IndexAdapter.RequiredVisibilityWriter { def <init>(): <$anon: org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter with org.locationtech.geomesa.index.api.IndexAdapter.RequiredVisibilityWriter> = { $anon.super.<init>(HBaseIndexAdapter.this.ds, indices, wrapper, partition); () } }; new $anon() }
355 90254 18010 - 18013 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.$anon.<init> new $anon()
357 90257 18113 - 18166 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.<init> new HBaseIndexAdapter.HBaseIndexWriter(HBaseIndexAdapter.this.ds, indices, wrapper, partition)
357 90256 18134 - 18136 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.ds HBaseIndexAdapter.this.ds
357 90258 18113 - 18166 Block org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.<init> new HBaseIndexAdapter.HBaseIndexWriter(HBaseIndexAdapter.this.ds, indices, wrapper, partition)
379 90259 18837 - 18892 Select scala.Option.get org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.ScannerBlockCaching.toBoolean.get
380 90260 18952 - 18994 Select org.locationtech.geomesa.utils.conf.GeoMesaSystemProperties.SystemProperty.toInt org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.ScannerCaching.toInt
384 90261 19106 - 19118 Select scala.Boolean.unary_! coprocessor.unary_!
384 90262 19097 - 19118 Apply scala.Boolean.&& small.&&(coprocessor.unary_!)
384 90284 19120 - 19730 Block <nosymbol> { val filter: Option[org.apache.hadoop.hbase.filter.Filter] = filters match { case scala.collection.immutable.Nil => scala.None case scala.collection.Seq.unapplySeq[org.apache.hadoop.hbase.filter.Filter](<unapply-selector>) <unapply> ((f @ _)) => scala.Some.apply[org.apache.hadoop.hbase.filter.Filter](f) case (f @ _) => scala.Some.apply[org.apache.hadoop.hbase.filter.FilterList](new org.apache.hadoop.hbase.filter.FilterList((f: _*))) }; tables.map[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan, Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]](((table: org.apache.hadoop.hbase.TableName) => { val scans: Seq[org.apache.hadoop.hbase.client.Scan] = ranges.map[org.apache.hadoop.hbase.client.Scan, Seq[org.apache.hadoop.hbase.client.Scan]](((r: org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange) => { val scan: org.apache.hadoop.hbase.client.Scan = new org.apache.hadoop.hbase.client.Scan(r.getStartRow(), r.getStopRow()); scan.addFamily(colFamily).setCacheBlocks(cacheBlocks).setSmall(true); filter.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: org.apache.hadoop.hbase.filter.Filter) => scan.setFilter(x$1)) }); cacheSize.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Int) => scan.setCaching(x$1)) }); HBaseIndexAdapter.this.ds.applySecurity(scan); scan }))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan]); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply(table, scans) }))(collection.this.Seq.canBuildFrom[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]) }
386 90263 19180 - 19184 Select scala.None scala.None
386 90264 19180 - 19184 Block scala.None scala.None
387 90265 19208 - 19215 Apply scala.Some.apply scala.Some.apply[org.apache.hadoop.hbase.filter.Filter](f)
387 90266 19208 - 19215 Block scala.Some.apply scala.Some.apply[org.apache.hadoop.hbase.filter.Filter](f)
388 90267 19244 - 19265 Apply org.apache.hadoop.hbase.filter.FilterList.<init> new org.apache.hadoop.hbase.filter.FilterList((f: _*))
388 90269 19239 - 19266 Block scala.Some.apply scala.Some.apply[org.apache.hadoop.hbase.filter.FilterList](new org.apache.hadoop.hbase.filter.FilterList((f: _*)))
388 90268 19239 - 19266 Apply scala.Some.apply scala.Some.apply[org.apache.hadoop.hbase.filter.FilterList](new org.apache.hadoop.hbase.filter.FilterList((f: _*)))
391 90283 19343 - 19724 ApplyToImplicitArgs scala.collection.TraversableLike.map tables.map[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan, Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]](((table: org.apache.hadoop.hbase.TableName) => { val scans: Seq[org.apache.hadoop.hbase.client.Scan] = ranges.map[org.apache.hadoop.hbase.client.Scan, Seq[org.apache.hadoop.hbase.client.Scan]](((r: org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange) => { val scan: org.apache.hadoop.hbase.client.Scan = new org.apache.hadoop.hbase.client.Scan(r.getStartRow(), r.getStopRow()); scan.addFamily(colFamily).setCacheBlocks(cacheBlocks).setSmall(true); filter.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: org.apache.hadoop.hbase.filter.Filter) => scan.setFilter(x$1)) }); cacheSize.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Int) => scan.setCaching(x$1)) }); HBaseIndexAdapter.this.ds.applySecurity(scan); scan }))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan]); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply(table, scans) }))(collection.this.Seq.canBuildFrom[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan])
391 90282 19354 - 19354 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]
392 90279 19396 - 19396 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan]
392 90280 19385 - 19684 ApplyToImplicitArgs scala.collection.TraversableLike.map ranges.map[org.apache.hadoop.hbase.client.Scan, Seq[org.apache.hadoop.hbase.client.Scan]](((r: org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange) => { val scan: org.apache.hadoop.hbase.client.Scan = new org.apache.hadoop.hbase.client.Scan(r.getStartRow(), r.getStopRow()); scan.addFamily(colFamily).setCacheBlocks(cacheBlocks).setSmall(true); filter.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: org.apache.hadoop.hbase.filter.Filter) => scan.setFilter(x$1)) }); cacheSize.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Int) => scan.setCaching(x$1)) }); HBaseIndexAdapter.this.ds.applySecurity(scan); scan }))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan])
393 90271 19448 - 19460 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.BasicRowRange.getStopRow r.getStopRow()
393 90270 19433 - 19446 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.BasicRowRange.getStartRow r.getStartRow()
393 90272 19424 - 19461 Apply org.apache.hadoop.hbase.client.Scan.<init> new org.apache.hadoop.hbase.client.Scan(r.getStartRow(), r.getStopRow())
394 90273 19472 - 19540 Apply org.apache.hadoop.hbase.client.Scan.setSmall scan.addFamily(colFamily).setCacheBlocks(cacheBlocks).setSmall(true)
395 90275 19551 - 19581 Apply scala.Option.foreach filter.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: org.apache.hadoop.hbase.filter.Filter) => scan.setFilter(x$1)) })
395 90274 19566 - 19580 Apply org.apache.hadoop.hbase.client.Scan.setFilter scan.setFilter(x$1)
396 90277 19592 - 19626 Apply scala.Option.foreach cacheSize.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Int) => scan.setCaching(x$1)) })
396 90276 19610 - 19625 Apply org.apache.hadoop.hbase.client.Scan.setCaching scan.setCaching(x$1)
397 90278 19637 - 19659 Apply org.locationtech.geomesa.hbase.data.HBaseDataStore.applySecurity HBaseIndexAdapter.this.ds.applySecurity(scan)
400 90281 19693 - 19716 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply(table, scans)
402 90351 19736 - 22094 Block <nosymbol> { val rangesPerTable: Seq[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])] = tables.map[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]]), Seq[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])]](((t: org.apache.hadoop.hbase.TableName) => scala.Predef.ArrowAssoc[org.apache.hadoop.hbase.TableName](t).->[scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]]](HBaseIndexAdapter.this.groupRangesByRegion(t, ranges))))(collection.this.Seq.canBuildFrom[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])]); def createGroup(group: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]): org.apache.hadoop.hbase.client.Scan = { val scan: org.apache.hadoop.hbase.client.Scan = new org.apache.hadoop.hbase.client.Scan(group.get(0).getStartRow(), group.get(group.size().-(1)).getStopRow()); val mrrf: Seq[org.apache.hadoop.hbase.filter.Filter] = if (group.size().<(2)) filters else filters.+:[org.apache.hadoop.hbase.filter.Filter, Seq[org.apache.hadoop.hbase.filter.Filter]](new org.apache.hadoop.hbase.filter.MultiRowRangeFilter(group))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.Filter]); scan.setFilter(if (mrrf.lengthCompare(1).>(0)) new org.apache.hadoop.hbase.filter.FilterList((mrrf: _*)) else mrrf.headOption.orNull[org.apache.hadoop.hbase.filter.Filter](scala.Predef.$conforms[Null])); scan.addFamily(colFamily).setCacheBlocks(cacheBlocks); cacheSize.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Int) => scan.setCaching(x$1)) }); HBaseIndexAdapter.this.ds.applySecurity(scan); scan }; rangesPerTable.map[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan, Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]](((x0$1: (org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])) => x0$1 match { case (_1: org.apache.hadoop.hbase.TableName, _2: scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])((table @ _), (rangesPerRegion @ _)) => { val maxRangesPerGroup: Int = { def calcMax(maxPerGroup: Int, threads: Int): Int = { val totalRanges: Int = rangesPerRegion.values.map[Int, Iterable[Int]](((x$15: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]) => x$15.size()))(collection.this.Iterable.canBuildFrom[Int]).sum[Int](math.this.Numeric.IntIsIntegral); scala.math.`package`.min(maxPerGroup, scala.math.`package`.max(1, scala.math.`package`.ceil(totalRanges.toDouble./(threads)).toInt)) }; if (coprocessor) calcMax(HBaseIndexAdapter.this.ds.config.coprocessors.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.coprocessors.threads) else calcMax(HBaseIndexAdapter.this.ds.config.queries.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.queries.threads) }; val groupedScans: scala.collection.mutable.Builder[org.apache.hadoop.hbase.client.Scan,Seq[org.apache.hadoop.hbase.client.Scan]] = scala.collection.Seq.newBuilder[org.apache.hadoop.hbase.client.Scan]; rangesPerRegion.foreach[Unit](((x0$2: (String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])) => x0$2 match { case (_1: String, _2: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(_, (list @ _)) => { java.util.Collections.sort[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange](list); var i: Int = 0; while$2(){ if (i.<(list.size())) { { val groupSize: Int = scala.math.`package`.min(maxRangesPerGroup, list.size().-(i)); groupedScans.+=(createGroup(list.subList(i, i.+(groupSize)))); i = i.+(groupSize) }; while$2() } else () } } })); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply(table, scala.util.Random.shuffle[org.apache.hadoop.hbase.client.Scan, Seq](groupedScans.result())(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan])) } }))(collection.this.Seq.canBuildFrom[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]) }
406 90285 19980 - 20010 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.groupRangesByRegion HBaseIndexAdapter.this.groupRangesByRegion(t, ranges)
406 90287 19969 - 19969 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])]
406 90286 19975 - 20010 Apply scala.Predef.ArrowAssoc.-> scala.Predef.ArrowAssoc[org.apache.hadoop.hbase.TableName](t).->[scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]]](HBaseIndexAdapter.this.groupRangesByRegion(t, ranges))
406 90288 19959 - 20011 ApplyToImplicitArgs scala.collection.TraversableLike.map tables.map[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]]), Seq[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])]](((t: org.apache.hadoop.hbase.TableName) => scala.Predef.ArrowAssoc[org.apache.hadoop.hbase.TableName](t).->[scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]]](HBaseIndexAdapter.this.groupRangesByRegion(t, ranges))))(collection.this.Seq.canBuildFrom[(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])])
409 90289 20106 - 20130 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.BasicRowRange.getStartRow group.get(0).getStartRow()
409 90291 20097 - 20171 Apply org.apache.hadoop.hbase.client.Scan.<init> new org.apache.hadoop.hbase.client.Scan(group.get(0).getStartRow(), group.get(group.size().-(1)).getStopRow())
409 90290 20132 - 20170 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.BasicRowRange.getStopRow group.get(group.size().-(1)).getStopRow()
410 90293 20215 - 20222 Ident org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.filters filters
410 90292 20195 - 20211 Apply scala.Int.< group.size().<(2)
415 90295 20489 - 20489 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.Filter]
415 90294 20490 - 20520 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.<init> new org.apache.hadoop.hbase.filter.MultiRowRangeFilter(group)
415 90297 20479 - 20521 Block scala.collection.SeqLike.+: filters.+:[org.apache.hadoop.hbase.filter.Filter, Seq[org.apache.hadoop.hbase.filter.Filter]](new org.apache.hadoop.hbase.filter.MultiRowRangeFilter(group))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.Filter])
415 90296 20479 - 20521 ApplyToImplicitArgs scala.collection.SeqLike.+: filters.+:[org.apache.hadoop.hbase.filter.Filter, Seq[org.apache.hadoop.hbase.filter.Filter]](new org.apache.hadoop.hbase.filter.MultiRowRangeFilter(group))(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.filter.Filter])
417 90299 20588 - 20612 Apply org.apache.hadoop.hbase.filter.FilterList.<init> new org.apache.hadoop.hbase.filter.FilterList((mrrf: _*))
417 90298 20559 - 20584 Apply scala.Int.> mrrf.lengthCompare(1).>(0)
417 90301 20638 - 20638 TypeApply scala.Predef.$conforms scala.Predef.$conforms[Null]
417 90300 20588 - 20612 Block org.apache.hadoop.hbase.filter.FilterList.<init> new org.apache.hadoop.hbase.filter.FilterList((mrrf: _*))
417 90303 20622 - 20644 Block scala.Option.orNull mrrf.headOption.orNull[org.apache.hadoop.hbase.filter.Filter](scala.Predef.$conforms[Null])
417 90302 20622 - 20644 ApplyToImplicitArgs scala.Option.orNull mrrf.headOption.orNull[org.apache.hadoop.hbase.filter.Filter](scala.Predef.$conforms[Null])
417 90304 20540 - 20647 Apply org.apache.hadoop.hbase.client.Scan.setFilter scan.setFilter(if (mrrf.lengthCompare(1).>(0)) new org.apache.hadoop.hbase.filter.FilterList((mrrf: _*)) else mrrf.headOption.orNull[org.apache.hadoop.hbase.filter.Filter](scala.Predef.$conforms[Null]))
418 90305 20656 - 20709 Apply org.apache.hadoop.hbase.client.Scan.setCacheBlocks scan.addFamily(colFamily).setCacheBlocks(cacheBlocks)
419 90307 20718 - 20752 Apply scala.Option.foreach cacheSize.foreach[org.apache.hadoop.hbase.client.Scan]({ ((x$1: Int) => scan.setCaching(x$1)) })
419 90306 20736 - 20751 Apply org.apache.hadoop.hbase.client.Scan.setCaching scan.setCaching(x$1)
422 90308 20792 - 20814 Apply org.locationtech.geomesa.hbase.data.HBaseDataStore.applySecurity HBaseIndexAdapter.this.ds.applySecurity(scan)
427 90349 20863 - 20863 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]
427 90348 20895 - 22080 Block <nosymbol> { val maxRangesPerGroup: Int = { def calcMax(maxPerGroup: Int, threads: Int): Int = { val totalRanges: Int = rangesPerRegion.values.map[Int, Iterable[Int]](((x$15: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]) => x$15.size()))(collection.this.Iterable.canBuildFrom[Int]).sum[Int](math.this.Numeric.IntIsIntegral); scala.math.`package`.min(maxPerGroup, scala.math.`package`.max(1, scala.math.`package`.ceil(totalRanges.toDouble./(threads)).toInt)) }; if (coprocessor) calcMax(HBaseIndexAdapter.this.ds.config.coprocessors.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.coprocessors.threads) else calcMax(HBaseIndexAdapter.this.ds.config.queries.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.queries.threads) }; val groupedScans: scala.collection.mutable.Builder[org.apache.hadoop.hbase.client.Scan,Seq[org.apache.hadoop.hbase.client.Scan]] = scala.collection.Seq.newBuilder[org.apache.hadoop.hbase.client.Scan]; rangesPerRegion.foreach[Unit](((x0$2: (String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])) => x0$2 match { case (_1: String, _2: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(_, (list @ _)) => { java.util.Collections.sort[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange](list); var i: Int = 0; while$2(){ if (i.<(list.size())) { { val groupSize: Int = scala.math.`package`.min(maxRangesPerGroup, list.size().-(i)); groupedScans.+=(createGroup(list.subList(i, i.+(groupSize)))); i = i.+(groupSize) }; while$2() } else () } } })); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply(table, scala.util.Random.shuffle[org.apache.hadoop.hbase.client.Scan, Seq](groupedScans.result())(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan])) }
427 90350 20844 - 22088 ApplyToImplicitArgs scala.collection.TraversableLike.map rangesPerTable.map[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan, Seq[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan]](((x0$1: (org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])) => x0$1 match { case (_1: org.apache.hadoop.hbase.TableName, _2: scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])(org.apache.hadoop.hbase.TableName, scala.collection.Map[String,java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]])((table @ _), (rangesPerRegion @ _)) => { val maxRangesPerGroup: Int = { def calcMax(maxPerGroup: Int, threads: Int): Int = { val totalRanges: Int = rangesPerRegion.values.map[Int, Iterable[Int]](((x$15: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]) => x$15.size()))(collection.this.Iterable.canBuildFrom[Int]).sum[Int](math.this.Numeric.IntIsIntegral); scala.math.`package`.min(maxPerGroup, scala.math.`package`.max(1, scala.math.`package`.ceil(totalRanges.toDouble./(threads)).toInt)) }; if (coprocessor) calcMax(HBaseIndexAdapter.this.ds.config.coprocessors.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.coprocessors.threads) else calcMax(HBaseIndexAdapter.this.ds.config.queries.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.queries.threads) }; val groupedScans: scala.collection.mutable.Builder[org.apache.hadoop.hbase.client.Scan,Seq[org.apache.hadoop.hbase.client.Scan]] = scala.collection.Seq.newBuilder[org.apache.hadoop.hbase.client.Scan]; rangesPerRegion.foreach[Unit](((x0$2: (String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])) => x0$2 match { case (_1: String, _2: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(_, (list @ _)) => { java.util.Collections.sort[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange](list); var i: Int = 0; while$2(){ if (i.<(list.size())) { { val groupSize: Int = scala.math.`package`.min(maxRangesPerGroup, list.size().-(i)); groupedScans.+=(createGroup(list.subList(i, i.+(groupSize)))); i = i.+(groupSize) }; while$2() } else () } } })); org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply(table, scala.util.Random.shuffle[org.apache.hadoop.hbase.client.Scan, Seq](groupedScans.result())(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan])) } }))(collection.this.Seq.canBuildFrom[org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan])
430 90309 21052 - 21058 Apply java.util.List.size x$15.size()
430 90311 21060 - 21060 Select scala.math.Numeric.IntIsIntegral math.this.Numeric.IntIsIntegral
430 90310 21051 - 21051 TypeApply scala.collection.Iterable.canBuildFrom collection.this.Iterable.canBuildFrom[Int]
430 90312 21025 - 21063 ApplyToImplicitArgs scala.collection.TraversableOnce.sum rangesPerRegion.values.map[Int, Iterable[Int]](((x$15: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]) => x$15.size()))(collection.this.Iterable.canBuildFrom[Int]).sum[Int](math.this.Numeric.IntIsIntegral)
431 90313 21107 - 21108 Literal <nosymbol> 1
431 90315 21110 - 21157 Select scala.Double.toInt scala.math.`package`.ceil(totalRanges.toDouble./(threads)).toInt
431 90314 21120 - 21150 Apply scala.Double./ totalRanges.toDouble./(threads)
431 90317 21076 - 21159 Apply scala.math.min scala.math.`package`.min(maxPerGroup, scala.math.`package`.max(1, scala.math.`package`.ceil(totalRanges.toDouble./(threads)).toInt))
431 90316 21098 - 21158 Apply scala.math.max scala.math.`package`.max(1, scala.math.`package`.ceil(totalRanges.toDouble./(threads)).toInt)
434 90319 21270 - 21300 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.CoprocessorConfig.threads HBaseIndexAdapter.this.ds.config.coprocessors.threads
434 90318 21221 - 21268 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.CoprocessorConfig.maxRangesPerExtendedScan HBaseIndexAdapter.this.ds.config.coprocessors.maxRangesPerExtendedScan
434 90321 21213 - 21301 Block org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.calcMax calcMax(HBaseIndexAdapter.this.ds.config.coprocessors.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.coprocessors.threads)
434 90320 21213 - 21301 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.calcMax calcMax(HBaseIndexAdapter.this.ds.config.coprocessors.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.coprocessors.threads)
436 90323 21385 - 21410 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.HBaseQueryConfig.threads HBaseIndexAdapter.this.ds.config.queries.threads
436 90322 21341 - 21383 Select org.locationtech.geomesa.hbase.data.HBaseDataStoreFactory.HBaseQueryConfig.maxRangesPerExtendedScan HBaseIndexAdapter.this.ds.config.queries.maxRangesPerExtendedScan
436 90325 21333 - 21411 Block org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.calcMax calcMax(HBaseIndexAdapter.this.ds.config.queries.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.queries.threads)
436 90324 21333 - 21411 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.calcMax calcMax(HBaseIndexAdapter.this.ds.config.queries.maxRangesPerExtendedScan, HBaseIndexAdapter.this.ds.config.queries.threads)
440 90326 21462 - 21482 TypeApply scala.collection.Seq.newBuilder scala.collection.Seq.newBuilder[org.apache.hadoop.hbase.client.Scan]
442 90343 21492 - 21912 Apply scala.collection.IterableLike.foreach rangesPerRegion.foreach[Unit](((x0$2: (String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])) => x0$2 match { case (_1: String, _2: java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange])(_, (list @ _)) => { java.util.Collections.sort[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange](list); var i: Int = 0; while$2(){ if (i.<(list.size())) { { val groupSize: Int = scala.math.`package`.min(maxRangesPerGroup, list.size().-(i)); groupedScans.+=(createGroup(list.subList(i, i.+(groupSize)))); i = i.+(groupSize) }; while$2() } else () } } }))
442 90342 21533 - 21902 Block <nosymbol> { java.util.Collections.sort[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange](list); var i: Int = 0; while$2(){ if (i.<(list.size())) { { val groupSize: Int = scala.math.`package`.min(maxRangesPerGroup, list.size().-(i)); groupedScans.+=(createGroup(list.subList(i, i.+(groupSize)))); i = i.+(groupSize) }; while$2() } else () } }
444 90327 21639 - 21661 Apply java.util.Collections.sort java.util.Collections.sort[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange](list)
446 90328 21681 - 21682 Literal <nosymbol> 0
447 90329 21704 - 21715 Apply java.util.List.size list.size()
447 90330 21700 - 21715 Apply scala.Int.< i.<(list.size())
447 90339 21717 - 21902 Block <nosymbol> { { val groupSize: Int = scala.math.`package`.min(maxRangesPerGroup, list.size().-(i)); groupedScans.+=(createGroup(list.subList(i, i.+(groupSize)))); i = i.+(groupSize) }; while$2() }
447 90338 21717 - 21717 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.while$2 while$2()
447 90341 21693 - 21693 Block <nosymbol> ()
447 90340 21693 - 21693 Literal <nosymbol> ()
448 90331 21775 - 21790 Apply scala.Int.- list.size().-(i)
448 90332 21747 - 21791 Apply scala.math.min scala.math.`package`.min(maxRangesPerGroup, list.size().-(i))
449 90333 21848 - 21861 Apply scala.Int.+ i.+(groupSize)
449 90335 21820 - 21863 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.createGroup createGroup(list.subList(i, i.+(groupSize)))
449 90334 21832 - 21862 Apply java.util.List.subList list.subList(i, i.+(groupSize))
449 90336 21804 - 21863 Apply scala.collection.mutable.Builder.+= groupedScans.+=(createGroup(list.subList(i, i.+(groupSize))))
450 90337 21876 - 21890 Apply scala.Int.+ i.+(groupSize)
455 90345 22058 - 22058 TypeApply scala.collection.Seq.canBuildFrom collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan]
455 90344 22059 - 22078 Apply scala.collection.mutable.Builder.result groupedScans.result()
455 90347 22027 - 22080 Apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply org.locationtech.geomesa.hbase.data.HBaseQueryPlan.TableScan.apply(table, scala.util.Random.shuffle[org.apache.hadoop.hbase.client.Scan, Seq](groupedScans.result())(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan]))
455 90346 22044 - 22079 ApplyToImplicitArgs scala.util.Random.shuffle scala.util.Random.shuffle[org.apache.hadoop.hbase.client.Scan, Seq](groupedScans.result())(collection.this.Seq.canBuildFrom[org.apache.hadoop.hbase.client.Scan])
470 90352 22422 - 22490 TypeApply scala.collection.mutable.Map.empty scala.collection.mutable.Map.empty[String, java.util.List[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]]
471 90353 22505 - 22542 Apply org.apache.hadoop.hbase.client.Connection.getRegionLocator HBaseIndexAdapter.this.ds.connection.getRegionLocator(table)
471 90357 22495 - 22624 ApplyToImplicitArgs org.locationtech.geomesa.utils.io.WithClose.apply org.locationtech.geomesa.utils.io.`package`.WithClose.apply[org.apache.hadoop.hbase.client.RegionLocator, Unit](HBaseIndexAdapter.this.ds.connection.getRegionLocator(table))(((locator: org.apache.hadoop.hbase.client.RegionLocator) => ranges.foreach[Unit](((x$16: org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange) => HBaseIndexAdapter.this.groupRange(locator, x$16, rangesPerRegion)))))(io.this.IsCloseable.closeableIsCloseable)
471 90356 22544 - 22544 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.closeableIsCloseable io.this.IsCloseable.closeableIsCloseable
472 90355 22563 - 22618 Apply scala.collection.IterableLike.foreach ranges.foreach[Unit](((x$16: org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange) => HBaseIndexAdapter.this.groupRange(locator, x$16, rangesPerRegion)))
472 90354 22578 - 22617 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.groupRange HBaseIndexAdapter.this.groupRange(locator, x$16, rangesPerRegion)
490 90358 23120 - 23124 Literal <nosymbol> null
491 90359 23154 - 23158 Literal <nosymbol> null
492 90373 23175 - 23956 Block <nosymbol> { val regionInfo: org.apache.hadoop.hbase.HRegionInfo = locator.getRegionLocation(range.getStartRow()).getRegionInfo(); encodedName = regionInfo.getEncodedName(); val regionEndKey: Array[Byte] = regionInfo.getEndKey(); if (scala.Predef.byteArrayOps(regionEndKey).nonEmpty.&&(scala.Predef.byteArrayOps(range.getStopRow()).isEmpty.||(org.locationtech.geomesa.utils.index.ByteArrays.ByteOrdering.compare(regionEndKey, range.getStopRow()).<=(0)))) if (org.locationtech.geomesa.utils.index.ByteArrays.ByteOrdering.compare(range.getStartRow(), regionEndKey).<(0)) split = regionEndKey else (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn(scala.StringContext.apply("HBase region location does not correspond to requested range:\\n").s().+(scala.StringContext.apply(" requested row: ", "\\n").s(org.locationtech.geomesa.utils.index.ByteArrays.toHex(range.getStartRow()))).+(scala.StringContext.apply(" region: ", " ", " :: ", "").s(encodedName, org.locationtech.geomesa.utils.index.ByteArrays.toHex(regionInfo.getStartKey()), org.locationtech.geomesa.utils.index.ByteArrays.toHex(regionEndKey)))) else (): Unit) else () }
493 90360 23192 - 23250 Apply org.apache.hadoop.hbase.HRegionLocation.getRegionInfo locator.getRegionLocation(range.getStartRow()).getRegionInfo()
494 90361 23271 - 23296 Apply org.apache.hadoop.hbase.HRegionInfo.getEncodedName regionInfo.getEncodedName()
495 90362 23322 - 23342 Apply org.apache.hadoop.hbase.HRegionInfo.getEndKey regionInfo.getEndKey()
496 90366 23380 - 23513 Apply scala.Boolean.&& scala.Predef.byteArrayOps(regionEndKey).nonEmpty.&&(scala.Predef.byteArrayOps(range.getStopRow()).isEmpty.||(org.locationtech.geomesa.utils.index.ByteArrays.ByteOrdering.compare(regionEndKey, range.getStopRow()).<=(0)))
496 90371 23376 - 23376 Literal <nosymbol> ()
496 90372 23376 - 23376 Block <nosymbol> ()
497 90363 23416 - 23432 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.BasicRowRange.getStopRow range.getStopRow()
497 90365 23416 - 23512 Apply scala.Boolean.|| scala.Predef.byteArrayOps(range.getStopRow()).isEmpty.||(org.locationtech.geomesa.utils.index.ByteArrays.ByteOrdering.compare(regionEndKey, range.getStopRow()).<=(0))
497 90364 23444 - 23512 Apply scala.Int.<= org.locationtech.geomesa.utils.index.ByteArrays.ByteOrdering.compare(regionEndKey, range.getStopRow()).<=(0)
498 90367 23529 - 23597 Apply scala.Int.< org.locationtech.geomesa.utils.index.ByteArrays.ByteOrdering.compare(range.getStartRow(), regionEndKey).<(0)
498 90370 23525 - 23948 If <nosymbol> if (org.locationtech.geomesa.utils.index.ByteArrays.ByteOrdering.compare(range.getStartRow(), regionEndKey).<(0)) split = regionEndKey else (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn(scala.StringContext.apply("HBase region location does not correspond to requested range:\\n").s().+(scala.StringContext.apply(" requested row: ", "\\n").s(org.locationtech.geomesa.utils.index.ByteArrays.toHex(range.getStartRow()))).+(scala.StringContext.apply(" region: ", " ", " :: ", "").s(encodedName, org.locationtech.geomesa.utils.index.ByteArrays.toHex(regionInfo.getStartKey()), org.locationtech.geomesa.utils.index.ByteArrays.toHex(regionEndKey)))) else (): Unit)
499 90368 23611 - 23631 Assign <nosymbol> split = regionEndKey
501 90369 23659 - 23938 Typed <nosymbol> (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn(scala.StringContext.apply("HBase region location does not correspond to requested range:\\n").s().+(scala.StringContext.apply(" requested row: ", "\\n").s(org.locationtech.geomesa.utils.index.ByteArrays.toHex(range.getStartRow()))).+(scala.StringContext.apply(" region: ", " ", " :: ", "").s(encodedName, org.locationtech.geomesa.utils.index.ByteArrays.toHex(regionInfo.getStartKey()), org.locationtech.geomesa.utils.index.ByteArrays.toHex(regionEndKey)))) else (): Unit)
507 90374 23997 - 24059 Typed <nosymbol> (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn(scala.StringContext.apply("Error checking range location for \'", "\'\'").s(range), e) else (): Unit)
509 90375 24119 - 24144 Apply java.util.ArrayList.<init> new java.util.ArrayList[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]()
509 90376 24083 - 24145 Apply scala.collection.mutable.MapLike.getOrElseUpdate result.getOrElseUpdate(encodedName, new java.util.ArrayList[org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange]())
510 90377 24154 - 24167 Apply java.lang.Object.== split.==(null)
511 90379 24187 - 24187 Literal <nosymbol> ()
511 90378 24177 - 24194 Apply java.util.List.add buffer.add(range)
511 90380 24177 - 24194 Block <nosymbol> { buffer.add(range); () }
512 90391 24206 - 24422 Block <nosymbol> { buffer.add(new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(range.getStartRow(), true, split, false)); HBaseIndexAdapter.this.groupRange(locator, new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(split, true, range.getStopRow(), false), result) }
514 90381 24291 - 24308 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.BasicRowRange.getStartRow range.getStartRow()
514 90383 24323 - 24328 Literal <nosymbol> false
514 90382 24310 - 24314 Literal <nosymbol> true
514 90385 24267 - 24330 Apply java.util.List.add buffer.add(new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(range.getStartRow(), true, split, false))
514 90384 24278 - 24329 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange.<init> new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(range.getStartRow(), true, split, false)
515 90387 24383 - 24399 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.BasicRowRange.getStopRow range.getStopRow()
515 90386 24377 - 24381 Literal <nosymbol> true
515 90389 24357 - 24407 Apply org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange.<init> new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(split, true, range.getStopRow(), false)
515 90388 24401 - 24406 Literal <nosymbol> false
515 90390 24337 - 24416 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.groupRange HBaseIndexAdapter.this.groupRange(locator, new org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange(split, true, range.getStopRow(), false), result)
522 90392 24520 - 24582 Apply java.util.regex.Pattern.compile java.util.regex.Pattern.compile("^geomesa-hbase-distributed-runtime.*\\.jar$")
525 90393 24687 - 24757 Literal <nosymbol> "org.locationtech.geomesa.hbase.server.coprocessor.GeoMesaCoprocessor"
526 90394 24784 - 24830 Literal <nosymbol> "org.locationtech.geomesa.hbase.server.common"
528 90395 24863 - 24905 Select org.locationtech.geomesa.utils.conf.GeoMesaSystemProperties.SystemProperty.option org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.WalDurability.option
530 90397 24967 - 25001 Apply java.lang.String.equalsIgnoreCase x$17.toString().equalsIgnoreCase(value)
530 90396 24944 - 24961 Apply org.apache.hadoop.hbase.client.Durability.values org.apache.hadoop.hbase.client.Durability.values()
530 90399 24944 - 25159 Apply scala.Option.getOrElse scala.Predef.refArrayOps[org.apache.hadoop.hbase.client.Durability](org.apache.hadoop.hbase.client.Durability.values()).find(((x$17: org.apache.hadoop.hbase.client.Durability) => x$17.toString().equalsIgnoreCase(value))).getOrElse[org.apache.hadoop.hbase.client.Durability]({ (if (HBaseIndexAdapter.this.logger.underlying.isErrorEnabled()) HBaseIndexAdapter.this.logger.underlying.error("Invalid HBase WAL durability setting: {}. Falling back to default durability", (value: AnyRef)) else (): Unit); USE_DEFAULT })
530 90400 24944 - 25159 Block scala.Option.getOrElse scala.Predef.refArrayOps[org.apache.hadoop.hbase.client.Durability](org.apache.hadoop.hbase.client.Durability.values()).find(((x$17: org.apache.hadoop.hbase.client.Durability) => x$17.toString().equalsIgnoreCase(value))).getOrElse[org.apache.hadoop.hbase.client.Durability]({ (if (HBaseIndexAdapter.this.logger.underlying.isErrorEnabled()) HBaseIndexAdapter.this.logger.underlying.error("Invalid HBase WAL durability setting: {}. Falling back to default durability", (value: AnyRef)) else (): Unit); USE_DEFAULT })
532 90398 25129 - 25151 Literal <nosymbol> USE_DEFAULT
534 90401 25177 - 25199 Literal <nosymbol> USE_DEFAULT
534 90402 25177 - 25199 Block <nosymbol> USE_DEFAULT
544 90403 25408 - 25438 Select scala.Boolean.unary_! admin.isTableAvailable(table).unary_!
544 90419 25404 - 25404 Literal <nosymbol> ()
544 90418 25440 - 25890 Block <nosymbol> { val timeout: Option[scala.concurrent.duration.Duration] = org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.TableAvailabilityTimeout.toUnboundedDuration.filter(((x$18: scala.concurrent.duration.Duration) => x$18.isFinite())); (if (HBaseIndexAdapter.this.logger.underlying.isDebugEnabled()) HBaseIndexAdapter.this.logger.underlying.debug(scala.StringContext.apply("Waiting for table \'", "\' to become available with ").s(table).+(scala.StringContext.apply("", "").s(timeout.map[String](((t: scala.concurrent.duration.Duration) => scala.StringContext.apply("a timeout of ", "").s(t))).getOrElse[String]("no timeout")))) else (): Unit); val stop: Option[Long] = timeout.map[Long](((t: scala.concurrent.duration.Duration) => java.lang.System.currentTimeMillis().+(t.toMillis))); while$3(){ if (admin.isTableAvailable(table).unary_!.&&(stop.forall(((x$19: Long) => x$19.>(java.lang.System.currentTimeMillis()))))) { java.lang.Thread.sleep(1000L); while$3() } else () } }
544 90420 25404 - 25404 Block <nosymbol> ()
545 90405 25462 - 25525 Apply scala.Option.filter org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.TableAvailabilityTimeout.toUnboundedDuration.filter(((x$18: scala.concurrent.duration.Duration) => x$18.isFinite()))
545 90404 25514 - 25524 Apply scala.concurrent.duration.Duration.isFinite x$18.isFinite()
548 90407 25715 - 25754 Apply scala.Long.+ java.lang.System.currentTimeMillis().+(t.toMillis)
548 90406 25744 - 25754 Select scala.concurrent.duration.Duration.toMillis t.toMillis
548 90408 25698 - 25755 Apply scala.Option.map timeout.map[Long](((t: scala.concurrent.duration.Duration) => java.lang.System.currentTimeMillis().+(t.toMillis)))
549 90409 25819 - 25845 Apply java.lang.System.currentTimeMillis java.lang.System.currentTimeMillis()
549 90411 25803 - 25846 Apply scala.Option.forall stop.forall(((x$19: Long) => x$19.>(java.lang.System.currentTimeMillis())))
549 90410 25815 - 25845 Apply scala.Long.> x$19.>(java.lang.System.currentTimeMillis())
549 90412 25769 - 25846 Apply scala.Boolean.&& admin.isTableAvailable(table).unary_!.&&(stop.forall(((x$19: Long) => x$19.>(java.lang.System.currentTimeMillis()))))
549 90415 25858 - 25876 Block <nosymbol> { java.lang.Thread.sleep(1000L); while$3() }
549 90417 25762 - 25762 Block <nosymbol> ()
549 90416 25762 - 25762 Literal <nosymbol> ()
550 90413 25858 - 25876 Apply java.lang.Thread.sleep java.lang.Thread.sleep(1000L)
550 90414 25870 - 25870 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.while$3 while$3()
564 90421 26181 - 26197 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseResultsToFeatures.<init> HBaseResultsToFeatures.this.<init>(null, null)
564 90422 26181 - 26181 Literal <nosymbol> ()
567 90423 26323 - 26343 Apply scala.Array.apply result.rawCells().apply(0)
568 90425 26396 - 26413 Apply org.apache.hadoop.hbase.Cell.getRowOffset cell.getRowOffset()
568 90424 26378 - 26394 Apply org.apache.hadoop.hbase.Cell.getRowArray cell.getRowArray()
568 90427 26434 - 26438 Literal <nosymbol> null
568 90426 26415 - 26432 Select scala.Short.toInt cell.getRowLength().toInt
568 90428 26359 - 26439 Apply org.locationtech.geomesa.index.api.GeoMesaFeatureIndex.getIdFromRow HBaseResultsToFeatures.this.index.getIdFromRow(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength().toInt, null)
569 90429 26473 - 26491 Apply org.apache.hadoop.hbase.Cell.getValueArray cell.getValueArray()
569 90431 26514 - 26533 Apply org.apache.hadoop.hbase.Cell.getValueLength cell.getValueLength()
569 90430 26493 - 26512 Apply org.apache.hadoop.hbase.Cell.getValueOffset cell.getValueOffset()
569 90432 26446 - 26534 Apply org.locationtech.geomesa.features.SimpleFeatureSerializer.deserialize HBaseResultsToFeatures.this.serializer.deserialize(id, cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())
589 90433 27054 - 27097 Select org.locationtech.geomesa.utils.conf.GeoMesaSystemProperties.SystemProperty.toLong org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.WriteBatchSize.toLong
590 90434 27129 - 27175 Select org.locationtech.geomesa.utils.conf.GeoMesaSystemProperties.SystemProperty.toLong org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.WriteFlushTimeout.toLong
591 90435 27247 - 27268 Apply org.apache.hadoop.hbase.security.visibility.CellVisibility.<init> new org.apache.hadoop.hbase.security.visibility.CellVisibility(x$20)
591 90436 27204 - 27269 Apply scala.Option.map org.locationtech.geomesa.hbase.`package`.HBaseSystemProperties.DeleteVis.option.map[org.apache.hadoop.hbase.security.visibility.CellVisibility](((x$20: String) => new org.apache.hadoop.hbase.security.visibility.CellVisibility(x$20)))
595 90437 27381 - 27382 Literal <nosymbol> 1
595 90439 27372 - 27464 Apply scala.math.max scala.math.`package`.max(1, HBaseIndexWriter.this.ds.connection.getConfiguration().getInt("hbase.htable.threads.max", 2147483647))
595 90438 27384 - 27463 Apply org.apache.hadoop.conf.Configuration.getInt HBaseIndexWriter.this.ds.connection.getConfiguration().getInt("hbase.htable.threads.max", 2147483647)
596 90441 27498 - 27530 Apply org.locationtech.geomesa.utils.concurrent.CachedThreadPool.<init> new org.locationtech.geomesa.utils.concurrent.CachedThreadPool(maxThreads)
596 90440 27482 - 27496 Select scala.collection.SeqLike.length HBaseIndexWriter.this.indices.length
596 90442 27471 - 27531 ApplyToImplicitArgs scala.Array.fill scala.Array.fill[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](HBaseIndexWriter.this.indices.length)(new org.locationtech.geomesa.utils.concurrent.CachedThreadPool(maxThreads))((ClassTag.apply[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](classOf[org.locationtech.geomesa.utils.concurrent.CachedThreadPool]): scala.reflect.ClassTag[org.locationtech.geomesa.utils.concurrent.CachedThreadPool]))
599 90443 27566 - 27581 ApplyToImplicitArgs scala.collection.TraversableOnce.toArray HBaseIndexWriter.this.indices.toArray[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]]((ClassTag.apply[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]](classOf[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex]): scala.reflect.ClassTag[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]]))
599 90457 27566 - 28242 ApplyToImplicitArgs scala.collection.TraversableLike.map scala.Predef.refArrayOps[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]](HBaseIndexWriter.this.indices.toArray[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]]((ClassTag.apply[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]](classOf[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex]): scala.reflect.ClassTag[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]]))).map[org.apache.hadoop.hbase.client.BufferedMutator, Array[org.apache.hadoop.hbase.client.BufferedMutator]](((index: org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]) => { val table: String = index.getTableName(HBaseIndexWriter.this.partition); val params: org.apache.hadoop.hbase.client.BufferedMutatorParams = new org.apache.hadoop.hbase.client.BufferedMutatorParams(org.apache.hadoop.hbase.TableName.valueOf(table)); HBaseIndexWriter.this.batchSize.foreach[org.apache.hadoop.hbase.client.BufferedMutatorParams]({ ((x$1: Long) => params.writeBufferSize(x$1)) }); HBaseIndexWriter.this.flushTimeout.foreach[org.apache.hadoop.hbase.client.BufferedMutatorParams]({ ((x$1: Long) => params.setWriteBufferPeriodicFlushTimeoutMs(x$1)) }); params.pool(HBaseIndexWriter.this.pools.apply(HBaseIndexWriter.this.indices.indexOf[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]](index))); HBaseIndexWriter.this.ds.connection.getBufferedMutator(params) }))(scala.this.Array.canBuildFrom[org.apache.hadoop.hbase.client.BufferedMutator]((ClassTag.apply[org.apache.hadoop.hbase.client.BufferedMutator](classOf[org.apache.hadoop.hbase.client.BufferedMutator]): scala.reflect.ClassTag[org.apache.hadoop.hbase.client.BufferedMutator])))
599 90456 27586 - 27586 ApplyToImplicitArgs scala.Array.canBuildFrom scala.this.Array.canBuildFrom[org.apache.hadoop.hbase.client.BufferedMutator]((ClassTag.apply[org.apache.hadoop.hbase.client.BufferedMutator](classOf[org.apache.hadoop.hbase.client.BufferedMutator]): scala.reflect.ClassTag[org.apache.hadoop.hbase.client.BufferedMutator]))
601 90445 27672 - 27701 Apply org.locationtech.geomesa.index.api.GeoMesaFeatureIndex.getTableName index.getTableName(HBaseIndexWriter.this.partition)
601 90444 27691 - 27700 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.partition HBaseIndexWriter.this.partition
602 90447 27721 - 27772 Apply org.apache.hadoop.hbase.client.BufferedMutatorParams.<init> new org.apache.hadoop.hbase.client.BufferedMutatorParams(org.apache.hadoop.hbase.TableName.valueOf(table))
602 90446 27747 - 27771 Apply org.apache.hadoop.hbase.TableName.valueOf org.apache.hadoop.hbase.TableName.valueOf(table)
603 90449 27779 - 27820 Apply scala.Option.foreach HBaseIndexWriter.this.batchSize.foreach[org.apache.hadoop.hbase.client.BufferedMutatorParams]({ ((x$1: Long) => params.writeBufferSize(x$1)) })
603 90448 27797 - 27819 Apply org.apache.hadoop.hbase.client.BufferedMutatorParams.writeBufferSize params.writeBufferSize(x$1)
604 90451 27827 - 27892 Apply scala.Option.foreach HBaseIndexWriter.this.flushTimeout.foreach[org.apache.hadoop.hbase.client.BufferedMutatorParams]({ ((x$1: Long) => params.setWriteBufferPeriodicFlushTimeoutMs(x$1)) })
604 90450 27848 - 27891 Apply org.apache.hadoop.hbase.client.BufferedMutatorParams.setWriteBufferPeriodicFlushTimeoutMs params.setWriteBufferPeriodicFlushTimeoutMs(x$1)
609 90453 28159 - 28188 Apply scala.Array.apply HBaseIndexWriter.this.pools.apply(HBaseIndexWriter.this.indices.indexOf[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]](index))
609 90452 28165 - 28187 Apply scala.collection.GenSeqLike.indexOf HBaseIndexWriter.this.indices.indexOf[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]](index)
609 90454 28147 - 28189 Apply org.apache.hadoop.hbase.client.BufferedMutatorParams.pool params.pool(HBaseIndexWriter.this.pools.apply(HBaseIndexWriter.this.indices.indexOf[org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]](index)))
610 90455 28196 - 28236 Apply org.apache.hadoop.hbase.client.Connection.getBufferedMutator HBaseIndexWriter.this.ds.connection.getBufferedMutator(params)
613 90459 28300 - 28326 Select org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType.getFeatureExpiration org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType(x$21.sft).getFeatureExpiration
613 90458 28300 - 28305 Select org.locationtech.geomesa.index.api.GeoMesaFeatureIndex.sft x$21.sft
613 90461 28273 - 28334 ApplyToImplicitArgs scala.Option.orNull HBaseIndexWriter.this.indices.headOption.flatMap[org.locationtech.geomesa.utils.conf.FeatureExpiration](((x$21: org.locationtech.geomesa.index.api.GeoMesaFeatureIndex[_, _]) => org.locationtech.geomesa.utils.geotools.RichSimpleFeatureType.RichSimpleFeatureType(x$21.sft).getFeatureExpiration)).orNull[org.locationtech.geomesa.utils.conf.FeatureExpiration](scala.Predef.$conforms[Null])
613 90460 28328 - 28328 TypeApply scala.Predef.$conforms scala.Predef.$conforms[Null]
615 90462 28356 - 28357 Literal <nosymbol> 0
618 90463 28480 - 28498 Apply java.lang.Object.!= HBaseIndexWriter.this.expiration.!=(null)
618 90471 28500 - 28753 Block <nosymbol> { val t: Long = HBaseIndexWriter.this.expiration.expires(feature.feature).-(java.lang.System.currentTimeMillis()); if (t.>(0)) t else { (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn("Feature is already past its TTL; not added to database") else (): Unit); return () } }
619 90465 28556 - 28580 Apply java.lang.System.currentTimeMillis java.lang.System.currentTimeMillis()
619 90464 28537 - 28552 Select org.locationtech.geomesa.index.api.WritableFeature.feature feature.feature
619 90466 28518 - 28580 Apply scala.Long.- HBaseIndexWriter.this.expiration.expires(feature.feature).-(java.lang.System.currentTimeMillis())
620 90467 28593 - 28598 Apply scala.Long.> t.>(0)
621 90468 28612 - 28613 Ident org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.t t
623 90470 28637 - 28745 Block <nosymbol> { (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn("Feature is already past its TTL; not added to database") else (): Unit); return () }
625 90469 28729 - 28735 Literal <nosymbol> ()
628 90473 28769 - 28771 Block <nosymbol> 0L
628 90472 28769 - 28771 Literal <nosymbol> 0L
631 90474 28787 - 28792 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i_= HBaseIndexWriter.this.i_=(0)
632 90475 28810 - 28823 Select scala.Array.length values.length
632 90476 28806 - 28823 Apply scala.Int.< HBaseIndexWriter.this.i.<(values.length)
632 90533 28825 - 28825 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.while$4 while$4()
632 90535 28799 - 28799 Literal <nosymbol> ()
632 90534 28825 - 29958 Block <nosymbol> { { val mutator: org.apache.hadoop.hbase.client.BufferedMutator = HBaseIndexWriter.this.mutators.apply(HBaseIndexWriter.this.i); values.apply(HBaseIndexWriter.this.i) match { case (kv @ (_: org.locationtech.geomesa.index.api.SingleRowKeyValue[_])) => kv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val put: org.apache.hadoop.hbase.client.Put = new org.apache.hadoop.hbase.client.Put(kv.row); put.addImmutable(value.cf, value.cq, value.value); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else (); put.setDurability(HBaseIndexAdapter.this.durability); if (ttl.>(0)) put.setTTL(ttl) else (); mutator.mutate(put) })) case (mkv @ (_: org.locationtech.geomesa.index.api.MultiRowKeyValue[_])) => mkv.rows.foreach[Unit](((row: Array[Byte]) => mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val put: org.apache.hadoop.hbase.client.Put = new org.apache.hadoop.hbase.client.Put(row); put.addImmutable(value.cf, value.cq, value.value); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else (); put.setDurability(HBaseIndexAdapter.this.durability); if (ttl.>(0)) put.setTTL(ttl) else (); mutator.mutate(put) })))) }; HBaseIndexWriter.this.i_=(HBaseIndexWriter.this.i.+(1)) }; while$4() }
632 90536 28799 - 28799 Block <nosymbol> ()
633 90477 28858 - 28859 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i HBaseIndexWriter.this.i
633 90478 28849 - 28860 Apply scala.Array.apply HBaseIndexWriter.this.mutators.apply(HBaseIndexWriter.this.i)
634 90479 28876 - 28877 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i HBaseIndexWriter.this.i
634 90480 28869 - 28878 Apply scala.Array.apply values.apply(HBaseIndexWriter.this.i)
636 90505 28942 - 29370 Block scala.collection.IterableLike.foreach kv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val put: org.apache.hadoop.hbase.client.Put = new org.apache.hadoop.hbase.client.Put(kv.row); put.addImmutable(value.cf, value.cq, value.value); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else (); put.setDurability(HBaseIndexAdapter.this.durability); if (ttl.>(0)) put.setTTL(ttl) else (); mutator.mutate(put) }))
636 90504 28942 - 29370 Apply scala.collection.IterableLike.foreach kv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val put: org.apache.hadoop.hbase.client.Put = new org.apache.hadoop.hbase.client.Put(kv.row); put.addImmutable(value.cf, value.cq, value.value); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else (); put.setDurability(HBaseIndexAdapter.this.durability); if (ttl.>(0)) put.setTTL(ttl) else (); mutator.mutate(put) }))
637 90481 29003 - 29009 Select org.locationtech.geomesa.index.api.SingleRowKeyValue.row kv.row
637 90482 28995 - 29010 Apply org.apache.hadoop.hbase.client.Put.<init> new org.apache.hadoop.hbase.client.Put(kv.row)
638 90483 29042 - 29050 Select org.locationtech.geomesa.index.api.KeyValue.cf value.cf
638 90485 29025 - 29074 Apply org.apache.hadoop.hbase.client.Put.addImmutable put.addImmutable(value.cf, value.cq, value.value)
638 90484 29052 - 29060 Select org.locationtech.geomesa.index.api.KeyValue.cq value.cq
639 90487 29093 - 29111 Select scala.Boolean.unary_! scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!
639 90486 29094 - 29103 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
639 90495 29089 - 29089 Block <nosymbol> ()
639 90494 29089 - 29089 Literal <nosymbol> ()
640 90489 29194 - 29216 Select java.nio.charset.StandardCharsets.UTF_8 java.nio.charset.StandardCharsets.UTF_8
640 90488 29183 - 29192 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
640 90491 29153 - 29218 Apply org.apache.hadoop.hbase.security.visibility.CellVisibility.<init> new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))
640 90490 29172 - 29217 Apply java.lang.String.<init> new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)
640 90493 29131 - 29219 Block org.apache.hadoop.hbase.client.Put.setCellVisibility put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
640 90492 29131 - 29219 Apply org.apache.hadoop.hbase.client.Put.setCellVisibility put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
642 90497 29250 - 29279 Apply org.apache.hadoop.hbase.client.Put.setDurability put.setDurability(HBaseIndexAdapter.this.durability)
642 90496 29268 - 29278 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.durability HBaseIndexAdapter.this.durability
643 90499 29307 - 29322 Apply org.apache.hadoop.hbase.client.Put.setTTL put.setTTL(ttl)
643 90498 29298 - 29305 Apply scala.Long.> ttl.>(0)
643 90501 29294 - 29294 Literal <nosymbol> ()
643 90500 29307 - 29322 Block org.apache.hadoop.hbase.client.Put.setTTL put.setTTL(ttl)
643 90502 29294 - 29294 Block <nosymbol> ()
644 90503 29337 - 29356 Apply org.apache.hadoop.hbase.client.BufferedMutator.mutate mutator.mutate(put)
648 90529 29427 - 29925 Apply scala.collection.IterableLike.foreach mkv.rows.foreach[Unit](((row: Array[Byte]) => mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val put: org.apache.hadoop.hbase.client.Put = new org.apache.hadoop.hbase.client.Put(row); put.addImmutable(value.cf, value.cq, value.value); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else (); put.setDurability(HBaseIndexAdapter.this.durability); if (ttl.>(0)) put.setTTL(ttl) else (); mutator.mutate(put) }))))
648 90530 29427 - 29925 Block scala.collection.IterableLike.foreach mkv.rows.foreach[Unit](((row: Array[Byte]) => mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val put: org.apache.hadoop.hbase.client.Put = new org.apache.hadoop.hbase.client.Put(row); put.addImmutable(value.cf, value.cq, value.value); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else (); put.setDurability(HBaseIndexAdapter.this.durability); if (ttl.>(0)) put.setTTL(ttl) else (); mutator.mutate(put) }))))
649 90528 29467 - 29911 Apply scala.collection.IterableLike.foreach mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val put: org.apache.hadoop.hbase.client.Put = new org.apache.hadoop.hbase.client.Put(row); put.addImmutable(value.cf, value.cq, value.value); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else (); put.setDurability(HBaseIndexAdapter.this.durability); if (ttl.>(0)) put.setTTL(ttl) else (); mutator.mutate(put) }))
650 90506 29523 - 29535 Apply org.apache.hadoop.hbase.client.Put.<init> new org.apache.hadoop.hbase.client.Put(row)
651 90507 29569 - 29577 Select org.locationtech.geomesa.index.api.KeyValue.cf value.cf
651 90509 29552 - 29601 Apply org.apache.hadoop.hbase.client.Put.addImmutable put.addImmutable(value.cf, value.cq, value.value)
651 90508 29579 - 29587 Select org.locationtech.geomesa.index.api.KeyValue.cq value.cq
652 90511 29622 - 29640 Select scala.Boolean.unary_! scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!
652 90510 29623 - 29632 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
652 90519 29618 - 29618 Block <nosymbol> ()
652 90518 29618 - 29618 Literal <nosymbol> ()
653 90513 29725 - 29747 Select java.nio.charset.StandardCharsets.UTF_8 java.nio.charset.StandardCharsets.UTF_8
653 90512 29714 - 29723 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
653 90515 29684 - 29749 Apply org.apache.hadoop.hbase.security.visibility.CellVisibility.<init> new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))
653 90514 29703 - 29748 Apply java.lang.String.<init> new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)
653 90517 29662 - 29750 Block org.apache.hadoop.hbase.client.Put.setCellVisibility put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
653 90516 29662 - 29750 Apply org.apache.hadoop.hbase.client.Put.setCellVisibility put.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
655 90521 29785 - 29814 Apply org.apache.hadoop.hbase.client.Put.setDurability put.setDurability(HBaseIndexAdapter.this.durability)
655 90520 29803 - 29813 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.durability HBaseIndexAdapter.this.durability
656 90523 29844 - 29859 Apply org.apache.hadoop.hbase.client.Put.setTTL put.setTTL(ttl)
656 90522 29835 - 29842 Apply scala.Long.> ttl.>(0)
656 90525 29831 - 29831 Literal <nosymbol> ()
656 90524 29844 - 29859 Block org.apache.hadoop.hbase.client.Put.setTTL put.setTTL(ttl)
656 90526 29831 - 29831 Block <nosymbol> ()
657 90527 29876 - 29895 Apply org.apache.hadoop.hbase.client.BufferedMutator.mutate mutator.mutate(put)
661 90531 29944 - 29950 Apply scala.Int.+ HBaseIndexWriter.this.i.+(1)
661 90532 29944 - 29950 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i_= HBaseIndexWriter.this.i_=(HBaseIndexWriter.this.i.+(1))
670 90537 30172 - 30204 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.delete HBaseIndexWriter.this.delete(previous, previousValues)
672 90538 30285 - 30292 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.flush HBaseIndexWriter.this.flush()
673 90539 30299 - 30314 Apply java.lang.Thread.sleep java.lang.Thread.sleep(1L)
674 90540 30321 - 30344 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.append HBaseIndexWriter.this.append(feature, values)
678 90541 30459 - 30464 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i_= HBaseIndexWriter.this.i_=(0)
679 90543 30478 - 30495 Apply scala.Int.< HBaseIndexWriter.this.i.<(values.length)
679 90542 30482 - 30495 Select scala.Array.length values.length
679 90587 30497 - 31722 Block <nosymbol> { { val mutator: org.apache.hadoop.hbase.client.BufferedMutator = HBaseIndexWriter.this.mutators.apply(HBaseIndexWriter.this.i); values.apply(HBaseIndexWriter.this.i) match { case (kv @ (_: org.locationtech.geomesa.index.api.SingleRowKeyValue[_])) => kv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val del: org.apache.hadoop.hbase.client.Delete = new org.apache.hadoop.hbase.client.Delete(kv.row); del.addFamily(value.cf); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) }); mutator.mutate(del) })) case (mkv @ (_: org.locationtech.geomesa.index.api.MultiRowKeyValue[_])) => mkv.rows.foreach[Unit](((row: Array[Byte]) => mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val del: org.apache.hadoop.hbase.client.Delete = new org.apache.hadoop.hbase.client.Delete(row); del.addFamily(value.cf); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) }); mutator.mutate(del) })))) }; HBaseIndexWriter.this.i_=(HBaseIndexWriter.this.i.+(1)) }; while$5() }
679 90586 30497 - 30497 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.while$5 while$5()
679 90589 30471 - 30471 Block <nosymbol> ()
679 90588 30471 - 30471 Literal <nosymbol> ()
680 90545 30521 - 30532 Apply scala.Array.apply HBaseIndexWriter.this.mutators.apply(HBaseIndexWriter.this.i)
680 90544 30530 - 30531 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i HBaseIndexWriter.this.i
681 90547 30541 - 30550 Apply scala.Array.apply values.apply(HBaseIndexWriter.this.i)
681 90546 30548 - 30549 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i HBaseIndexWriter.this.i
683 90565 30614 - 31088 Block scala.collection.IterableLike.foreach kv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val del: org.apache.hadoop.hbase.client.Delete = new org.apache.hadoop.hbase.client.Delete(kv.row); del.addFamily(value.cf); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) }); mutator.mutate(del) }))
683 90564 30614 - 31088 Apply scala.collection.IterableLike.foreach kv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val del: org.apache.hadoop.hbase.client.Delete = new org.apache.hadoop.hbase.client.Delete(kv.row); del.addFamily(value.cf); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) }); mutator.mutate(del) }))
684 90549 30667 - 30685 Apply org.apache.hadoop.hbase.client.Delete.<init> new org.apache.hadoop.hbase.client.Delete(kv.row)
684 90548 30678 - 30684 Select org.locationtech.geomesa.index.api.SingleRowKeyValue.row kv.row
685 90551 30700 - 30723 Apply org.apache.hadoop.hbase.client.Delete.addFamily del.addFamily(value.cf)
685 90550 30714 - 30722 Select org.locationtech.geomesa.index.api.KeyValue.cf value.cf
686 90553 30818 - 30836 Select scala.Boolean.unary_! scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!
686 90552 30819 - 30828 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
687 90555 30919 - 30941 Select java.nio.charset.StandardCharsets.UTF_8 java.nio.charset.StandardCharsets.UTF_8
687 90554 30908 - 30917 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
687 90557 30878 - 30943 Apply org.apache.hadoop.hbase.security.visibility.CellVisibility.<init> new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))
687 90556 30897 - 30942 Apply java.lang.String.<init> new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)
687 90559 30856 - 30944 Block org.apache.hadoop.hbase.client.Delete.setCellVisibility del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
687 90558 30856 - 30944 Apply org.apache.hadoop.hbase.client.Delete.setCellVisibility del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
689 90561 30984 - 31024 Apply scala.Option.foreach HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) })
689 90560 31002 - 31023 Apply org.apache.hadoop.hbase.client.Delete.setCellVisibility del.setCellVisibility(x$1)
689 90562 30984 - 31024 Block scala.Option.foreach HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) })
691 90563 31055 - 31074 Apply org.apache.hadoop.hbase.client.BufferedMutator.mutate mutator.mutate(del)
695 90583 31145 - 31689 Block scala.collection.IterableLike.foreach mkv.rows.foreach[Unit](((row: Array[Byte]) => mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val del: org.apache.hadoop.hbase.client.Delete = new org.apache.hadoop.hbase.client.Delete(row); del.addFamily(value.cf); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) }); mutator.mutate(del) }))))
695 90582 31145 - 31689 Apply scala.collection.IterableLike.foreach mkv.rows.foreach[Unit](((row: Array[Byte]) => mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val del: org.apache.hadoop.hbase.client.Delete = new org.apache.hadoop.hbase.client.Delete(row); del.addFamily(value.cf); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) }); mutator.mutate(del) }))))
696 90581 31185 - 31675 Apply scala.collection.IterableLike.foreach mkv.values.foreach[Unit](((value: org.locationtech.geomesa.index.api.KeyValue) => { val del: org.apache.hadoop.hbase.client.Delete = new org.apache.hadoop.hbase.client.Delete(row); del.addFamily(value.cf); if (scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!) del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))) else HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) }); mutator.mutate(del) }))
697 90566 31241 - 31256 Apply org.apache.hadoop.hbase.client.Delete.<init> new org.apache.hadoop.hbase.client.Delete(row)
698 90567 31287 - 31295 Select org.locationtech.geomesa.index.api.KeyValue.cf value.cf
698 90568 31273 - 31296 Apply org.apache.hadoop.hbase.client.Delete.addFamily del.addFamily(value.cf)
699 90569 31394 - 31403 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
699 90570 31393 - 31411 Select scala.Boolean.unary_! scala.Predef.byteArrayOps(value.vis).isEmpty.unary_!
700 90571 31485 - 31494 Select org.locationtech.geomesa.index.api.KeyValue.vis value.vis
700 90573 31474 - 31519 Apply java.lang.String.<init> new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)
700 90572 31496 - 31518 Select java.nio.charset.StandardCharsets.UTF_8 java.nio.charset.StandardCharsets.UTF_8
700 90575 31433 - 31521 Apply org.apache.hadoop.hbase.client.Delete.setCellVisibility del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
700 90574 31455 - 31520 Apply org.apache.hadoop.hbase.security.visibility.CellVisibility.<init> new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8))
700 90576 31433 - 31521 Block org.apache.hadoop.hbase.client.Delete.setCellVisibility del.setCellVisibility(new org.apache.hadoop.hbase.security.visibility.CellVisibility(new scala.Predef.String(value.vis, java.nio.charset.StandardCharsets.UTF_8)))
702 90577 31583 - 31604 Apply org.apache.hadoop.hbase.client.Delete.setCellVisibility del.setCellVisibility(x$1)
702 90579 31565 - 31605 Block scala.Option.foreach HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) })
702 90578 31565 - 31605 Apply scala.Option.foreach HBaseIndexWriter.this.deleteVis.foreach[org.apache.hadoop.hbase.client.Delete]({ ((x$1: org.apache.hadoop.hbase.security.visibility.CellVisibility) => del.setCellVisibility(x$1)) })
704 90580 31640 - 31659 Apply org.apache.hadoop.hbase.client.BufferedMutator.mutate mutator.mutate(del)
708 90585 31708 - 31714 Apply org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.i_= HBaseIndexWriter.this.i_=(HBaseIndexWriter.this.i.+(1))
708 90584 31708 - 31714 Apply scala.Int.+ HBaseIndexWriter.this.i.+(1)
712 90591 31796 - 31839 Select org.locationtech.geomesa.utils.io.IsFlushableImplicits.arrayIsFlushable HBaseIndexAdapter.this.BufferedMutatorIsFlushable.arrayIsFlushable
712 90590 31786 - 31794 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.mutators HBaseIndexWriter.this.mutators
712 90592 31763 - 31840 Apply org.locationtech.geomesa.utils.io.SafeFlush.raise org.locationtech.geomesa.utils.io.`package`.FlushWithLogging.raise[Array[org.apache.hadoop.hbase.client.BufferedMutator]](HBaseIndexWriter.this.mutators)(HBaseIndexAdapter.this.BufferedMutatorIsFlushable.arrayIsFlushable)
715 90593 31912 - 31920 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.mutators HBaseIndexWriter.this.mutators
715 90595 31889 - 31921 ApplyToImplicitArgs org.locationtech.geomesa.utils.io.SafeClose.raise org.locationtech.geomesa.utils.io.`package`.CloseWithLogging.raise[Array[org.apache.hadoop.hbase.client.BufferedMutator]](HBaseIndexWriter.this.mutators)(io.this.IsCloseable.arrayIsCloseable)
715 90594 31911 - 31911 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.arrayIsCloseable io.this.IsCloseable.arrayIsCloseable
715 90596 31889 - 31921 Block org.locationtech.geomesa.utils.io.SafeClose.raise org.locationtech.geomesa.utils.io.`package`.CloseWithLogging.raise[Array[org.apache.hadoop.hbase.client.BufferedMutator]](HBaseIndexWriter.this.mutators)(io.this.IsCloseable.arrayIsCloseable)
716 90597 31942 - 31947 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.pools HBaseIndexWriter.this.pools
716 90599 31956 - 32015 Apply org.locationtech.geomesa.utils.io.CloseWithLogging.apply org.locationtech.geomesa.utils.io.`package`.CloseWithLogging.apply[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](x$22)(org.locationtech.geomesa.utils.io.IsCloseable.executorServiceIsCloseable)
716 90598 31976 - 32014 Select org.locationtech.geomesa.utils.io.IsCloseableImplicits.executorServiceIsCloseable org.locationtech.geomesa.utils.io.IsCloseable.executorServiceIsCloseable
716 90601 31942 - 32016 Block scala.collection.IndexedSeqOptimized.foreach scala.Predef.refArrayOps[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](HBaseIndexWriter.this.pools).foreach[Option[Throwable]](((x$22: org.locationtech.geomesa.utils.concurrent.CachedThreadPool) => org.locationtech.geomesa.utils.io.`package`.CloseWithLogging.apply[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](x$22)(org.locationtech.geomesa.utils.io.IsCloseable.executorServiceIsCloseable)))
716 90600 31942 - 32016 Apply scala.collection.IndexedSeqOptimized.foreach scala.Predef.refArrayOps[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](HBaseIndexWriter.this.pools).foreach[Option[Throwable]](((x$22: org.locationtech.geomesa.utils.concurrent.CachedThreadPool) => org.locationtech.geomesa.utils.io.`package`.CloseWithLogging.apply[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](x$22)(org.locationtech.geomesa.utils.io.IsCloseable.executorServiceIsCloseable)))
718 90603 32051 - 32055 Literal <nosymbol> true
718 90602 32036 - 32041 Select org.locationtech.geomesa.hbase.data.HBaseIndexAdapter.HBaseIndexWriter.pools HBaseIndexWriter.this.pools
718 90605 32086 - 32143 Apply scala.Boolean.&& terminated.&&(pool.awaitTermination(60L, SECONDS))
718 90604 32100 - 32143 Apply org.locationtech.geomesa.utils.concurrent.CachedThreadPool.awaitTermination pool.awaitTermination(60L, SECONDS)
718 90607 32035 - 32145 Select scala.Boolean.unary_! scala.Predef.refArrayOps[org.locationtech.geomesa.utils.concurrent.CachedThreadPool](HBaseIndexWriter.this.pools).foldLeft[Boolean](true)(((x0$1: Boolean, x1$1: org.locationtech.geomesa.utils.concurrent.CachedThreadPool) => scala.Tuple2.apply[Boolean, org.locationtech.geomesa.utils.concurrent.CachedThreadPool](x0$1, x1$1) match { case (_1: Boolean, _2: org.locationtech.geomesa.utils.concurrent.CachedThreadPool)(Boolean, org.locationtech.geomesa.utils.concurrent.CachedThreadPool)((terminated @ _), (pool @ _)) => terminated.&&(pool.awaitTermination(60L, SECONDS)) })).unary_!
718 90606 32086 - 32143 Block scala.Boolean.&& terminated.&&(pool.awaitTermination(60L, SECONDS))
718 90609 32031 - 32031 Literal <nosymbol> ()
718 90610 32031 - 32031 Block <nosymbol> ()
719 90608 32157 - 32220 Typed <nosymbol> (if (HBaseIndexAdapter.this.logger.underlying.isWarnEnabled()) HBaseIndexAdapter.this.logger.underlying.warn("Failed to terminate thread pool after 60 seconds") else (): Unit)
725 90611 32394 - 32403 Apply org.apache.hadoop.hbase.client.BufferedMutator.flush f.flush()
725 90612 32390 - 32404 Apply scala.util.Try.apply scala.util.Try.apply[Unit](f.flush())