From 67995e6d4d20b5c8ea277f291646364f506a1daa Mon Sep 17 00:00:00 2001 From: peng Date: Thu, 19 Oct 2023 18:01:39 -0400 Subject: [PATCH] fix deprecation Caching impl is simplified CachingUtils renamed to Caching cleanup StateLock --- .../spookystuff/SpookyViewsConst.scala | 2 +- .../caching/ExploreRunnerCache.scala | 12 +-- .../caching/InMemoryDocCache.scala | 5 +- .../spookystuff/caching/Memoize.scala | 4 +- .../spookystuff/conf/DriverFactory.scala | 2 +- .../spookystuff/conf/ParametricPoly1.scala | 4 +- .../tribbloids/spookystuff/doc/CSVBlock.scala | 2 +- .../com/tribbloids/spookystuff/doc/Doc.scala | 7 +- .../spookystuff/dsl/ExploreAlgorithm.scala | 3 +- .../spookystuff/execution/ExploreRunner.scala | 2 +- .../spookystuff/python/ref/PyRef.scala | 2 +- .../utils/io/URLConnectionResolver.scala | 2 +- .../rdd/spookystuff/UncleanTaskContext.scala | 6 +- .../testutils/SpookyEnvFixture.scala | 2 +- .../integration/SnapshotRunner.scala | 4 +- .../fetch/FetchInteractionsIT.scala | 4 +- .../integration/fetch/FetchVisitIT.scala | 4 +- .../fetch/FetchWgetAndSaveIT.scala | 4 +- .../tribbloids/spookystuff/graph/Layout.scala | 2 +- .../spookystuff/graph/Visualisation.scala | 2 +- .../xml/ElementToArrayDeserializer.scala | 2 +- .../relay/xml/XMLWeakDeserializer.scala | 6 +- .../utils/BufferedShuffleIteratorV1.scala | 4 +- .../spookystuff/utils/Caching.scala | 102 ++++++++++++++++++ .../spookystuff/utils/CachingUtils.scala | 83 -------------- .../spookystuff/utils/CircularDeque.scala | 2 +- .../spookystuff/utils/io/LocalResolver.scala | 2 +- .../spookystuff/utils/io/lock/Lock.scala | 6 +- .../utils/lifespan/Cleanable.scala | 10 +- .../spookystuff/utils/lifespan/LeafType.scala | 2 +- .../ml/dsl/utils/refl/FromClassMixin.scala | 5 +- ...ingUtilsSuite.scala => CachingSuite.scala} | 13 ++- .../spookystuff/testutils/TestHelper.scala | 3 +- .../planning/VRPOptimizers/JSpritRunner.scala | 2 +- .../spookystuff/uav/sim/APMQuadFixture.scala | 2 +- .../unused/ExternalAppendOnlyArray.scala | 4 +- .../spookystuff/IncrementallyCachedRDD.scala | 8 +- .../unused/ExternalAppendOnlyArraySuite.scala | 2 +- .../web/actions/WebInteraction.scala | 2 +- .../web/conf/WebDriverFactory.scala | 2 +- 40 files changed, 176 insertions(+), 161 deletions(-) create mode 100644 parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/Caching.scala delete mode 100644 parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CachingUtils.scala rename parent/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/{CachingUtilsSuite.scala => CachingSuite.scala} (84%) diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/SpookyViewsConst.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/SpookyViewsConst.scala index 012cbe451..71273ff1d 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/SpookyViewsConst.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/SpookyViewsConst.scala @@ -1,6 +1,6 @@ package com.tribbloids.spookystuff -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap object SpookyViewsConst { diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/ExploreRunnerCache.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/ExploreRunnerCache.scala index 1ab626950..643be5fec 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/ExploreRunnerCache.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/ExploreRunnerCache.scala @@ -3,7 +3,8 @@ package com.tribbloids.spookystuff.caching import com.tribbloids.spookystuff.execution.ExplorePlan.ExeID import com.tribbloids.spookystuff.execution.{ExploreRunner, NodeKey} import com.tribbloids.spookystuff.row.{DataRow, RowReducer} -import com.tribbloids.spookystuff.utils.CachingUtils.{ConcurrentCache, ConcurrentMap, ConcurrentSet} +import com.tribbloids.spookystuff.utils.Caching +import com.tribbloids.spookystuff.utils.Caching.{ConcurrentCache, ConcurrentMap, ConcurrentSet} /** * Singleton, always in the JVM and shared by all executors on the same machine This is a makeshift implementation, @@ -13,10 +14,10 @@ object ExploreRunnerCache { // (NodeKey, ExecutionID) -> Squashed Rows // exeID is used to segment Squashed Rows from different jobs - val committedVisited: ConcurrentCache[(NodeKey, ExeID), Iterable[DataRow]] = ConcurrentCache() + val committedVisited: ConcurrentCache[(NodeKey, ExeID), Iterable[DataRow]] = Caching.ConcurrentCache() val onGoings: ConcurrentMap[ExeID, ConcurrentSet[ExploreRunner]] = - ConcurrentMap() // executionID -> running ExploreStateView + Caching.ConcurrentMap() // executionID -> running ExploreStateView def getOnGoingRunners(exeID: ExeID): ConcurrentSet[ExploreRunner] = { // onGoings.synchronized{ @@ -58,7 +59,7 @@ object ExploreRunnerCache { } def register(v: ExploreRunner, exeID: ExeID): Unit = { - getOnGoingRunners(exeID) += v -> {} + getOnGoingRunners(exeID) += v } def deregister(v: ExploreRunner, exeID: ExeID): Unit = { @@ -68,7 +69,6 @@ object ExploreRunnerCache { def get(key: (NodeKey, ExeID)): Set[Iterable[DataRow]] = { val onGoing = this .getOnGoingRunners(key._2) - .keySet val onGoingVisitedSet = onGoing .flatMap { v => @@ -81,7 +81,7 @@ object ExploreRunnerCache { def getAll(exeID: ExeID): Map[NodeKey, Iterable[DataRow]] = { val onGoing: Map[NodeKey, Iterable[DataRow]] = this .getOnGoingRunners(exeID) - .map(_._1.visited.toMap) + .map(_.visited.toMap) .reduceOption { (v1, v2) => v1 ++ v2 } diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/InMemoryDocCache.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/InMemoryDocCache.scala index 617999b50..c54fcb4ef 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/InMemoryDocCache.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/InMemoryDocCache.scala @@ -3,14 +3,15 @@ package com.tribbloids.spookystuff.caching import com.tribbloids.spookystuff.SpookyContext import com.tribbloids.spookystuff.actions.{Trace, TraceView} import com.tribbloids.spookystuff.doc.DocOption -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentCache +import com.tribbloids.spookystuff.utils.Caching +import com.tribbloids.spookystuff.utils.Caching.ConcurrentCache /** * Backed by a WeakHashMap, the web cache temporarily store all trace -> Array[Page] until next GC. Always enabled */ object InMemoryDocCache extends AbstractDocCache { - val internal: ConcurrentCache[Trace, Seq[DocOption]] = ConcurrentCache() + val internal: ConcurrentCache[Trace, Seq[DocOption]] = Caching.ConcurrentCache() def cacheable(v: Seq[DocOption]): Boolean = { v.exists(v => v.cacheLevel.isInstanceOf[DocCacheLevel.InMemory]) diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/Memoize.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/Memoize.scala index 5e5520dbf..e8bc7023e 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/Memoize.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/caching/Memoize.scala @@ -16,14 +16,14 @@ package com.tribbloids.spookystuff.caching -import com.tribbloids.spookystuff.utils.CachingUtils +import com.tribbloids.spookystuff.utils.Caching //TODO: not efficient should be replaced trait Memoize[T, R] extends (T => R) with Serializable { def f(v: T): R - val cache = CachingUtils.ConcurrentCache[T, (R, Long)]() + val cache = Caching.ConcurrentCache[T, (R, Long)]() def get(x: T, condition: ((R, Long)) => Boolean): R = { if (cache.contains(x)) { diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala index 51d1f6284..4b29938f8 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala @@ -16,7 +16,7 @@ limitations under the License. package com.tribbloids.spookystuff.conf import com.tribbloids.spookystuff.session.{DriverLike, DriverStatus, Session} -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap import com.tribbloids.spookystuff.utils.lifespan.Cleanable.{BatchID, Lifespan} import com.tribbloids.spookystuff.SpookyContext import org.apache.spark.TaskContext diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/ParametricPoly1.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/ParametricPoly1.scala index ca844f9b3..e38627fae 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/ParametricPoly1.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/conf/ParametricPoly1.scala @@ -1,6 +1,6 @@ package com.tribbloids.spookystuff.conf -import com.tribbloids.spookystuff.utils.CachingUtils +import com.tribbloids.spookystuff.utils.Caching trait ParametricPoly1 extends GenParametricPoly1 { @@ -11,7 +11,7 @@ object ParametricPoly1 { trait Cached extends ParametricPoly1 { - lazy val cache: CachingUtils.ConcurrentMap[UB, Out[_ <: UB]] = CachingUtils.ConcurrentMap() + lazy val cache: Caching.ConcurrentMap[UB, Out[_ <: UB]] = Caching.ConcurrentMap() def get[T <: UB](k: T): Option[Out[T]] = { cache diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/CSVBlock.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/CSVBlock.scala index b9c0f77eb..76349ec9e 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/CSVBlock.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/CSVBlock.scala @@ -13,7 +13,7 @@ case class CSVBlock( csvFormat: CSVFormat ) extends Unstructured { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val parsed: CSVParser = CSVParser.parse(_text, csvFormat) val parsedList: List[CSVRecord] = parsed.asScala.toList diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/Doc.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/Doc.scala index a2862daed..8981741ea 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/Doc.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/doc/Doc.scala @@ -18,6 +18,7 @@ import org.mozilla.universalchardet.UniversalDetector import java.sql.{Date, Time, Timestamp} import java.util.UUID +import scala.collection.mutable class DocOptionUDT extends ScalaUDT[DocOption] @@ -122,15 +123,15 @@ case class Doc( declaredContentType: Option[String] = None, // cookie: Seq[SerializableCookie] = Nil, override val timeMillis: Long = System.currentTimeMillis(), - saved: scala.collection.mutable.Set[String] = scala.collection.mutable.Set(), // TODO: move out of constructor override val cacheLevel: DocCacheLevel.Value = DocCacheLevel.All, httpStatus: Option[StatusLine] = None, override val metadata: ResourceMetadata = - ResourceMetadata.empty // for customizing parsing TODO: remove, delegate to CSVElement. + ResourceMetadata.empty, // for customizing parsing TODO: remove, delegate to CSVElement. + saved: mutable.Set[String] = mutable.Set() // TODO: move out of constructor ) extends DocOption with EqualBy { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ lazy val _equalBy: Any = (uid, uri, declaredContentType, timeMillis, httpStatus.toString) diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/dsl/ExploreAlgorithm.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/dsl/ExploreAlgorithm.scala index 9403d850b..a1c72b0fb 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/dsl/ExploreAlgorithm.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/dsl/ExploreAlgorithm.scala @@ -4,7 +4,7 @@ import com.tribbloids.spookystuff.caching.ExploreRunnerCache import com.tribbloids.spookystuff.execution.ExplorePlan.Params import com.tribbloids.spookystuff.execution.NodeKey import com.tribbloids.spookystuff.row._ -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap sealed trait ExploreAlgorithm { @@ -110,7 +110,6 @@ object ExploreAlgorithms { override val ordering: RowOrdering = Ordering.by { tuple: (NodeKey, Iterable[DataRow]) => val inProgress = ExploreRunnerCache .getOnGoingRunners(params.executionID) - .keySet .flatMap(_.fetchingInProgressOpt) val result = if (inProgress contains tuple._1) { diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/execution/ExploreRunner.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/execution/ExploreRunner.scala index 182515fae..84acbc829 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/execution/ExploreRunner.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/execution/ExploreRunner.scala @@ -6,7 +6,7 @@ import com.tribbloids.spookystuff.dsl.ExploreAlgorithm import com.tribbloids.spookystuff.execution.ExplorePlan.Open_Visited import com.tribbloids.spookystuff.extractors.Resolved import com.tribbloids.spookystuff.row._ -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap import com.tribbloids.spookystuff.utils.serialization.NOTSerializable import com.tribbloids.spookystuff.{dsl, SpookyContext} diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala index 8a0de00f8..234980af4 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala @@ -4,7 +4,7 @@ import com.tribbloids.spookystuff.SpookyContext import com.tribbloids.spookystuff.conf.Python import com.tribbloids.spookystuff.python.PyConverter import com.tribbloids.spookystuff.session.{PythonDriver, Session} -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap import com.tribbloids.spookystuff.utils.TreeThrowable import com.tribbloids.spookystuff.utils.lifespan.Cleanable import org.apache.spark.ml.dsl.utils._ diff --git a/parent/core/src/main/scala/com/tribbloids/spookystuff/utils/io/URLConnectionResolver.scala b/parent/core/src/main/scala/com/tribbloids/spookystuff/utils/io/URLConnectionResolver.scala index 11b6c87cf..a6f174fb6 100644 --- a/parent/core/src/main/scala/com/tribbloids/spookystuff/utils/io/URLConnectionResolver.scala +++ b/parent/core/src/main/scala/com/tribbloids/spookystuff/utils/io/URLConnectionResolver.scala @@ -29,7 +29,7 @@ case class URLConnectionResolver( override val retry: Retry = Retry.ExponentialBackoff(8, 16000) ) extends URIResolver { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ case class _Execution(pathStr: String) extends Execution { diff --git a/parent/core/src/main/scala/org/apache/spark/rdd/spookystuff/UncleanTaskContext.scala b/parent/core/src/main/scala/org/apache/spark/rdd/spookystuff/UncleanTaskContext.scala index 44dc62b22..5ae8f815e 100644 --- a/parent/core/src/main/scala/org/apache/spark/rdd/spookystuff/UncleanTaskContext.scala +++ b/parent/core/src/main/scala/org/apache/spark/rdd/spookystuff/UncleanTaskContext.scala @@ -1,7 +1,7 @@ package org.apache.spark.rdd.spookystuff import java.util.EventListener -import com.tribbloids.spookystuff.utils.CachingUtils +import com.tribbloids.spookystuff.utils.Caching import org.apache.spark.TaskContext import org.apache.spark.executor.TaskMetrics import org.apache.spark.metrics.source.Source @@ -26,8 +26,8 @@ case class UncleanTaskContext( override def resourcesJMap(): util.Map[String, ResourceInformation] = self.resourcesJMap() - lazy val listeners: CachingUtils.ConcurrentMap[Long, EventListener] = - CachingUtils.ConcurrentMap[Long, EventListener]() + lazy val listeners: Caching.ConcurrentMap[Long, EventListener] = + Caching.ConcurrentMap[Long, EventListener]() override def addTaskCompletionListener(listener: TaskCompletionListener): TaskContext = { listeners += (System.currentTimeMillis() -> listener) diff --git a/parent/core/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/SpookyEnvFixture.scala b/parent/core/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/SpookyEnvFixture.scala index 3a7621f90..1ac0886e7 100644 --- a/parent/core/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/SpookyEnvFixture.scala +++ b/parent/core/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/SpookyEnvFixture.scala @@ -21,7 +21,7 @@ import scala.util.Try object SpookyEnvFixture { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ // def cleanDriverInstances(): Unit = { // CleanMixin.unclean.foreach { diff --git a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/SnapshotRunner.scala b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/SnapshotRunner.scala index 1ac9f03c6..29b109910 100644 --- a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/SnapshotRunner.scala +++ b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/SnapshotRunner.scala @@ -23,7 +23,7 @@ object SnapshotRunner extends SpookyEnvFixture.EnvBase { import com.tribbloids.spookystuff.dsl.DSL._ import com.tribbloids.spookystuff.utils.CommonViews.StringView - val pathEncoding: GenExtractor[FR, String] = S.uri + val pathTemplate: GenExtractor[FR, String] = S.uri .andFn { uri => val base = uri.split(SPLITTER).last CommonConst.USER_TEMP_DIR \\ "test-sites" \\ base @@ -45,7 +45,7 @@ object SnapshotRunner extends SpookyEnvFixture.EnvBase { cooldown = coolDown ) originalVersion - .savePages_!(pathEncoding, overwrite = true) + .savePages_!(pathTemplate, overwrite = true) fd } diff --git a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchInteractionsIT.scala b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchInteractionsIT.scala index cdeb0a972..b3fbc8911 100644 --- a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchInteractionsIT.scala +++ b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchInteractionsIT.scala @@ -56,8 +56,8 @@ class FetchInteractionsIT extends IntegrationFixture { assert(unionRows.length === 2) assert( - unionRows(0).docs.head.copy(timeMillis = 0, raw = null, saved = null) - === unionRows(1).docs.head.copy(timeMillis = 0, raw = null, saved = null) + unionRows(0).docs.head.copy(timeMillis = 0, raw = null) + === unionRows(1).docs.head.copy(timeMillis = 0, raw = null) ) assert(unionRows(0).docs.head.timeMillis === unionRows(1).docs.head.timeMillis) diff --git a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchVisitIT.scala b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchVisitIT.scala index cbc5fbf4a..2fb37d208 100644 --- a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchVisitIT.scala +++ b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchVisitIT.scala @@ -36,8 +36,8 @@ class FetchVisitIT extends IntegrationFixture { assert(unionRows.length === 2) assert( - unionRows(0).docs.head.copy(timeMillis = 0, raw = null, saved = null) - === unionRows(1).docs.head.copy(timeMillis = 0, raw = null, saved = null) + unionRows(0).docs.head.copy(timeMillis = 0, raw = null) + === unionRows(1).docs.head.copy(timeMillis = 0, raw = null) ) assert(unionRows(0).docs.head.timeMillis === unionRows(1).docs.head.timeMillis) diff --git a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchWgetAndSaveIT.scala b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchWgetAndSaveIT.scala index c5fbdf359..c42a8d851 100644 --- a/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchWgetAndSaveIT.scala +++ b/parent/integration/src/test/scala/com/tribbloids/spookystuff/integration/fetch/FetchWgetAndSaveIT.scala @@ -64,8 +64,8 @@ class FetchWgetAndSaveIT extends IntegrationFixture { assert(unionRows.length === 2) assert( - unionRows(0).docs.head.copy(timeMillis = 0, raw = null, saved = null) === - unionRows(1).docs.head.copy(timeMillis = 0, raw = null, saved = null) + unionRows(0).docs.head.copy(timeMillis = 0, raw = null) === + unionRows(1).docs.head.copy(timeMillis = 0, raw = null) ) assert(unionRows(0).docs.head.timeMillis === unionRows(1).docs.head.timeMillis) diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Layout.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Layout.scala index d8dfa2411..a07f99959 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Layout.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Layout.scala @@ -3,7 +3,7 @@ package com.tribbloids.spookystuff.graph import com.tribbloids.spookystuff.graph.IDAlgebra.Rotator import com.tribbloids.spookystuff.graph.Layout.Facet import com.tribbloids.spookystuff.graph.Module.{Heads, Tails} -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap import scala.language.implicitConversions diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Visualisation.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Visualisation.scala index 005c3137c..a9f6f5725 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Visualisation.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/graph/Visualisation.scala @@ -71,7 +71,7 @@ case class Visualisation[D <: Domain]( def compileASCII( endWith: Seq[Element[D]] ): Graph[ElementView[D]#WFormat] = { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ // val buffer = mutable.LinkedHashSet.empty[ElementView[D]#WFormat] val buffer = { diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/ElementToArrayDeserializer.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/ElementToArrayDeserializer.scala index e2f2e6e98..ab001938d 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/ElementToArrayDeserializer.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/ElementToArrayDeserializer.scala @@ -27,7 +27,7 @@ object ElementToArrayDeserializer extends XMLWeakDeserializer[Any] { extractInner(ti, jv, format).toSet case (ti @ TypeInfo(this.arrayListClass, _), jv) if !jv.isInstanceOf[JArray] => - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ new java.util.ArrayList[Any](extractInner(ti, jv, format).toList.asJava) diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/XMLWeakDeserializer.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/XMLWeakDeserializer.scala index ffc1863ce..ce32485d3 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/XMLWeakDeserializer.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/relay/xml/XMLWeakDeserializer.scala @@ -1,7 +1,7 @@ package com.tribbloids.spookystuff.relay.xml -import com.tribbloids.spookystuff.utils.CachingUtils -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentCache +import com.tribbloids.spookystuff.utils.Caching +import com.tribbloids.spookystuff.utils.Caching.ConcurrentCache import org.apache.spark.ml.dsl.utils.Verbose import com.tribbloids.spookystuff.relay.MessageAPI import org.json4s._ @@ -21,7 +21,7 @@ object XMLWeakDeserializer { custom: Seq[String] = Nil ) - val cached: ConcurrentCache[Long, ParsingException] = CachingUtils.ConcurrentCache[Long, ParsingException]() + val cached: ConcurrentCache[Long, ParsingException] = Caching.ConcurrentCache[Long, ParsingException]() trait ExceptionLike extends Throwable with Verbose { diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/BufferedShuffleIteratorV1.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/BufferedShuffleIteratorV1.scala index e9f9fad3d..c4bf5f8f7 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/BufferedShuffleIteratorV1.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/BufferedShuffleIteratorV1.scala @@ -1,6 +1,6 @@ package com.tribbloids.spookystuff.utils -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap import scala.collection.mutable.ArrayBuffer import scala.util.Random @@ -10,7 +10,7 @@ object BufferedShuffleIteratorV1 { /** * (instanceID -> partitionID) -> iterator */ - val seeds: ConcurrentMap[(String, Int), Long] = CachingUtils.ConcurrentMap[(String, Int), Long]() + val seeds: ConcurrentMap[(String, Int), Long] = Caching.ConcurrentMap[(String, Int), Long]() } case class BufferedShuffleIteratorV1[T]( diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/Caching.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/Caching.scala new file mode 100644 index 000000000..faad79e4c --- /dev/null +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/Caching.scala @@ -0,0 +1,102 @@ +package com.tribbloids.spookystuff.utils + +import org.sparkproject.guava.cache.CacheBuilder + +import scala.collection.mutable +import scala.language.implicitConversions + +trait Caching { + + import scala.jdk.CollectionConverters._ + + trait CacheTag + + def guavaBuilder: CacheBuilder[AnyRef, AnyRef] + + /** + * A cache designed for multithreaded usage in cases where values (not keys) contained in this map will not + * necessarily be cleanly removed. This map uses weak references for contained values (not keys) in order to ensure + * that the existance of a reference to that object in this cache will not prevent the garbage collection of the + * contained object. + * + *

Warning: DO NOT use .weakKeys()!. Otherwise, the resulting map will use identity ({@code ==}) comparison + * to determine equality of keys, which is a technical violation of the {@link Map} specification, and may not be + * what you expect. + * + * @throws IllegalStateException + * if the key strength was already set + * @see + * WeakReference + */ + type ConcurrentCache[K, V] = scala.collection.concurrent.Map[K, V] with CacheTag + + def ConcurrentCache[K, V](): ConcurrentCache[K, V] = { + + // TODO: switching to https://github.com/blemale/scaffeine if faster? + val base = guavaBuilder + .asInstanceOf[CacheBuilder[K, V]] + .build[K, V]() + .asMap() + + val asScala = base.asScala + + asScala.asInstanceOf[ConcurrentCache[K, V]] + } +} + +object Caching { + + import scala.jdk.CollectionConverters._ + + trait ConcurrentTag + + def javaConcurrentMap[K, V]() = new java.util.concurrent.ConcurrentHashMap[K, V]() + + type ConcurrentMap[K, V] = scala.collection.concurrent.Map[K, V] with ConcurrentTag + def ConcurrentMap[K, V](): ConcurrentMap[K, V] = { + javaConcurrentMap[K, V]().asScala.asInstanceOf[ConcurrentMap[K, V]] + } + + type ConcurrentSet[V] = mutable.Set[V] with ConcurrentTag + def ConcurrentSet[V](): ConcurrentSet[V] = { + + val proto: mutable.Set[V] = javaConcurrentMap[V, Unit]().keySet(()).asScala + + // val proto: mutable.Set[V] = javaConcurrentMap[V, Unit]().keySet().asScala// don't use, Java doesn't have a default value for Unit + + proto.asInstanceOf[ConcurrentSet[V]] + } + + object Strong extends Caching { + + override lazy val guavaBuilder: CacheBuilder[AnyRef, AnyRef] = { + CacheBuilder + .newBuilder() + .concurrencyLevel(CommonUtils.numLocalCores) + } + } + + object Weak extends Caching { + + override lazy val guavaBuilder: CacheBuilder[AnyRef, AnyRef] = { + CacheBuilder + .newBuilder() + .concurrencyLevel(CommonUtils.numLocalCores) + .weakValues() // This ensures that being present in this map will not prevent garbage collection/finalization + } + } + + object Soft extends Caching { + + override lazy val guavaBuilder: CacheBuilder[AnyRef, AnyRef] = { + CacheBuilder + .newBuilder() + .concurrencyLevel(CommonUtils.numLocalCores) + .softValues() // values are only GC'd when closing to max heap + } + } + + type ConcurrentCache[K, V] = Soft.ConcurrentCache[K, V] + implicit def defaultImpl(v: this.type): Soft.type = Soft + +} diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CachingUtils.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CachingUtils.scala deleted file mode 100644 index 3f4b55fbd..000000000 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CachingUtils.scala +++ /dev/null @@ -1,83 +0,0 @@ -package com.tribbloids.spookystuff.utils - -import org.sparkproject.guava.cache.CacheBuilder - -import scala.collection.mutable - -object CachingUtils { - - import scala.collection.JavaConverters._ - - /** - * A cache designed for multithreaded usage in cases where values (not keys) contained in this map will not - * necessarily be cleanly removed. This map uses weak references for contained values (not keys) in order to ensure - * that the existance of a reference to that object in this cache will not prevent the garbage collection of the - * contained object. - * - *

Warning: DO NOT use .weakKeys()!. Otherwise, the resulting map will use identity ({@code ==}) comparison - * to determine equality of keys, which is a technical violation of the {@link Map} specification, and may not be - * what you expect. - * - * @throws IllegalStateException - * if the key strength was already set - * @see - * WeakReference - */ - type ConcurrentCache[K, V] = scala.collection.concurrent.Map[K, V] - def ConcurrentCache[K, V](): ConcurrentCache[K, V] = { - - // TODO: switching to https://github.com/blemale/scaffeine if faster? - val base = CacheBuilder - .newBuilder() - .concurrencyLevel(CommonUtils.numLocalCores) - .weakValues() // This ensures that being present in this map will not prevent garbage collection/finalization - .build[Object, Object]() - .asMap() - - val asScala = base.asScala - - asScala.asInstanceOf[ConcurrentCache[K, V]] - } - - type ConcurrentSoftCache[K, V] = scala.collection.concurrent.Map[K, V] - def ConcurrentSoftCache[K, V](): ConcurrentCache[K, V] = { - - val base = CacheBuilder - .newBuilder() - .concurrencyLevel(CommonUtils.numLocalCores) - .softValues() - .build[Object, Object]() - .asMap() - - val asScala = base.asScala - - asScala.asInstanceOf[ConcurrentCache[K, V]] - } - - type ConcurrentMap[K, V] = scala.collection.concurrent.Map[K, V] - def ConcurrentMap[K, V](): ConcurrentMap[K, V] = { - new java.util.concurrent.ConcurrentHashMap[K, V]().asScala - } - - type ConcurrentSet[V] = ConcurrentMap[V, Unit] - def ConcurrentSet[V](): ConcurrentSet[V] = { - ConcurrentMap[V, Unit]() - } - - implicit class MapView[K, V](self: mutable.Map[K, V]) { - - def getOrUpdateSync(key: K)(value: => V): V = { - - self.getOrElse( - key, { - self.synchronized { - self.getOrElseUpdate( - key, - value - ) - } - } - ) - } - } -} diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CircularDeque.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CircularDeque.scala index 79aae010c..19ee9f749 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CircularDeque.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CircularDeque.scala @@ -6,7 +6,7 @@ import scala.language.implicitConversions case class CircularDeque[T](size: Int = 10) { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ lazy val delegate: LinkedBlockingDeque[T] = new LinkedBlockingDeque[T](size) diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/LocalResolver.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/LocalResolver.scala index 7a0235c4a..b2d2b62bf 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/LocalResolver.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/LocalResolver.scala @@ -21,7 +21,7 @@ case class LocalResolver( val path: Path = Paths.get(pathStr) - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ // CAUTION: resolving is different on each driver or executors val absolutePath: Path = path.toAbsolutePath diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala index 6b857911b..8a31f0f7f 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala @@ -1,6 +1,6 @@ package com.tribbloids.spookystuff.utils.io.lock -import com.tribbloids.spookystuff.utils.CachingUtils +import com.tribbloids.spookystuff.utils.Caching import com.tribbloids.spookystuff.utils.io.{URIExecution, URIResolver} import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable @@ -90,6 +90,6 @@ object Lock { case class InMemoryLock() {} - lazy val inMemoryLocks: CachingUtils.ConcurrentCache[(Class[_], String), InMemoryLock] = - CachingUtils.ConcurrentCache() + lazy val inMemoryLocks: Caching.ConcurrentCache[(Class[_], String), InMemoryLock] = + Caching.ConcurrentCache() } diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala index 21156da88..ce2cdefcd 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala @@ -1,14 +1,12 @@ package com.tribbloids.spookystuff.utils.lifespan -import com.tribbloids.spookystuff.utils.CachingUtils._ +import com.tribbloids.spookystuff.utils.Caching._ import org.slf4j.{Logger, LoggerFactory} import scala.reflect.ClassTag object Cleanable { - case class StateLock() - import com.tribbloids.spookystuff.utils.CommonViews._ type Lifespan = LifespanInternal#ForShipping @@ -92,8 +90,6 @@ trait Cleanable extends AutoCloseable { import Cleanable._ - @transient lazy val stateLock: StateLock = StateLock() - /** * taskOrThreadOnCreation is incorrect in withDeadline or threads not created by Spark Override this to correct such * problem @@ -111,7 +107,7 @@ trait Cleanable extends AutoCloseable { // each can only be cleaned once @volatile protected var _isCleaned: Boolean = false - def isCleaned: Boolean = stateLock.synchronized { + def isCleaned: Boolean = this.synchronized { _isCleaned } @@ -151,7 +147,7 @@ trait Cleanable extends AutoCloseable { ) } - lazy val doCleanOnce: Unit = stateLock.synchronized { + lazy val doCleanOnce: Unit = this.synchronized { stacktraceAtCleaning = Some(Thread.currentThread().getStackTrace) try { diff --git a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LeafType.scala b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LeafType.scala index 7ff1051af..7eafb383b 100644 --- a/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LeafType.scala +++ b/parent/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LeafType.scala @@ -1,6 +1,6 @@ package com.tribbloids.spookystuff.utils.lifespan -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Batch abstract class LeafType extends LifespanType { diff --git a/parent/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/refl/FromClassMixin.scala b/parent/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/refl/FromClassMixin.scala index 1c2d502cf..79b8161ec 100644 --- a/parent/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/refl/FromClassMixin.scala +++ b/parent/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/refl/FromClassMixin.scala @@ -1,6 +1,7 @@ package org.apache.spark.ml.dsl.utils.refl -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentCache +import com.tribbloids.spookystuff.utils.Caching +import com.tribbloids.spookystuff.utils.Caching.ConcurrentCache import scala.language.{existentials, implicitConversions} import scala.reflect.ClassTag @@ -60,7 +61,7 @@ abstract class FromClassMixin { protected def createNew[T](v: I[T]): TypeMagnet[T] - protected lazy val cache: ConcurrentCache[I[_], TypeMagnet[_]] = ConcurrentCache[I[_], TypeMagnet[_]]() + protected lazy val cache: ConcurrentCache[I[_], TypeMagnet[_]] = Caching.ConcurrentCache[I[_], TypeMagnet[_]]() final def apply[T]( implicit diff --git a/parent/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/CachingUtilsSuite.scala b/parent/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/CachingSuite.scala similarity index 84% rename from parent/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/CachingUtilsSuite.scala rename to parent/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/CachingSuite.scala index f42314cd5..62480d244 100644 --- a/parent/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/CachingUtilsSuite.scala +++ b/parent/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/CachingSuite.scala @@ -1,15 +1,14 @@ package com.tribbloids.spookystuff.utils import com.tribbloids.spookystuff.testutils.FunSpecx -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentCache import org.scalatest.BeforeAndAfterEach import scala.concurrent.duration.Duration import scala.concurrent.{Await, ExecutionContextExecutor, Future} -class CachingUtilsSuite extends FunSpecx with BeforeAndAfterEach { +class CachingSuite extends FunSpecx with BeforeAndAfterEach { - import CachingUtilsSuite._ + import CachingSuite._ implicit def global: ExecutionContextExecutor = scala.concurrent.ExecutionContext.global override def beforeEach(): Unit = { @@ -52,12 +51,12 @@ class CachingUtilsSuite extends FunSpecx with BeforeAndAfterEach { } } - describe("ConcurrentCache") { + describe("Weak ConcurrentCache") { describe("should remove value on garbage collection") { it("if the value is de-referenced") { - val cache = ConcurrentCache[String, CacheTestData]() + val cache = Caching.Weak.ConcurrentCache[String, CacheTestData]() var myVal = CacheTestData("myString") @@ -72,7 +71,7 @@ class CachingUtilsSuite extends FunSpecx with BeforeAndAfterEach { it("if the value is not in scope") { - val cache = CachingUtils.ConcurrentCache[String, CacheTestData]() + val cache = Caching.Weak.ConcurrentCache[String, CacheTestData]() val f: Future[Unit] = Future { @@ -91,7 +90,7 @@ class CachingUtilsSuite extends FunSpecx with BeforeAndAfterEach { } } -object CachingUtilsSuite { +object CachingSuite { @volatile var count: Int = 0 diff --git a/parent/mldsl/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala b/parent/mldsl/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala index a61675ef3..29908e28b 100644 --- a/parent/mldsl/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala +++ b/parent/mldsl/src/testFixtures/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala @@ -3,9 +3,8 @@ package com.tribbloids.spookystuff.testutils import com.tribbloids.spookystuff.utils.classpath.ClasspathResolver import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable -import com.tribbloids.spookystuff.utils.{CommonConst, CommonUtils, ConfUtils} +import com.tribbloids.spookystuff.utils.{CommonConst, CommonUtils} import org.apache.hadoop.fs.FileUtil -import org.apache.spark.launcher.InProcessLauncher import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.{SQLContext, SparkSession} import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException} diff --git a/parent/uav/src/main/scala/com/tribbloids/spookystuff/uav/planning/VRPOptimizers/JSpritRunner.scala b/parent/uav/src/main/scala/com/tribbloids/spookystuff/uav/planning/VRPOptimizers/JSpritRunner.scala index 2809cc51f..ff6ad0f08 100644 --- a/parent/uav/src/main/scala/com/tribbloids/spookystuff/uav/planning/VRPOptimizers/JSpritRunner.scala +++ b/parent/uav/src/main/scala/com/tribbloids/spookystuff/uav/planning/VRPOptimizers/JSpritRunner.scala @@ -300,7 +300,7 @@ case class JSpritRunner( lazy val getUAV2TraceMap: Map[LinkStatus, Seq[TraceView]] = { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val routes = solve.getRoutes.asScala.toList val status_KVs: Seq[(LinkStatus, List[TraceView])] = routes.map { route => diff --git a/parent/uav/src/test/scala/com/tribbloids/spookystuff/uav/sim/APMQuadFixture.scala b/parent/uav/src/test/scala/com/tribbloids/spookystuff/uav/sim/APMQuadFixture.scala index 8714cda6c..5fb7460b5 100644 --- a/parent/uav/src/test/scala/com/tribbloids/spookystuff/uav/sim/APMQuadFixture.scala +++ b/parent/uav/src/test/scala/com/tribbloids/spookystuff/uav/sim/APMQuadFixture.scala @@ -30,7 +30,7 @@ class TestAPMQuad extends APMQuadFixture { assert(fleet.size == fleet.distinct.size) assert(fleet.size == iNums.size) - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val processes = JProcesses.getProcessList().asScala diff --git a/parent/unused/src/main/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArray.scala b/parent/unused/src/main/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArray.scala index b93369e4f..340fc454b 100644 --- a/parent/unused/src/main/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArray.scala +++ b/parent/unused/src/main/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArray.scala @@ -3,7 +3,7 @@ package com.tribbloids.spookystuff.unused import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable import com.tribbloids.spookystuff.utils.serialization.NOTSerializable -import com.tribbloids.spookystuff.utils.{CachingUtils, CommonConst, CommonUtils, ThreadLocal} +import com.tribbloids.spookystuff.utils.{Caching, CommonConst, CommonUtils, ThreadLocal} import org.apache.spark.rdd.spookystuff.{ConsumedIterator, FallbackIterator, FastForwardingIterator} import org.apache.spark.serializer import org.apache.spark.serializer.SerializerInstance @@ -340,7 +340,7 @@ object ExternalAppendOnlyArray { val processID: String = UUID.randomUUID().toString - val existing: CachingUtils.ConcurrentCache[String, ExternalAppendOnlyArray[_]] = CachingUtils.ConcurrentCache() + val existing: Caching.ConcurrentCache[String, ExternalAppendOnlyArray[_]] = Caching.ConcurrentCache() def apply[T: ClassTag]( id: String, diff --git a/parent/unused/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala b/parent/unused/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala index 7ec9134b9..e05393529 100644 --- a/parent/unused/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala +++ b/parent/unused/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala @@ -1,11 +1,11 @@ package org.apache.spark.rdd.spookystuff import com.tribbloids.spookystuff.unused.ExternalAppendOnlyArray -import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap +import com.tribbloids.spookystuff.utils.Caching.ConcurrentMap import com.tribbloids.spookystuff.utils.accumulator.MapAccumulator import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan import com.tribbloids.spookystuff.utils.lifespan.{Cleanable, LocalCleanable} -import com.tribbloids.spookystuff.utils.{CachingUtils, EqualBy, Retry, SCFunctions} +import com.tribbloids.spookystuff.utils.{Caching, EqualBy, Retry, SCFunctions} import org.apache.spark import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging @@ -394,8 +394,8 @@ object IncrementallyCachedRDD { rddID: Int ) { - @transient lazy val existing: CachingUtils.ConcurrentMap[Int, T] = - CachingUtils.ConcurrentMap() + @transient lazy val existing: Caching.ConcurrentMap[Int, T] = + Caching.ConcurrentMap() def getOrCreate(id: Int)(create: => T): T = { diff --git a/parent/unused/src/test/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArraySuite.scala b/parent/unused/src/test/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArraySuite.scala index a59d89497..5e2dc142b 100644 --- a/parent/unused/src/test/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArraySuite.scala +++ b/parent/unused/src/test/scala/com/tribbloids/spookystuff/unused/ExternalAppendOnlyArraySuite.scala @@ -19,7 +19,7 @@ abstract class ExternalAppendOnlyArraySuite( import com.tribbloids.spookystuff.testutils.TestHelper._ - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val e_resultSize: Int = parallelism * numTestRuns * taskSize val e_computeTimes: Int = parallelism * taskSize // * TestHelper.numWorkers diff --git a/parent/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala b/parent/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala index 070149b7c..ed6fdfdd9 100644 --- a/parent/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala +++ b/parent/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala @@ -187,7 +187,7 @@ case class ClickNext( val elements = this.getElements(selector, session) - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ elements.asScala.foreach { element => { diff --git a/parent/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala b/parent/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala index f0bbd8195..641f1ad0f 100644 --- a/parent/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala +++ b/parent/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala @@ -112,7 +112,7 @@ object WebDriverFactory { asSeleniumProxy(v) } - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ var builder = PhantomJS.defaultBuilder .usingAnyFreePort()