From 53a3db8e92566d585119cb75823f0692709e46d0 Mon Sep 17 00:00:00 2001 From: Brian O'Neill Date: Wed, 20 Feb 2019 13:21:59 -0800 Subject: [PATCH 01/20] feat(bootstrap): Reduce log spew when running the test suite. --- .../filodb/akkabootstrapper/ClusterSeedDiscovery.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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) From 77bcd8069f8937f7d7d9cdfcc950f5bd0e8c7b56 Mon Sep 17 00:00:00 2001 From: Brian O'Neill Date: Wed, 20 Feb 2019 13:24:42 -0800 Subject: [PATCH 02/20] bug(coordinator): Add a sleep before allocation of ActorSystem when running the test suite. This improves the likelihood that port 2552 can be bound again. --- .../filodb.coordinator/ActorSpecConfig.scala | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) 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))) } From 29dd630f0f5263da9117ae1549c7644c076daa83 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 20 Feb 2019 17:57:42 -0800 Subject: [PATCH 03/20] feat(memory,core): Replace OffheapLFSortedIDMap with ChunkMap (#213) * feat(memory,core): Replace OffheapLFSortedIDMap with ChunkMap, which isn't designed to be lock-free, and is simpler. TimeSeriesPartition extends this new class instead of relying on MapHolder, which was confusing and required more special state to track shared lock ownership. --- .../memstore/TimeSeriesPartition.scala | 65 +- .../memstore/TimeSeriesShard.scala | 12 +- .../scala/filodb.core/query/RangeVector.scala | 6 +- .../memstore/TimeSeriesPartitionSpec.scala | 8 +- .../main/scala/filodb.memory/MemFactory.scala | 2 +- .../scala/filodb.memory/data/ChunkMap.scala | 628 ++++++++++++ .../data/OffheapLFSortedIDMap.scala | 934 ------------------ .../filodb.memory/data/ChunkMapTest.scala | 710 +++++++++++++ .../data/OffheapLFSortedIDMapTest.scala | 641 ------------ .../query/exec/AggrOverRangeVectors.scala | 6 +- 10 files changed, 1384 insertions(+), 1628 deletions(-) create mode 100644 memory/src/main/scala/filodb.memory/data/ChunkMap.scala delete mode 100644 memory/src/main/scala/filodb.memory/data/OffheapLFSortedIDMap.scala create mode 100644 memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala delete mode 100644 memory/src/test/scala/filodb.memory/data/OffheapLFSortedIDMapTest.scala diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 921419529f..965de6a189 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,7 +303,7 @@ extends ReadablePartition with MapHolder { def nextInfo: ChunkSetInfo = { if (closed) throw new NoSuchElementException() if (!valueSeen) { - offheapInfoMap.acquireShared(TimeSeriesPartition.this) + chunkmapAcquireShared() valueSeen = true } return info @@ -316,7 +315,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 +330,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 +361,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..4fdf5d07dc 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -25,7 +25,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,9 +268,7 @@ 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) /** @@ -955,11 +952,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) @@ -1068,7 +1064,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/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..900dca7bd2 --- /dev/null +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -0,0 +1,628 @@ +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] + } + + // 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) { + 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) { + // 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 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/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/test/scala/filodb.memory/data/ChunkMapTest.scala b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala new file mode 100644 index 0000000000..d0b14b7871 --- /dev/null +++ b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala @@ -0,0 +1,710 @@ +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 + + startNanos = System.nanoTime() + for (i <- 1 to 2) { + map.chunkmapAcquireShared() + map.chunkmapReleaseShared() + Thread.sleep(100) + } + 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() + } +} 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..2ccf85370f 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._ @@ -531,7 +531,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 +548,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 From 2abb512fd63f76dd5a4c5231258cf7df7b746e53 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 20 Feb 2019 20:25:05 -0800 Subject: [PATCH 04/20] bug(query): fix bug in aggregate over time functions for NaN (#238) Current behavior : sum_over_time, avg_over_time and count_over_time return default values as 0 when there is no sample in the window. min_over_time returns Double.MaxValue and max_over_time returns Double.MinValue when there is no sample in the window New behavior : Aggregate over time functions return NaN when no sample is present --- .../exec/rangefn/AggrOverTimeFunctions.scala | 107 +++++++++--- .../query/exec/rangefn/RangeFunction.scala | 9 +- .../query/exec/WindowIteratorSpec.scala | 152 ++++++++++++++++-- 3 files changed, 228 insertions(+), 40 deletions(-) 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..27877cef6e 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)) + } } /** diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index 40c8e30163..8a7ecf8386 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,26 @@ 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 + 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") { @@ -330,4 +324,138 @@ class WindowIteratorSpec extends RawDataWindowingSpec { .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 + } } From 5cd27766743ca368200783207e17d1ff000770ca Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Fri, 22 Feb 2019 10:34:23 -0800 Subject: [PATCH 05/20] query: fix for resets function (#246) query: fix for resets function --- .../scala/filodb/query/exec/rangefn/RangeFunction.scala | 2 +- .../query/exec/rangefn/RangeInstantFunctions.scala | 4 ++-- .../filodb/query/exec/rangefn/RateFunctionsSpec.scala | 9 +++++---- 3 files changed, 8 insertions(+), 7 deletions(-) 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 27877cef6e..972b620b03 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -240,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/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 } From c6434e748378af28cf8e7e4fd7efe999e43c2bcd Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Fri, 22 Feb 2019 14:19:16 -0800 Subject: [PATCH 06/20] feat(config): Increase query-timeout in filodb (#251) --- core/src/main/resources/filodb-defaults.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 1f5b2fc77c..9b5a6138a7 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -29,7 +29,7 @@ filodb { query { # Timeout for query engine subtree/ExecPlans for requests to sub nodes - ask-timeout = 30 seconds + ask-timeout = 60 seconds stale-sample-after = 5 minutes From 1ec43319182ce7e6f97ecc489ef7cac97d33c2fe Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 22 Feb 2019 14:35:52 -0800 Subject: [PATCH 07/20] feat(memory): Support reentrant shared lock when exclusive lock is requested and waiting. (#248) --- .../scala/filodb.memory/data/ChunkMap.scala | 5 +- .../filodb.memory/data/ChunkMapTest.scala | 57 +++++++++++++++++-- 2 files changed, 56 insertions(+), 6 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 900dca7bd2..523b940d4b 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -58,6 +58,9 @@ object ChunkMap extends StrictLogging { 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 @@ -327,7 +330,7 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) { while (true) { lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) - if (lockState < 0) { + if (lockState < 0 && !hasSharedLock(this)) { // Wait for exclusive lock to be released. Thread.`yield` } else if (UnsafeUtils.unsafe.compareAndSwapInt(this, lockStateOffset, lockState, lockState + 1)) { diff --git a/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala index d0b14b7871..8fa43f08ca 100644 --- a/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala +++ b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala @@ -594,12 +594,20 @@ class ChunkMapTest extends NativeVectorTest with ScalaFutures { acquired shouldBe false - startNanos = System.nanoTime() - for (i <- 1 to 2) { - map.chunkmapAcquireShared() - map.chunkmapReleaseShared() - Thread.sleep(100) + // 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 @@ -707,4 +715,43 @@ class ChunkMapTest extends NativeVectorTest with ScalaFutures { 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() + } } From 5ec306e582ff0d7db54959f78d668b1ffdf6b0e3 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 22 Feb 2019 14:38:52 -0800 Subject: [PATCH 08/20] feat(standalone): Profiler tracks calls to Thread.yield, to indicate how much time is spent contended for ChunkMap locks. (#250) --- standalone/src/main/java/filodb/standalone/SimpleProfiler.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/standalone/src/main/java/filodb/standalone/SimpleProfiler.java b/standalone/src/main/java/filodb/standalone/SimpleProfiler.java index 54d36b0907..be2608c56e 100644 --- a/standalone/src/main/java/filodb/standalone/SimpleProfiler.java +++ b/standalone/src/main/java/filodb/standalone/SimpleProfiler.java @@ -312,10 +312,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; From 3d3325026d4ecac144735fb845605374eba4c162 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Sun, 24 Feb 2019 22:52:51 -0800 Subject: [PATCH 09/20] feat(query): Fix broken tracing (#252) --- query/src/main/scala/filodb/query/exec/ExecPlan.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 92df9a991c..c19b088e3b 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,13 @@ abstract class NonLeafExecPlan extends ExecPlan { queryConfig: QueryConfig) (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = { + val spanFromHelper = Kamon.currentSpan() 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) + 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) + } } } compose(childTasks, queryConfig) From 67a47acffdac0a0ad6f180d8e6f64a4e88be8697 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Sun, 24 Feb 2019 23:59:40 -0800 Subject: [PATCH 10/20] feat(query): Add additional info in tracing (#253) --- .../src/main/scala/filodb.coordinator/QueryActor.scala | 4 +++- .../scala/filodb.coordinator/queryengine2/QueryEngine.scala | 5 ++++- 2 files changed, 7 insertions(+), 2 deletions(-) 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) + } } /** From 0dd11232823e9f087e6b20b562f01c36cc1d6fa7 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 25 Feb 2019 13:09:52 -0800 Subject: [PATCH 11/20] bug(memory, query): Window iteration mustn't release the shared chunk map lock too soon.(#249) --- .../memstore/TimeSeriesPartition.scala | 3 ++ .../filodb.core/store/ChunkSetInfo.scala | 18 +++++++- .../scala/filodb.memory/data/ChunkMap.scala | 4 ++ .../filodb.memory/data/ElementIterator.scala | 15 +++++++ .../query/exec/PeriodicSamplesMapper.scala | 43 +++++++++++++------ .../query/exec/LastSampleFunctionSpec.scala | 12 +++--- .../query/exec/WindowIteratorSpec.scala | 15 +++---- .../rangefn/AggrOverTimeFunctionsSpec.scala | 2 +- 8 files changed, 84 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 965de6a189..198a02be6b 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -308,6 +308,9 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { } return info } + + final def lock(): Unit = chunkmapAcquireShared() + final def unlock(): Unit = chunkmapReleaseShared() } final def earliestTime: Long = { 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/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 523b940d4b..7935bee96b 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -620,6 +620,10 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) { next } + final def lock(): Unit = chunkmapAcquireShared() + + final def unlock(): Unit = chunkmapReleaseShared() + final def getNextElem: NativePointer = nextElem /** 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/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/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 8a7ecf8386..3cb87b0be4 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -198,9 +198,8 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 1100000L, 100000, RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() + .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") { @@ -292,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 } @@ -321,7 +320,7 @@ 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 } @@ -354,7 +353,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() + .asInstanceOf[ChunkedRangeFunction], queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -388,7 +387,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() + .asInstanceOf[ChunkedRangeFunction], queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -421,7 +420,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() + .asInstanceOf[ChunkedRangeFunction], queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -455,7 +454,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig)() + .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], From 40feec11eade6f6467947c076056bae8d7adae45 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 25 Feb 2019 22:11:17 -0800 Subject: [PATCH 12/20] feat(memory): Add basic deadlock detection when shared lock owner tries to upgrade to exclusive. (#255) --- memory/src/main/scala/filodb.memory/data/ChunkMap.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 7935bee96b..c852b89d16 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -246,6 +246,12 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) { 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 From 415250ada1de5233f08f7ae45b538571f4ac5546 Mon Sep 17 00:00:00 2001 From: jackson-paul Date: Tue, 26 Feb 2019 15:38:59 -0800 Subject: [PATCH 13/20] Update version to 0.8.3-integration-SNAPSHOT --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index f8c8a91384..1b0cb1bf45 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.8.2-integration-SNAPSHOT" +version in ThisBuild := "0.8.3-integration-SNAPSHOT" From 69d2544e08f4857dd31f41b1b32b3da84ba25ff1 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 27 Feb 2019 09:54:43 -0800 Subject: [PATCH 14/20] feat(standalone): Add package-level summary to profiling report. (#258) --- .../filodb/standalone/SimpleProfiler.java | 128 +++++++++++++----- 1 file changed, 97 insertions(+), 31 deletions(-) diff --git a/standalone/src/main/java/filodb/standalone/SimpleProfiler.java b/standalone/src/main/java/filodb/standalone/SimpleProfiler.java index be2608c56e..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": @@ -403,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. @@ -417,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; @@ -433,6 +498,7 @@ private class Sampler extends Thread { } mSamples = new HashMap<>(); + mSummaries = new HashMap<>(); } @Override @@ -466,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) { From d23791e15c1b33fb3ae38798c995ec5c3d70f1fa Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 27 Feb 2019 11:33:27 -0800 Subject: [PATCH 15/20] bug(core): Add a lock to protect access to the activelyIngesting bitmap. (#254) --- .../memstore/TimeSeriesShard.scala | 64 +++++++++++++------ 1 file changed, 46 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 4fdf5d07dc..9dbf047b8e 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -468,7 +468,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 } @@ -644,13 +646,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 @@ -666,12 +669,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() @@ -757,11 +765,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()) @@ -824,6 +845,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)) @@ -849,18 +871,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) + } } } @@ -910,13 +935,16 @@ 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") } @@ -1027,7 +1055,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()) From cc9d9109bd2256d8f4bd5dc71324a0f1307290fd Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 27 Feb 2019 14:06:02 -0800 Subject: [PATCH 16/20] bug(core): currentIndexTimeBucket should be volatile ... since it is read from multiple threads --- .../filodb.core/memstore/TimeSeriesShard.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 9dbf047b8e..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 @@ -273,9 +274,12 @@ class TimeSeriesShard(val dataset: Dataset, /** * 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. @@ -603,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. @@ -935,9 +939,7 @@ 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.synchronized { activelyIngesting.set(partId) } - val stamp = partSetLock.writeLock() try { partSet.add(newPart) From ed65e8744108b208769114a4e4626030c910b938 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 27 Feb 2019 14:39:12 -0800 Subject: [PATCH 17/20] bug(query): mapAsync of child nodes causes incorrect order of child results (#261) One of the recent changes to execute child plans in parallel with mapAsync(numProcs) has caused regression in Binary Join. Since LHS and RHS results can arrive different order, results are sometimes incorrect. Fix is to keep track of original order of ExecPlan nodes and use the order when composing the results. --- .../filodb/query/exec/AggrOverRangeVectors.scala | 6 +++--- .../scala/filodb/query/exec/BinaryJoinExec.scala | 10 +++++----- .../scala/filodb/query/exec/DistConcatExec.scala | 6 +++--- .../src/main/scala/filodb/query/exec/ExecPlan.scala | 7 ++++--- .../scala/filodb/query/exec/MetadataExecPlan.scala | 12 ++++++------ .../scala/filodb/query/exec/BinaryJoinExecSpec.scala | 9 +++++---- 6 files changed, 26 insertions(+), 24 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 2ccf85370f..0236719f80 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -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) } 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 c19b088e3b..a2562f8198 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -235,12 +235,13 @@ abstract class NonLeafExecPlan extends ExecPlan { (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = { val spanFromHelper = Kamon.currentSpan() - val childTasks = Observable.fromIterable(children).mapAsync(Runtime.getRuntime.availableProcessors()) { plan => + 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) @@ -257,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/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] } From 76a5a62287e7af2c31c30a3b67f970757c0b4446 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Wed, 27 Feb 2019 14:53:43 -0800 Subject: [PATCH 18/20] fix(memory): Fix bug where binarySearch of DDVs was reading memory that was out of bounds (#260) --- .../scala/filodb.memory/format/vectors/DeltaDeltaVector.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 = { From 4f47698d3dc026497456894fdd7c96f00ac557ca Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Wed, 27 Feb 2019 15:38:43 -0800 Subject: [PATCH 19/20] misc: fix for On Demand Chunk page-in span (#256) misc: fix for On Demand Chunk page-in span --- .../memstore/OnDemandPagingShard.scala | 65 +++++++++++-------- 1 file changed, 37 insertions(+), 28 deletions(-) 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. From 3f331b9bfaddce62e1fb9684f97390411ead6a07 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Fri, 15 Mar 2019 15:30:29 -0700 Subject: [PATCH 20/20] bug(query): fix bug in Count to add input value in reduce only when it is not NaN --- .../filodb/query/exec/AggrOverRangeVectors.scala | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 0236719f80..010261b522 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -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)