Skip to content

Commit

Permalink
Merge pull request #30 from cmu-pasta/hashcode-remap
Browse files Browse the repository at this point in the history
Allow Fray to control Object.hashCode.
  • Loading branch information
aoli-al authored Aug 18, 2024
2 parents 4e607d2 + 2f54bdf commit 5c47600
Show file tree
Hide file tree
Showing 23 changed files with 384 additions and 73 deletions.
2 changes: 1 addition & 1 deletion build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ plugins {

allprojects {
group = "org.pastalab.fray"
version = "0.1.3"
version = "0.1.4-SNAPSHOT"
}

repositories {
Expand Down
6 changes: 6 additions & 0 deletions core/build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ tasks.withType<JavaExec> {
jvmArgs("--add-opens", "java.base/java.io=ALL-UNNAMED")
jvmArgs("--add-opens", "java.base/sun.nio.ch=ALL-UNNAMED")
jvmArgs("--add-opens", "java.base/java.lang.reflect=ALL-UNNAMED")
jvmArgs("-Dfray.recordSchedule=true")
doFirst {
// Printing the full command
println("Executing command: ${executable} ${jvmArgs!!.joinToString(" ")} -cp ${classpath.asPath} ${mainClass.get()} ${args!!.joinToString(" ")}")
Expand Down Expand Up @@ -101,3 +102,8 @@ tasks.create("genRunner") {
file.setExecutable(true)
}
}

tasks.register<Copy>("copyDependencies") {
from(configurations.runtimeClasspath)
into("${layout.buildDirectory.get().asFile}/dependency")
}
16 changes: 15 additions & 1 deletion core/src/main/kotlin/org/pastalab/fray/core/RunContext.kt
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import org.pastalab.fray.core.concurrency.HelperThread
import org.pastalab.fray.core.concurrency.SynchronizationManager
import org.pastalab.fray.core.concurrency.locks.CountDownLatchManager
import org.pastalab.fray.core.concurrency.locks.LockManager
import org.pastalab.fray.core.concurrency.locks.ReferencedContextManager
import org.pastalab.fray.core.concurrency.locks.SemaphoreManager
import org.pastalab.fray.core.concurrency.operations.*
import org.pastalab.fray.instrumentation.base.memory.VolatileManager
Expand All @@ -33,6 +34,7 @@ class RunContext(val config: Configuration) {
var nanoTime = System.nanoTime()
val terminatingThread = mutableSetOf<Int>()
val logger = config.loggerContext.getLogger(RunContext::class.java)
val hashCodeMapper = ReferencedContextManager<Int>({ config.randomnessProvider.nextInt() })
private val lockManager = LockManager()
private val semaphoreManager = SemaphoreManager()
private val volatileManager = VolatileManager(true)
Expand Down Expand Up @@ -141,6 +143,7 @@ class RunContext(val config: Configuration) {
// We need to submit a dummy task to trigger the executor
// thread creation
executor.submit {}
config.scheduleObservers.forEach { it.onExecutionStart() }
step = 0
bugFound = null
mainExiting = false
Expand All @@ -156,6 +159,8 @@ class RunContext(val config: Configuration) {
assert(syncManager.synchronizationPoints.isEmpty())
lockManager.done()
registeredThreads.clear()
config.scheduleObservers.forEach { it.onExecutionDone() }
hashCodeMapper.done()
}

fun shutDown() {
Expand Down Expand Up @@ -837,7 +842,7 @@ class RunContext(val config: Configuration) {
}

val nextThread = config.scheduler.scheduleNextOperation(enabledOperations)
val index = enabledOperations.indexOf(nextThread)
config.scheduleObservers.forEach { it.onNewSchedule(enabledOperations, nextThread) }
currentThreadId = nextThread.thread.id
nextThread.state = ThreadState.Running
unblockThread(currentThread, nextThread)
Expand Down Expand Up @@ -868,6 +873,15 @@ class RunContext(val config: Configuration) {
}
}

fun hashCode(obj: Any): Int {
val hashCodeMethod = obj.javaClass.getMethod("hashCode")
return if (hashCodeMethod.declaringClass == Object::class.java) {
hashCodeMapper.getContext(obj)
} else {
obj.hashCode()
}
}

fun nanoTime(): Long {
nanoTime += 1000L
return nanoTime
Expand Down
15 changes: 5 additions & 10 deletions core/src/main/kotlin/org/pastalab/fray/core/RuntimeDelegate.kt
Original file line number Diff line number Diff line change
Expand Up @@ -742,15 +742,10 @@ class RuntimeDelegate(val context: RunContext) : org.pastalab.fray.runtime.Deleg
return context.nanoTime()
}

override fun onThreadHashCode(t: Any): Int {
if (t is Thread) {
val context = context.registeredThreads[t.id]
if (context != null) {
return 0
} else {
return t.hashCode()
}
}
return t.hashCode()
override fun onObjectHashCode(t: Any): Int {
if (checkEntered()) return t.hashCode()
val hashCode = context.hashCode(t)
entered.set(false)
return hashCode
}
}
6 changes: 6 additions & 0 deletions core/src/main/kotlin/org/pastalab/fray/core/TestRunner.kt
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,12 @@ class TestRunner(val config: Configuration) {
val start = timeSource.markNow()
var i = 0
var bugsFound = 0
if (config.dummyRun) {
// We want to do a dummy-run first to make sure all variables are initialized
try {
config.executionInfo.executor.execute()
} catch (e: Throwable) {}
}
while (i != config.iter) {
reportProgress(i, bugsFound)
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,10 @@ import kotlinx.serialization.modules.subclass
import org.apache.logging.log4j.core.LoggerContext
import org.apache.logging.log4j.core.config.Configurator
import org.apache.logging.log4j.core.config.builder.api.ConfigurationBuilderFactory
import org.pastalab.fray.core.observers.ScheduleObserver
import org.pastalab.fray.core.observers.ScheduleRecorder
import org.pastalab.fray.core.observers.ScheduleRecording
import org.pastalab.fray.core.observers.ScheduleVerifier
import org.pastalab.fray.core.randomness.ControlledRandom
import org.pastalab.fray.core.scheduler.*

Expand Down Expand Up @@ -90,46 +94,55 @@ class JsonExecutionConfig : ExecutionConfig("json") {
}

sealed class ScheduleAlgorithm(name: String) : OptionGroup(name) {
open fun getScheduler(): Pair<Scheduler, ControlledRandom> {
return Pair(FifoScheduler(), ControlledRandom())
open fun getScheduler(): Triple<Scheduler, ControlledRandom, ScheduleVerifier?> {
return Triple(FifoScheduler(), ControlledRandom(), null)
}
}

class Fifo : ScheduleAlgorithm("fifo") {
override fun getScheduler(): Pair<Scheduler, ControlledRandom> {
return Pair(FifoScheduler(), ControlledRandom())
override fun getScheduler(): Triple<Scheduler, ControlledRandom, ScheduleVerifier?> {
return Triple(FifoScheduler(), ControlledRandom(), null)
}
}

class POS : ScheduleAlgorithm("pos") {
override fun getScheduler(): Pair<Scheduler, ControlledRandom> {
return Pair(POSScheduler(), ControlledRandom())
override fun getScheduler(): Triple<Scheduler, ControlledRandom, ScheduleVerifier?> {
return Triple(POSScheduler(), ControlledRandom(), null)
}
}

class Replay : ScheduleAlgorithm("replay") {
val path by option("--path").file().required()

override fun getScheduler(): Pair<Scheduler, ControlledRandom> {
override fun getScheduler(): Triple<Scheduler, ControlledRandom, ScheduleVerifier?> {
val randomPath = "${path.absolutePath}/random.json"
val schedulerPath = "${path.absolutePath}/schedule.json"
val randomnessProvider = Json.decodeFromString<ControlledRandom>(File(randomPath).readText())
val scheduler = Json.decodeFromString<Scheduler>(File(schedulerPath).readText())
return Pair(scheduler, randomnessProvider)
val scheduleVerifier =
if (System.getProperty("fray.verifySchedule", "true").toBoolean()) {
val recordingPath = "${path.absolutePath}/recording.json"
val scheduleRecordings =
Json.decodeFromString<List<ScheduleRecording>>(File(recordingPath).readText())
ScheduleVerifier(scheduleRecordings)
} else {
null
}
return Triple(scheduler, randomnessProvider, scheduleVerifier)
}
}

class Rand : ScheduleAlgorithm("random") {
override fun getScheduler(): Pair<Scheduler, ControlledRandom> {
return Pair(RandomScheduler(), ControlledRandom())
override fun getScheduler(): Triple<Scheduler, ControlledRandom, ScheduleVerifier?> {
return Triple(RandomScheduler(), ControlledRandom(), null)
}
}

class PCT : ScheduleAlgorithm("pct") {
val numSwitchPoints by option().int().default(3)

override fun getScheduler(): Pair<Scheduler, ControlledRandom> {
return Pair(PCTScheduler(ControlledRandom(), numSwitchPoints, 0), ControlledRandom())
override fun getScheduler(): Triple<Scheduler, ControlledRandom, ScheduleVerifier?> {
return Triple(PCTScheduler(ControlledRandom(), numSwitchPoints, 0), ControlledRandom(), null)
}
}

Expand Down Expand Up @@ -169,23 +182,37 @@ class MainCommand : CliktCommand() {
"json" to JsonExecutionConfig(),
)
.defaultByName("cli")
val dummyRun by
option(
"--no-dummy-run",
help =
"Run the target application without dummy run. The dummy run (run target once " +
"before launching Fray) helps Fray to prune out non-determinism " +
"introduced by the constructors and initializers.")
.flag(default = true)

override fun run() {}

fun toConfiguration(): Configuration {
val executionInfo = runConfig.getExecutionInfo()
val s = scheduler.getScheduler()
return Configuration(
executionInfo,
report,
iter,
s.first,
s.second,
fullSchedule,
exploreMode,
noExitWhenBugFound,
scheduler is Replay,
noFray)
val configuration =
Configuration(
executionInfo,
report,
iter,
s.first,
s.second,
fullSchedule,
exploreMode,
noExitWhenBugFound,
scheduler is Replay,
noFray,
dummyRun)
if (s.third != null) {
configuration.scheduleObservers.add(s.third!!)
}
return configuration
}
}

Expand All @@ -200,11 +227,15 @@ data class Configuration(
val noExitWhenBugFound: Boolean,
val isReplay: Boolean,
val noFray: Boolean,
val dummyRun: Boolean,
) {
val scheduleObservers = mutableListOf<ScheduleObserver>()

fun saveToReportFolder(index: Int) {
Paths.get("$report/recording_$index").createDirectories()
File("$report/recording_$index/schedule.json").writeText(Json.encodeToString(scheduler))
File("$report/recording_$index/random.json").writeText(Json.encodeToString(randomnessProvider))
scheduleObservers.forEach { it.saveToReportFolder("$report/recording_$index") }
}

val loggerContext by lazy {
Expand All @@ -231,6 +262,9 @@ data class Configuration(
if (!isReplay) {
prepareReportPath(report)
}
if (System.getProperty("fray.recordSchedule", "false").toBoolean()) {
scheduleObservers.add(ScheduleRecorder())
}
}

@OptIn(ExperimentalPathApi::class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,17 +13,17 @@ class CountDownLatchManager {
}

fun await(latch: CountDownLatch, canInterrupt: Boolean, thread: ThreadContext): Boolean {
return latchStore.getLockContext(latch).await(canInterrupt, thread)
return latchStore.getContext(latch).await(canInterrupt, thread)
}

/*
* Returns number of unblocked threads.
*/
fun countDown(latch: CountDownLatch): Int {
return latchStore.getLockContext(latch).countDown()
return latchStore.getContext(latch).countDown()
}

fun release(latch: CountDownLatch): Int {
return latchStore.getLockContext(latch).release()
return latchStore.getContext(latch).release()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ class LockManager {
}

fun getLockContext(lock: Any): LockContext {
return lockContextManager.getLockContext(lock)
return lockContextManager.getContext(lock)
}

fun reentrantReadWriteLockInit(readLock: ReadLock, writeLock: WriteLock) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,33 +10,33 @@ class IdentityPhantomReference<T>(referent: T, queue: ReferenceQueue<in T>) :

class ReferencedContextManager<T>(val contextProducer: (Any) -> T) {
val queue = ReferenceQueue<Any>()
val lockMap = mutableMapOf<Int, Pair<T, IdentityPhantomReference<*>>>()
val objMap = mutableMapOf<Int, Pair<T, IdentityPhantomReference<*>>>()

fun getLockContext(lock: Any): T {
val id = System.identityHashCode(lock)
if (!lockMap.containsKey(id)) {
lockMap[id] = Pair(contextProducer(lock), IdentityPhantomReference(lock, queue))
fun getContext(obj: Any): T {
val id = System.identityHashCode(obj)
if (!objMap.containsKey(id)) {
objMap[id] = Pair(contextProducer(obj), IdentityPhantomReference(obj, queue))
gc()
}
return lockMap[id]!!.first
return objMap[id]!!.first
}

fun addContext(lock: Any, context: T) {
val id = System.identityHashCode(lock)
lockMap[id] = Pair(context, IdentityPhantomReference(lock, queue))
objMap[id] = Pair(context, IdentityPhantomReference(lock, queue))
gc()
}

fun done() {
gc()
lockMap.clear()
objMap.clear()
}

fun gc() {
var ref = queue.poll()
while (ref != null) {
val id = (ref as IdentityPhantomReference<*>).id
lockMap.remove(id)
objMap.remove(id)
ref = queue.poll()
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,19 +25,19 @@ class SemaphoreManager {
threadContext: ThreadContext
): Boolean {
return lockContextManager
.getLockContext(sem)
.getContext(sem)
.acquire(permits, shouldBlock, canInterrupt, threadContext)
}

fun release(sem: Semaphore, permits: Int) {
lockContextManager.getLockContext(sem).release(permits)
lockContextManager.getContext(sem).release(permits)
}

fun drainPermits(sem: Semaphore): Int {
return lockContextManager.getLockContext(sem).drainPermits()
return lockContextManager.getContext(sem).drainPermits()
}

fun reducePermits(sem: Semaphore, permits: Int) {
lockContextManager.getLockContext(sem).reducePermits(permits)
lockContextManager.getContext(sem).reducePermits(permits)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
package org.pastalab.fray.core.observers

import org.pastalab.fray.core.ThreadContext

interface ScheduleObserver {
fun onExecutionStart()

fun onNewSchedule(enabledSchedules: List<ThreadContext>, scheduled: ThreadContext)

fun onExecutionDone()

fun saveToReportFolder(path: String)
}
Loading

0 comments on commit 5c47600

Please sign in to comment.