diff --git a/akka-bootstrapper/src/main/scala/filodb/akkabootstrapper/ClusterSeedDiscovery.scala b/akka-bootstrapper/src/main/scala/filodb/akkabootstrapper/ClusterSeedDiscovery.scala index 853a9af46e..5bbeac1d11 100644 --- a/akka-bootstrapper/src/main/scala/filodb/akkabootstrapper/ClusterSeedDiscovery.scala +++ b/akka-bootstrapper/src/main/scala/filodb/akkabootstrapper/ClusterSeedDiscovery.scala @@ -38,8 +38,13 @@ abstract class ClusterSeedDiscovery(val cluster: Cluster, response = Http(seedsEndpoint).timeout(2000, 2000).asString logger.info(s"Seeds endpoint returned a ${response.code}. Response body was ${response.body}") } catch { - case NonFatal(e) => + case NonFatal(e) => { + if (e.isInstanceOf[java.net.ConnectException]) { + // Don't bother logging the full the trace for something which is expected. + e.setStackTrace(new Array[StackTraceElement](0)) + } logger.info(s"Seeds endpoint $seedsEndpoint failed. This is expected on cluster bootstrap", e) + } } retriesRemaining -= 1 if (retriesRemaining > 0) Thread.sleep(settings.seedsHttpSleepBetweenRetries.toMillis) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index f3d3f87f63..7f458dc617 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -67,7 +67,9 @@ final class QueryActor(memStore: MemStore, def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = { epRequests.increment Kamon.currentSpan().tag("query", q.getClass.getSimpleName) - val span = Kamon.buildSpan(s"execplan2-${q.getClass.getSimpleName}").start() + val span = Kamon.buildSpan(s"execplan2-${q.getClass.getSimpleName}") + .withTag("query-id", q.id) + .start() implicit val _ = queryConfig.askTimeout q.execute(memStore, dataset, queryConfig) .foreach { res => diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index b4f04eef1c..2c9682a842 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -40,7 +40,10 @@ class QueryEngine(dataset: Dataset, def dispatchExecPlan(execPlan: ExecPlan) (implicit sched: ExecutionContext, timeout: FiniteDuration): Task[QueryResponse] = { - execPlan.dispatcher.dispatch(execPlan) + val currentSpan = Kamon.currentSpan() + Kamon.withSpan(currentSpan) { + execPlan.dispatcher.dispatch(execPlan) + } } /** diff --git a/coordinator/src/test/scala/filodb.coordinator/ActorSpecConfig.scala b/coordinator/src/test/scala/filodb.coordinator/ActorSpecConfig.scala index 35e119b7ca..50681f031a 100644 --- a/coordinator/src/test/scala/filodb.coordinator/ActorSpecConfig.scala +++ b/coordinator/src/test/scala/filodb.coordinator/ActorSpecConfig.scala @@ -10,6 +10,15 @@ import org.scalatest.concurrent.{Eventually, IntegrationPatience, ScalaFutures} import filodb.core.AbstractSpec +object ActorSpecConfig { + def getNewSystem(name: String, config: Config): ActorSystem = { + // Delay between each test, to provide some allowance for the port binding to succeed. + // Ideally the port binding should automatically retry, but this is a simpler fix. + Thread.sleep(5000) + ActorSystem(name, config) + } +} + trait ActorSpecConfig { val defaultConfig = """ @@ -27,8 +36,7 @@ trait ActorSpecConfig { .withFallback(ConfigFactory.parseResources("application_test.conf")) .withFallback(ConfigFactory.load("filodb-defaults.conf")) - def getNewSystem = ActorSystem("test", config) - + def getNewSystem = ActorSpecConfig.getNewSystem("test", config) } trait SeedNodeConfig { @@ -85,8 +93,9 @@ object AkkaSpec extends SeedNodeConfig { val settings = new FilodbSettings(userConfig.withFallback(serverConfig)) - def getNewSystem(c: Option[Config] = None): ActorSystem = - ActorSystem("test", c.map(_.withFallback(settings.allConfig)) getOrElse settings.allConfig) + def getNewSystem(c: Option[Config] = None): ActorSystem = { + ActorSpecConfig.getNewSystem("test", c.map(_.withFallback(settings.allConfig)) getOrElse settings.allConfig) + } } abstract class AkkaSpec(system: ActorSystem) extends AbstractTestKit(system) @@ -97,8 +106,9 @@ abstract class AkkaSpec(system: ActorSystem) extends AbstractTestKit(system) with Matchers with ScalaFutures { - def this() = this(ActorSystem("akka-test", AkkaSpec.settings.allConfig)) - def this(config: Config) = this(ActorSystem("akka-test", config.withFallback(AkkaSpec.settings.allConfig))) + def this() = this(ActorSpecConfig.getNewSystem("akka-test", AkkaSpec.settings.allConfig)) + def this(config: Config) = + this(ActorSpecConfig.getNewSystem("akka-test", config.withFallback(AkkaSpec.settings.allConfig))) } diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index fcf642e290..7d4375984b 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -5,6 +5,7 @@ import scala.concurrent.ExecutionContext import debox.Buffer import kamon.Kamon +import kamon.trace.Span import monix.eval.Task import monix.execution.Scheduler import monix.reactive.{Observable, OverflowStrategy} @@ -38,6 +39,10 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol // TODO: make this configurable private val strategy = OverflowStrategy.BackPressure(1000) + private def startODPSpan: Span = Kamon.buildSpan(s"odp-cassandra-latency") + .withTag("dataset", dataset.name) + .withTag("shard", shardNum) + .start() // NOTE: the current implementation is as follows // 1. Fetch partitions from memStore // 2. Determine, one at a time, what chunks are missing and could be fetched from disk @@ -73,42 +78,46 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol } } shardStats.partitionsQueried.increment(inMemoryPartitions.length) - val span = Kamon.buildSpan(s"odp-cassandra-latency") - .withTag("dataset", dataset.name) - .withTag("shard", shardNum) - .start() logger.debug(s"dataset=${dataset.ref} shard=$shardNum Querying ${inMemoryPartitions.length} in memory " + s"partitions, ODPing ${methods.length}") // NOTE: multiPartitionODP mode does not work with AllChunkScan and unit tests; namely missing partitions will not // return data that is in memory. TODO: fix - (if (storeConfig.multiPartitionODP) { - Observable.fromIterable(inMemoryPartitions) ++ - Observable.fromTask(odpPartTask(indexIt, partKeyBytesToPage, methods, chunkMethod)).flatMap { odpParts => - val multiPart = MultiPartitionScan(partKeyBytesToPage, shardNum) - shardStats.partitionsQueried.increment(partKeyBytesToPage.length) - if (partKeyBytesToPage.nonEmpty) { - rawStore.readRawPartitions(dataset, allDataCols, multiPart, computeBoundingMethod(methods)) - // NOTE: this executes the partMaker single threaded. Needed for now due to concurrency constraints. - // In the future optimize this if needed. - .mapAsync { rawPart => partitionMaker.populateRawChunks(rawPart).executeOn(singleThreadPool) } - // This is needed so future computations happen in a different thread - .asyncBoundary(strategy) - } else { Observable.empty } - } - } else { - Observable.fromIterable(inMemoryPartitions) ++ - Observable.fromTask(odpPartTask(indexIt, partKeyBytesToPage, methods, chunkMethod)).flatMap { odpParts => - shardStats.partitionsQueried.increment(partKeyBytesToPage.length) - Observable.fromIterable(partKeyBytesToPage.zip(methods)) - .mapAsync(storeConfig.demandPagingParallelism) { case (partBytes, method) => - rawStore.readRawPartitions(dataset, allDataCols, SinglePartitionScan(partBytes, shardNum), method) + Observable.fromIterable(inMemoryPartitions) ++ { + if (storeConfig.multiPartitionODP) { + Observable.fromTask(odpPartTask(indexIt, partKeyBytesToPage, methods, chunkMethod)).flatMap { odpParts => + val multiPart = MultiPartitionScan(partKeyBytesToPage, shardNum) + shardStats.partitionsQueried.increment(partKeyBytesToPage.length) + if (partKeyBytesToPage.nonEmpty) { + val span = startODPSpan + rawStore.readRawPartitions(dataset, allDataCols, multiPart, computeBoundingMethod(methods)) + // NOTE: this executes the partMaker single threaded. Needed for now due to concurrency constraints. + // In the future optimize this if needed. .mapAsync { rawPart => partitionMaker.populateRawChunks(rawPart).executeOn(singleThreadPool) } - .defaultIfEmpty(getPartition(partBytes).get) - .headL + .doOnComplete(() => span.finish()) + // This is needed so future computations happen in a different thread + .asyncBoundary(strategy) + } else { Observable.empty } + } + } else { + Observable.fromTask(odpPartTask(indexIt, partKeyBytesToPage, methods, chunkMethod)).flatMap { odpParts => + shardStats.partitionsQueried.increment(partKeyBytesToPage.length) + if (partKeyBytesToPage.nonEmpty) { + val span = startODPSpan + Observable.fromIterable(partKeyBytesToPage.zip(methods)) + .mapAsync(storeConfig.demandPagingParallelism) { case (partBytes, method) => + rawStore.readRawPartitions(dataset, allDataCols, SinglePartitionScan(partBytes, shardNum), method) + .mapAsync { rawPart => partitionMaker.populateRawChunks(rawPart).executeOn(singleThreadPool) } + .defaultIfEmpty(getPartition(partBytes).get) + .headL + } + .doOnComplete(() => span.finish()) + } else { + Observable.empty } + } } - }).doOnComplete(() => span.finish()) + } } // 3. Deal with partitions no longer in memory but still indexed in Lucene. diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 921419529f..198a02be6b 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -6,8 +6,8 @@ import scalaxy.loops._ import filodb.core.Types._ import filodb.core.metadata.Dataset import filodb.core.store._ -import filodb.memory.{BinaryRegion, BinaryRegionLarge, BlockMemFactory} -import filodb.memory.data.{MapHolder, OffheapLFSortedIDMapMutator} +import filodb.memory.{BinaryRegion, BinaryRegionLarge, BlockMemFactory, MemFactory} +import filodb.memory.data.ChunkMap import filodb.memory.format._ object TimeSeriesPartition extends StrictLogging { @@ -44,11 +44,14 @@ final case class InfoAppenders(info: ChunkSetInfo, appenders: TimeSeriesPartitio * This allows for safe and cheap write buffer churn without losing any data. * switchBuffers() is called before flush() is called in another thread, possibly. * - * The main data structure used is the "infoMap" - an OffheapLFSortedIDMap, an extremely efficient, offheap sorted map - * Note that other than the variables used in this class, there is NO heap memory used for managing chunks. Thus - * the amount of heap memory used for a partition is O(1) constant regardless of the number of chunks in a TSPartition. - * The partition key and infoMap are both in offheap write buffers, and chunks and chunk metadata are kept in - * offheap block memory. + * The main data structure used is inherited from ChunkMap, an efficient, offheap sorted map. + * Note that other than the variables used in this class, there is NO JVM-managed memory used + * for managing chunks. Thus the amount of JVM-managed memory used for a partition is constant + * regardless of the number of chunks in a TSPartition. The partition key and infoMap are both + * in offheap write buffers, and chunks and chunk metadata are kept in offheap block memory. + * + * Note: Inheritance is chosen over composition to avoid an extra object allocation, which + * speeds up GC and reduces memory overhead a bit. */ class TimeSeriesPartition(val partID: Int, val dataset: Dataset, @@ -56,13 +59,9 @@ class TimeSeriesPartition(val partID: Int, val shard: Int, bufferPool: WriteBufferPool, val shardStats: TimeSeriesShardStats, - // Volatile pointer to infoMap structure. Name of field MUST match mapKlazz method above - var mapPtr: BinaryRegion.NativePointer, - // Shared class for mutating the infoMap / OffheapLFSortedIDMap given mapPtr above - offheapInfoMap: OffheapLFSortedIDMapMutator, - // Lock state used by OffheapLFSortedIDMap. - var lockState: Int = 0) -extends ReadablePartition with MapHolder { + memFactory: MemFactory, + initMapSize: Int) +extends ChunkMap(memFactory, initMapSize) with ReadablePartition { import TimeSeriesPartition._ require(bufferPool.dataset == dataset) // Really important that buffer pool schema matches @@ -237,7 +236,7 @@ extends ReadablePartition with MapHolder { appenders = appenders.filterNot(_ == ia) } - def numChunks: Int = offheapInfoMap.length(this) + def numChunks: Int = chunkmapSize // inherited from ChunkMap def appendingChunkLen: Int = if (currentInfo != nullInfo) currentInfo.numRows else 0 /** @@ -245,13 +244,13 @@ extends ReadablePartition with MapHolder { * when flushes happen. Computed dynamically from current infosChunks state. * NOTE: since sliceToEnd is inclusive, we need to start just past the newestFlushedID */ - def unflushedChunksets: Int = offheapInfoMap.sliceToEnd(this, newestFlushedID + 1).count + def unflushedChunksets: Int = chunkmapSliceToEnd(newestFlushedID + 1).count - private def allInfos: ChunkInfoIterator = new ElementChunkInfoIterator(offheapInfoMap.iterate(this)) + private def allInfos: ChunkInfoIterator = new ElementChunkInfoIterator(chunkmapIterate) // NOT including currently flushing writeBuffer chunks if there are any private[memstore] def infosToBeFlushed: ChunkInfoIterator = - new ElementChunkInfoIterator(offheapInfoMap.sliceToEnd(this, newestFlushedID + 1)) + new ElementChunkInfoIterator(chunkmapSliceToEnd(newestFlushedID + 1)) .filter(_ != currentInfo) // filter out the appending chunk def infos(method: ChunkScanMethod): ChunkInfoIterator = method match { @@ -266,7 +265,7 @@ extends ReadablePartition with MapHolder { try { new OneChunkInfo(currentInfo) } catch { - case e: Throwable => offheapInfoMap.releaseShared(this); throw e; + case e: Throwable => chunkmapReleaseShared(); throw e; } } } @@ -293,7 +292,7 @@ extends ReadablePartition with MapHolder { private def doClose(): Unit = { closed = true - offheapInfoMap.releaseShared(TimeSeriesPartition.this) + chunkmapReleaseShared() } def hasNext: Boolean = { @@ -304,11 +303,14 @@ extends ReadablePartition with MapHolder { def nextInfo: ChunkSetInfo = { if (closed) throw new NoSuchElementException() if (!valueSeen) { - offheapInfoMap.acquireShared(TimeSeriesPartition.this) + chunkmapAcquireShared() valueSeen = true } return info } + + final def lock(): Unit = chunkmapAcquireShared() + final def unlock(): Unit = chunkmapReleaseShared() } final def earliestTime: Long = { @@ -316,7 +318,7 @@ extends ReadablePartition with MapHolder { Long.MinValue } else { // Acquire shared lock to safely access the native pointer. - offheapInfoMap.withShared(this, ChunkSetInfo(offheapInfoMap.first(this)).startTime) + chunkmapWithShared(ChunkSetInfo(chunkmapDoGetFirst).startTime) } } @@ -331,27 +333,27 @@ extends ReadablePartition with MapHolder { currentInfo.endTime } else if (numChunks > 0) { // Acquire shared lock to safely access the native pointer. - offheapInfoMap.withShared(this, infoLast.endTime) + chunkmapWithShared(infoLast.endTime) } else { -1 } } - // Disabled for now. Requires a shared lock on offheapInfoMap. + // Disabled for now. Requires a shared lock on the inherited map. //def dataChunkPointer(id: ChunkID, columnID: Int): BinaryVector.BinaryVectorPtr = infoGet(id).vectorPtr(columnID) final def removeChunksAt(id: ChunkID): Unit = { - offheapInfoMap.withExclusive(this, offheapInfoMap.remove(this, id)) + chunkmapWithExclusive(chunkmapDoRemove(id)) shardStats.chunkIdsEvicted.increment() } - final def hasChunksAt(id: ChunkID): Boolean = offheapInfoMap.contains(this, id) + final def hasChunksAt(id: ChunkID): Boolean = chunkmapContains(id) // Used for adding chunksets that are paged in, ie that are already persisted // Atomic and multi-thread safe; only mutates state if chunkID not present final def addChunkInfoIfAbsent(id: ChunkID, infoAddr: BinaryRegion.NativePointer): Boolean = { - offheapInfoMap.withExclusive(this, { - val inserted = offheapInfoMap.putIfAbsent(this, id, infoAddr) + chunkmapWithExclusive({ + val inserted = chunkmapDoPutIfAbsent(infoAddr) // Make sure to update newestFlushedID so that flushes work correctly and don't try to flush these chunksets if (inserted) updateFlushedID(infoGet(id)) inserted @@ -362,14 +364,14 @@ extends ReadablePartition with MapHolder { newestFlushedID = Math.max(newestFlushedID, info.id) } - // Caller must hold lock on offheapInfoMap. - private def infoGet(id: ChunkID): ChunkSetInfo = ChunkSetInfo(offheapInfoMap(this, id)) + // Caller must hold lock on the inherited map. + private def infoGet(id: ChunkID): ChunkSetInfo = ChunkSetInfo(chunkmapDoGet(id)) - // Caller must hold lock on offheapInfoMap. - private[core] def infoLast(): ChunkSetInfo = ChunkSetInfo(offheapInfoMap.last(this)) + // Caller must hold lock on the inherited map. + private[core] def infoLast(): ChunkSetInfo = ChunkSetInfo(chunkmapDoGetLast) private def infoPut(info: ChunkSetInfo): Unit = { - offheapInfoMap.withExclusive(this, offheapInfoMap.put(this, info.infoAddr)) + chunkmapWithExclusive(chunkmapDoPut(info.infoAddr)) } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index a6becedb50..5d742fdc74 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1,5 +1,7 @@ package filodb.core.memstore +import java.util.concurrent.locks.StampedLock + import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ import scala.util.{Random, Try} @@ -7,7 +9,6 @@ import scala.util.{Random, Try} import com.googlecode.javaewah.{EWAHCompressedBitmap, IntIterator} import com.typesafe.scalalogging.StrictLogging import debox.Buffer -import java.util.concurrent.locks.StampedLock import kamon.Kamon import kamon.metric.MeasurementUnit import monix.eval.Task @@ -25,7 +26,6 @@ import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, ColumnInfo} import filodb.core.store._ import filodb.memory._ -import filodb.memory.data.{OffheapLFSortedIDMap, OffheapLFSortedIDMapMutator} import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} import filodb.memory.format.BinaryVector.BinaryVectorPtr import filodb.memory.format.ZeroCopyUTF8String._ @@ -269,16 +269,17 @@ class TimeSeriesShard(val dataset: Dataset, private final val numTimeBucketsToRetain = Math.ceil(chunkRetentionHours.hours / storeConfig.flushInterval).toInt - // Not really one instance of a map; more like an accessor class shared amongst all TSPartition instances - private val offheapInfoMap = new OffheapLFSortedIDMapMutator(bufferMemoryManager, classOf[TimeSeriesPartition]) - // Use 1/4 of max # buckets for initial ChunkInfo map size + // Use 1/4 of max # buckets for initial ChunkMap size private val initInfoMapSize = Math.max((numTimeBucketsToRetain / 4) + 4, 20) /** * Current time bucket number. Time bucket number is initialized from last value stored in metastore - * and is incremented each time a new bucket is prepared for flush + * and is incremented each time a new bucket is prepared for flush. + * + * This value is mutated only from the ingestion thread, but read from both flush and ingestion threads. */ - private var currentIndexTimeBucket: Int = _ + @volatile + private var currentIndexTimeBucket = 0 /** * Timestamp to start searching for partitions to evict. Advances as more and more partitions are evicted. @@ -471,7 +472,9 @@ class TimeSeriesShard(val dataset: Dataset, partKeyIndex.upsertPartKey(partKeyBaseOnHeap, partId, startTime, endTime, PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset(partKeyOffset))(partKeyNumBytes) timeBucketBitmaps.get(segment.timeBucket).set(partId) - if (endTime == Long.MaxValue) activelyIngesting.set(partId) else activelyIngesting.clear(partId) + activelyIngesting.synchronized { + if (endTime == Long.MaxValue) activelyIngesting.set(partId) else activelyIngesting.clear(partId) + } } numRecordsProcessed += 1 } @@ -604,7 +607,7 @@ class TimeSeriesShard(val dataset: Dataset, } /** - * Prepare to flush current index records, switch current currentIndexTimeBucketPartIds with new one. + * Prepare to flush current index records, switch current currentIndexTimeBucket partId bitmap with new one. * Return Some if part keys need to be flushed (happens for last flush group). Otherwise, None. * * NEEDS TO RUN ON INGESTION THREAD since it removes entries from the partition data structures. @@ -647,13 +650,14 @@ class TimeSeriesShard(val dataset: Dataset, shardStats.indexEntries.set(partKeyIndex.indexNumEntries) shardStats.indexBytes.set(partKeyIndex.indexRamBytes) shardStats.numPartitions.set(numActivePartitions) - shardStats.numActivelyIngestingParts.set(activelyIngesting.cardinality()) + val cardinality = activelyIngesting.synchronized { activelyIngesting.cardinality() } + shardStats.numActivelyIngestingParts.set(cardinality) } private def addPartKeyToTimebucket(indexRb: RecordBuilder, p: TimeSeriesPartition) = { var startTime = partKeyIndex.startTimeFromPartId(p.partID) if (startTime == -1) startTime = p.earliestTime// can remotely happen since lucene reads are eventually consistent - val endTime = if (activelyIngesting.get(p.partID)) { + val endTime = if (isActivelyIngesting(p.partID)) { Long.MaxValue } else { val et = p.timestampOfLatestSample // -1 can be returned if no sample after reboot @@ -669,12 +673,17 @@ class TimeSeriesShard(val dataset: Dataset, indexRb.endRecord(false) } + private def isActivelyIngesting(partID: Integer): Boolean = { + activelyIngesting.synchronized { activelyIngesting.get(partID) } + } + // scalastyle:off method.length private def doFlushSteps(flushGroup: FlushGroup, partitionIt: Iterator[TimeSeriesPartition]): Task[Response] = { val tracer = Kamon.buildSpan("chunk-flush-task-latency-after-retries") .withTag("dataset", dataset.name) .withTag("shard", shardNum).start() + // Only allocate the blockHolder when we actually have chunks/partitions to flush val blockHolder = blockFactoryPool.checkout() @@ -760,11 +769,24 @@ class TimeSeriesShard(val dataset: Dataset, .withTag("dataset", dataset.name) .withTag("shard", shardNum).start() + /* Note regarding thread safety of accessing time bucket bitmaps: + + Every flush task reads bits on the earliest time bucket bitmap and sets bits on the + latest timeBucket, both of which are uniquely associated with the flush group. Since + each flush group is associated with different latest and earliest time buckets, + concurrent threads should not be reading or writing to same time bucket bitmaps, or + even setting the same time bucket in the collection. This can in theory happen only if + a flush task lasts more than the retention period (not possible). + */ + /* Add to timeBucketRb partKeys for (earliestTimeBucketBitmap && ~stoppedIngesting). These keys are from earliest time bucket that are still ingesting */ val earliestTimeBucket = cmd.timeBucket - numTimeBucketsToRetain if (earliestTimeBucket >= 0) { - val partIdsToRollOver = timeBucketBitmaps.get(earliestTimeBucket).and(activelyIngesting) + var partIdsToRollOver = timeBucketBitmaps.get(earliestTimeBucket) + activelyIngesting.synchronized { + partIdsToRollOver = partIdsToRollOver.and(activelyIngesting) + } val newBitmap = timeBucketBitmaps.get(cmd.timeBucket).or(partIdsToRollOver) timeBucketBitmaps.put(cmd.timeBucket, newBitmap) shardStats.numRolledKeysInLatestTimeBucket.increment(partIdsToRollOver.cardinality()) @@ -827,6 +849,7 @@ class TimeSeriesShard(val dataset: Dataset, logger.error(s"Critical! Chunk persistence failed after retries and skipped in dataset=${dataset.ref} " + s"shard=$shardNum", e) shardStats.flushesFailedChunkWrite.increment + // Encode and free up the remainder of the WriteBuffers that have not been flushed yet. Otherwise they will // never be freed. partitionIt.foreach(_.encodeAndReleaseBuffers(blockHolder)) @@ -852,18 +875,21 @@ class TimeSeriesShard(val dataset: Dataset, private def updateIndexWithEndTime(p: TimeSeriesPartition, partFlushChunks: Iterator[ChunkSet], timeBucket: Int) = { - if (partFlushChunks.isEmpty && activelyIngesting.get(p.partID)) { - var endTime = p.timestampOfLatestSample - if (endTime == -1) endTime = System.currentTimeMillis() // this can happen if no sample after reboot - updatePartEndTimeInIndex(p, endTime) - timeBucketBitmaps.get(timeBucket).set(p.partID) - activelyIngesting.clear(p.partID) - } else if (partFlushChunks.nonEmpty && !activelyIngesting.get(p.partID)) { - // Partition started re-ingesting. - // TODO: we can do better than this for intermittent time series. Address later. - updatePartEndTimeInIndex(p, Long.MaxValue) - timeBucketBitmaps.get(timeBucket).set(p.partID) - activelyIngesting.set(p.partID) + // Synchronize for safe read-modify-write behavior. + activelyIngesting.synchronized { + if (partFlushChunks.isEmpty && activelyIngesting.get(p.partID)) { + var endTime = p.timestampOfLatestSample + if (endTime == -1) endTime = System.currentTimeMillis() // this can happen if no sample after reboot + updatePartEndTimeInIndex(p, endTime) + timeBucketBitmaps.get(timeBucket).set(p.partID) + activelyIngesting.clear(p.partID) + } else if (partFlushChunks.nonEmpty && !activelyIngesting.get(p.partID)) { + // Partition started re-ingesting. + // TODO: we can do better than this for intermittent time series. Address later. + updatePartEndTimeInIndex(p, Long.MaxValue) + timeBucketBitmaps.get(timeBucket).set(p.partID) + activelyIngesting.set(p.partID) + } } } @@ -913,13 +939,14 @@ class TimeSeriesShard(val dataset: Dataset, val startTime = dataset.ingestionSchema.getLong(recordBase, recordOff, timestampColId) partKeyIndex.addPartKey(newPart.partKeyBytes, partId, startTime)() timeBucketBitmaps.get(currentIndexTimeBucket).set(partId) - activelyIngesting.set(partId) + activelyIngesting.synchronized { activelyIngesting.set(partId) } val stamp = partSetLock.writeLock() try { partSet.add(newPart) } finally { partSetLock.unlockWrite(stamp) } + logger.trace(s"Created new partition ${newPart.stringPartition} on dataset=${dataset.ref} " + s"shard $shardNum at offset $ingestOffset") } @@ -955,11 +982,10 @@ class TimeSeriesShard(val dataset: Dataset, // NOTE: allocateAndCopy and allocNew below could fail if there isn't enough memory. It is CRUCIAL // that min-write-buffers-free setting is large enough to accommodate the below use cases ALWAYS val (_, partKeyAddr, _) = BinaryRegionLarge.allocateAndCopy(partKeyBase, partKeyOffset, bufferMemoryManager) - val infoMapAddr = OffheapLFSortedIDMap.allocNew(bufferMemoryManager, initMapSize) val partId = nextPartitionID incrementPartitionID() - val newPart = new TimeSeriesPartition(partId, dataset, partKeyAddr, shardNum, bufferPool, shardStats, - infoMapAddr, offheapInfoMap) + val newPart = new TimeSeriesPartition( + partId, dataset, partKeyAddr, shardNum, bufferPool, shardStats, bufferMemoryManager, initMapSize) partitions.put(partId, newPart) shardStats.partitionsCreated.increment partitionGroups(group).set(partId) @@ -1031,7 +1057,7 @@ class TimeSeriesShard(val dataset: Dataset, if (partitionObj != UnsafeUtils.ZeroPointer) { // TODO we can optimize fetching of endTime by getting them along with top-k query val endTime = partKeyIndex.endTimeFromPartId(partitionObj.partID) - if (activelyIngesting.get(partitionObj.partID)) + if (isActivelyIngesting(partitionObj.partID)) logger.warn(s"Partition ${partitionObj.partID} is ingesting, but it was eligible for eviction. How?") if (endTime == PartKeyLuceneIndex.NOT_FOUND || endTime == Long.MaxValue) { logger.warn(s"endTime ${endTime} was not correct. how?", new IllegalStateException()) @@ -1068,7 +1094,7 @@ class TimeSeriesShard(val dataset: Dataset, } finally { partSetLock.unlockWrite(stamp) } - offheapInfoMap.free(partitionObj) + partitionObj.chunkmapFree() bufferMemoryManager.freeMemory(partitionObj.partKeyOffset) partitions.remove(partitionObj.partID) } diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 64cd021693..80aca5eea8 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -10,7 +10,7 @@ import filodb.core.metadata.Column import filodb.core.metadata.Column.ColumnType._ import filodb.core.store._ import filodb.memory.{MemFactory, UTF8StringMedium} -import filodb.memory.data.OffheapLFSortedIDMap +import filodb.memory.data.ChunkMap import filodb.memory.format.{RowReader, ZeroCopyUTF8String => UTF8Str} /** @@ -177,7 +177,7 @@ object SerializableRangeVector extends StrictLogging { // Important TODO / TechDebt: We need to replace Iterators with cursors to better control // the chunk iteration, lock acquisition and release. This is much needed for safe memory access. try { - OffheapLFSortedIDMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks() val rows = rv.rows while (rows.hasNext) { numRows += 1 @@ -185,7 +185,7 @@ object SerializableRangeVector extends StrictLogging { } } finally { // When the query is done, clean up lingering shared locks caused by iterator limit. - OffheapLFSortedIDMap.releaseAllSharedLocks() + ChunkMap.releaseAllSharedLocks() } // If there weren't containers before, then take all of them. If there were, discard earlier ones, just // start with the most recent one we started adding to diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index d3fc318534..9867f18c6c 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -225,6 +225,12 @@ trait ChunkInfoIterator { base: ChunkInfoIterator => def hasNext: Boolean def nextInfo: ChunkSetInfo + /** + * Explicit locking to guard access to native memory. See ElementIterator. + */ + def lock(): Unit + def unlock(): Unit + /** * Returns a new ChunkInfoIterator which filters items from this iterator */ @@ -267,6 +273,8 @@ object ChunkInfoIterator { def close(): Unit = {} def hasNext: Boolean = false def nextInfo: ChunkSetInfo = ChunkSetInfo(0) + def lock(): Unit = {} + def unlock(): Unit = {} } } @@ -274,6 +282,8 @@ class ElementChunkInfoIterator(elIt: ElementIterator) extends ChunkInfoIterator def close(): Unit = elIt.close() final def hasNext: Boolean = elIt.hasNext final def nextInfo: ChunkSetInfo = ChunkSetInfo(elIt.next) + final def lock(): Unit = elIt.lock() + final def unlock(): Unit = elIt.unlock() } class FilteredChunkInfoIterator(base: ChunkInfoIterator, @@ -302,6 +312,9 @@ class FilteredChunkInfoIterator(base: ChunkInfoIterator, require(nextnext.infoAddr != 0, s"nextInfo called before hasNext!!") nextnext } + + final def lock(): Unit = base.lock() + final def unlock(): Unit = base.unlock() } /** @@ -378,6 +391,9 @@ extends ChunkInfoIterator { readIndex += 1 ChunkSetInfo(next) } + + final def lock(): Unit = infos.lock() + final def unlock(): Unit = infos.unlock() } /** @@ -425,4 +441,4 @@ class ChunkInfoRowReader(column: Column) extends RowReader { def getBlobBase(columnNo: Int): Any = ??? def getBlobOffset(columnNo: Int): Long = ??? def getBlobNumBytes(columnNo: Int): Int = ??? -} \ No newline at end of file +} diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index 220f7778fd..b8834d88cf 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -11,7 +11,6 @@ import filodb.core._ import filodb.core.metadata.Dataset import filodb.core.store._ import filodb.memory._ -import filodb.memory.data.{OffheapLFSortedIDMap, OffheapLFSortedIDMapMutator} import filodb.memory.format.UnsafeUtils object TimeSeriesPartitionSpec { @@ -19,16 +18,15 @@ object TimeSeriesPartitionSpec { import BinaryRegion.NativePointer val memFactory = new NativeMemoryManager(10 * 1024 * 1024) - val offheapInfoMapKlass = new OffheapLFSortedIDMapMutator(memFactory, classOf[TimeSeriesPartition]) + val maxChunkSize = 100 protected val myBufferPool = new WriteBufferPool(memFactory, dataset1, maxChunkSize, 50) def makePart(partNo: Int, dataset: Dataset, partKey: NativePointer = defaultPartKey, bufferPool: WriteBufferPool = myBufferPool): TimeSeriesPartition = { - val infoMapAddr = OffheapLFSortedIDMap.allocNew(memFactory, 40) new TimeSeriesPartition(partNo, dataset, partKey, 0, bufferPool, - new TimeSeriesShardStats(dataset.ref, 0), infoMapAddr, offheapInfoMapKlass) + new TimeSeriesShardStats(dataset.ref, 0), memFactory, 40) } } @@ -399,4 +397,4 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { readData2.toBuffer shouldEqual (timestamps take 5) ++ (timestamps drop 7) } -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index 9d49851ad4..b02fc2701c 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -69,7 +69,7 @@ object MemFactory { /** * Native (off heap) memory manager, allocating using MemoryIO with every call to allocateWithMagicHeader * and relying on a cap to not allocate more than upperBoundSizeInBytes - * Synchronized to be multi-thread safe -- for example, the OffheapLFSortedIDMap will cause concurrent free/allocates + * Synchronized to be multi-thread safe -- for example, the OffheapSortedIDMap will cause concurrent free/allocates * TODO: we don't really need freeAll(), consider not needing the map and just storing the size of allocation in * first four bytes. That in fact matches what is needed for BinaryVector and BinaryRecord allocations. * Have an allocateOffheapWithSizeHeader which just returns the address to the size bytes :) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala new file mode 100644 index 0000000000..c852b89d16 --- /dev/null +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -0,0 +1,641 @@ +package filodb.memory.data + +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map +import scala.concurrent.duration._ + +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon + +import filodb.memory.BinaryRegion.NativePointer +import filodb.memory.MemFactory +import filodb.memory.format.UnsafeUtils + +/** + * Specialized map/set implementation which maps long keys to native pointers. The key must be + * derived from the object referenced by the native pointer, hence this map behaves somewhat + * like a set. The bulk of the map contents is maintained in an off-heap (native) array, + * managed like a circular buffer. The elements are sorted, and so lookup operations perform a + * binary search. + * + * To safely use this class from multiple threads, a lock must be acquired and released. Some + * of the methods do this already, and only those that require explicit locks are documented as + * such. The lock protects the map and it ensures that referenced memory isn't reclaimed too + * soon by the block manager. Hold the shared lock while reading elements, and release it when + * the memory can be reclaimed. To be effective, all writes into the map must acquire an + * exclusive lock. The lock implementation spins if necessary, but it yields the current thread + * to be fair with other threads. To help reduce the likelihood of deadlocks, a thread which is + * waiting to acquire the exclusive lock times out and retries while waiting, to help advance + * threads which are stuck behind the exclusive lock request. A warning is logged by the + * exclusive waiter when it's timeout has reached one second. This indicates that a deadlock + * likely exists and cannot be auto resolved. + * + * NOTE: By convention, methods which require that the caller obtain the lock are denoted with + * a "Do" in the name, such as `chunkmapDoGet`. All other methods acquire the lock + * automatically. Care must be take with respect to reentrancy. An exclusive lock cannot be + * acquired again once held, and the current thread will deadlock with itself. + * + * The implementation stores elements in a sorted circular buffer, assuming that most inserts + * are higher than all other keys, and that most deletes are against the lowest key. These + * operations can perform in constant time as a result. For keys not at the extremities, the + * the contents of the circular buffer must shift around, leading to a very high cost when the + * map is very large. + */ +object ChunkMap extends StrictLogging { + private val _logger = logger + + private val lockStateOffset = UnsafeUtils.unsafe.objectFieldOffset( + classOf[ChunkMap].getDeclaredField("lockState")) + + private val InitialExclusiveRetryTimeoutNanos = 1.millisecond.toNanos + private val MaxExclusiveRetryTimeoutNanos = 1.second.toNanos + + private val exclusiveLockWait = Kamon.counter("memory-exclusive-lock-waits") + private val sharedLockLingering = Kamon.counter("memory-shared-lock-lingering") + + // Tracks all the shared locks held, by each thread. + private val sharedLockCounts = new ThreadLocal[Map[ChunkMap, Int]] { + override def initialValue() = new HashMap[ChunkMap, Int] + } + + // Returns true if the current thread has acquired the shared lock at least once. + private def hasSharedLock(inst: ChunkMap): Boolean = sharedLockCounts.get.contains(inst) + + // Updates the shared lock count, for the current thread. + private def adjustSharedLockCount(inst: ChunkMap, amt: Int): Unit = { + val countMap = sharedLockCounts.get + if (!countMap.contains(inst)) { + if (amt > 0) { + countMap.put(inst, amt) + } + } else { + val newCount = countMap(inst) + amt + if (newCount <= 0) { + countMap.remove(inst) + } else { + countMap.put(inst, newCount) + } + } + } + + /** + * Releases all shared locks, against all ChunkMap instances, for the current thread. + */ + //scalastyle:off null + def releaseAllSharedLocks(): Int = { + var total = 0 + val countMap = sharedLockCounts.get + if (countMap != null) { + for ((inst, amt) <- countMap) { + if (amt > 0) { + total += amt + sharedLockLingering.increment(amt) + _logger.warn(s"Releasing all shared locks for: $inst, amount: $amt") + var lockState = 0 + do { + lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) + } while (!UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState - amt)) + } + } + + countMap.clear + } + total + } + //scalastyle:on null + + /** + * Validate no locks are held by the thread. Typically invoked prior to + * consumption from a query iterator. If there are lingering locks, + * it is quite possible a lock acquire or release bug exists + */ + def validateNoSharedLocks(): Unit = { + val numLocksReleased = ChunkMap.releaseAllSharedLocks() + if (numLocksReleased > 0) { + logger.warn(s"Number of locks was non-zero: $numLocksReleased. " + + s"This is indicative of a possible lock acquisition/release bug.") + } + } +} + +/** + * @param memFactory a THREAD-SAFE factory for allocating offheap space + * @param capacity initial capacity of the map; must be more than 0 + */ +class ChunkMap(val memFactory: MemFactory, var capacity: Int) { + require(capacity > 0) + + private var lockState: Int = 0 + private var size: Int = 0 + private var first: Int = 0 + private var arrayPtr = memFactory.allocateOffheap(capacity << 3, zero=true) + + import ChunkMap._ + + /** + * Returns the number of total elements in the map. + */ + final def chunkmapSize(): Int = { + chunkmapWithShared(size) + } + + /** + * Returns the element at the given key, or NULL (0) if the key isn't found. Takes O(log n) + * time. Caller must hold any lock. + */ + final def chunkmapDoGet(key: Long): NativePointer = { + val index = doBinarySearch(key) + if (index >= 0) arrayGet(realIndex(index)) else 0 + } + + /** + * Returns true if the given key exists in this map. Takes O(log n) time. + */ + final def chunkmapContains(key: Long): Boolean = { + chunkmapWithShared(doBinarySearch(key) >= 0) + } + + /** + * Returns the first element, the one with the lowest key. Caller must hold any lock. + * Throws IndexOutOfBoundsException if there are no elements. + */ + final def chunkmapDoGetFirst(): NativePointer = { + if (size <= 0) { + throw new IndexOutOfBoundsException + } + arrayGet(first) + } + + /** + * Returns the last element, the one with the highest key. Caller must hold any lock. + * Throws IndexOutOfBoundsException if there are no elements. + */ + final def chunkmapDoGetLast(): NativePointer = { + if (size <= 0) { + throw new IndexOutOfBoundsException + } + arrayGet(realIndex(first + size - 1)) + } + + /** + * Produces an ElementIterator for going through every element of the map in increasing key order. + */ + final def chunkmapIterate(): ElementIterator = { + new LazyElementIterator(() => { + chunkmapAcquireShared() + try { + new MapIterator(first, first + size) + } catch { + case e: Throwable => chunkmapReleaseShared(); throw e; + } + }) + } + + /** + * Produces an ElementIterator for iterating elements in increasing key order from startKey + * to endKey. + * @param startKey start at element whose key is equal or immediately greater than startKey + * @param endKey end iteration when element is greater than endKey. endKey is inclusive. + */ + final def chunkmapSlice(startKey: Long, endKey: Long): ElementIterator = { + new LazyElementIterator(() => { + chunkmapAcquireShared() + try { + new MapIterator(doBinarySearch(startKey) & 0x7fffffff, first + size) { + override def isPastEnd: Boolean = chunkmapKeyRetrieve(getNextElem) > endKey + } + } catch { + case e: Throwable => chunkmapReleaseShared(); throw e; + } + }) + } + + /** + * Produces an ElementIterator for iterating elements in increasing key order starting + * with startKey. + * @param startKey start at element whose key is equal or immediately greater than startKey + */ + final def chunkmapSliceToEnd(startKey: Long): ElementIterator = { + new LazyElementIterator(() => { + chunkmapAcquireShared() + try { + new MapIterator(doBinarySearch(startKey) & 0x7fffffff, first + size) + } catch { + case e: Throwable => chunkmapReleaseShared(); throw e; + } + }) + } + + /** + * Acquire exclusive access to this map, spinning if necessary. Exclusive lock isn't re-entrant. + */ + final def chunkmapAcquireExclusive(): Unit = { + // Spin-lock implementation. Because the owner of the shared lock might be blocked by this + // thread as it waits for an exclusive lock, deadlock is possible. To mitigate this problem, + // timeout and retry, allowing shared lock waiters to make progress. The timeout doubles + // for each retry, up to a limit, but the retries continue indefinitely. + + var timeoutNanos = InitialExclusiveRetryTimeoutNanos + var warned = false + + while (true) { + if (tryAcquireExclusive(timeoutNanos)) { + return + } + + timeoutNanos = Math.min(timeoutNanos << 1, MaxExclusiveRetryTimeoutNanos) + + if (!warned && timeoutNanos >= MaxExclusiveRetryTimeoutNanos) { + if (hasSharedLock(this)) { + // Self deadlock. Upgrading the shared lock to an exclusive lock is possible if the + // current thread is the only shared lock owner, but this isn't that common. Instead, + // this is a bug which needs to be fixed. + throw new IllegalStateException("Cannot acquire exclusive lock because thread already owns a shared lock") + } + exclusiveLockWait.increment() + _logger.warn(s"Waiting for exclusive lock: $this") + warned = true + } + } + } + + /** + * Acquire exclusive access to this map, spinning if necessary. Exclusive lock isn't re-entrant. + * + * @return false if timed out + */ + private def tryAcquireExclusive(timeoutNanos: Long): Boolean = { + // Spin-lock implementation. + + var lockState = 0 + + // First set the high bit, to signal an exclusive lock request. + + var done = false + do { + lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) + if (lockState < 0) { + // Wait for exclusive lock to be released. + Thread.`yield` + } else if (UnsafeUtils.unsafe.compareAndSwapInt(this, lockStateOffset, lockState, lockState | 0x80000000)) { + if (lockState == 0) { + return true + } + done = true + } + } while (!done) + + // Wait for shared lock owners to release the lock. + + val endNanos = System.nanoTime + timeoutNanos + + do { + Thread.`yield` + lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) + if ((lockState & 0x7fffffff) == 0) { + return true + } + } while (System.nanoTime() < endNanos) + + // Timed out. Release the exclusive lock request signal and yield (to permit shared access again). + + while(!UnsafeUtils.unsafe.compareAndSwapInt(this, lockStateOffset, lockState, lockState & 0x7fffffff)) { + lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) + } + + Thread.`yield` + return false + } + + /** + * Release an acquired exclusive lock. + */ + final def chunkmapReleaseExclusive(): Unit = { + UnsafeUtils.setIntVolatile(this, lockStateOffset, 0) + } + + /** + * Run the given function body with the exclusive lock held, which isn't re-entrant. + */ + final def chunkmapWithExclusive[T](body: => T): T = { + chunkmapAcquireExclusive() + try { + body + } finally { + chunkmapReleaseExclusive() + } + } + + /** + * Acquire shared access to this map, spinning if necessary. + */ + final def chunkmapAcquireShared(): Unit = { + // Spin-lock implementation. + + var lockState = 0 + + while (true) { + lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) + if (lockState < 0 && !hasSharedLock(this)) { + // Wait for exclusive lock to be released. + Thread.`yield` + } else if (UnsafeUtils.unsafe.compareAndSwapInt(this, lockStateOffset, lockState, lockState + 1)) { + adjustSharedLockCount(this, +1) + return + } + } + } + + /** + * Release an acquired shared lock. + */ + final def chunkmapReleaseShared(): Unit = { + var lockState = 0 + do { + lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) + } while (!UnsafeUtils.unsafe.compareAndSwapInt(this, lockStateOffset, lockState, lockState - 1)) + adjustSharedLockCount(this, -1) + } + + /** + * Run the given function body with the shared lock held. + */ + final def chunkmapWithShared[T](body: => T): T = { + chunkmapAcquireShared() + try { + body + } finally { + chunkmapReleaseShared() + } + } + + /** + * Inserts/replaces the element into the map using the key computed from the element. + * In case of replacing existing value for same key - then the last write wins. + * Takes O(1) time if key is the highest in the map, or O(n) otherwise. Caller must hold + * exclusive lock. + * @param element the native pointer to the offheap element; must be able to apply + * chunkmapKeyRetrieve to it to get the key + */ + final def chunkmapDoPut(element: NativePointer): Unit = { + require(element != 0) + chunkmapDoPut(chunkmapKeyRetrieve(element), element) + } + + /** + * Atomically inserts the element it returns IF AND ONLY IF the element isn't + * already in the map. Caller must hold exclusive lock. + * @return true if the element was inserted, false otherwise + */ + final def chunkmapDoPutIfAbsent(element: NativePointer): Boolean = { + require(element != 0) + val key = chunkmapKeyRetrieve(element) + if (doBinarySearch(key) >= 0) { + return false + } + chunkmapDoPut(key, element) + true + } + + //scalastyle:off + private def chunkmapDoPut(key: Long, element: NativePointer): Unit = { + if (size == 0) { + arraySet(0, element) + first = 0 + size = 1 + return + } + + // Ensure enough capacity, under the assumption that in most cases the element is + // inserted and not simply replaced. + if (size >= capacity) { + val newArrayPtr = memFactory.allocateOffheap(capacity << 4, zero=true) + if (first == 0) { + // No wraparound. + UnsafeUtils.unsafe.copyMemory(arrayPtr, newArrayPtr, size << 3) + } else { + // Wraparound correction. + val len = (capacity - first) << 3 + UnsafeUtils.unsafe.copyMemory(arrayPtr + (first << 3), newArrayPtr, len) + UnsafeUtils.unsafe.copyMemory(arrayPtr, newArrayPtr + len, first << 3) + first = 0 + } + memFactory.freeMemory(arrayPtr) + arrayPtr = newArrayPtr + capacity <<= 1 + } + + { + val last = first + size - 1 + val rlast = realIndex(last) + val lastKey = chunkmapKeyRetrieve(arrayGet(rlast)) + + if (key > lastKey) { + // New highest key; this is the expected common case. + arraySet(realIndex(last + 1), element) + size += 1 + return + } + + if (key == lastKey) { + // Replacing the last element. + arraySet(rlast, element) + return + } + } + + var index = doBinarySearch(key) + + if (index >= 0) { + // Replacing an existing element. + arraySet(realIndex(index), element) + return + } + + // Convert to insertion index. + index &= 0x7fffffff + + val ri = realIndex(index) + val rlast = realIndex(first + size) // rlast is the new index after insertion + val amt = rlast - ri + if (amt >= 0) { + // Shift the elements; no wraparound correction is required. + arrayCopy(ri, ri + 1, amt) // src, dst, len + } else { + // Shift the elements with wraparound correction. + arrayCopy(0, 1, rlast) + arraySet(0, arrayGet(capacity - 1)) + arrayCopy(ri, ri + 1, capacity - index - 1) + } + + arraySet(ri, element) + size += 1 + } + //scalastyle:on + + /** + * Removes the element at key from the map. Takes O(1) time if the key is the first, + * otherwise O(n) time on average. Caller must hold exclusive lock. + * @param key the key to remove. If key isn't present, then nothing is changed. + */ + final def chunkmapDoRemove(key: Long): Unit = { + if (size <= 0) { + return + } + + // Check if matches the first key. + if (key == chunkmapKeyRetrieve(arrayGet(first))) { + first += 1 + if (first >= capacity) { + // Wraparound. + first = 0 + } + } else { + val index = doBinarySearch(key) + if (index < 0) { + // Not found. + return + } + val ri = realIndex(index) + val rlast = realIndex(first + size - 1) + val amt = rlast - ri + if (amt >= 0) { + // Shift the elements; no wraparound correction is required. + arrayCopy(ri + 1, ri, amt) // src, dst, len + } else { + // Shift the elements with wraparound correction. + arrayCopy(ri + 1, ri, capacity - index - 1) + arraySet(capacity - 1, arrayGet(0)) + arrayCopy(1, 0, rlast) + } + } + + size -= 1 + } + + final def chunkmapFree(): Unit = { + chunkmapWithExclusive({ + if (arrayPtr != 0) { + memFactory.freeMemory(arrayPtr) + capacity = 0 + size = 0 + first = 0 + arrayPtr = 0 + } + }) + } + + /** + * Method which retrieves a pointer to the key/ID within the element. It just reads the first + * eight bytes from the element as the ID. Please override to implement custom functionality. + */ + private def chunkmapKeyRetrieve(elementPtr: NativePointer): Long = UnsafeUtils.getLong(elementPtr) + + /** + * Does a binary search for the element with the given key. Caller must hold any lock. + * @param key the key to search for + * @return found index, or index with bit 31 set if not found + */ + private def doBinarySearch(key: Long): Int = { + var low = first + var high = first + size - 1 + + while (low <= high) { + var mid = (low + high) >>> 1 + var midKey = chunkmapKeyRetrieve(arrayGet(realIndex(mid))) + if (midKey < key) { + low = mid + 1 + } else if (midKey > key) { + high = mid - 1 + } else { + return mid + } + } + + return low | 0x80000000 + } + + /** + * Returns the real index in the array, correcting for circular buffer wraparound. + */ + private def realIndex(index: Int): Int = { + var ix = index + if (ix >= capacity) { + ix -= capacity + } + ix + } + + private def arrayGet(index: Int): NativePointer = { + UnsafeUtils.getLong(arrayPtr + (index << 3)) + } + + private def arraySet(index: Int, value: NativePointer): Unit = { + UnsafeUtils.setLong(arrayPtr + (index << 3), value) + } + + private def arrayCopy(srcIndex: Int, dstIndex: Int, len: Int): Unit = { + UnsafeUtils.unsafe.copyMemory(arrayPtr + (srcIndex << 3), arrayPtr + (dstIndex << 3), len << 3) + } + + /** + * @param index initialized to first index to read from + * @param lastIndex last index to read from (exclusive) + */ + private class MapIterator(var index: Int, val lastIndex: Int) extends ElementIterator { + private var closed: Boolean = false + private var nextElem: NativePointer = 0 + + final def close(): Unit = { + if (!closed) doClose() + } + + private def doClose(): Unit = { + closed = true + nextElem = 0 + chunkmapReleaseShared() + } + + final def hasNext: Boolean = { + if (nextElem == 0) { + if (closed) return false + if (index >= lastIndex) { + doClose() + return false + } + nextElem = arrayGet(realIndex(index)) + if (isPastEnd) { + doClose() + return false + } + } + return true + } + + final def next: NativePointer = { + var next = nextElem + if (next == 0) { + if (hasNext) { + next = nextElem + } else { + throw new NoSuchElementException() + } + } + nextElem = 0 + index += 1 + next + } + + final def lock(): Unit = chunkmapAcquireShared() + + final def unlock(): Unit = chunkmapReleaseShared() + + final def getNextElem: NativePointer = nextElem + + /** + * Check if the current element is just past the end, and iteration should stop. + * Override this method to actually do something. + */ + def isPastEnd: Boolean = false + } +} diff --git a/memory/src/main/scala/filodb.memory/data/ElementIterator.scala b/memory/src/main/scala/filodb.memory/data/ElementIterator.scala index 0ca4d5fa8b..f0089d5141 100644 --- a/memory/src/main/scala/filodb.memory/data/ElementIterator.scala +++ b/memory/src/main/scala/filodb.memory/data/ElementIterator.scala @@ -29,6 +29,17 @@ trait ElementIterator { } _count } + + /** + * ElementIterators obtain a lock to protect access to native memory, and the lock is + * released when the iterator is closed. As a convenience (or not), the iterator is + * automatically closed when the hasNext method returns true. To protect native memory access + * even longer, call the lock method before performing any iteration. When done, call unlock. + * The lock method can be called multiple times, but be sure to call unlock the same amount. + */ + def lock(): Unit + + def unlock(): Unit } /** @@ -48,6 +59,10 @@ class LazyElementIterator(source: () => ElementIterator) extends ElementIterator override def next: NativePointer = sourceIt().next + override def lock(): Unit = sourceIt().lock() + + override def unlock(): Unit = sourceIt().unlock() + private def sourceIt(): ElementIterator = { if (it == null) it = source() it diff --git a/memory/src/main/scala/filodb.memory/data/OffheapLFSortedIDMap.scala b/memory/src/main/scala/filodb.memory/data/OffheapLFSortedIDMap.scala deleted file mode 100644 index bdea174d0f..0000000000 --- a/memory/src/main/scala/filodb.memory/data/OffheapLFSortedIDMap.scala +++ /dev/null @@ -1,934 +0,0 @@ -package filodb.memory.data - -import java.util.concurrent.ConcurrentHashMap - -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map -import scala.concurrent.duration._ - -import com.typesafe.scalalogging.StrictLogging -import kamon.Kamon - -import filodb.memory.BinaryRegion.NativePointer -import filodb.memory.MemFactory -import filodb.memory.format.UnsafeUtils - -/** - * Offheap (mostly) Lock-Free Sorted ID Map. - * The OffheapLFSortedIDMap was written to replace the ConcurrentSkipListMap and use 1/10th of CSLM overhead, - * be completely offheap, and fit our use cases much better. - * - * Note: As currently used within FiloDB, locks must be acquired on the map to ensure that - * referenced memory isn't reclaimed too soon by the block manager. Hold the shared lock - * while reading elements, and release it when the memory can be reclaimed. To be effective, - * all writes into the map must acquire an exclusive lock. The lock implementation spins if - * necessary, but it yields the current thread to be fair with other threads. To help reduce - * the likelihood of deadlocks, a thread which is waiting to acquire the exclusive lock times - * out and retries while waiting, to help advance threads which are stuck behind the exclusive - * lock request. A warning is logged by the exclusive waiter when it's timeout has reached one - * second. This indicates that a deadlock likely exists and cannot be auto resolved. - * - * The OffheapLFSortedIDMap is a data structure with the following properties: - * - Everything (except for the Long pointer reference) both data and metadata is completely offheap - * - Memory efficiency is (much) more important than CPU or safety - * - Each value's key must be derivable from the value using a function - * - Data is kept sorted by key - * - Fast lookups/contains in <= O(lg n) - * - Is optimized for small n (say n usually < 100) - * - Must be multi-thread safe for modifications, except OK for replacements of existing elements to not be MTsafe - * - but optimized for low concurrency/conflicts - * - Most insertions are at head (newest keys are usually highest) but a few are near the tail - * - Most deletions are near the tail - * - * Design: - * AtomicLong reference to array memory location - * Ring-structured array with head and tail - * Head and tail modifications involve lockfree pointer updates mostly - * Modifications in middle involve copying array and atomic reference swap - * Lookups involve binary search - * - * Memory layout: - * +0000 (uint16) head: long array position of head (highest ordered) element - * +0002 (uint16) tail: long array position of tail (lowest ordered) element - * +0004 (bool/uint8) flag, if nonzero, means copying of data structure is happening. Retry later please! - * +0006 (uint16) maximum number of elements this can hold - * +0008 64-bit/Long array of pointers to the actual offheap "value" for each element - * - * The metadata must fit in 64-bits so that it can be atomically swapped (esp head, tail, flag) - * The key must be a Long, so the function to derive a Key from a Value is a Long => Long. - * The maximum number of elements that can be accommodated is 65535, but it's really designed for much smaller n. - * This is because if the head is one less than the tail, that is interpreted as an empty Map. - */ -object OffheapLFSortedIDMap extends StrictLogging { - val OffsetHead = 0 // Head is the highest keyed index/element number in the ring, -1 initially - val OffsetTail = 2 - val OffsetCopyFlag = 4 - val OffsetMaxElements = 6 - val OffsetElementPtrs = 8 - - val CopyFlagMask = 0xff00000000L - val MinMaxElements = 4 // Must be more than a few - val MaxMaxElements = 65535 // Top/absolute limit on # of elements - - val IllegalStateResult = -1 // Returned from binarySearch when state changed underneath - - val InitialExclusiveRetryTimeoutNanos = 1.millisecond.toNanos - val MaxExclusiveRetryTimeoutNanos = 1.second.toNanos - - val _logger = logger - - val sharedLockLingering = Kamon.counter("memory-shared-lock-lingering") - - // Tracks all the shared locks held, by each thread. - val sharedLockCounts = new ThreadLocal[Map[MapHolder, Int]] - - // Lock state memory offsets for all known MapHolder classes. - val lockStateOffsets = new ConcurrentHashMap[Class[_ <: MapHolder], Long] - - // Updates the shared lock count, for the current thread. - //scalastyle:off - def adjustSharedLockCount(inst: MapHolder, amt: Int): Unit = { - var countMap = sharedLockCounts.get - - if (countMap == null) { - if (amt <= 0) { - return - } - countMap = new HashMap[MapHolder, Int] - sharedLockCounts.set(countMap) - } - - var newCount = amt - - countMap.get(inst) match { - case None => if (newCount <= 0) return - case Some(count) => { - newCount += count - if (newCount <= 0) { - countMap.remove(inst) - return - } - } - } - - countMap.put(inst, newCount) - } - - /** - * Releases all shared locks, against all OffheapLFSortedIDMap instances, for the current thread. - */ - def releaseAllSharedLocks(): Int = { - var total = 0 - val countMap = sharedLockCounts.get - if (countMap != null) { - var lastKlass: Class[_ <: MapHolder] = null - var lockStateOffset = 0L - - for ((inst, amt) <- countMap) { - if (amt > 0) { - val holderKlass = inst.getClass - - if (holderKlass != lastKlass) { - lockStateOffset = lockStateOffsets.get(holderKlass) - lastKlass = holderKlass - } - - total += amt - sharedLockLingering.increment(amt) - - _logger.warn(s"Releasing all shared locks for: $inst, amount: $amt") - - var lockState = 0 - do { - lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) - } while (!UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState - amt)) - } - } - - countMap.clear - } - total - } - //scalastyle:on - - /** - * Validate no locks are held by the thread. Typically invoked prior to - * consumption from a query iterator. If there are lingering locks, - * it is quite possible a lock acquire or release bug exists - */ - def validateNoSharedLocks(): Unit = { - val numLocksReleased = OffheapLFSortedIDMap.releaseAllSharedLocks() - if (numLocksReleased > 0) { - logger.warn(s"Number of locks was non-zero: $numLocksReleased. " + - s"This is indicative of a possible lock acquisition/release bug.") - } - } - - def bytesNeeded(maxElements: Int): Int = { - require(maxElements <= MaxMaxElements) - OffsetElementPtrs + 8 * maxElements - } - - /** - * Initializes a new OffheapLFSortedIDMap in offheap memory with initial capacity maxElements - * @param memFactory an offheap MemFactory for allocating memory - * @param maxElements the maximum initial capacity + 1 - * @return pointer to the initialized memory region - */ - def allocNew(memFactory: MemFactory, maxElements: Int): NativePointer = { - require(maxElements >= MinMaxElements) - val mapPtr = memFactory.allocateOffheap(bytesNeeded(maxElements), zero=true) - UnsafeUtils.setInt(mapPtr, maxElements - 1) // head = maxElemnts - 1, one logical less than tail - UnsafeUtils.setInt(mapPtr + OffsetCopyFlag, maxElements << 16) - // TODO: do we need to initialize rest of memory? Should be initialized already - mapPtr - } -} - -// A MapHolder is the class instance that contains the pointer to the location of the offheap map. It is a var -// as the OffheapLFSortedIDMap code will mutate it atomically as the map grows/changes. -trait MapHolder { - var mapPtr: NativePointer - var lockState: Int -} - -/** - * This is a reader class because we don't want to impose the restriction that each instance of this - * offheap data structure requires an onheap class instance. That would in the case of core FiloDB use up too - * much heap memory. Instead, what we rely on is that some other class instance (for example, each instance - * of TSPartition) implements MapHolder. Thus all the methods of this class are passed a Mapholder - * and we can thus reuse one instance of this class across many many "instances" of the offheap map where - * each offheap map pointer resides in the mapPtr field of the MapHolder. - * - * @param memFactory a THREAD-SAFE factory for allocating offheap space - * @param holderKlass the Class of the MapHolder used to hold the mapPtr pointer - */ -class OffheapLFSortedIDMapReader(memFactory: MemFactory, holderKlass: Class[_ <: MapHolder]) { - import OffheapLFSortedIDMap._ - - val exclusiveLockWait = Kamon.counter("memory-exclusive-lock-waits") - - /** - * Default keyFunc which maps pointer to element to the Long keyID. It just reads the first eight bytes - * from the element as the ID. Please override to implement custom functionality. - */ - def keyFunc(elementPtr: NativePointer): Long = UnsafeUtils.getLongVolatile(elementPtr) - - protected val mapPtrOffset = UnsafeUtils.unsafe.objectFieldOffset(holderKlass.getDeclaredField("mapPtr")) - protected val lockStateOffset = UnsafeUtils.unsafe.objectFieldOffset(holderKlass.getDeclaredField("lockState")) - - lockStateOffsets.putIfAbsent(holderKlass, lockStateOffset) - - // basic accessor classes; caller must hold a lock. - @inline final def head(inst: MapHolder): Int = state(inst).head - @inline final def tail(inst: MapHolder): Int = state(inst).tail - @inline final def maxElements(inst: MapHolder): Int = state(inst).maxElem - - /** - * Number of total elements in the map - */ - final def length(inst: MapHolder): Int = { - withShared(inst, doLength(inst)) - } - - // Caller must hold a lock. - @inline private final def doLength(inst: MapHolder): Int = state(inst).length - - /** - * Accesses the element at index index, where 0 = tail or lowest element and (length - 1) is the head or highest - * Returns the pointer to the value. Caller must hold a lock. - */ - @inline final def at(inst: MapHolder, index: Int): NativePointer = { - val _mapPtr = mapPtr(inst) - val _state = state(_mapPtr) - if (_state == MapState.empty) 0 else getElem(_mapPtr, realIndex(_state, index)) - } - - /** - * Returns the element at the given key, or NULL (0) if the key is not found. Takes O(log n) time. - * Caller must hold a lock. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - * @param key the key to search for - */ - final def apply(inst: MapHolder, key: Long): NativePointer = { - val res = binarySearch(inst, key) - if (res >= 0) at(inst, res) else 0 - } - - /** - * Returns true if the given key exists in this map. Takes O(log n) time. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - * @param key the key to search for - */ - final def contains(inst: MapHolder, key: Long): Boolean = { - withShared(inst, if (doLength(inst) > 0) binarySearch(inst, key) >= 0 else false) - } - - /** - * Returns the first element, the one with the lowest key. Caller must hold a lock. - * Throws IndexOutOfBoundsException if there are no elements. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - */ - final def first(inst: MapHolder): NativePointer = { - if (doLength(inst) > 0) { getElem(mapPtr(inst), tail(inst)) } - else { throw new IndexOutOfBoundsException } - } - - /** - * Returns the last element, the one with the highest key. Caller must hold a lock. - * Throws IndexOutOfBoundsException if there are no elements. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - */ - final def last(inst: MapHolder): NativePointer = { - if (doLength(inst) > 0) { getElem(mapPtr(inst), head(inst)) } - else { throw new IndexOutOfBoundsException } - } - - /** - * Produces an ElementIterator for going through every element of the map in increasing key order. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - */ - final def iterate(inst: MapHolder): ElementIterator = { - new LazyElementIterator(() => { - acquireShared(inst) - try { - makeElemIterator(inst, 0)(alwaysContinue) - } catch { - case e: Throwable => releaseShared(inst); throw e; - } - }) - } - - /** - * Produces an ElementIterator for iterating elements in increasing key order from startKey to endKey - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - * @param startKey start at element whose key is equal or immediately greater than startKey - * @param endKey end iteration when element is greater than endKey. endKey is inclusive. - */ - final def slice(inst: MapHolder, startKey: Long, endKey: Long): ElementIterator = { - new LazyElementIterator(() => { - acquireShared(inst) - try { - val _mapPtr = mapPtr(inst) - val _state = state(_mapPtr) - val logicalStart = binarySearch(_mapPtr, _state, startKey) & 0x7fffffff - makeElemIterator(inst, logicalStart) { elem: NativePointer => keyFunc(elem) <= endKey } - } catch { - case e: Throwable => releaseShared(inst); throw e; - } - }) - } - - /** - * Produces an ElementIterator for iterating elements in increasing key order starting with startKey. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - * @param startKey start at element whose key is equal or immediately greater than startKey - */ - final def sliceToEnd(inst: MapHolder, startKey: Long): ElementIterator = { - new LazyElementIterator(() => { - acquireShared(inst) - try { - val _mapPtr = mapPtr(inst) - val _state = state(_mapPtr) - val logicalStart = binarySearch(_mapPtr, _state, startKey) & 0x7fffffff - makeElemIterator(inst, logicalStart)(alwaysContinue) - } catch { - case e: Throwable => releaseShared(inst); throw e; - } - }) - } - - /** - * Does a binary search for the element with the given key. Caller must hold a lock. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - * @param key the key to search for - * @return the element number (that can be passed to at) if exact match found, or - * element number BEFORE the element to insert, with bit 31 set, if not exact match. - * 0 if key is lower than tail/first element, and length if key is higher than last element - * IllegalStateResult if state changed underneath - */ - def binarySearch(inst: MapHolder, key: Long): Int = { - var result = IllegalStateResult - do { - val _mapPtr = mapPtr(inst) - val _state = state(_mapPtr) - if (_state == MapState.empty) return IllegalStateResult - require(_state.length > 0, "Cannot binarySearch inside an empty map") - result = binarySearch(_mapPtr, _state, key) - } while (result == IllegalStateResult) - result - } - - def binarySearch(_mapPtr: NativePointer, _state: MapState, key: Long): Int = { - val mapLen = _state.length - if (!check(_mapPtr, _state)) return IllegalStateResult - - @annotation.tailrec def innerBinSearch(first: Int, len: Int): Int = - if (first >= mapLen) { - // Past the last element. Return mapLen with not found bit set - mapLen | 0x80000000 - } else if (len == 0) { - val elem = getElem(_mapPtr, realIndex(_state, first)) - if (keyFunc(elem) == key) first else first | 0x80000000 - } else { - val half = len >>> 1 - val middle = first + half - val elem = getElem(_mapPtr, realIndex(_state, middle)) - if (!check(_mapPtr, _state)) { IllegalStateResult } - else { - val elementKey = keyFunc(elem) - if (elementKey == key) { - middle - } else if (elementKey < key) { - innerBinSearch(middle + 1, len - half - 1) - } else { - innerBinSearch(first, half) - } - } - } - - innerBinSearch(0, mapLen) - } - - /** - * Acquire exclusive access to this map, spinning if necessary. Exclusive lock isn't re-entrant. - */ - def acquireExclusive(inst: MapHolder): Unit = { - // Spin-lock implementation. Because the owner of the shared lock might be blocked by this - // thread as it waits for an exclusive lock, deadlock is possible. To mitigate this problem, - // timeout and retry, allowing shared lock waiters to make progress. The timeout doubles - // for each retry, up to a limit, but the retries continue indefinitely. - - var timeoutNanos = InitialExclusiveRetryTimeoutNanos - var warned = false - - while (true) { - if (tryAcquireExclusive(inst, timeoutNanos)) { - return - } - - timeoutNanos = Math.min(timeoutNanos << 1, MaxExclusiveRetryTimeoutNanos) - - if (!warned && timeoutNanos >= MaxExclusiveRetryTimeoutNanos) { - exclusiveLockWait.increment() - _logger.warn(s"Waiting for exclusive lock: $inst") - warned = true - } - } - } - - /** - * Acquire exclusive access to this map, spinning if necessary. Exclusive lock isn't re-entrant. - * - * @return false if timed out - */ - private def tryAcquireExclusive(inst: MapHolder, timeoutNanos: Long): Boolean = { - // Spin-lock implementation. - - var lockState = 0 - - // First set the high bit, to signal an exclusive lock request. - - var done = false - do { - lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) - if (lockState < 0) { - // Wait for exclusive lock to be released. - Thread.`yield` - } else if (UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState | 0x80000000)) { - if (lockState == 0) { - return true - } - done = true - } - } while (!done) - - // Wait for shared lock owners to release the lock. - - val endNanos = System.nanoTime + timeoutNanos - - do { - Thread.`yield` - lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) - if ((lockState & 0x7fffffff) == 0) { - return true - } - } while (System.nanoTime() < endNanos) - - // Timed out. Release the exclusive lock request signal and yield (to permit shared access again). - - while(!UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState & 0x7fffffff)) { - lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) - } - - Thread.`yield` - return false - } - - /** - * Release an acquired exclusive lock. - */ - def releaseExclusive(inst: MapHolder): Unit = { - UnsafeUtils.setIntVolatile(inst, lockStateOffset, 0) - } - - /** - * Run the given function body with the exclusive lock held, which isn't re-entrant. - */ - def withExclusive[T](inst: MapHolder, body: => T): T = { - acquireExclusive(inst) - try { - body - } finally { - releaseExclusive(inst) - } - } - - /** - * Acquire shared access to this map, spinning if necessary. - */ - def acquireShared(inst: MapHolder): Unit = { - // Spin-lock implementation. - - var lockState = 0 - - while (true) { - lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) - if (lockState < 0) { - // Wait for exclusive lock to be released. - Thread.`yield` - } else if (UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState + 1)) { - adjustSharedLockCount(inst, +1) - return - } - } - } - - /** - * Release an acquired shared lock. - */ - def releaseShared(inst: MapHolder): Unit = { - var lockState = 0 - do { - lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) - } while (!UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState - 1)) - adjustSharedLockCount(inst, -1) - } - - /** - * Run the given function body with the shared lock held. - */ - def withShared[T](inst: MapHolder, body: => T): T = { - acquireShared(inst) - try { - body - } finally { - releaseShared(inst) - } - } - - // curIdx has to be initialized to one less than the starting logical index - // NOTE: This always fetches items using the official API. This is slower, but guarantees that no matter - // how slowly the iterator user pulls, it will always be pulling the right thing even if state/mapPtr changes. - private class SortedIDMapElemIterator(inst: MapHolder, - var curIdx: Int, - continue: NativePointer => Boolean, - var closed: Boolean = false, - var nextElem: NativePointer = 0L) - extends ElementIterator { - final def close(): Unit = { - if (!closed) doClose() - } - - private def doClose(): Unit = { - closed = true; - releaseShared(inst) - } - - final def hasNext: Boolean = { - if (closed) return false - nextElem = at(inst, curIdx + 1) - val result = curIdx < (doLength(inst) - 1) && continue(nextElem) - if (!result) doClose() - result - } - - final def next: NativePointer = { - if (closed) throw new NoSuchElementException() - curIdx += 1 - nextElem - } - } - - // Note: Caller must have acquired shared lock. It's released when iterator is closed. - private def makeElemIterator(inst: MapHolder, logicalStart: Int) - (continue: NativePointer => Boolean): ElementIterator = - new SortedIDMapElemIterator(inst, logicalStart - 1, continue) - - private def alwaysContinue(p: NativePointer): Boolean = true - - // "real" index adjusting for position of head/tail - @inline protected def mapPtr(inst: MapHolder): NativePointer = UnsafeUtils.getLongVolatile(inst, mapPtrOffset) - private[memory] def state(inst: MapHolder): MapState = state(mapPtr(inst)) - protected def state(mapPtr: NativePointer): MapState = - if (isPtrNull(mapPtr)) MapState.empty else MapState(UnsafeUtils.getLongVolatile(mapPtr)) - @inline protected final def realIndex(state: MapState, index: Int): Int = - if (state.maxElem == 0) 0 else (index + state.tail) % state.maxElem - @inline protected final def elemPtr(mapPtr: NativePointer, realIndex: Int): NativePointer = - mapPtr + OffsetElementPtrs + 8 * realIndex - @inline protected final def getElem(mapPtr: NativePointer, realIndex: Int): NativePointer = - UnsafeUtils.getLongVolatile(elemPtr(mapPtr, realIndex)) - - // For some reason, occasionally the upper 4 bits of a read pointer can be nonzero even though the rest of it is. - // So this is a "safer" null check - @inline final def isPtrNull(ptr: NativePointer): Boolean = (ptr & 0xffffffffffffffL) == 0 - - @inline protected def check(mapPtr: NativePointer, state: MapState): Boolean = - !isPtrNull(mapPtr) && UnsafeUtils.getLongVolatile(mapPtr) == state.state && state.state != 0 -} - -class OffheapLFSortedIDMapMutator(memFactory: MemFactory, holderKlass: Class[_ <: MapHolder]) -extends OffheapLFSortedIDMapReader(memFactory, holderKlass) { - import OffheapLFSortedIDMap._ - - /** - * Inserts/replaces the element into the Map using the key computed from the element, - * atomically changing the state, and retrying until compare and swap succeeds. - * In case of replacing existing value for same key - then the last write wins. - * Takes O(1) time if key is the highest in the map, or O(n) otherwise. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - * @param element the native pointer to the offheap element; must be able to apply keyFunc to it to get the key - */ - //scalastyle:off - final def put(inst: MapHolder, element: NativePointer): Unit = { - // Inner method to retry for optimistic locking/lockfree loop - def innerPut(key: Long, newElem: NativePointer): Boolean = { - // Get current state in one shot and make decisions. All decisions are based on the initial state and - // subsequent operations always use CAS techniques to ensure that there is no possibility of corruption/change - // of state from time of initial entry until the CAS. If CAS fails then something changed underneath and - // we return false and retry. - val _mapPtr = mapPtr(inst) - val initState = state(_mapPtr) - // All of these should be using registers for super fast calculations - val _head = initState.head - val _maxElem = initState.maxElem - // Below should never happen with exclusive locking during writes - require(initState != MapState.empty && !initState.copyFlag) - val _len = initState.length - // If empty, add to head - if (_len == 0) { atomicHeadAdd(_mapPtr, initState, newElem) } - else { - // Problem with checking head is that new element might not have been written just after CAS succeeds - val headElem = getElem(_mapPtr, _head) - val headKey = keyFunc(headElem) - // If key == head element key, directly replace w/o binary search - if (key == headKey) { atomicReplace(_mapPtr, _head, headElem, newElem) } - else if (key > headKey) { - // If higher than head (common case), and room, atomic switch and add to head - if (_len < (_maxElem - 1)) { atomicHeadAdd(_mapPtr, initState, newElem) } - // ... or copy, insert, atomic switch ref. Remember to release current memory block. - else { copyInsertAtomicSwitch(inst, initState, _head + 1, newElem, _maxElem * 2) } - } else { - // key < headKey, Binary search. If match, atomic compareAndSwap - val res = binarySearch(_mapPtr, initState, key) // TODO: make this based on original state - if (res >= 0) { - // NOTE: replaces do not need to be atomic, but we can try anyways - // We need to "adjust" the binarySearch result, which is a logical index, into one adjusted for the tail - val insertIndex = (res + initState.tail) % _maxElem - val origElem = getElem(_mapPtr, insertIndex) - atomicReplace(_mapPtr, insertIndex, origElem, newElem) - } else if (res == IllegalStateResult) { - false - // No match. Copy, insert, atomic switch ref. Release cur mem block - } else { - val insertIndex = ((res & 0x7fffffff) + initState.tail) % _maxElem - copyInsertAtomicSwitch(inst, initState, insertIndex, newElem, - if (_len < (_maxElem - 1)) _maxElem else _maxElem * 2) - } - } - } - } - - require(element != 0, s"Cannot insert/put NULL elements") - val newKey = keyFunc(element) - while (!innerPut(newKey, element)) { - if (state(inst) == MapState.empty) return // maxElems cannot be zero - } - } - //scalastyle:on - - /** - * Atomically inserts the element it returns IF AND ONLY IF - * the item with the given key is not already in the map. - * To achieve the above goals, a new copy of the map is always made and the copyFlag CAS is used to - * guarantee only one party can do the insertion at a time. - * Thus using this function has some side effects: - * - No O(1) head optimization - * @return true if the item was inserted, false otherwise - */ - final def putIfAbsent(inst: MapHolder, key: Long, element: NativePointer): Boolean = { - var inserted = false - while (!inserted) { - val _mapPtr = mapPtr(inst) - val initState = state(_mapPtr) - if (initState == MapState.empty) return false - val _maxElem = initState.maxElem - if (initState.length == 0) { inserted = atomicHeadAdd(_mapPtr, initState, element) } - else { - val res = binarySearch(_mapPtr, initState, key) // TODO: make this based on original state - if (res >= 0) { - // key already present in map, just return false - return false - } else if (res != IllegalStateResult) { - val insertIndex = ((res & 0x7fffffff) + initState.tail) % _maxElem - inserted = copyInsertAtomicSwitch(inst, initState, insertIndex, element, - if (initState.length < (_maxElem - 1)) _maxElem else _maxElem * 2) - } - } - } - inserted - } - - /** - * Removes the element at key atomically from the map, retrying in case of conflict. - * Takes O(1) time if the key is at the tail, otherwise O(n) time on average. - * @param inst the instance (eg TSPartition) with the mapPtr field containing the map address - * @param key the key to remove. If key is not present then nothing is changed. - */ - final def remove(inst: MapHolder, key: Long): Unit = { - def innerRemove(key: Long): Boolean = { - val _mapPtr = mapPtr(inst) - val _state = state(_mapPtr) - if (_state == MapState.empty || _state.copyFlag || _state.maxElem == 0) return false - if (_state.length <= 0) { - true - } else if (check(_mapPtr, _state) && _mapPtr == mapPtr(inst)) { - // Is the tail the element to be removed? Then remove it, this is O(1) - val tailElem = getElem(_mapPtr, _state.tail) - if (key == keyFunc(tailElem)) { - atomicTailRemove(_mapPtr, _state) - } else { - // Not the tail, do a binary search, O(log n) - val res = binarySearch(_mapPtr, _state, key) - if (res == IllegalStateResult) { false } - // if key not found, just return not found - else if (res < 0) { true } - else { - copyRemoveAtomicSwitch(inst, _state, realIndex(_state, res)) - } - } - } else { false } - } - - while (!innerRemove(key)) { - if (state(inst) == MapState.empty) return // don't modify a null/absent map - } - } - - /** - * Frees the memory used by the map pointed to by inst.mapPtr, using CAS such that it will wait for concurrent - * modifications occurring to finish first. - * First the state is reset to 0, then the mapPtr itself is reset to 0, then the memory is finally freed. - * After this is called, concurrent modifications and reads of the map in inst will fail gracefully. - */ - final def free(inst: MapHolder): Unit = { - withExclusive(inst, { - var curState = state(inst) - while (curState != MapState.empty) { - val mapPtr = inst.mapPtr - if (casState(mapPtr, curState, MapState.empty)) - if (UnsafeUtils.unsafe.compareAndSwapLong(inst, mapPtrOffset, mapPtr, 0)) - memFactory.freeMemory(mapPtr) - curState = state(inst) - } - }) - } - - private def casLong(mapPtr: NativePointer, mapOffset: Int, oldLong: Long, newLong: Long): Boolean = - UnsafeUtils.unsafe.compareAndSwapLong(UnsafeUtils.ZeroPointer, mapPtr + mapOffset, oldLong, newLong) - - private def casLong(pointer: NativePointer, oldLong: Long, newLong: Long): Boolean = - UnsafeUtils.unsafe.compareAndSwapLong(UnsafeUtils.ZeroPointer, pointer, oldLong, newLong) - - private def casState(mapPtr: NativePointer, oldState: MapState, newState: MapState): Boolean = - casLong(mapPtr, 0, oldState.state, newState.state) - - private def atomicHeadAdd(mapPtr: NativePointer, initState: MapState, newElem: NativePointer): Boolean = - !initState.copyFlag && { - // compute new head - val newHead = (initState.head + 1) % initState.maxElem - - // Check the new spot is uninitialized, then use CAS to protect state while we write new element - // After CAS we can directly write stuff as we are essentially protected - getElem(mapPtr, newHead) == 0L && - atomicEnableCopyFlag(mapPtr, initState) && { - UnsafeUtils.setLong(elemPtr(mapPtr, newHead), newElem) - UnsafeUtils.setLong(mapPtr, initState.withHead(newHead).state) - true - } - } - - private def atomicTailRemove(mapPtr: NativePointer, state: MapState): Boolean = - !state.copyFlag && { - // compute new tail - val oldTail = state.tail - val newTail = (oldTail + 1) % state.maxElem - val swapped = casState(mapPtr, state, state.withTail(newTail)) - // If CAS succeeds, clear out element at old tail. This helps prevent problems during inserts. - if (swapped) UnsafeUtils.setLong(elemPtr(mapPtr, oldTail), 0) - swapped - } - - private def atomicReplace(mapPtr: NativePointer, - index: Int, - oldElem: NativePointer, - newElem: NativePointer): Boolean = - casLong(mapPtr, OffsetElementPtrs + 8 * index, oldElem, newElem) - - private def copyInsertAtomicSwitch(inst: MapHolder, - initState: MapState, - insertIndex: Int, - newElem: NativePointer, - newMaxElems: Int): Boolean = - copyMutateSwitchMap(inst, initState, newMaxElems) { case (startIndex, endIndex, movePtr) => - val _mapPtr = mapPtr(inst) - // If insertIndex = endIndex + 1, that == inserting at end - if (insertIndex >= startIndex && insertIndex <= (endIndex + 1)) { - val insertOffset = 8 * (insertIndex - startIndex) // # of bytes to copy before insertion point - UnsafeUtils.copy(elemPtr(_mapPtr, startIndex), movePtr, insertOffset) - UnsafeUtils.setLong(movePtr + insertOffset, newElem) - UnsafeUtils.copy(elemPtr(_mapPtr, insertIndex), movePtr + insertOffset + 8, 8 * (endIndex - insertIndex + 1)) - endIndex - startIndex + 2 // include endIndex and also include new element - } else { - // no need to insert within this range, just copy everything and return same # of elements - val numElemsToCopy = endIndex - startIndex + 1 - UnsafeUtils.copy(elemPtr(_mapPtr, startIndex), movePtr, numElemsToCopy * 8) - numElemsToCopy - } - } - - private def copyRemoveAtomicSwitch(inst: MapHolder, initState: MapState, removeIndex: Int): Boolean = { - // Shrink if the new length will be less than half of current maxElems - val _maxElem = initState.maxElem - val newMaxElems = if (_maxElem > 8 && initState.length <= (_maxElem / 2)) _maxElem/2 else _maxElem - copyMutateSwitchMap(inst, initState, newMaxElems) { case (startIndex, endIndex, movePtr) => - val _mapPtr = mapPtr(inst) - if (removeIndex >= startIndex && removeIndex <= endIndex) { - val removeOffset = 8 * (removeIndex - startIndex) // # of bytes to copy before insertion point - UnsafeUtils.copy(elemPtr(_mapPtr, startIndex), movePtr, removeOffset) - UnsafeUtils.copy(elemPtr(_mapPtr, removeIndex + 1), movePtr + removeOffset, 8 * (endIndex - removeIndex)) - endIndex - startIndex - } else { - val numElemsToCopy = endIndex - startIndex + 1 - UnsafeUtils.copy(elemPtr(_mapPtr, startIndex), movePtr, numElemsToCopy * 8) - numElemsToCopy - } - } - } - - private def atomicEnableCopyFlag(mapPtr: NativePointer, initState: MapState): Boolean = - !initState.copyFlag && casState(mapPtr, initState, initState.withCopyFlagOn) - - /** - * This takes care of atomically copying/transforming current array to a new location and swapping out the - * map pointers in the map address field of the instance. - * In theory the copy flag is not necessary but helps minimize # of concurrent modifications attempted. - * @param xformFunc function for copying to new address is passed (startIndex, endIndex, newElemPtr) - * where newElemPtr is a pointer to the address where the function is supposed to start copying/transforming - * original elements starting at startIndex (0-based, not tail based). - * The # of elements written is returned. - */ - private def copyMutateSwitchMap(inst: MapHolder, initState: MapState, newMaxElems: Int) - (xformFunc: (Int, Int, NativePointer) => Int): Boolean = { - val oldMapPtr = mapPtr(inst) // Should we pass this in to avoid this changing under us too? - // 1. must not be already copying and must atomically enable copying - atomicEnableCopyFlag(oldMapPtr, initState) && { - // 2. allocate new space - val newMapPtr = memFactory.allocateOffheap(bytesNeeded(newMaxElems), zero=true) - - // 3. first part: from tail to head/end - val _head = initState.head - val endIndex = if (_head < initState.tail) initState.maxElem - 1 else _head - val firstPartElems = xformFunc(initState.tail, endIndex, newMapPtr + OffsetElementPtrs) - - // 4. second part, optional: from 0 to head, if head was orig < tail - val secondPartElems = if (_head < initState.tail) { - xformFunc(0, _head, elemPtr(newMapPtr, firstPartElems)) - } else { - 0 - } - - // 5. write new state at new loc, atomic switch mapPtr, free old loc - // New tail = 0, new head = total # elements - 1 - val newState = (firstPartElems + secondPartElems - 1).toLong | (newMaxElems.toLong << 48) - UnsafeUtils.setLong(newMapPtr, newState) - // NOTE: state has to be valid before the CAS, once the CAS is done another thread will try to read it - // It is safe to write the state since we are only ones who know about the new mem region - if (UnsafeUtils.unsafe.compareAndSwapLong(inst, mapPtrOffset, oldMapPtr, newMapPtr)) { - UnsafeUtils.setLong(oldMapPtr, 0) // zero old state so those still reading will stop - memFactory.freeMemory(oldMapPtr) - true - } else { - // CAS of map pointer failed, free new map memory and try again. Also unset copy flag? - // Though we should really never reach this state, since we had the original lock on copy flag - memFactory.freeMemory(newMapPtr) - UnsafeUtils.setLong(oldMapPtr, initState.state) - false - } - } - } -} - -/** - * This is a value class, it should not occupy any heap objects. It is used to encapsulate and make the Map state - * bitfield access easier, and make the state access more typesafe. - */ -final case class MapState(state: Long) extends AnyVal { - def head: Int = (state & 0x0ffff).toInt - def tail: Int = ((state >> 16) & 0x0ffff).toInt - def copyFlag: Boolean = (state & OffheapLFSortedIDMap.CopyFlagMask) != 0 - def maxElem: Int = ((state >> 48) & 0x0ffff).toInt - def length: Int = if (maxElem > 0) (head - tail + 1 + maxElem) % maxElem else 0 - - def details: String = s"MapState(head=$head tail=$tail copyFlag=$copyFlag maxElem=$maxElem len=$length)" - - def withHead(newHead: Int): MapState = MapState(state & ~0x0ffffL | newHead) - def withTail(newTail: Int): MapState = MapState(state & ~0x0ffff0000L | (newTail.toLong << 16)) - def withCopyFlagOn: MapState = MapState(state | OffheapLFSortedIDMap.CopyFlagMask) -} - -object MapState { - val empty = MapState(0) -} - -/** - * A convenient class which uses one onheap object for each offheap SortedIDMap, probably around 50 bytes. - * If you want to save more space, it's better to share an implementation of OffheapLFSortedIDMap amongst multiple - * actual maps. The API in here is pretty convenient though. - */ -class OffheapLFSortedIDMap(memFactory: MemFactory, var mapPtr: NativePointer, var lockState: Int = 0) -extends OffheapLFSortedIDMapMutator(memFactory, classOf[OffheapLFSortedIDMap]) with MapHolder { - final def length: Int = length(this) - final def apply(key: Long): NativePointer = apply(this, key) - final def contains(key: Long): Boolean = contains(this, key) - final def first: NativePointer = first(this) - final def last: NativePointer = last(this) - final def iterate: ElementIterator = iterate(this) - final def slice(startKey: Long, endKey: Long): ElementIterator = slice(this, startKey, endKey) - final def sliceToEnd(startKey: Long): ElementIterator = sliceToEnd(this, startKey) - final def put(elem: NativePointer): Unit = put(this, elem) - final def remove(key: Long): Unit = remove(this, key) - - // Locking methods. - final def acquireExclusive(): Unit = acquireExclusive(this) - final def releaseExclusive(): Unit = releaseExclusive(this) - final def withExclusive[T](body: => T): T = withExclusive(this, body) - final def acquireShared(): Unit = acquireShared(this) - final def releaseShared(): Unit = releaseShared(this) - final def withShared[T](body: => T): T = withShared(this, body) -} - -object SingleOffheapLFSortedIDMap { - def apply(memFactory: MemFactory, maxElements: Int): OffheapLFSortedIDMap = - new OffheapLFSortedIDMap(memFactory, OffheapLFSortedIDMap.allocNew(memFactory, maxElements)) -} \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala index 789424337b..b95b1f4fa0 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala @@ -182,7 +182,7 @@ object DeltaDeltaDataReader extends LongVectorDataReader { curBase += _slope } - if (item == (curBase + inReader(inner, elemNo))) elemNo else elemNo | 0x80000000 + if (elemNo < _len && item == (curBase + inReader(inner, elemNo))) elemNo else elemNo | 0x80000000 } final def sum(vector: BinaryVectorPtr, start: Int, end: Int): Double = { diff --git a/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala new file mode 100644 index 0000000000..8fa43f08ca --- /dev/null +++ b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala @@ -0,0 +1,757 @@ +package filodb.memory.data + +import scala.collection.mutable.HashSet +import scala.concurrent.Future +import scala.util.Random + +import debox.Buffer +import org.scalatest.concurrent.ScalaFutures + +import filodb.memory.BinaryRegion.NativePointer +import filodb.memory.format.UnsafeUtils +import filodb.memory.format.vectors.NativeVectorTest + +class ChunkMapTest extends NativeVectorTest with ScalaFutures { + def makeElementWithID(id: Long): NativePointer = { + val newElem = memFactory.allocateOffheap(16) + UnsafeUtils.setLong(newElem, id) + // Ignore the second eight bytes + newElem + } + + def makeElems(ids: Seq[Long]): Array[NativePointer] = ids.toArray.map(makeElementWithID) + + def checkElems(ids: Seq[Long], elems: Buffer[Long]): Unit = { + elems.map(UnsafeUtils.getLong).toVector shouldEqual ids + } + + it("should be empty when first starting") { + val map = new ChunkMap(memFactory, 8) + map.chunkmapSize shouldEqual 0 + map.chunkmapContains(5L) shouldEqual false + intercept[IndexOutOfBoundsException] { map.chunkmapDoGetFirst } + intercept[IndexOutOfBoundsException] { map.chunkmapDoGetLast } + map.chunkmapDoGet(5L) shouldEqual 0 + map.chunkmapIterate.toBuffer shouldEqual Buffer.empty[Long] + + map.chunkmapFree() + } + + it("should insert and read back properly in various places") { + val map = new ChunkMap(memFactory, 8) + val elems = makeElems((0 to 11).map(_.toLong)) + + // when empty + map.chunkmapDoPut(elems(5)) + map.chunkmapSize shouldEqual 1 + map.chunkmapContains(5L) shouldEqual true + map.chunkmapDoGet(5L) shouldEqual elems(5) + map.chunkmapDoGetLast shouldEqual elems(5) + map.chunkmapDoGetFirst shouldEqual elems(5) + checkElems(Seq(5L), map.chunkmapIterate.toBuffer) + + // last, not empty and not full + map.chunkmapDoPut(elems(8)) + map.chunkmapSize shouldEqual 2 + map.chunkmapContains(8L) shouldEqual true + map.chunkmapDoGetLast shouldEqual elems(8) + map.chunkmapDoGetFirst shouldEqual elems(5) + checkElems(Seq(5L, 8L), map.chunkmapIterate.toBuffer) + + // middle, not empty and not full (no resize) + map.chunkmapDoPut(elems(6)) + map.chunkmapSize shouldEqual 3 + map.chunkmapContains(6L) shouldEqual true + map.chunkmapDoGetLast shouldEqual elems(8) + map.chunkmapDoGetFirst shouldEqual elems(5) + checkElems(Seq(5L, 6L, 8L), map.chunkmapIterate.toBuffer) + + // Should be no resizing as long as length/# elements < 7 + Seq(2, 3, 9, 7).foreach { n => + map.chunkmapDoPut(elems(n)) + map.chunkmapContains(n.toLong) shouldEqual true + } + map.chunkmapSize shouldEqual 7 + map.chunkmapDoGetLast shouldEqual elems(9) + checkElems(Seq(2L, 3L, 5L, 6L, 7L, 8L, 9L), map.chunkmapIterate.toBuffer) + + // last, full (should resize) + map.chunkmapDoPut(elems(10)) + map.chunkmapSize shouldEqual 8 + checkElems(Seq(2L, 3L, 5L, 6L, 7L, 8L, 9L, 10L), map.chunkmapIterate.toBuffer) + + // middle, full (should resize) + // should not resize until # elements = 15 + val elems2 = makeElems((21 to 27).map(_.toLong)) + elems2.foreach { elem => + map.chunkmapDoPut(elem) + map.chunkmapContains(UnsafeUtils.getLong(elem)) shouldEqual true + } + map.chunkmapSize shouldEqual 15 + map.chunkmapDoGetLast shouldEqual elems2.last + + map.chunkmapDoPut(elems(4)) + map.chunkmapSize shouldEqual 16 + checkElems(((2 to 10) ++ (21 to 27)).map(_.toLong), map.chunkmapIterate.toBuffer) + + map.chunkmapFree() + } + + it("should replace existing elements in various places") { + // pre-populate with elements 2 to 10 + val map = new ChunkMap(memFactory, 8) + val elems = makeElems((2 to 10).map(_.toLong)) + elems.foreach { elem => + map.chunkmapDoPut(elem) + map.chunkmapContains(UnsafeUtils.getLong(elem)) shouldEqual true + } + map.chunkmapSize shouldEqual 9 + map.chunkmapDoGetLast shouldEqual elems.last + map.chunkmapDoGet(4L) shouldEqual elems(2) + + // replace in middle + val newElem4 = makeElementWithID(4L) + map.chunkmapDoPut(newElem4) + map.chunkmapSize shouldEqual 9 + map.chunkmapDoGet(4L) shouldEqual newElem4 + map.chunkmapDoGet(4L) should not equal (elems(2)) + checkElems((2 to 10).map(_.toLong), map.chunkmapIterate.toBuffer) + + // replace at head + val newElem10 = makeElementWithID(10L) + map.chunkmapDoPut(newElem10) + map.chunkmapSize shouldEqual 9 + map.chunkmapDoGet(10L) shouldEqual newElem10 + map.chunkmapDoGetLast shouldEqual newElem10 + map.chunkmapDoGet(10L) should not equal (elems.last) + checkElems((2 to 10).map(_.toLong), map.chunkmapIterate.toBuffer) + + map.chunkmapFree() + } + + it("should putIfAbsent only if item doesn't already exist") { + // pre-populate with elements 2 to 10 + val map = new ChunkMap(memFactory, 8) + val elems = makeElems((2 to 10).map(_.toLong)) + map.chunkmapSize shouldEqual 0 + + map.chunkmapDoPutIfAbsent(elems(0)) shouldEqual true + map.chunkmapSize shouldEqual 1 + + val twoElem = makeElementWithID(2) + map.chunkmapDoPutIfAbsent(twoElem) shouldEqual false + map.chunkmapSize shouldEqual 1 + + map.chunkmapDoPutIfAbsent(elems(3)) shouldEqual true + map.chunkmapSize shouldEqual 2 + + map.chunkmapDoPutIfAbsent(elems(3)) shouldEqual false + map.chunkmapSize shouldEqual 2 + + val elemIt = map.chunkmapIterate + try { + elemIt.hasNext shouldEqual true + elemIt.next shouldEqual elems(0) + elemIt.hasNext shouldEqual true + elemIt.next shouldEqual elems(3) + } finally { + elemIt.close() + } + + map.chunkmapFree() + } + + it("should not be able to put NULL elements") { + val map = new ChunkMap(memFactory, 8) + intercept[IllegalArgumentException] { + map.chunkmapDoPut(0) + } + map.chunkmapFree() + } + + it("should insert, delete, and reinsert") { + // insert 1 item, then delete it, test map is truly empty + val map = new ChunkMap(memFactory, 8) + map.chunkmapDoPut(makeElementWithID(1)) + map.chunkmapSize shouldEqual 1 + map.chunkmapDoRemove(1L) + map.chunkmapSize shouldEqual 0 + checkElems(Nil, map.chunkmapIterate.toBuffer) + + // pre-populate with various elements + val elems = makeElems((2 to 10).map(_.toLong)) + elems.foreach { elem => + map.chunkmapDoPut(elem) + map.chunkmapContains(UnsafeUtils.getLong(elem)) shouldEqual true + } + map.chunkmapSize shouldEqual 9 + map.chunkmapDoGetLast shouldEqual elems.last + map.chunkmapDoGetFirst shouldEqual elems.head + map.chunkmapDoGet(4L) shouldEqual elems(2) + + // remove at tail. No resizing should occur. + map.chunkmapDoRemove(2L) + map.chunkmapDoGetFirst shouldEqual elems(1) + map.chunkmapSize shouldEqual 8 + checkElems((3 to 10).map(_.toLong), map.chunkmapIterate.toBuffer) + + // remove in middle. Resizing because 8 -> 7? + map.chunkmapDoRemove(6L) + map.chunkmapSize shouldEqual 7 + checkElems(Seq(3L, 4L, 5L, 7L, 8L, 9L, 10L), map.chunkmapIterate.toBuffer) + + // re-insert removed element + map.chunkmapDoPut(elems(4)) + map.chunkmapSize shouldEqual 8 + checkElems((3 to 10).map(_.toLong), map.chunkmapIterate.toBuffer) + + map.chunkmapFree() + } + + import scala.concurrent.ExecutionContext.Implicits.global + + it("should handle concurrent inserts in various places") { + // Let's have 1 thread inserting at head, and another one inserting in middle + val map = new ChunkMap(memFactory, 32) + val headElems = makeElems((100 to 199).map(_.toLong)) + val midElems = makeElems((0 to 99).map(_.toLong)) + + val headThread = Future { + headElems.foreach { elem => + map.chunkmapWithExclusive(map.chunkmapDoPut(elem)) + map.chunkmapContains(UnsafeUtils.getLong(elem)) shouldEqual true + } + } + val midThread = Future { + midElems.foreach { elem => + map.chunkmapWithExclusive(map.chunkmapDoPut(elem)) + map.chunkmapContains(UnsafeUtils.getLong(elem)) shouldEqual true + } + } + Future.sequence(Seq(headThread, midThread)).futureValue + + map.chunkmapSize shouldEqual (headElems.length + midElems.length) + checkElems((0 to 199).map(_.toLong), map.chunkmapIterate.toBuffer) + + map.chunkmapFree() + } + + it("should handle concurrent inserts and ensure slice/iterations return sane data") { + // 1 thread inserts random elem. Another allocates random strings in the buffer, just to + // increase chances of reading random crap + val map = new ChunkMap(memFactory, 32) + val elems = makeElems((0 to 99).map(_.toLong)).toSeq + + val insertThread = Future { + Random.shuffle(elems).foreach { elem => + map.chunkmapWithExclusive(map.chunkmapDoPut(elem)) + map.chunkmapContains(UnsafeUtils.getLong(elem)) shouldEqual true + } + } + val stringThread = Future { + (0 to 199).foreach { n => + val addr = memFactory.allocateOffheap(12) + UnsafeUtils.setInt(addr, Random.nextInt(1000000)) + UnsafeUtils.setInt(addr + 4, Random.nextInt(1000000)) + UnsafeUtils.setInt(addr + 8, Random.nextInt(1000000)) + } + } + val readThread = Future { + (0 to 30).foreach { n => + map.chunkmapSlice(25, 75).toBuffer.map(UnsafeUtils.getLong).foreach { key => + // This cannot always be guaranteed, esp if inserts change things underneath + //key should be >= 25L + key should be <= 75L + } + } + } + Future.sequence(Seq(insertThread, stringThread, readThread)).futureValue + + map.chunkmapSize shouldEqual elems.length + checkElems((0 to 99).map(_.toLong), map.chunkmapIterate.toBuffer) + + map.chunkmapFree() + } + + it("should handle concurrent inserts and deletes in various places") { + // First insert 0 to 99 single threaded + val map = new ChunkMap(memFactory, 32) + val elems = makeElems((0 to 99).map(_.toLong)) + elems.foreach { elem => + map.chunkmapWithExclusive(map.chunkmapDoPut(elem)) + } + map.chunkmapSize shouldEqual elems.length + + val moreElems = makeElems((100 to 199).map(_.toLong)) + val toDelete = util.Random.shuffle(0 to 99) + + // Now, have one thread deleting 0-99, while second one inserts 100-199 + val deleteThread = Future { + toDelete.foreach { n => + map.chunkmapWithExclusive(map.chunkmapDoRemove(n)) + map.chunkmapContains(n) shouldEqual false + } + } + + val insertThread = Future { + moreElems.foreach { elem => + map.chunkmapWithExclusive(map.chunkmapDoPut(elem)) + // once in a while this could fail + //map.chunkmapContains(UnsafeUtils.getLong(elem)) shouldEqual true + } + } + + Future.sequence(Seq(deleteThread, insertThread)).futureValue + + // Final map should have ONLY 100-199 + map.chunkmapSize shouldEqual moreElems.length + checkElems((100 to 199).map(_.toLong), map.chunkmapIterate.toBuffer) + + map.chunkmapFree() + } + + it("should slice correctly") { + val map = new ChunkMap(memFactory, 32) + val elems = makeElems((0 to 30 by 3).map(_.toLong)) + elems.foreach { elem => + map.chunkmapDoPut(elem) + } + map.chunkmapSize shouldEqual elems.length + + // slice: match startKey, but not endKey + checkElems(Seq(9L, 12L, 15L), map.chunkmapSlice(9L, 16L).toBuffer) + checkElems((0 to 15 by 3).map(_.toLong), map.chunkmapSlice(0L, 16L).toBuffer) + checkElems((18 to 30 by 3).map(_.toLong), map.chunkmapSlice(18L, 31L).toBuffer) + checkElems(Seq(30L), map.chunkmapSlice(30L, 30L).toBuffer) + + // slice: not match startKey, match endKey + checkElems((0 to 12 by 3).map(_.toLong), map.chunkmapSlice(-1L, 12L).toBuffer) + checkElems((12 to 18 by 3).map(_.toLong), map.chunkmapSlice(10L, 18L).toBuffer) + checkElems(Nil, map.chunkmapSlice(19L, 18L).toBuffer) + + // slice: no match for either + checkElems((12 to 18 by 3).map(_.toLong), map.chunkmapSlice(10L, 19L).toBuffer) + checkElems((0 to 15 by 3).map(_.toLong), map.chunkmapSlice(-2L, 17L).toBuffer) + checkElems((21 to 30 by 3).map(_.toLong), map.chunkmapSlice(20L, 33L).toBuffer) + checkElems(Nil, map.chunkmapSlice(16L, 17L).toBuffer) + + map.chunkmapFree() + } + + it("should sliceToEnd correctly") { + val map = new ChunkMap(memFactory, 32) + val elems = makeElems((0 to 30 by 3).map(_.toLong)) + elems.foreach { elem => + map.chunkmapDoPut(elem) + } + map.chunkmapSize shouldEqual elems.length + + checkElems((18 to 30 by 3).map(_.toLong), map.chunkmapSliceToEnd(18L).toBuffer) + checkElems((0 to 30 by 3).map(_.toLong), map.chunkmapSliceToEnd(0L).toBuffer) + + checkElems((18 to 30 by 3).map(_.toLong), map.chunkmapSliceToEnd(17L).toBuffer) + checkElems(Nil, map.chunkmapSliceToEnd(31L).toBuffer) + checkElems(Seq(30L), map.chunkmapSliceToEnd(30L).toBuffer) + + map.chunkmapFree() + } + + it("should behave gracefully once map is freed") { + val map = new ChunkMap(memFactory, 32) + val elems = makeElems((0 to 30 by 3).map(_.toLong)) + elems.foreach { elem => + map.chunkmapDoPut(elem) + } + map.chunkmapSize shouldEqual elems.length + + map.chunkmapFree() + map.chunkmapSize shouldEqual 0 + map.chunkmapDoGet(2L) shouldEqual 0 + map.chunkmapContains(3L) shouldEqual false + intercept[IndexOutOfBoundsException] { map.chunkmapDoGetFirst } + intercept[IndexOutOfBoundsException] { map.chunkmapDoGetLast } + map.chunkmapIterate.toBuffer shouldEqual Buffer.empty[Long] + map.chunkmapSliceToEnd(18L).toBuffer shouldEqual Buffer.empty[Long] + map.chunkmapSize shouldEqual 0 + map.chunkmapDoRemove(6L) + + // Double free does nothing. + map.chunkmapFree() + } + + it("should handle random access") { + // This test exercises more circular buffer cases. This can be verified with simple code + // coverage examination. The important cases deal with wraparound when adding and removing + // elements, which do end up getting tested with these parameters. + + val rnd = new java.util.Random(8675309) + val map = new ChunkMap(memFactory, 4) + val set = new HashSet[Long]() + + var size = 0 + + for (i <- 1 to 1000) { + val id = rnd.nextInt(50) + if (rnd.nextBoolean()) { + set.add(id) + if (map.chunkmapDoPutIfAbsent(makeElementWithID(id))) { + size += 1 + } + map.chunkmapContains(id) shouldEqual true + } else { + if (map.chunkmapContains(id)) { + set.remove(id) shouldEqual true + map.chunkmapDoRemove(id) + map.chunkmapContains(id) shouldEqual false + size -= 1 + } else { + set.remove(id) shouldEqual false + } + } + map.chunkmapSize shouldEqual size + set.size shouldEqual size + } + + map.chunkmapFree() + } + + it("should support uncontended locking behavior") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireExclusive() + map.chunkmapReleaseExclusive() + + map.chunkmapAcquireShared() + map.chunkmapReleaseShared() + + // Shouldn't stall. + map.chunkmapAcquireExclusive() + map.chunkmapReleaseExclusive() + + // Re-entrant shared lock. + map.chunkmapAcquireShared() + map.chunkmapAcquireShared() + map.chunkmapReleaseShared() + map.chunkmapReleaseShared() + + // Shouldn't stall. + map.chunkmapAcquireExclusive() + map.chunkmapReleaseExclusive() + + map.chunkmapFree() + } + + it("should support exclusive lock") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireExclusive() + + @volatile var acquired = false + + val stuck = new Thread { + override def run(): Unit = { + map.chunkmapAcquireExclusive() + acquired = true + map.chunkmapReleaseExclusive() + } + } + + stuck.start() + + val startNanos = System.nanoTime() + stuck.join(500) + val durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L + + acquired shouldBe false + + // Now let the second lock request complete. + map.chunkmapReleaseExclusive() + Thread.`yield` + + stuck.join(10000) + + acquired shouldBe true + + map.chunkmapFree() + } + + it("should block exclusive lock when shared lock is held") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireShared() + map.chunkmapAcquireShared() + + @volatile var acquired = false + + val stuck = new Thread { + override def run(): Unit = { + map.chunkmapAcquireExclusive() + acquired = true + map.chunkmapReleaseExclusive() + } + } + + stuck.start() + + val startNanos = System.nanoTime() + stuck.join(500) + var durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L + + acquired shouldBe false + + map.chunkmapReleaseShared() + + stuck.join(500) + durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L * 2 + + acquired shouldBe false + + // Now let the exclusive lock request complete. + map.chunkmapReleaseShared() + Thread.`yield` + + stuck.join(10000) + + acquired shouldBe true + + map.chunkmapFree() + } + + it("should block shared lock when exclusive lock is held") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireExclusive() + + @volatile var acquired = false + + val stuck = new Thread { + override def run(): Unit = { + map.chunkmapAcquireShared() + acquired = true + } + } + + stuck.start() + + val startNanos = System.nanoTime() + stuck.join(500) + var durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L + + acquired shouldBe false + + // Now let the shared lock request complete. + map.chunkmapReleaseExclusive() + Thread.`yield` + + stuck.join(10000) + + acquired shouldBe true + + // Can acquire more shared locks. + map.chunkmapAcquireShared() + map.chunkmapAcquireShared() + + // Release all shared locks. + for (i <- 1 to 3) map.chunkmapReleaseShared + + // Exclusive can be acquired again. + map.chunkmapAcquireExclusive() + map.chunkmapReleaseExclusive() + + map.chunkmapFree() + } + + it("should delay shared lock when exclusive lock is waiting") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireShared() + + @volatile var acquired = false + + val stuck = new Thread { + override def run(): Unit = { + map.chunkmapAcquireExclusive() + acquired = true + map.chunkmapReleaseExclusive() + } + } + + stuck.start() + + var startNanos = System.nanoTime() + stuck.join(1000) + var durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 1000000000L + + acquired shouldBe false + + // Need to start in another thread due to reentrancy check. + val delayed = new Thread { + override def run(): Unit = { + for (i <- 1 to 2) { + map.chunkmapAcquireShared() + map.chunkmapReleaseShared() + Thread.sleep(100) + } + } + } + + startNanos = System.nanoTime() + delayed.start() + delayed.join() + durationNanos = System.nanoTime() - startNanos + + durationNanos should be > 1000000000L + acquired shouldBe false + + // Now let the exclusive lock request complete. + map.chunkmapReleaseShared() + Thread.`yield` + + stuck.join(10000) + + acquired shouldBe true + + map.chunkmapFree() + } + + it("should release all shared locks held by the current thread") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireShared() + map.chunkmapAcquireShared() + + @volatile var acquired = false + + val stuck = new Thread { + override def run(): Unit = { + map.chunkmapAcquireExclusive() + acquired = true + map.chunkmapReleaseExclusive() + } + } + + stuck.start() + + val startNanos = System.nanoTime() + stuck.join(500) + var durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L + + acquired shouldBe false + + // Releasing all shared locks allows the exclusive lock request to complete. + ChunkMap.releaseAllSharedLocks() + Thread.`yield` + + stuck.join(10000) + + acquired shouldBe true + + map.chunkmapFree() + } + + it("should release all shared locks held for only the current thread") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireShared() + map.chunkmapAcquireShared() + + // Acquire another share, in another thread. + val shareThread = new Thread { + override def run(): Unit = map.chunkmapAcquireShared() + } + + shareThread.start() + shareThread.join() + + @volatile var acquired = false + + val stuck = new Thread { + override def run(): Unit = { + map.chunkmapAcquireExclusive() + acquired = true + map.chunkmapReleaseExclusive() + } + } + + stuck.start() + + var startNanos = System.nanoTime() + stuck.join(500) + var durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L + + acquired shouldBe false + + // Releasing one thread shared locks isn't sufficient. + ChunkMap.releaseAllSharedLocks() + Thread.`yield` + + startNanos = System.nanoTime() + stuck.join(500) + durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L + + // Now let the exclusive lock request complete. + map.chunkmapReleaseShared() + Thread.`yield` + + stuck.join(10000) + + acquired shouldBe true + + map.chunkmapFree() + } + + it("should support reentrant shared lock when exclusive lock is requested") { + val map = new ChunkMap(memFactory, 32) + + map.chunkmapAcquireShared() + + @volatile var acquired = false + + val stuck = new Thread { + override def run(): Unit = { + map.chunkmapAcquireExclusive() + acquired = true + } + } + + stuck.start() + + val startNanos = System.nanoTime() + stuck.join(500) + var durationNanos = System.nanoTime() - startNanos + + durationNanos should be >= 500000000L + + acquired shouldBe false + + // Shared lock is held by current thread, so it can easily acquire more. + for (i <- 1 to 100) map.chunkmapAcquireShared() + + acquired shouldBe false + + // Release all shared locks. + for (i <- 1 to 101) map.chunkmapReleaseShared() + + stuck.join() + acquired shouldBe true + map.chunkmapReleaseExclusive() + + map.chunkmapFree() + } +} diff --git a/memory/src/test/scala/filodb.memory/data/OffheapLFSortedIDMapTest.scala b/memory/src/test/scala/filodb.memory/data/OffheapLFSortedIDMapTest.scala deleted file mode 100644 index bf9ccab28b..0000000000 --- a/memory/src/test/scala/filodb.memory/data/OffheapLFSortedIDMapTest.scala +++ /dev/null @@ -1,641 +0,0 @@ -package filodb.memory.data - -import scala.concurrent.Future -import scala.util.Random - -import debox.Buffer -import org.scalatest.concurrent.ScalaFutures - -import filodb.memory.BinaryRegion.NativePointer -import filodb.memory.format.UnsafeUtils -import filodb.memory.format.vectors.NativeVectorTest - -class OffheapLFSortedIDMapTest extends NativeVectorTest with ScalaFutures { - def makeElementWithID(id: Long): NativePointer = { - val newElem = memFactory.allocateOffheap(16) - UnsafeUtils.setLong(newElem, id) - // Ignore the second eight bytes - newElem - } - - def makeElems(ids: Seq[Long]): Array[NativePointer] = ids.toArray.map(makeElementWithID) - - def checkElems(ids: Seq[Long], elems: Buffer[Long]): Unit = { - elems.map(UnsafeUtils.getLong).toVector shouldEqual ids - } - - it("should be empty when first starting") { - val map = SingleOffheapLFSortedIDMap(memFactory, 8) - map.length shouldEqual 0 - map.maxElements(map) shouldEqual 8 - map.tail(map) shouldEqual 0 - map.contains(5L) shouldEqual false - intercept[IndexOutOfBoundsException] { map.first } - intercept[IndexOutOfBoundsException] { map.last } - intercept[IllegalArgumentException] { map(5L) } - map.iterate.toBuffer shouldEqual Buffer.empty[Long] - } - - it("should insert and read back properly in various places") { - val map = SingleOffheapLFSortedIDMap(memFactory, 8) - val elems = makeElems((0 to 11).map(_.toLong)) - - // when empty - map.put(elems(5)) - map.length shouldEqual 1 - map.contains(5L) shouldEqual true - map(5L) shouldEqual elems(5) - map.last shouldEqual elems(5) - map.first shouldEqual elems(5) - checkElems(Seq(5L), map.iterate.toBuffer) - - // at head, not empty and not full - map.put(elems(8)) - map.length shouldEqual 2 - map.contains(8L) shouldEqual true - map.last shouldEqual elems(8) - map.first shouldEqual elems(5) - checkElems(Seq(5L, 8L), map.iterate.toBuffer) - - // in middle, not empty and not full (no resize) - map.put(elems(6)) - map.length shouldEqual 3 - map.contains(6L) shouldEqual true - map.last shouldEqual elems(8) - map.first shouldEqual elems(5) - map.maxElements(map) shouldEqual 8 - checkElems(Seq(5L, 6L, 8L), map.iterate.toBuffer) - - // Should be no resizing as long as length/# elements < 7 - Seq(2, 3, 9, 7).foreach { n => - map.put(elems(n)) - map.contains(n.toLong) shouldEqual true - } - map.length shouldEqual 7 - map.maxElements(map) shouldEqual 8 // still not resized hopefully - map.last shouldEqual elems(9) - checkElems(Seq(2L, 3L, 5L, 6L, 7L, 8L, 9L), map.iterate.toBuffer) - - // at head, full (should resize) - val origPtr = map.mapPtr - map.put(elems(10)) - map.length shouldEqual 8 - map.maxElements(map) shouldEqual 16 - map.mapPtr should not equal (origPtr) - checkElems(Seq(2L, 3L, 5L, 6L, 7L, 8L, 9L, 10L), map.iterate.toBuffer) - - // in middle, full (should resize) - // should not resize until # elements = 15 - val elems2 = makeElems((21 to 27).map(_.toLong)) - elems2.foreach { elem => - map.put(elem) - map.contains(UnsafeUtils.getLong(elem)) shouldEqual true - } - map.length shouldEqual 15 - map.maxElements(map) shouldEqual 16 // still not resized hopefully - map.last shouldEqual elems2.last - - map.put(elems(4)) - map.length shouldEqual 16 - map.maxElements(map) shouldEqual 32 - checkElems(((2 to 10) ++ (21 to 27)).map(_.toLong), map.iterate.toBuffer) - } - - it("should replace existing elements in various places") { - // pre-populate with elements 2 to 10 - val map = SingleOffheapLFSortedIDMap(memFactory, 8) - val elems = makeElems((2 to 10).map(_.toLong)) - elems.foreach { elem => - map.put(elem) - map.contains(UnsafeUtils.getLong(elem)) shouldEqual true - } - map.length shouldEqual 9 - map.last shouldEqual elems.last - map(4L) shouldEqual elems(2) - - // replace in middle - val newElem4 = makeElementWithID(4L) - map.put(newElem4) - map.length shouldEqual 9 - map(4L) shouldEqual newElem4 - map(4L) should not equal (elems(2)) - checkElems((2 to 10).map(_.toLong), map.iterate.toBuffer) - - // replace at head - val newElem10 = makeElementWithID(10L) - map.put(newElem10) - map.length shouldEqual 9 - map(10L) shouldEqual newElem10 - map.last shouldEqual newElem10 - map(10L) should not equal (elems.last) - checkElems((2 to 10).map(_.toLong), map.iterate.toBuffer) - } - - it("should putIfAbsent only if item doesn't already exist") { - // pre-populate with elements 2 to 10 - val map = SingleOffheapLFSortedIDMap(memFactory, 8) - val elems = makeElems((2 to 10).map(_.toLong)) - map.length shouldEqual 0 - - map.putIfAbsent(map, 2, elems(0)) shouldEqual true - map.length shouldEqual 1 - - val twoElem = makeElementWithID(2) - map.putIfAbsent(map, 2, twoElem) shouldEqual false - map.length shouldEqual 1 - - map.putIfAbsent(map, 5, elems(3)) shouldEqual true - map.length shouldEqual 2 - - map.putIfAbsent(map, 5, elems(3)) shouldEqual false - map.length shouldEqual 2 - - val elemIt = map.iterate - try { - elemIt.hasNext shouldEqual true - elemIt.next shouldEqual elems(0) - elemIt.hasNext shouldEqual true - elemIt.next shouldEqual elems(3) - } finally { - elemIt.close() - } - - // TODO: add concurrency tests... - } - - it("should not be able to put NULL elements") { - val map = SingleOffheapLFSortedIDMap(memFactory, 8) - intercept[IllegalArgumentException] { - map.put(0) - } - } - - it("should insert, delete, and reinsert") { - // insert 1 item, then delete it, test map is truly empty - val map = SingleOffheapLFSortedIDMap(memFactory, 8) - map.put(makeElementWithID(1)) - map.length shouldEqual 1 - map.remove(1L) - map.length shouldEqual 0 - checkElems(Nil, map.iterate.toBuffer) - - // pre-populate with various elements - val elems = makeElems((2 to 10).map(_.toLong)) - elems.foreach { elem => - map.put(elem) - map.contains(UnsafeUtils.getLong(elem)) shouldEqual true - } - map.length shouldEqual 9 - map.maxElements(map) shouldEqual 16 - map.last shouldEqual elems.last - map.first shouldEqual elems.head - map(4L) shouldEqual elems(2) - - // remove at tail. No resizing should occur. - map.remove(2L) - map.first shouldEqual elems(1) - map.length shouldEqual 8 - map.maxElements(map) shouldEqual 16 - checkElems((3 to 10).map(_.toLong), map.iterate.toBuffer) - - // remove in middle. Resizing because 8 -> 7? - map.remove(6L) - map.length shouldEqual 7 - map.maxElements(map) shouldEqual 8 - checkElems(Seq(3L, 4L, 5L, 7L, 8L, 9L, 10L), map.iterate.toBuffer) - - // re-insert removed element - map.put(elems(4)) - map.length shouldEqual 8 - checkElems((3 to 10).map(_.toLong), map.iterate.toBuffer) - } - - import scala.concurrent.ExecutionContext.Implicits.global - - it("should handle concurrent inserts in various places") { - // Let's have 1 thread inserting at head, and another one inserting in middle - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - val headElems = makeElems((100 to 199).map(_.toLong)) - val midElems = makeElems((0 to 99).map(_.toLong)) - - val headThread = Future { - headElems.foreach { elem => - map.withExclusive(map.put(elem)) - map.contains(UnsafeUtils.getLong(elem)) shouldEqual true - } - } - val midThread = Future { - midElems.foreach { elem => - map.withExclusive(map.put(elem)) - map.contains(UnsafeUtils.getLong(elem)) shouldEqual true - } - } - Future.sequence(Seq(headThread, midThread)).futureValue - - map.length shouldEqual (headElems.length + midElems.length) - checkElems((0 to 199).map(_.toLong), map.iterate.toBuffer) - } - - it("should handle concurrent inserts and ensure slice/iterations return sane data") { - // 1 thread inserts random elem. Another allocates random strings in the buffer, just to increase - // chances of reading random crap - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - val elems = makeElems((0 to 99).map(_.toLong)).toSeq - - val insertThread = Future { - Random.shuffle(elems).foreach { elem => - map.withExclusive(map.put(elem)) - map.contains(UnsafeUtils.getLong(elem)) shouldEqual true - } - } - val stringThread = Future { - (0 to 199).foreach { n => - val addr = memFactory.allocateOffheap(12) - UnsafeUtils.setInt(addr, Random.nextInt(1000000)) - UnsafeUtils.setInt(addr + 4, Random.nextInt(1000000)) - UnsafeUtils.setInt(addr + 8, Random.nextInt(1000000)) - } - } - val readThread = Future { - (0 to 30).foreach { n => - map.slice(25, 75).toBuffer.map(UnsafeUtils.getLong).foreach { key => - // key should be >= 25L // This cannot always be guaranteed, esp if inserts change things underneath - key should be <= 75L - } - } - } - Future.sequence(Seq(insertThread, stringThread, readThread)).futureValue - - map.length shouldEqual elems.length - checkElems((0 to 99).map(_.toLong), map.iterate.toBuffer) - } - - it("should handle concurrent inserts and deletes in various places") { - // First insert 0 to 99 single threaded - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - val elems = makeElems((0 to 99).map(_.toLong)) - elems.foreach { elem => - map.withExclusive(map.put(elem)) - } - map.length shouldEqual elems.length - - val moreElems = makeElems((100 to 199).map(_.toLong)) - val toDelete = util.Random.shuffle(0 to 99) - - // Now, have one thread deleting 0-99, while second one inserts 100-199 - val deleteThread = Future { - toDelete.foreach { n => - map.withExclusive(map.remove(n)) - map.contains(n) shouldEqual false - } - } - - val insertThread = Future { - moreElems.foreach { elem => - map.withExclusive(map.put(elem)) - // map.contains(UnsafeUtils.getLong(elem)) shouldEqual true // once in a while this could fail - } - } - - Future.sequence(Seq(deleteThread, insertThread)).futureValue - - // Final map should have ONLY 100-199 - map.length shouldEqual moreElems.length - checkElems((100 to 199).map(_.toLong), map.iterate.toBuffer) - } - - it("should slice correctly") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - val elems = makeElems((0 to 30 by 3).map(_.toLong)) - elems.foreach { elem => - map.put(elem) - } - map.length shouldEqual elems.length - - // slice: match startKey, but not endKey - checkElems(Seq(9L, 12L, 15L), map.slice(9L, 16L).toBuffer) - checkElems((0 to 15 by 3).map(_.toLong), map.slice(0L, 16L).toBuffer) - checkElems((18 to 30 by 3).map(_.toLong), map.slice(18L, 31L).toBuffer) - checkElems(Seq(30L), map.slice(30L, 30L).toBuffer) - - // slice: not match startKey, match endKey - checkElems((0 to 12 by 3).map(_.toLong), map.slice(-1L, 12L).toBuffer) - checkElems((12 to 18 by 3).map(_.toLong), map.slice(10L, 18L).toBuffer) - checkElems(Nil, map.slice(19L, 18L).toBuffer) - - // slice: no match for either - checkElems((12 to 18 by 3).map(_.toLong), map.slice(10L, 19L).toBuffer) - checkElems((0 to 15 by 3).map(_.toLong), map.slice(-2L, 17L).toBuffer) - checkElems((21 to 30 by 3).map(_.toLong), map.slice(20L, 33L).toBuffer) - checkElems(Nil, map.slice(16L, 17L).toBuffer) - } - - it("should sliceToEnd correctly") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - val elems = makeElems((0 to 30 by 3).map(_.toLong)) - elems.foreach { elem => - map.put(elem) - } - map.length shouldEqual elems.length - - checkElems((18 to 30 by 3).map(_.toLong), map.sliceToEnd(18L).toBuffer) - checkElems((0 to 30 by 3).map(_.toLong), map.sliceToEnd(0L).toBuffer) - - checkElems((18 to 30 by 3).map(_.toLong), map.sliceToEnd(17L).toBuffer) - checkElems(Nil, map.sliceToEnd(31L).toBuffer) - checkElems(Seq(30L), map.sliceToEnd(30L).toBuffer) - } - - it("should behave gracefully once map is freed") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - val elems = makeElems((0 to 30 by 3).map(_.toLong)) - elems.foreach { elem => - map.put(elem) - } - map.length shouldEqual elems.length - - map.free(map) - map.length shouldEqual 0 - map(2L) shouldEqual 0 - map.contains(3L) shouldEqual false - intercept[IndexOutOfBoundsException] { map.first } - intercept[IndexOutOfBoundsException] { map.last } - map.iterate.toBuffer shouldEqual Buffer.empty[Long] - map.sliceToEnd(18L).toBuffer shouldEqual Buffer.empty[Long] - map.length shouldEqual 0 - map.remove(6L) - } - - it("should support uncontended locking behavior") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - - map.acquireExclusive() - map.releaseExclusive() - - map.acquireShared() - map.releaseShared() - - // Shouldn't stall. - map.acquireExclusive() - map.releaseExclusive() - - // Re-entrant shared lock. - map.acquireShared() - map.acquireShared() - map.releaseShared() - map.releaseShared() - - // Shouldn't stall. - map.acquireExclusive() - map.releaseExclusive() - } - - it("should support exclusive lock") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - - map.acquireExclusive() - - @volatile var acquired = false - - val stuck = new Thread { - override def run(): Unit = { - map.acquireExclusive() - acquired = true - } - } - - stuck.start() - - val startNanos = System.nanoTime() - stuck.join(500) - val durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 500000000L - - acquired shouldBe false - - // Now let the second lock request complete. - map.releaseExclusive() - Thread.`yield` - - stuck.join(10000) - - acquired shouldBe true - } - - it("should block exclusive lock when shared lock is held") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - - map.acquireShared() - map.acquireShared() - - @volatile var acquired = false - - val stuck = new Thread { - override def run(): Unit = { - map.acquireExclusive() - acquired = true - } - } - - stuck.start() - - val startNanos = System.nanoTime() - stuck.join(500) - var durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 500000000L - - acquired shouldBe false - - map.releaseShared() - - stuck.join(500) - durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 500000000L * 2 - - acquired shouldBe false - - // Now let the exclusive lock request complete. - map.releaseShared() - Thread.`yield` - - stuck.join(10000) - - acquired shouldBe true - } - - it("should block shared lock when exclusive lock is held") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - - map.acquireExclusive() - - @volatile var acquired = false - - val stuck = new Thread { - override def run(): Unit = { - map.acquireShared() - acquired = true - } - } - - stuck.start() - - val startNanos = System.nanoTime() - stuck.join(500) - var durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 500000000L - - acquired shouldBe false - - // Now let the shared lock request complete. - map.releaseExclusive() - Thread.`yield` - - stuck.join(10000) - - acquired shouldBe true - - // Can acquire more shared locks. - map.acquireShared() - map.acquireShared() - - // Release all shared locks. - for (i <- 1 to 3) map.releaseShared - - // Exclusive can be acquired again. - map.acquireExclusive() - } - - it("should delay shared lock when exclusive lock is waiting") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - - map.acquireShared() - - @volatile var acquired = false - - val stuck = new Thread { - override def run(): Unit = { - map.acquireExclusive() - acquired = true - } - } - - stuck.start() - - var startNanos = System.nanoTime() - stuck.join(1000) - var durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 1000000000L - - acquired shouldBe false - - startNanos = System.nanoTime() - for (i <- 1 to 2) { - map.acquireShared() - map.releaseShared() - Thread.sleep(100) - } - durationNanos = System.nanoTime() - startNanos - - durationNanos should be > 1000000000L - acquired shouldBe false - - // Now let the exclusive lock request complete. - map.releaseShared() - Thread.`yield` - - stuck.join(10000) - - acquired shouldBe true - } - - it("should release all shared locks held by the current thread") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - - map.acquireShared() - map.acquireShared() - - @volatile var acquired = false - - val stuck = new Thread { - override def run(): Unit = { - map.acquireExclusive() - acquired = true - } - } - - stuck.start() - - val startNanos = System.nanoTime() - stuck.join(500) - var durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 500000000L - - acquired shouldBe false - - // Releasing all shared locks allows the exclusive lock request to complete. - OffheapLFSortedIDMap.releaseAllSharedLocks() - Thread.`yield` - - stuck.join(10000) - - acquired shouldBe true - } - - it("should release all shared locks held for only the current thread") { - val map = SingleOffheapLFSortedIDMap(memFactory, 32) - - map.acquireShared() - map.acquireShared() - - // Acquire another share, in another thread. - val shareThread = new Thread { - override def run(): Unit = map.acquireShared() - } - - shareThread.start() - shareThread.join() - - @volatile var acquired = false - - val stuck = new Thread { - override def run(): Unit = { - map.acquireExclusive() - acquired = true - } - } - - stuck.start() - - var startNanos = System.nanoTime() - stuck.join(500) - var durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 500000000L - - acquired shouldBe false - - // Releasing one thread shared locks isn't sufficient. - OffheapLFSortedIDMap.releaseAllSharedLocks() - Thread.`yield` - - startNanos = System.nanoTime() - stuck.join(500) - durationNanos = System.nanoTime() - startNanos - - durationNanos should be >= 500000000L - - // Now let the exclusive lock request complete. - map.releaseShared() - Thread.`yield` - - stuck.join(10000) - - acquired shouldBe true - } -} \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index f431429017..010261b522 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -11,7 +11,7 @@ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.Dataset import filodb.core.query._ -import filodb.memory.data.OffheapLFSortedIDMap +import filodb.memory.data.ChunkMap import filodb.memory.format.{RowReader, UnsafeUtils, ZeroCopyUTF8String} import filodb.query._ import filodb.query.AggregationOperator._ @@ -31,11 +31,11 @@ final case class ReduceAggregateExec(id: String, protected def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" - protected def compose(childResponses: Observable[QueryResponse], + protected def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { val results = childResponses.flatMap { - case QueryResult(_, _, result) => Observable.fromIterable(result) - case QueryError(_, ex) => throw ex + case (QueryResult(_, _, result), _) => Observable.fromIterable(result) + case (QueryError(_, ex), _) => throw ex } RangeVectorAggregator.mapReduce(aggrOp, aggrParams, skipMapPhase = true, results, rv => rv.key) } @@ -387,20 +387,15 @@ object CountRowAggregator extends RowAggregator { def zero: CountHolder = new CountHolder() def newRowToMapInto: MutableRowReader = new TransientRow() def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = { - if (!item.getDouble(1).isNaN) { - mapInto.setLong(0, item.getLong(0)) - mapInto.setDouble(1, 1d) - } - else { - mapInto.setLong(0, item.getLong(0)) - mapInto.setDouble(1, 0d) - } + mapInto.setLong(0, item.getLong(0)) + mapInto.setDouble(1, if (item.getDouble(1).isNaN) 0d else 1d) mapInto } def reduceAggregate(acc: CountHolder, aggRes: RowReader): CountHolder = { if (acc.count.isNaN && aggRes.getDouble(1) > 0) acc.count = 0d; acc.timestamp = aggRes.getLong(0) - acc.count += aggRes.getDouble(1) + if (!aggRes.getDouble(1).isNaN) + acc.count += aggRes.getDouble(1) acc } def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) @@ -531,7 +526,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { // Important TODO / TechDebt: We need to replace Iterators with cursors to better control // the chunk iteration, lock acquisition and release. This is much needed for safe memory access. try { - OffheapLFSortedIDMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks() // We limit the results wherever it is materialized first. So it is done here. aggRangeVector.rows.take(limit).foreach { row => var i = 1 @@ -548,7 +543,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { } } } finally { - OffheapLFSortedIDMap.releaseAllSharedLocks() + ChunkMap.releaseAllSharedLocks() } resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords).sum diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index f8253238ae..68f37a8c5b 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -58,15 +58,15 @@ final case class BinaryJoinExec(id: String, protected def args: String = s"binaryOp=$binaryOp, on=$on, ignoring=$ignoring" - protected[exec] def compose(childResponses: Observable[QueryResponse], + protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { val taskOfResults = childResponses.map { - case QueryResult(_, _, result) => result - case QueryError(_, ex) => throw ex + case (QueryResult(_, _, result), i) => (result, i) + case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") - val lhsRvs = resp.take(lhs.size).flatten - val rhsRvs = resp.drop(lhs.size).flatten + val lhsRvs = resp.filter(_._2 < lhs.size).map(_._1).flatten + val rhsRvs = resp.filter(_._2 >= lhs.size).map(_._1).flatten // figure out which side is the "one" side val (oneSide, otherSide, lhsIsOneSide) = diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index 3e8c2154c9..9795d50f7d 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -28,12 +28,12 @@ final case class DistConcatExec(id: String, protected def schemaOfCompose(dataset: Dataset): ResultSchema = children.head.schema(dataset) - protected def compose(childResponses: Observable[QueryResponse], + protected def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { qLogger.debug(s"DistConcatExec: Concatenating results") childResponses.flatMap { - case qr: QueryResult => Observable.fromIterable(qr.result) - case qe: QueryError => throw qe.t + case (QueryResult(_, _, result), _) => Observable.fromIterable(result) + case (QueryError(_, ex), _) => throw ex } } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 92df9a991c..a2562f8198 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.FiniteDuration import scala.util.control.NonFatal +import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -233,10 +234,14 @@ abstract class NonLeafExecPlan extends ExecPlan { queryConfig: QueryConfig) (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = { - val childTasks = Observable.fromIterable(children).mapAsync(Runtime.getRuntime.availableProcessors()) { plan => - plan.dispatcher.dispatch(plan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${id} Execution failed for sub-query ${plan.printTree()}", ex) - QueryError(id, ex) + val spanFromHelper = Kamon.currentSpan() + val childTasks = Observable.fromIterable(children.zipWithIndex) + .mapAsync(Runtime.getRuntime.availableProcessors()) { case (plan, i) => + Kamon.withSpan(spanFromHelper) { + plan.dispatcher.dispatch(plan).onErrorHandle { case ex: Throwable => + qLogger.error(s"queryId: ${id} Execution failed for sub-query ${plan.printTree()}", ex) + QueryError(id, ex) + }.map((_, i)) } } compose(childTasks, queryConfig) @@ -253,7 +258,7 @@ abstract class NonLeafExecPlan extends ExecPlan { * Sub-class non-leaf nodes should provide their own implementation of how * to compose the sub-query results here. */ - protected def compose(childResponses: Observable[QueryResponse], + protected def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index a572776c77..0bbc9dcde8 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -39,13 +39,13 @@ final case class PartKeysDistConcatExec(id: String, /** * Compose the sub-query/leaf results here. */ - override protected def compose(childResponses: Observable[QueryResponse], queryConfig: QueryConfig): + override protected def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { - case QueryResult(_, _, result) => result - case QueryError(_, ex) => throw ex + case (QueryResult(_, _, result), _) => result + case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), rowIterAccumulator(resp)) } @@ -74,13 +74,13 @@ final case class LabelValuesDistConcatExec(id: String, /** * Compose the sub-query/leaf results here. */ - override protected def compose(childResponses: Observable[QueryResponse], queryConfig: QueryConfig): + override protected def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { - case QueryResult(_, _, result) => result - case QueryError(_, ex) => throw ex + case (QueryResult(_, _, result), _) => result + case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => var metadataResult = scala.collection.mutable.Set.empty[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] resp.foreach(rv => { diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index b9246809ab..eb90478c85 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -57,7 +57,7 @@ final case class PeriodicSamplesMapper(start: Long, IteratorBackedRangeVector(rv.key, new ChunkedWindowIterator(rv.asInstanceOf[RawDataRangeVector], start, step, end, windowLength, rangeFuncGen().asInstanceOf[ChunkedRangeFunction], - queryConfig)()) + queryConfig)) } // Iterator-based: Wrap long columns to yield a double value case f: RangeFunction if valColType == ColumnType.LongColumn => @@ -105,36 +105,55 @@ class ChunkedWindowIterator(rv: RawDataRangeVector, end: Long, window: Long, rangeFunction: ChunkedRangeFunction, - queryConfig: QueryConfig) - (windowIt: WindowedChunkIterator = - new WindowedChunkIterator(rv.chunkInfos(start - window, end), start, step, end, window) - ) extends Iterator[TransientRow] with StrictLogging { + queryConfig: QueryConfig) extends Iterator[TransientRow] with StrictLogging { + // Lazily open the iterator and obtain the lock. This allows one thread to create the + // iterator, but the lock is owned by the thread actually performing the iteration. + private lazy val windowIt = { + val it = new WindowedChunkIterator(rv.chunkInfos(start - window, end), start, step, end, window) + // Need to hold the shared lock explicitly, because the window iterator needs to + // pre-fetch chunks to determine the window. This pre-fetching can force the internal + // iterator to close, which would release the lock too soon. + it.lock() + it + } + private val sampleToEmit = new TransientRow() override def hasNext: Boolean = windowIt.hasMoreWindows + override def next: TransientRow = { rangeFunction.reset() // TODO: detect if rangeFunction needs items completely sorted. For example, it is possible // to do rate if only each chunk is sorted. Also check for counter correction here - windowIt.nextWindow() - while (windowIt.hasNext) { - val nextInfo = windowIt.nextInfo + // Lazy variables have an extra lookup cost, due to a volatile bitmap field generated by + // the compiler. Copy to a local variable to reduce some overhead. + val wit = windowIt + + wit.nextWindow() + while (wit.hasNext) { + val nextInfo = wit.nextInfo try { rangeFunction.addChunks(rv.timestampColID, rv.valueColID, nextInfo, - windowIt.curWindowStart, windowIt.curWindowEnd, queryConfig) + wit.curWindowStart, wit.curWindowEnd, queryConfig) } catch { case e: Exception => val timestampVector = nextInfo.vectorPtr(rv.timestampColID) val tsReader = bv.LongBinaryVector(timestampVector) logger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + - s"info.endTime=${nextInfo.endTime} curWindowEnd=${windowIt.curWindowEnd} " + + s"info.endTime=${nextInfo.endTime} curWindowEnd=${wit.curWindowEnd} " + s"tsReader=$tsReader timestampVectorLength=${tsReader.length(timestampVector)}") throw e } } - rangeFunction.apply(windowIt.curWindowEnd, sampleToEmit) - if (!windowIt.hasMoreWindows) windowIt.close() // release shared lock proactively + rangeFunction.apply(wit.curWindowEnd, sampleToEmit) + + if (!wit.hasMoreWindows) { + // Release the shared lock and close the iterator, in case it also holds a lock. + wit.unlock() + wit.close() + } + sampleToEmit } } diff --git a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala index cf7b443938..1b9c924daa 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -28,8 +28,8 @@ class MinMaxOverTimeFunction(ord: Ordering[Double]) extends RangeFunction { } } -class MinOverTimeChunkedFunctionD(var min: Double = Double.MaxValue) extends ChunkedDoubleRangeFunction { - override final def reset(): Unit = { min = Double.MaxValue } +class MinOverTimeChunkedFunctionD(var min: Double = Double.NaN) extends ChunkedDoubleRangeFunction { + override final def reset(): Unit = { min = Double.NaN } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { sampleToEmit.setValues(endTimestamp, min) } @@ -41,7 +41,7 @@ class MinOverTimeChunkedFunctionD(var min: Double = Double.MaxValue) extends Chu val it = doubleReader.iterate(doubleVect, startRowNum) while (rowNum <= endRowNum) { val nextVal = it.next - if (!JLDouble.isNaN(nextVal)) min = Math.min(min, nextVal) // cannot compare NaN, always < anything else + min = if (min.isNaN) nextVal else Math.min(min, nextVal) rowNum += 1 } } @@ -65,8 +65,8 @@ class MinOverTimeChunkedFunctionL(var min: Long = Long.MaxValue) extends Chunked } } -class MaxOverTimeChunkedFunctionD(var max: Double = Double.MinValue) extends ChunkedDoubleRangeFunction { - override final def reset(): Unit = { max = Double.MinValue } +class MaxOverTimeChunkedFunctionD(var max: Double = Double.NaN) extends ChunkedDoubleRangeFunction { + override final def reset(): Unit = { max = Double.NaN } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { sampleToEmit.setValues(endTimestamp, max) } @@ -78,7 +78,7 @@ class MaxOverTimeChunkedFunctionD(var max: Double = Double.MinValue) extends Chu val it = doubleReader.iterate(doubleVect, startRowNum) while (rowNum <= endRowNum) { val nextVal = it.next - if (!JLDouble.isNaN(nextVal)) max = Math.max(max, nextVal) // cannot compare NaN, always < anything else + max = if (max.isNaN) nextVal else Math.max(max, nextVal) // cannot compare NaN, always < anything else rowNum += 1 } } @@ -102,13 +102,28 @@ class MaxOverTimeChunkedFunctionL(var max: Long = Long.MinValue) extends Chunked } } -class SumOverTimeFunction(var sum: Double = 0d) extends RangeFunction { +class SumOverTimeFunction(var sum: Double = Double.NaN, var count: Int = 0) extends RangeFunction { override def addedToWindow(row: TransientRow, window: Window): Unit = { - sum += row.value + if (!JLDouble.isNaN(row.value)) { + if (sum.isNaN) { + sum = 0d + } + sum += row.value + count += 1 + } } override def removedFromWindow(row: TransientRow, window: Window): Unit = { - sum -= row.value + if (!JLDouble.isNaN(row.value)) { + if (sum.isNaN) { + sum = 0d + } + sum -= row.value + count -= 1 + if (count == 0) { // There is no value in window + sum = Double.NaN + } + } } override def apply(startTimestamp: Long, endTimestamp: Long, window: Window, @@ -118,8 +133,8 @@ class SumOverTimeFunction(var sum: Double = 0d) extends RangeFunction { } } -abstract class SumOverTimeChunkedFunction(var sum: Double = 0d) extends ChunkedRangeFunction { - override final def reset(): Unit = { sum = 0d } +abstract class SumOverTimeChunkedFunction(var sum: Double = Double.NaN )extends ChunkedRangeFunction { + override final def reset(): Unit = { sum = Double.NaN } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { sampleToEmit.setValues(endTimestamp, sum) } @@ -131,6 +146,9 @@ class SumOverTimeChunkedFunctionD extends SumOverTimeChunkedFunction() with Chun startRowNum: Int, endRowNum: Int): Unit = { // NaN values are ignored by default in the sum method + if (sum.isNaN) { + sum = 0d + } sum += doubleReader.sum(doubleVect, startRowNum, endRowNum) } } @@ -140,23 +158,39 @@ class SumOverTimeChunkedFunctionL extends SumOverTimeChunkedFunction() with Chun longReader: bv.LongVectorDataReader, startRowNum: Int, endRowNum: Int): Unit = { + if (sum.isNaN) { + sum = 0d + } sum += longReader.sum(longVect, startRowNum, endRowNum) } } -class CountOverTimeFunction(var count: Int = 0) extends RangeFunction { +class CountOverTimeFunction(var count: Double = Double.NaN) extends RangeFunction { override def addedToWindow(row: TransientRow, window: Window): Unit = { - count += 1 + if (!JLDouble.isNaN(row.value)) { + if (count.isNaN) { + count = 0d + } + count += 1 + } } override def removedFromWindow(row: TransientRow, window: Window): Unit = { - count -= 1 + if (!JLDouble.isNaN(row.value)) { + if (count.isNaN) { + count = 0d + } + count -= 1 + if (count==0) { //Reset count as no sample is present + count = Double.NaN + } + } } override def apply(startTimestamp: Long, endTimestamp: Long, window: Window, sampleToEmit: TransientRow, queryConfig: QueryConfig): Unit = { - sampleToEmit.setValues(endTimestamp, count.toDouble) + sampleToEmit.setValues(endTimestamp, count) } } @@ -181,28 +215,41 @@ class CountOverTimeChunkedFunction(var count: Int = 0) extends ChunkedRangeFunct // Special count_over_time chunked function for doubles needed to not count NaNs whih are used by // Prometheus to mark end of a time series. // TODO: handle end of time series a different, better way. This function shouldn't be needed. -class CountOverTimeChunkedFunctionD(var count: Int = 0) extends ChunkedDoubleRangeFunction { - override final def reset(): Unit = { count = 0 } +class CountOverTimeChunkedFunctionD(var count: Double = Double.NaN) extends ChunkedDoubleRangeFunction { + override final def reset(): Unit = { count = Double.NaN } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { - sampleToEmit.setValues(endTimestamp, count.toDouble) + sampleToEmit.setValues(endTimestamp, count) } final def addTimeDoubleChunks(doubleVect: BinaryVector.BinaryVectorPtr, doubleReader: bv.DoubleVectorDataReader, startRowNum: Int, endRowNum: Int): Unit = { + if (count.isNaN) { + count = 0d + } count += doubleReader.count(doubleVect, startRowNum, endRowNum) } } -class AvgOverTimeFunction(var sum: Double = 0d, var count: Int = 0) extends RangeFunction { +class AvgOverTimeFunction(var sum: Double = Double.NaN, var count: Int = 0) extends RangeFunction { override def addedToWindow(row: TransientRow, window: Window): Unit = { - sum += row.value - count += 1 + if (!JLDouble.isNaN(row.value)) { + if (sum.isNaN) { + sum = 0d; + } + sum += row.value + count += 1 + } } override def removedFromWindow(row: TransientRow, window: Window): Unit = { - sum -= row.value - count -= 1 + if (!JLDouble.isNaN(row.value)) { + if (sum.isNaN) { + sum = 0d; + } + sum -= row.value + count -= 1 + } } override def apply(startTimestamp: Long, endTimestamp: Long, window: Window, @@ -212,10 +259,15 @@ class AvgOverTimeFunction(var sum: Double = 0d, var count: Int = 0) extends Rang } } -abstract class AvgOverTimeChunkedFunction(var sum: Double = 0d, var count: Int = 0) extends ChunkedRangeFunction { - override final def reset(): Unit = { sum = 0d; count = 0 } +abstract class AvgOverTimeChunkedFunction(var sum: Double = Double.NaN, var count: Double = 0) + extends ChunkedRangeFunction { + override final def reset(): Unit = { + sum = Double.NaN; + count = 0d + } + final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { - sampleToEmit.setValues(endTimestamp, if (count > 0) sum/count else 0d) + sampleToEmit.setValues(endTimestamp, if (count > 0) sum/count else if (sum.isNaN()) sum else 0d) } } @@ -224,6 +276,9 @@ class AvgOverTimeChunkedFunctionD extends AvgOverTimeChunkedFunction() with Chun doubleReader: bv.DoubleVectorDataReader, startRowNum: Int, endRowNum: Int): Unit = { + if (sum.isNaN) { + sum = 0d + } sum += doubleReader.sum(doubleVect, startRowNum, endRowNum) count += doubleReader.count(doubleVect, startRowNum, endRowNum) } diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala index 08b04286f0..972b620b03 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -123,7 +123,10 @@ trait ChunkedDoubleRangeFunction extends ChunkedRangeFunction { val startRowNum = tsReader.binarySearch(timestampVector, startTime) & 0x7fffffff val endRowNum = tsReader.ceilingIndex(timestampVector, endTime) - addTimeDoubleChunks(doubleVector, dblReader, startRowNum, Math.min(endRowNum, info.numRows - 1)) + // At least one sample is present + if (startRowNum <= endRowNum) { + addTimeDoubleChunks(doubleVector, dblReader, startRowNum, Math.min(endRowNum, info.numRows - 1)) + } } /** @@ -150,7 +153,9 @@ trait ChunkedLongRangeFunction extends ChunkedRangeFunction { val startRowNum = tsReader.binarySearch(timestampVector, startTime) & 0x7fffffff val endRowNum = tsReader.ceilingIndex(timestampVector, endTime) - addTimeLongChunks(longVector, longReader, startRowNum, Math.min(endRowNum, info.numRows - 1)) + if (startRowNum <= endRowNum) { + addTimeLongChunks(longVector, longReader, startRowNum, Math.min(endRowNum, info.numRows - 1)) + } } /** @@ -235,7 +240,7 @@ object RangeFunction { case Some(Rate) => () => RateFunction case Some(Increase) => () => IncreaseFunction case Some(Delta) => () => DeltaFunction - case Some(Resets) => () => ResetsFunction + case Some(Resets) => () => new ResetsFunction() case Some(Irate) => () => IRateFunction case Some(Idelta) => () => IDeltaFunction case Some(Deriv) => () => DerivFunction diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala index 7c63392380..b6972a3285 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala @@ -118,7 +118,7 @@ object DerivFunction extends RangeFunction { } } -object ResetsFunction extends RangeFunction { +class ResetsFunction extends RangeFunction { var resets = 0 def addedToWindow(row: TransientRow, window: Window): Unit = { @@ -129,7 +129,7 @@ object ResetsFunction extends RangeFunction { } def removedFromWindow(row: TransientRow, window: Window): Unit = { - if (row.value > window.head.value) { + if (window.size > 0 && row.value > window.head.value) { resets -= 1 } } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index aed3c4c524..6e1b62c19b 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -70,7 +70,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, samplesLhs.map(rv => SerializableRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq(lhs, rhs)), queryConfig).toListL.runAsync.futureValue + // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), queryConfig).toListL.runAsync.futureValue result.foreach { rv => rv.key.labelValues.contains("__name__".utf8) shouldEqual false @@ -101,7 +102,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, samplesLhs.map(rv => SerializableRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq(lhs, rhs)), queryConfig).toListL.runAsync.futureValue + val result = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig).toListL.runAsync.futureValue result.foreach { rv => rv.key.labelValues.contains("__name__".utf8) shouldEqual false @@ -141,7 +142,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq(lhs, rhs)), queryConfig).toListL.runAsync + val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig).toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e shouldBe a[BadQueryException] } @@ -174,7 +175,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq(lhs, rhs)), queryConfig).toListL.runAsync + val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig).toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e shouldBe a[BadQueryException] } diff --git a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala index fd89ced58a..aac8f1da8f 100644 --- a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala @@ -27,7 +27,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig)() + val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -39,7 +39,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig)() + val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -52,7 +52,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig)() + val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -69,7 +69,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig)() + val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -84,7 +84,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig)() + val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -99,7 +99,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig)() + val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index 40c8e30163..3cb87b0be4 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -168,7 +168,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { } } - it ("should calculate SumOverTime correctly even after time series stops. " + + it("should calculate SumOverTime correctly even after time series stops. " + "It should exclude values at curWindowStart") { val samples = Seq( @@ -181,32 +181,25 @@ class WindowIteratorSpec extends RawDataWindowingSpec { 430000L->7d, 690000L->8d, 700000L->9d, - 710000L->Double.NaN // NOTE: Prom end of time series marker + 710000L->Double.NaN // NOTE: Prom end of time series marker ) val rv = timeValueRV(samples) val windowResults = Seq( - 50000->0.0, 150000->1.0, 250000->5.0, 350000->9.0, 450000->13.0, - 550000->0.0, - 650000->0.0, - 750000->17.0, - 850000->0.0, - 950000->0.0, - 1050000->0.0 + 750000->17.0 ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 1100000L, 100000, RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) - // NOTE: dum_over_time sliding iterator does not handle the NaN at the end correctly! - // slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).toList shouldEqual windowResults + slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 1100000L, 100000, RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() - chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).toList shouldEqual windowResults + .asInstanceOf[ChunkedRangeFunction], queryConfig) + chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } it("should calculate the rate of given samples matching the prometheus rate function") { @@ -298,7 +291,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIterator(rv, 1540845090000L, 15000, 1540855905000L, queryConfig.staleSampleAfterMs, RangeFunction(None, ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() + .asInstanceOf[ChunkedRangeFunction], queryConfig) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -327,7 +320,141 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIterator(rv, 100000L, 100000, 600000L, queryConfig.staleSampleAfterMs + 1, RangeFunction(None, ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() + .asInstanceOf[ChunkedRangeFunction], queryConfig) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } + + it("should calculate AvgOverTime correctly even for windows with no values") { + + val samples = Seq( + 100000L -> 1d, + 153000L -> 2d, + 250000L -> 3d, + 270000L -> 4d, + 280000L -> 5d, + 360000L -> 6d, + 430000L -> 7d, + 690000L -> 8d, + 700000L -> 9d, + 710000L -> Double.NaN // NOTE: Prom end of time series marker + ) + val rv = timeValueRV(samples) + + val windowResults = Seq( + 150000 -> 1.0, + 250000 -> 2.5, + 350000 -> 4.5, + 450000 -> 6.5 + ) + + val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + + val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, useChunked = true) + .asInstanceOf[ChunkedRangeFunction], queryConfig) + chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + } + + it("should calculate CountOverTime correctly even for windows with no values") { + + val samples = Seq( + 100000L -> 1d, + 153000L -> 2d, + 250000L -> 3d, + 270000L -> 4d, + 280000L -> 5d, + 360000L -> 6d, + 430000L -> 7d, + 690000L -> 8d, + 700000L -> 9d, + 710000L -> Double.NaN + ) + val rv = timeValueRV(samples) + + val windowResults = Seq( + 150000 -> 1.0, + 250000 -> 2.0, + 350000 -> 2.0, + 450000 -> 2.0 + ) + + val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + + + val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, useChunked = true) + .asInstanceOf[ChunkedRangeFunction], queryConfig) + chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + } + + it("should calculate MinOverTime correctly even for windows with no values") { + + val samples = Seq( + 100000L -> 1d, + 153000L -> 2d, + 250000L -> 3d, + 270000L -> 4d, + 280000L -> 5d, + 360000L -> 6d, + 430000L -> 7d, + 690000L -> 8d, + 700000L -> 9d, + 710000L -> Double.NaN + ) + val rv = timeValueRV(samples) + + val windowResults = Seq( + 150000 -> 1.0, + 250000 -> 2.0, + 350000 -> 4.0, + 450000 -> 6.0 + ) + + val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + + val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, useChunked = true) + .asInstanceOf[ChunkedRangeFunction], queryConfig) + chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + + } + + it("should calculate MaxOverTime correctly even for windows with no values") { + + val samples = Seq( + 100000L -> 1d, + 153000L -> 2d, + 250000L -> 3d, + 270000L -> 4d, + 280000L -> 5d, + 360000L -> 6d, + 430000L -> 7d, + 690000L -> 8d, + 700000L -> 9d, + 710000L -> Double.NaN + ) + val rv = timeValueRV(samples) + + val windowResults = Seq( + 150000 -> 1.0, + 250000 -> 3.0, + 350000 -> 5.0, + 450000 -> 7.0 + ) + + val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + + val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, useChunked = true) + .asInstanceOf[ChunkedRangeFunction], queryConfig) + chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index 71366d1698..5fc7336a42 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -68,7 +68,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val windowStartTS = defaultStartTS + windowTime val stepTimeMillis = step.toLong * pubFreq val windowEndTS = windowStartTS + (numWindows(data, windowSize, step) - 1) * stepTimeMillis - new ChunkedWindowIterator(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, queryConfig)() + new ChunkedWindowIterator(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, queryConfig) } def slidingWindowIt(data: Seq[Double], diff --git a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala index 479d91b2f2..24f19965c6 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -82,13 +82,14 @@ class RateFunctionsSpec extends FunSpec with Matchers { val toEmit = new TransientRow val q3 = new IndexedArrayQueue[TransientRow]() val gaugeWindowForReset = new QueueBasedWindow(q3) + val resetsFunction = new ResetsFunction gaugeSamples.foreach { case (t, v) => val s = new TransientRow(t, v) q3.add(s) - ResetsFunction.addedToWindow(s, gaugeWindowForReset) + resetsFunction.addedToWindow(s, gaugeWindowForReset) } - ResetsFunction.apply(startTs, endTs, gaugeWindowForReset, toEmit, queryConfig) + resetsFunction.apply(startTs, endTs, gaugeWindowForReset, toEmit, queryConfig) Math.abs(toEmit.value - expected) should be < errorOk // Window sliding case @@ -98,9 +99,9 @@ class RateFunctionsSpec extends FunSpec with Matchers { // 3 resets at the beginning - so resets count should drop only by 3 (4 - 3 = 1) even though we are removing 5 items for (i <- 0 until 5) { toEmit2 = q3.remove - ResetsFunction.removedFromWindow(toEmit2, gaugeWindowForReset)// old items being evicted for new window items + resetsFunction.removedFromWindow(toEmit2, gaugeWindowForReset)// old items being evicted for new window items } - ResetsFunction.apply(startTs, endTs, gaugeWindow, toEmit2, queryConfig) + resetsFunction.apply(startTs, endTs, gaugeWindow, toEmit2, queryConfig) Math.abs(toEmit2.value - expected2) should be < errorOk } diff --git a/standalone/src/main/java/filodb/standalone/SimpleProfiler.java b/standalone/src/main/java/filodb/standalone/SimpleProfiler.java index 54d36b0907..a2fb25dfcf 100644 --- a/standalone/src/main/java/filodb/standalone/SimpleProfiler.java +++ b/standalone/src/main/java/filodb/standalone/SimpleProfiler.java @@ -168,62 +168,111 @@ public void stop() { } } - private void analyze(Map samples, ThreadInfo[] infos) + private void analyze(Map samples, + Map summaries, + ThreadInfo[] infos) throws IOException { for (ThreadInfo info : infos) { StackTraceElement[] trace = examine(info); - if (trace != null) { - StackTraceElement elem = trace[0]; - Counter c = samples.get(elem); - if (c == null) { - c = new Counter(elem); - samples.put(elem, c); + if (trace == null) { + continue; + } + + StackTraceElement elem = trace[0]; + + TraceCounter tc = samples.get(elem); + if (tc == null) { + tc = new TraceCounter(elem); + samples.put(elem, tc); + } + tc.mValue++; + + // Choose the package name as the summary name. + String summaryName; + { + String className = elem.getClassName(); + int ix = className.lastIndexOf('.'); + if (ix <= 0) { + // No package at all, so use the class name instead. + summaryName = className; + } else { + summaryName = className.substring(0, ix); } - c.mValue++; } + + SummaryCounter sc = summaries.get(summaryName); + if (sc == null) { + sc = new SummaryCounter(summaryName); + summaries.put(summaryName, sc); + } + sc.mValue++; } synchronized (this) { long now = System.currentTimeMillis(); if (now >= mNextReportAtMillis && mSampler == Thread.currentThread()) { mNextReportAtMillis = Math.max(now, mNextReportAtMillis + mReportIntervalMillis); - report(samples); + report(samples, summaries); } } } - private void report(Map samples) throws IOException { + private void report(Map samples, + Map summaries) + throws IOException + { int size = samples.size(); if (size == 0) { return; } - Counter[] top = new Counter[size]; - samples.values().toArray(top); - Arrays.sort(top); + SummaryCounter[] allSummaries = new SummaryCounter[summaries.size()]; + summaries.values().toArray(allSummaries); + Arrays.sort(allSummaries); - double sum = 0; - for (Counter c : top) { - sum += c.mValue; + double summarySum = 0; + for (SummaryCounter sc : allSummaries) { + summarySum += sc.mValue; + } + + // Clear for next report. + summaries.clear(); + + TraceCounter[] topTraces = new TraceCounter[size]; + samples.values().toArray(topTraces); + Arrays.sort(topTraces); + + double traceSum = 0; + for (TraceCounter tc : topTraces) { + traceSum += tc.mValue; } int limit = Math.min(mTopCount, size); - StringBuilder b = new StringBuilder(limit * 80); + StringBuilder b = new StringBuilder((allSummaries.length + limit) * 80); b.append(Instant.now()).append(' ').append(getClass().getName()).append('\n'); + b.append("--- all profiled packages --- \n"); + + for (SummaryCounter sc : allSummaries) { + String percentStr = String.format("%1$7.3f%%", 100.0 * (sc.mValue / summarySum)); + b.append(percentStr).append(' ').append(sc.mName).append('\n'); + } + + b.append("--- top profiled methods --- \n"); + for (int i=0; i samples) throws IOException b.append('\n'); // Reset for next report. - c.mValue = 0; + tc.mValue = 0; } report(b.toString()); @@ -305,6 +354,10 @@ private StackTraceElement[] examine(ThreadInfo info) { if (elem.getMethodName().startsWith("notify")) { return null; } + // Sometimes the thread state is runnable for this method. Filter it out. + if (elem.getMethodName().equals("wait")) { + return null; + } break; case "java.lang.ref.Reference": @@ -312,10 +365,12 @@ private StackTraceElement[] examine(ThreadInfo info) { return null; case "java.lang.Thread": + /* Track yield, since it's used by the ChunkMap lock. // Reject threads which appeared as doing work only because they yielded. if (elem.getMethodName().equals("yield")) { return null; } + */ // Sometimes the thread state is runnable for this method. Filter it out. if (elem.getMethodName().equals("sleep")) { return null; @@ -401,13 +456,8 @@ private void shutdown() { } private static class Counter implements Comparable { - final StackTraceElement mElem; long mValue; - Counter(StackTraceElement elem) { - mElem = elem; - } - @Override public int compareTo(Counter other) { // Descending order. @@ -415,8 +465,25 @@ public int compareTo(Counter other) { } } + private static class TraceCounter extends Counter { + final StackTraceElement mElem; + + TraceCounter(StackTraceElement elem) { + mElem = elem; + } + } + + private static class SummaryCounter extends Counter { + final String mName; + + SummaryCounter(String name) { + mName = name; + } + } + private class Sampler extends Thread { - private final Map mSamples; + private final Map mSamples; + private final Map mSummaries; volatile boolean mShouldStop; @@ -431,6 +498,7 @@ private class Sampler extends Thread { } mSamples = new HashMap<>(); + mSummaries = new HashMap<>(); } @Override @@ -464,15 +532,15 @@ public void run() { ThreadInfo[] infos; if (dumpMethod == null) { // Use the slow version. - // lockMonitors=false, lockedSynchronizers=false + // lockedMonitors=false, lockedSynchronizers=false infos = tb.dumpAllThreads(false, false); } else { // Use the fast version. - // lockMonitors=false, lockedSynchronizers=false, maxDepth=1 + // lockedMonitors=false, lockedSynchronizers=false, maxDepth=1 infos = (ThreadInfo[]) dumpMethod.invoke(tb, false, false, 1); } - analyze(mSamples, infos); + analyze(mSamples, mSummaries, infos); } catch (InterruptedIOException e) { // Probably should stop. } catch (Throwable e) { diff --git a/version.sbt b/version.sbt index 3f0ac08901..8e8a39d958 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.8.2" +version in ThisBuild := "0.8.3"