diff --git a/README.md b/README.md index b924e69d69..12c086904c 100644 --- a/README.md +++ b/README.md @@ -17,19 +17,19 @@ While certain features have been stripped for improved modularity, development h Currently, the following learning algorithms with respective target models are supported: -| Algorithm (active) | Target models | | Algorithm (passive) | Models | -|---------------------|-----------------------------|-----|-----------------------|-----------------------| -| AAAR | `DFA` `Mealy` `Moore` | | OSTIA | `SST` | -| ADT | `Mealy` | | RPNI (incl. variants) | `DFA` `Mealy` `Moore` | -| DHC | `Mealy` | | | | -| Kearns & Vazirani | `DFA` `Mealy` | | | | -| Lambda | `DFA` `Mealy` | | | | -| L# | `Mealy` | | | | -| L* (incl. variants) | `DFA` `Mealy` `Moore` | | | | -| NL* | `NFA` | | | | -| Observation Pack | `DFA` `Mealy` `Moore` `VPA` | | | | -| Procedural | `SPA` `SBA` `SPMM` | | | | -| TTT | `DFA` `Mealy` `Moore` `VPA` | | | | +| Algorithm (active) | Target models | | Algorithm (passive) | Models | +|---------------------|------------------------------|-----|-----------------------|-----------------------| +| AAAR | `DFA` `Mealy` `Moore` | | OSTIA | `SST` | +| ADT | `Mealy` | | RPNI (incl. variants) | `DFA` `Mealy` `Moore` | +| DHC | `Mealy` | | | | +| Kearns & Vazirani | `DFA` `Mealy` | | | | +| Lambda | `DFA` `Mealy` | | | | +| L# | `Mealy` | | | | +| L* (incl. variants) | `DFA` `Mealy` `Moore` `MMLT` | | | | +| NL* | `NFA` | | | | +| Observation Pack | `DFA` `Mealy` `Moore` `VPA` | | | | +| Procedural | `SPA` `SBA` `SPMM` | | | | +| TTT | `DFA` `Mealy` `Moore` `VPA` | | | | Additionally, LearnLib offers a variety of tools to ease the practical application of automata learning on real-world systems. This includes drivers and mappers for interfacing software systems with the LearnLib API as well as caches and parallelization for improving the overall performance of the learning setup. diff --git a/algorithms/active/adt/src/test/java/de/learnlib/algorithm/adt/it/ADTIT.java b/algorithms/active/adt/src/test/java/de/learnlib/algorithm/adt/it/ADTIT.java index c1bead3f71..0ef688b114 100644 --- a/algorithms/active/adt/src/test/java/de/learnlib/algorithm/adt/it/ADTIT.java +++ b/algorithms/active/adt/src/test/java/de/learnlib/algorithm/adt/it/ADTIT.java @@ -42,6 +42,7 @@ import de.learnlib.oracle.equivalence.MealySimulatorEQOracle; import de.learnlib.oracle.membership.MealySimulatorOracle; import de.learnlib.oracle.membership.SULAdaptiveOracle; +import de.learnlib.statistic.Statistics; import de.learnlib.sul.SUL; import de.learnlib.testsupport.MQ2AQWrapper; import de.learnlib.testsupport.it.learner.AbstractMealyLearnerIT; @@ -130,6 +131,7 @@ public void testIssue137() throws IOException, FormatException { for (int seed = 0; seed < 50; seed++) { long last = 0; for (int iter = 0; iter < 5; iter++) { + Statistics.getCollector().clear(); final CounterAdaptiveQueryOracle counter = new CounterAdaptiveQueryOracle<>(aqo); final ADTLearner learner = new ADTLearner<>(alphabet, @@ -151,7 +153,8 @@ public void testIssue137() throws IOException, FormatException { exp.run(); - final long count = counter.getResetCounter().getCount(); + final long count = + Statistics.getCollector().getCount(CounterAdaptiveQueryOracle.RESET_KEY).orElse(0L); if (iter == 0) { last = count; diff --git a/algorithms/active/lstar/pom.xml b/algorithms/active/lstar/pom.xml index c785db71ea..2c7966646f 100644 --- a/algorithms/active/lstar/pom.xml +++ b/algorithms/active/lstar/pom.xml @@ -50,6 +50,10 @@ limitations under the License. de.learnlib learnlib-counterexamples + + de.learnlib + learnlib-symbol-filters + de.learnlib learnlib-util @@ -124,6 +128,21 @@ limitations under the License. org.testng testng + + net.automatalib + automata-serialization-dot + test + + + de.learnlib + learnlib-cache + test + + + de.learnlib + learnlib-statistics + test + @@ -134,7 +153,25 @@ limitations under the License. maven-surefire-plugin - @{argLine} --add-reads=de.learnlib.algorithm.lstar=net.automatalib.util + + + @{argLine} + --add-reads=de.learnlib.algorithm.lstar=net.automatalib.util + --add-reads=de.learnlib.algorithm.lstar=de.learnlib.filter.statistic + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + + + @{argLine} + --add-reads=de.learnlib.algorithm.lstar=net.automatalib.util + --add-reads=de.learnlib.algorithm.lstar=de.learnlib.filter.statistic + diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/ExtensibleLStarMMLT.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/ExtensibleLStarMMLT.java new file mode 100644 index 0000000000..bf1965171b --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/ExtensibleLStarMMLT.java @@ -0,0 +1,641 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import de.learnlib.acex.AcexAnalyzer; +import de.learnlib.acex.AcexAnalyzers; +import de.learnlib.algorithm.lstar.closing.ClosingStrategies; +import de.learnlib.algorithm.lstar.closing.ClosingStrategy; +import de.learnlib.algorithm.lstar.mmlt.cex.MMLTCounterexampleHandler; +import de.learnlib.algorithm.lstar.mmlt.cex.MMLTOutputInconsistency; +import de.learnlib.algorithm.lstar.mmlt.cex.results.CexAnalysisResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.FalseIgnoreResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.MissingDiscriminatorResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.MissingOneShotResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.MissingResetResult; +import de.learnlib.datastructure.observationtable.OTLearner; +import de.learnlib.datastructure.observationtable.ObservationTable; +import de.learnlib.datastructure.observationtable.Row; +import de.learnlib.filter.MutableSymbolFilter; +import de.learnlib.filter.symbol.AcceptAllSymbolFilter; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.DefaultQuery; +import de.learnlib.query.Query; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; +import de.learnlib.time.MMLTModelParams; +import de.learnlib.tooling.annotation.builder.GenerateBuilder; +import de.learnlib.util.mealy.MealyUtil; +import net.automatalib.alphabet.Alphabet; +import net.automatalib.alphabet.GrowingAlphabet; +import net.automatalib.alphabet.impl.GrowingMapAlphabet; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.TimerInfo; +import net.automatalib.common.util.HashUtil; +import net.automatalib.common.util.collection.IterableUtil; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An L*-based leaner for inferring {@link MMLT}s. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class ExtensibleLStarMMLT + implements OTLearner, TimedInput, Word>> { + + private static final Logger LOGGER = LoggerFactory.getLogger(ExtensibleLStarMMLT.class); + private final StatisticsCollector stats; + + private final ClosingStrategy, ? super Word>> closingStrategy; + + private final TimedQueryOracle timeOracle; + private final MutableSymbolFilter, InputSymbol> symbolFilter; + + private final MMLTHypDataContainer hypData; + + // ============================ + + private final List>> initialSuffixes; + private final MMLTCounterexampleHandler cexAnalyzer; + + /** + * Instantiates a new learner. + *

+ * This is a convenience constructor for + * {@link #ExtensibleLStarMMLT(Alphabet, MMLTModelParams, TimedQueryOracle, List, ClosingStrategy, + * MutableSymbolFilter, AcexAnalyzer)} which uses + *

    + *
  • {@link Collections#emptyList()} for {@code initialSuffixes},
  • + *
  • {@link ClosingStrategies#CLOSE_SHORTEST} for {@code closingStrategy},
  • + *
  • {@link AcceptAllSymbolFilter} for {@code symbolFilter}, and
  • + *
  • {@link AcexAnalyzers#BINARY_SEARCH_BWD} for {@code analyzer}.
  • + *
+ * + * @param alphabet + * alphabet (of non-delaying inputs) + * @param modelParams + * model parameters + * @param timeOracle + * the query oracle for MMLTs + */ + public ExtensibleLStarMMLT(Alphabet alphabet, + MMLTModelParams modelParams, + TimedQueryOracle timeOracle) { + this(alphabet, + modelParams, + timeOracle, + Collections.emptyList(), + ClosingStrategies.CLOSE_SHORTEST, + new AcceptAllSymbolFilter<>(), + AcexAnalyzers.BINARY_SEARCH_BWD); + } + + /** + * Instantiates a new learner. + * + * @param alphabet + * alphabet (of non-delaying inputs) + * @param modelParams + * model parameters + * @param timeOracle + * the query oracle for MMLTs + * @param initialSuffixes + * initial set of suffixes (may be empty) + * @param closingStrategy + * closing strategy for the observation table. + * @param symbolFilter + * the symbol filter + * @param analyzer + * the strategy for decomposing counterexamples. + */ + @GenerateBuilder(defaults = BuilderDefaults.class) + public ExtensibleLStarMMLT(Alphabet alphabet, + MMLTModelParams modelParams, + TimedQueryOracle timeOracle, + List>> initialSuffixes, + ClosingStrategy, ? super Word>> closingStrategy, + MutableSymbolFilter, InputSymbol> symbolFilter, + AcexAnalyzer analyzer) { + this.closingStrategy = closingStrategy; + this.timeOracle = timeOracle; + this.initialSuffixes = initialSuffixes; + this.stats = Statistics.getCollector(); + + // Prepare hyp data: + + // Internally, the learner also stores TimeStepSequences in its alphabet: + GrowingAlphabet> internalAlphabet = new GrowingMapAlphabet<>(); + alphabet.forEach(s -> internalAlphabet.add(TimedInput.input(s))); + + // Init hypothesis data: + this.hypData = new MMLTHypDataContainer<>(internalAlphabet, + modelParams, + new MMLTObservationTable<>(internalAlphabet, + modelParams.maxTimerQueryWaitingTime(), + symbolFilter, + modelParams.silentOutput())); + + this.cexAnalyzer = new MMLTCounterexampleHandler<>(timeOracle, analyzer, symbolFilter); + this.symbolFilter = symbolFilter; + } + + /** + * Heuristically chooses a new one-shot timer from the provided timers. Takes the timer with the highest initial + * value that + *
    + *
  • does not exceed {@code maxInitialValue} and
  • + *
  • has not timer with a lower initial value that times out at the same time.
  • + *
+ * + * @param sortedTimers + * timers, sorted ascendingly by their initial value + * @param maxInitialValue + * max. initial value to consider + * @param + * output type + * + * @return the index (in {@code sortedTimers}) of the new one-shot candidate + */ + public static int selectOneShotTimer(List> sortedTimers, long maxInitialValue) { + + // Filter relevant timers: + // Start at timer with the highest initial value. + // Ignore all timers whose initial value exceeds the maximum value. + // Also ignore timers whose timeout is the multiple of another timer's initial value. + timers: + for (int i = sortedTimers.size() - 1; i >= 0; i--) { + TimerInfo timer = sortedTimers.get(i); + + // could not have expired + if (timer.initial() <= maxInitialValue) { + + // Ignore timers whose initial value is a multiple of another one. + // When set to one-shot, these would expire at same time as periodic timer -> non-deterministic behavior! + for (int j = 0; j < i; j++) { + TimerInfo otherTimer = sortedTimers.get(j); + if (timer.initial() % otherTimer.initial() == 0) { + continue timers; + } + } + + return i; // not a multiple and within time + } + } + + throw new IllegalStateException("Max. initial value is too low; must include at least one timer."); + } + + @Override + public MMLT getHypothesisModel() { + return getInternalHypothesisModel(); + } + + /** + * Like {@link #getHypothesisModel()}, but returns an {@link MMLTHypothesis} object instead. This objects provides + * additional functions that are just intended for the learner but not the teacher. + * + * @return the internal hypothesis + */ + private MMLTHypothesis getInternalHypothesisModel() { + this.updateOutputs(); + return constructHypothesis(this.hypData); + } + + private List>> selectClosingRows(List>>> unclosed) { + return closingStrategy.selectClosingRows(unclosed, hypData.getTable(), timeOracle); + } + + private void updateOutputs() { + // Query output of newly-added transitions: + MMLTObservationTable ot = this.hypData.getTable(); + List> queries = new ArrayList<>(); + + for (Row> row : IterableUtil.concat(ot.getShortPrefixRows(), ot.getLongPrefixRows())) { + Word> label = row.getLabel(); + + if (label.isEmpty()) { + continue; // initial state + } + + if (this.hypData.getTransitionOutputMap().containsKey(label)) { + continue; // already queried + } + + Word> prefix = label.prefix(-1); + TimedInput inputSym = label.lastSymbol(); + + TimedOutput output; + if (inputSym instanceof TimeStepSequence ws) { + // Query timer output from table: + TimerInfo timerInfo = this.hypData.getTable().getTimerInfo(prefix, ws.timeSteps()); + assert timerInfo != null; + O combinedOutput = this.hypData.getModelParams().outputCombiner().combineSymbols(timerInfo.outputs()); + output = new TimedOutput<>(combinedOutput); + this.hypData.getTransitionOutputMap().put(label, output); + } else { + queries.add(new OutputQuery<>(label, prefix)); + } + } + + if (!queries.isEmpty()) { + timeOracle.processQueries(queries); + + for (OutputQuery q : queries) { + q.process(this.hypData.getTransitionOutputMap()); + } + } + } + + // ========================== + + @Override + public void startLearning() { + List>>> initialUnclosed = + this.hypData.getTable().initialize(Collections.emptyList(), this.initialSuffixes, timeOracle); + + // Ensure that closed: + this.completeConsistentTable(initialUnclosed); + } + + @Override + public boolean refineHypothesis(DefaultQuery, Word>> ceQuery) { + if (!refineHypothesisSingle(ceQuery)) { + return false; // no valid CEX + } + while (refineHypothesisSingle(ceQuery)) { + // analyze exhaustively + } + return true; + } + + /** + * Transforms the provided counterexample to an inconsistency object: First, checks if still a counterexample. If + * so, cuts the cex after the first output deviation. + * + * @param ceQuery + * Counterexample + * @param hypothesis + * Current hypothesis + * + * @return The resulting inconsistency, or null, if the counterexample is not a counterexample. + */ + private @Nullable MMLTOutputInconsistency toOutputInconsistency(DefaultQuery, Word>> ceQuery, + MMLTHypothesis hypothesis) { + // 1. Cut example after first deviation: + DefaultQuery, Word>> shortQuery = + MealyUtil.shortenCounterExample(hypothesis.getSemantics(), ceQuery); + if (shortQuery == null) { + return null; + } + + // 2. Calculate shortened hypothesis output: + Word> shortHypOutput = + hypothesis.getSemantics().computeSuffixOutput(shortQuery.getPrefix(), shortQuery.getSuffix()); + + assert !shortHypOutput.equals(shortQuery.getOutput()) : "Deviation lost after shortening."; + + return new MMLTOutputInconsistency<>(shortQuery.getPrefix(), + shortQuery.getSuffix(), + shortQuery.getOutput(), + shortHypOutput); + } + + private boolean refineHypothesisSingle(DefaultQuery, Word>> ceQuery) { + // 1. Update hypothesis (may have changed since last refinement): + MMLTHypothesis hypothesis = this.getInternalHypothesisModel(); + + // 2. Transform to output inconsistency: + MMLTOutputInconsistency outputIncons = this.toOutputInconsistency(ceQuery, hypothesis); + if (outputIncons == null) { + return false; + } + + LOGGER.debug("Refining with inconsistency {}", outputIncons); + + // 3. Identify source of deviation: + stats.startOrResumeClock("clk_cex_analysis", "Total cex analysis time"); + stats.increaseCounter("cnt_cex_analysis", "Cex analyses"); + CexAnalysisResult analysisResult = this.cexAnalyzer.analyzeInconsistency(outputIncons, hypothesis); + stats.pauseClock("clk_cex_analysis"); + + // 4. Refine: + if (analysisResult instanceof MissingDiscriminatorResult locSplit) { + stats.increaseCounter("INACC_MISSING_DISC", "Inaccuracies: missing discriminators"); + + // Add new discriminator as suffix: + assert !hypData.getTable().getSuffixes().contains(locSplit.getDiscriminator()); + List>> suffixes = Collections.singletonList(locSplit.getDiscriminator()); + List>>> unclosed = hypData.getTable().addSuffixes(suffixes, timeOracle); + + // Close transitions: + this.completeConsistentTable(unclosed); // no consistency check for RS + } else if (analysisResult instanceof MissingResetResult noReset) { + stats.increaseCounter("INACC_MISSING_RESETS", "Inaccuracies: missing resets"); + + // Add missing reset: + Word> resetTrans = hypothesis.getPrefix(noReset.getLocation()).append(noReset.getInput()); + this.hypData.getTransitionResetSet().add(resetTrans); + } else if (analysisResult instanceof MissingOneShotResult noAperiodic) { + stats.increaseCounter("INACC_MISSING_OS", "Inaccuracies: missing one-shot timers"); + + // Identify corresponding sp row: + Word> locPrefix = hypothesis.getPrefix(noAperiodic.getLocation()); + Row> spRow = hypData.getTable().getRow(locPrefix); + + assert spRow != null && spRow.isShortPrefixRow(); + + this.handleMissingTimeoutChange(spRow, noAperiodic.getTimeout()); + } else if (analysisResult instanceof FalseIgnoreResult falseIgnore) { + stats.increaseCounter("INACC_MISSING_FI", "Inaccuracies: false ignores"); + + // Identify corresponding sp row: + Word> locPrefix = hypothesis.getPrefix(falseIgnore.getLocation()); + Row> spRow = hypData.getTable().getRow(locPrefix); + + assert spRow != null && spRow.isShortPrefixRow(); + + // Update filter: + this.symbolFilter.accept(locPrefix, falseIgnore.getSymbol()); + + // Legalize symbol + close table: + List>>> unclosed = + hypData.getTable().addOutgoingTransition(spRow, falseIgnore.getSymbol(), this.timeOracle); + stats.increaseCounter("Count_legalized", "Legalized symbols"); + + this.completeConsistentTable(unclosed); + } else { + throw new IllegalStateException("Unknown inconsistency type."); + } + + return true; + } + + private void handleMissingTimeoutChange(Row> spRow, TimerInfo timeout) { + LocationTimerInfo locationTimerInfo = hypData.getTable().getLocationTimerInfo(spRow); + assert locationTimerInfo != null : "Location with missing one-shot timer must have timers."; + + // Only timer with highest initial value can be one-shot. + // If location already has a one-shot timer, prefix of its timeout-transition might be core or fringe prefix. + // If it is a fringe prefix, we need to remove it: + TimerInfo lastTimer = locationTimerInfo.getLastTimer(); + assert lastTimer != null; + if (!lastTimer.periodic()) { + Word> lastTimerTransPrefix = spRow.getLabel().append(TimedInput.step(lastTimer.initial())); + Row> row = hypData.getTable().getRow(lastTimerTransPrefix); + assert row != null; + if (!row.isShortPrefixRow()) { + // Last timer is one-shot + has fringe prefix: + this.hypData.getTable().removeLpRow(lastTimerTransPrefix); + } + } + + // Prefix for timeout-transition of new one-shot timer: + assert this.hypData.getTable().getRow(spRow.getLabel().append(TimedInput.step(timeout.initial()))) == null : + "Timer already appears to be one-shot."; + + // Remove all timers with greater timeout (are now redundant): + for (TimerInfo t : new ArrayList<>(locationTimerInfo.getSortedTimers())) { + if (t.initial() > timeout.initial()) { + locationTimerInfo.removeTimer(t.name()); + } + } + + // Change from periodic to one-shot: + locationTimerInfo.setOneShotTimer(timeout.name()); + + // Update fringe prefixes + close table: + List>>> unclosed = + this.hypData.getTable().addTimerTransition(spRow, timeout, this.timeOracle); + this.completeConsistentTable(unclosed); + } + + @Override + public ObservationTable, Word>> getObservationTable() { + return this.hypData.getTable(); + } + + /** + * Iteratively checks for unclosedness and inconsistencies in the table, and fixes any occurrences thereof. This + * process is repeated until the observation table is both closed and consistent. + *

+ * Simplified version for RS learner: assumes that OT is always consistent. + * + * @param unclosed + * the unclosed rows (equivalence classes) to start with. + */ + private void completeConsistentTable(List>>> unclosed) { + List>>> unclosedIter = unclosed; + while (!unclosedIter.isEmpty()) { + List>> closingRows = this.selectClosingRows(unclosedIter); + + // Add new states: + unclosedIter = hypData.getTable().toShortPrefixes(closingRows, timeOracle); + } + + } + + /** + * Constructs a hypothesis MMLT from an observation table, inferred local resets, and inferred local timers. + */ + private static MMLTHypothesis constructHypothesis(MMLTHypDataContainer hypData) { + + // 1. Create map that stores link between contentID and short-prefix row: + final Map>> locationContentIdMap = new HashMap<>(); // contentId -> sp location + for (Row> spRow : hypData.getTable().getShortPrefixRows()) { + // Multiple sp rows may have same contentID. Thus, assign each id only one location: + locationContentIdMap.putIfAbsent(spRow.getRowContentId(), spRow); + } + + // 2. Create untimed alphabet: + GrowingMapAlphabet alphabet = new GrowingMapAlphabet<>(); + for (TimedInput symbol : hypData.getAlphabet()) { + if (symbol instanceof InputSymbol ndi) { + alphabet.add(ndi.symbol()); + } + } + + // 3. Prepare objects for automaton, timers and resets: + int numLocations = hypData.getTable().numberOfShortPrefixRows(); + final Map stateMap = + new HashMap<>(HashUtil.capacity(numLocations)); // row content id -> state id + final Map>> prefixMap = + new HashMap<>(HashUtil.capacity(numLocations)); // state id -> location prefix + MMLTHypothesis hypothesis = new MMLTHypothesis<>(alphabet, + numLocations, + hypData.getModelParams().silentOutput(), + hypData.getModelParams().outputCombiner(), + prefixMap); // we pass the prefix map as reference so that we can fill it later + + // 4. Create one state per location: + for (Row> row : hypData.getTable().getShortPrefixRows()) { + int newStateId = hypothesis.addState(); + stateMap.putIfAbsent(row.getRowContentId(), newStateId); + prefixMap.put(newStateId, row.getLabel()); + + if (row.getLabel().equals(Word.epsilon())) { + hypothesis.setInitialState(newStateId); + } + } + // Ensure initial location: + assert hypothesis.getInitialState() != null : "Automaton must have an initial location."; + + // 5. Create outgoing transitions for non-delaying inputs: + for (Entry e : stateMap.entrySet()) { + Integer rowContentId = e.getKey(); + Row> spLocation = locationContentIdMap.get(rowContentId); + + for (I symbol : alphabet) { + int symIdx = hypData.getAlphabet().getSymbolIndex(TimedInput.input(symbol)); + + TimedOutput transOutput = hypData.getTransitionOutput(spLocation, symIdx); + O output = hypData.getModelParams().silentOutput(); // silent by default + if (transOutput != null) { + output = transOutput.symbol(); + } + + int successorId; + if (spLocation.getSuccessor(symIdx) == null) { + successorId = spLocation.getRowContentId(); // not in local alphabet -> self-loop + } else { + successorId = spLocation.getSuccessor(symIdx).getRowContentId(); + } + + // Add transition to automaton: + int sourceLocId = e.getValue(); + int successorLocId = stateMap.get(successorId); + hypothesis.addTransition(sourceLocId, symbol, successorLocId, output); + + // Check for local reset: + Word> targetTransition = spLocation.getLabel().append(TimedInput.input(symbol)); + if (hypData.getTransitionResetSet().contains(targetTransition) && sourceLocId == successorLocId) { + hypothesis.addLocalReset(sourceLocId, symbol); + } + } + + } + + // 6. Add timeout transitions: + for (Entry e : stateMap.entrySet()) { + Integer rowContentId = e.getKey(); + Row> spLocation = locationContentIdMap.get(rowContentId); + + assert spLocation != null; + + LocationTimerInfo timerInfo = hypData.getTable().getLocationTimerInfo(spLocation); + + if (timerInfo != null) { + for (TimerInfo timer : timerInfo.getLocalTimers().values()) { + if (timer.periodic()) { + hypothesis.addPeriodicTimer(e.getValue(), timer.name(), timer.initial(), timer.outputs()); + } else { + // One-shot: use successor from table + TimedInput symbol = new TimeStepSequence<>(timer.initial()); + + int symIdx = hypData.getAlphabet().getSymbolIndex(symbol); + int successorId = spLocation.getSuccessor(symIdx).getRowContentId(); + + hypothesis.addOneShotTimer(e.getValue(), + timer.name(), + timer.initial(), + timer.outputs(), + stateMap.get(successorId)); + } + } + } + } + + return hypothesis; + } + + private static final class OutputQuery extends Query, Word>> { + + private final Word> label; + private final Word> prefix; + private TimedOutput output; + + private OutputQuery(Word> label, Word> prefix) { + this.label = label; + this.prefix = prefix; + } + + @Override + public void answer(Word> output) { + assert output.size() == 1; + this.output = output.firstSymbol(); + } + + @Override + public Word> getPrefix() { + return prefix; + } + + @Override + public Word> getSuffix() { + return Word.fromLetter(label.lastSymbol()); + } + + /** + * Processes the query result by mapping the given label to the (single) response. + * + * @param outputs + * the output map to write the mapping to + */ + void process(Map>, TimedOutput> outputs) { + outputs.put(label, output); + } + } + + static final class BuilderDefaults { + + private BuilderDefaults() { + // prevent instantiation + } + + static List>> initialSuffixes() { + return Collections.emptyList(); + } + + static ClosingStrategy, ? super Word>> closingStrategy() { + return ClosingStrategies.CLOSE_SHORTEST; + } + + static MutableSymbolFilter, InputSymbol> symbolFilter() { + return new AcceptAllSymbolFilter<>(); + } + + static AcexAnalyzer analyzer() { + return AcexAnalyzers.BINARY_SEARCH_BWD; + } + } + +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/LocationTimerInfo.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/LocationTimerInfo.java new file mode 100644 index 0000000000..78d8bcc2c3 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/LocationTimerInfo.java @@ -0,0 +1,153 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import net.automatalib.automaton.mmlt.TimerInfo; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Stores information about local timers of a location. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class LocationTimerInfo { + + private static final Logger LOGGER = LoggerFactory.getLogger(LocationTimerInfo.class); + + private final Map> timers; // name -> info + + // Keep a list of timers sorted by their initial value. This lets us avoid redundant sort operations. + private final List> sortedTimers; + + private final Word> prefix; + + public LocationTimerInfo(Word> prefix) { + this.prefix = prefix; + this.timers = new HashMap<>(); + this.sortedTimers = new ArrayList<>(); + } + + public Word> getPrefix() { + return prefix; + } + + // ==================== + + /** + * Adds a local timer to this location. + * + * @param timer + * the timer to add + * + */ + public void addTimer(TimerInfo timer) { + this.timers.put(timer.name(), timer); + this.sortedTimers.add(timer); + this.sortedTimers.sort(Comparator.comparingLong(TimerInfo::initial)); + } + + public void removeTimer(String timerName) { + final TimerInfo removedTimer = this.timers.remove(timerName); + if (removedTimer == null) { + LOGGER.warn("Attempted to remove an unknown timer."); + } else { + this.sortedTimers.remove(removedTimer); + } + } + + /** + * Returns the timer with the given initial value. + * + * @param initial + * the queried initial value + * + * @return the timer with given timeout, {@code null} if no such timer exists + */ + public @Nullable TimerInfo getTimerInfo(long initial) { + for (TimerInfo t : this.sortedTimers) { + if (t.initial() == initial) { + return t; + } + } + return null; + } + + /** + * Returns the timer with the highest initial value. + * + * @return the timer with maximum timeout, {@code null} if no timers defined + */ + public @Nullable TimerInfo getLastTimer() { + if (this.timers.isEmpty()) { + return null; + } + return sortedTimers.get(sortedTimers.size() - 1); + } + + /** + * Sets the given timer to one-shot, ensuring that there is only one one-shot timer at a time. This is preferred + * over setting the timer property. + * + * @param name + * name of the new one-shot timer + */ + public void setOneShotTimer(String name) { + TimerInfo oneShotTimer = this.timers.get(name); + if (oneShotTimer == null) { + throw new IllegalArgumentException("Unknown one-shot timer name."); + } + if (!oneShotTimer.equals(sortedTimers.get(sortedTimers.size() - 1))) { + throw new IllegalArgumentException("Only the timer with maximum timeout can be one-shot."); + } + + // update references + TimerInfo newTimer = oneShotTimer.asOneShot(); + this.timers.put(name, newTimer); + this.sortedTimers.set(sortedTimers.size() - 1, newTimer); + } + + /** + * Returns a list of all timers defined in this location, sorted by their initial value. + * + * @return list of local timers, may be empty + */ + public List> getSortedTimers() { + return Collections.unmodifiableList(sortedTimers); + } + + /** + * Returns an unmodifiable view of the timers defined for this location. Format: name -> info + * + * @return map of local timers, may be empty + */ + public Map> getLocalTimers() { + return Collections.unmodifiableMap(this.timers); + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTHypDataContainer.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTHypDataContainer.java new file mode 100644 index 0000000000..5ce6d64a7d --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTHypDataContainer.java @@ -0,0 +1,89 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import de.learnlib.datastructure.observationtable.Row; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.alphabet.Alphabet; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Stores various data used for describing the MMLT hypothesis. This includes the observation table, a list of local + * resets, and a list of outputs. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +class MMLTHypDataContainer { + + private final Alphabet> alphabet; + + private final MMLTObservationTable table; + private final Map>, TimedOutput> transitionOutputMap; + private final Set>> transitionResetSet; // all transitions that trigger a reset + + private final MMLTModelParams modelParams; + + MMLTHypDataContainer(Alphabet> alphabet, + MMLTModelParams modelParams, + MMLTObservationTable table) { + this.alphabet = alphabet; + this.modelParams = modelParams; + this.table = table; + + this.transitionOutputMap = new HashMap<>(); + this.transitionResetSet = new HashSet<>(); + } + + @Nullable TimedOutput getTransitionOutput(Row> stateRow, int inputIdx) { + Row> transRow = stateRow.getSuccessor(inputIdx); + if (transRow == null) { + return null; + } + + return this.transitionOutputMap.get(transRow.getLabel()); + } + + MMLTModelParams getModelParams() { + return modelParams; + } + + Alphabet> getAlphabet() { + return alphabet; + } + + MMLTObservationTable getTable() { + return table; + } + + Map>, TimedOutput> getTransitionOutputMap() { + return transitionOutputMap; + } + + Set>> getTransitionResetSet() { + return transitionResetSet; + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTHypothesis.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTHypothesis.java new file mode 100644 index 0000000000..2509b1b7f3 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTHypothesis.java @@ -0,0 +1,98 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt; + +import java.util.Map; + +import net.automatalib.alphabet.Alphabet; +import net.automatalib.automaton.mmlt.State; +import net.automatalib.automaton.mmlt.SymbolCombiner; +import net.automatalib.automaton.mmlt.impl.CompactMMLT; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; + +/** + * An MMLT hypothesis that includes a prefix mapping. This mapping assigns a short prefix to each location. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MMLTHypothesis extends CompactMMLT { + + private final Map>> prefixMap; // location -> prefix + + MMLTHypothesis(Alphabet alphabet, + int sizeHint, + O silentOuput, + SymbolCombiner outputCombiner, + Map>> prefixMap) { + super(alphabet, sizeHint, silentOuput, outputCombiner); + this.prefixMap = prefixMap; + } + + /** + * Returns the prefix assigned to the provided configuration. The assigned prefix is the concatenation of the prefix + * assigned to the active location and the minimal number of time steps needed to reach the configuration after + * entering its location (= entry distance). + * + * @param configuration + * Considered configuration + * + * @return Assigned prefix + */ + public Word> getPrefix(State configuration) { + Word> locPrefix = getLocationPrefix(configuration); + if (configuration.isEntryConfig()) { + return locPrefix; // entry distance = 0 + } else { + return locPrefix.append(TimedInput.step(configuration.getEntryDistance())); + } + } + + /** + * Returns a prefix for the given location. This prefix is deterministic in the learner. + * + * @param location + * Location + * + * @return Location prefix + */ + public Word> getPrefix(Integer location) { + return prefixMap.get(location); + } + + public Word> getPrefix(Word> prefix) { + State resultingConfig = getSemantics().getState(prefix); + assert resultingConfig != null; + return getPrefix(resultingConfig); + } + + /** + * Returns the prefix assigned to the location that is active in the provided configuration. + * + * @param configuration + * Considered configuration + * + * @return Assigned prefix + */ + public Word> getLocationPrefix(State configuration) { + Word> locPrefix = this.prefixMap.get(configuration.getLocation()); + assert locPrefix != null; + return locPrefix; + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTObservationTable.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTObservationTable.java new file mode 100644 index 0000000000..61e639eebc --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/MMLTObservationTable.java @@ -0,0 +1,579 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import de.learnlib.datastructure.observationtable.ObservationTable; +import de.learnlib.datastructure.observationtable.Row; +import de.learnlib.datastructure.observationtable.RowImpl; +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.MutableSymbolFilter; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.oracle.TimedQueryOracle.TimerQueryResult; +import de.learnlib.query.DefaultQuery; +import net.automatalib.alphabet.Alphabet; +import net.automatalib.automaton.mmlt.TimerInfo; +import net.automatalib.common.util.HashUtil; +import net.automatalib.common.util.collection.IterableUtil; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The observation table used by the MMLT learner. + *

+ * Unlike an OT for standard Mealy learning, includes prefixes for the timeout transitions of one-shot timers. Intended + * to be used with a symbol filter. The filter is queried before adding a new transition for a non-delaying input. If + * the filter considers the transition to be a silent self-loop, the output of the transition is first verified. If it + * is actually silent the learner considers the transition to be a silent self-loop. Consequently, it does not add a + * transition for it. Transitions may be added later if an input was falsely ignored. + *

+ * Assumes that all short prefixes lead to different locations (-> no need to make canonical) + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +class MMLTObservationTable implements ObservationTable, Word>> { + + private static final Logger LOGGER = LoggerFactory.getLogger(MMLTObservationTable.class); + private static final int NO_CONTENT = -1; + + private final MutableSymbolFilter, InputSymbol> symbolFilter; + + private final Map>, LocationTimerInfo> timerInfoMap; // prefix -> timer info + + private final Map>, RowImpl>> shortPrefixRowMap; // label -> row info + private final Map>, RowImpl>> longPrefixRowMap; // label -> row info + + private final Map> rowContentMap; // contentID -> row content + + private final List>> suffixes; + private final Set>> suffixSet; + + private final Alphabet> alphabet; + private final long minTimerQueryWaitTime; + private final TimedOutput silentOutput; // used for symbol filtering + + MMLTObservationTable(Alphabet> alphabet, + long minTimerQueryWaitTime, + MutableSymbolFilter, InputSymbol> symbolFilter, + O silentOutput) { + this.alphabet = alphabet; + + this.symbolFilter = symbolFilter; + this.silentOutput = new TimedOutput<>(silentOutput); + this.minTimerQueryWaitTime = minTimerQueryWaitTime; + + this.timerInfoMap = new HashMap<>(); + + // use linked hashmaps for stable insertion-order + this.shortPrefixRowMap = new LinkedHashMap<>(); + this.longPrefixRowMap = new LinkedHashMap<>(); + + this.rowContentMap = new HashMap<>(); + this.suffixes = new ArrayList<>(); + this.suffixSet = new HashSet<>(); + } + + /** + * Infers local timers for the provided location. + * + * @param location + * source location + */ + private void identifyLocalTimers(LocationTimerInfo location, TimedQueryOracle timeOracle) { + TimerQueryResult timerQueryResponse = + timeOracle.queryTimers(location.getPrefix(), this.minTimerQueryWaitTime); + List> timers = timerQueryResponse.timers(); + + if (timerQueryResponse.aborted()) { + int end = ExtensibleLStarMMLT.selectOneShotTimer(timers, Long.MAX_VALUE); + timers.set(end, timers.get(end).asOneShot()); + } + + // Add timers up to one-shot: + for (TimerInfo timer : timerQueryResponse.timers()) { + location.addTimer(timer); + this.extendAlphabet(TimedInput.step(timer.initial())); + if (!timer.periodic()) { + break; + } + } + } + + /** + * Extends the global alphabet without adding new transitions. + * + * @param symbol + * new alphabet symbol + */ + private void extendAlphabet(TimeStepSequence symbol) { + if (!alphabet.containsSymbol(symbol)) { + alphabet.asGrowingAlphabetOrThrowException().addSymbol(symbol); + } + + for (RowImpl> prefix : this.shortPrefixRowMap.values()) { + prefix.ensureInputCapacity(alphabet.size()); + } + } + + /** + * Adds the initial location. + * + * @return corresponding row in the observation table + */ + private RowImpl> addInitialLocation() { + RowImpl> newRow = new RowImpl<>(Word.epsilon(), 0, alphabet.size()); + newRow.makeShort(alphabet.size()); + this.shortPrefixRowMap.put(Word.epsilon(), newRow); + + return newRow; + } + + /** + * Adds a new location that belongs to the provided short-prefix row. Infers timers for this location and creates + * outgoing transitions. + * + * @param newRow + * newly-added short prefix row + * @param timeOracle + * time oracle + */ + private void initLocation(RowImpl> newRow, TimedQueryOracle timeOracle) { + LocationTimerInfo timerInfo = new LocationTimerInfo<>(newRow.getLabel()); + this.identifyLocalTimers(timerInfo, timeOracle); + + if (timerInfo.getLastTimer() != null) { // location has timer + this.timerInfoMap.put(newRow.getLabel(), timerInfo); + } + + // Add outgoing transitions: + List>> transitions = this.createOutgoingTransitions(newRow, timeOracle); + this.queryAllSuffixes(transitions, timeOracle); + } + + /** + * Creates transitions for the provided short-prefix row. Adds transitions for non-delaying inputs and a transition + * for the one-shot timer of the location, if present. + *

+ * If a symbol filter is provided, the filter is queried before adding a transition for a non-delaying input. If the + * filter considers the input a silent self-loop, no transition is explicitly created for the input. + * + * @param spRow + * short prefix row + * @param timeOracle + * time query oracle + * + * @return new transitions + */ + private List>> createOutgoingTransitions(RowImpl> spRow, + TimedQueryOracle timeOracle) { + List>> transitions = new ArrayList<>(); + + Word> sp = spRow.getLabel(); + + // First, add transitions for non-delaying symbols: + for (int i = 0; i < alphabet.size(); i++) { + TimedInput sym = alphabet.getSymbol(i); + if (sym instanceof InputSymbol in) { + + Word> lp = sp.append(sym); + assert !this.shortPrefixRowMap.containsKey(lp); + + RowImpl> succRow = this.longPrefixRowMap.get(lp); + if (succRow == null) { + // Query symbol filter before adding transition: + FilterResponse filterResponse = this.symbolFilter.query(sp, in); + if (filterResponse == FilterResponse.IGNORE) { + // Verify that output is silent: + Word> response = timeOracle.answerQuery(sp, Word.fromLetter(sym)); + assert response.size() == 1; + if (!response.firstSymbol().equals(silentOutput)) { + // Not silent -> cannot be silent self-loop: + filterResponse = FilterResponse.ACCEPT; + + // Update filter: + this.symbolFilter.accept(sp, in); + } + } + + if (filterResponse == FilterResponse.ACCEPT) { + // Treat as usual: + succRow = this.createLpRow(lp); + } + } + + spRow.setSuccessor(i, succRow); + if (succRow != null) { + transitions.add(succRow); + } + } + + } + + // Second, add one-shot timer transition (if any): + LocationTimerInfo locTimers = timerInfoMap.get(spRow.getLabel()); + if (locTimers != null) { + TimerInfo lastTimer = locTimers.getLastTimer(); + if (lastTimer != null && !lastTimer.periodic()) { + TimedInput waitSym = new TimeStepSequence<>(lastTimer.initial()); + Word> lp = sp.append(waitSym); + assert !this.shortPrefixRowMap.containsKey(lp); + + RowImpl> succRow = this.longPrefixRowMap.get(lp); + if (succRow == null) { + succRow = this.createLpRow(lp); + } + spRow.setSuccessor(this.alphabet.getSymbolIndex(waitSym), succRow); + transitions.add(succRow); + } + } + + return transitions; + } + + private RowImpl> createLpRow(Word> prefix) { + RowImpl> newRow = new RowImpl<>(prefix, 0); + this.longPrefixRowMap.put(prefix, newRow); + + newRow.setLpIndex(0); // unused + + return newRow; + } + + /** + * Identify transitions that have not been closed, i.e., there is no state with the same suffix behavior. Also + * removes unused content ids. + * + * @return the list of unclosed transition, in a deterministic order + */ + List>>> findUnclosedTransitions() { + // Identify contentIds for locations: + Set spContentIds = new HashSet<>(this.shortPrefixRowMap.size()); + + for (RowImpl> row : this.shortPrefixRowMap.values()) { + spContentIds.add(row.getRowContentId()); + } + + // Identify ids that are not used by any SP and group them by their content id: + Map>>> lpContentMap = + new HashMap<>(HashUtil.capacity(this.longPrefixRowMap.size())); + + for (RowImpl> row : this.longPrefixRowMap.values()) { + int id = row.getRowContentId(); + + if (!spContentIds.contains(id)) { + lpContentMap.computeIfAbsent(id, k -> new ArrayList<>()).add(row); + } + } + + // Remove unused content ids: + this.rowContentMap.keySet().removeIf(key -> !(spContentIds.contains(key) || lpContentMap.containsKey(key))); + + return new ArrayList<>(lpContentMap.values()); + } + + List>>> initialize(List>> initialShortPrefixes, + List>> initialSuffixes, + TimedQueryOracle oracle) { + + assert this.shortPrefixRowMap.isEmpty() && this.longPrefixRowMap.isEmpty() && initialShortPrefixes.isEmpty(); + + // Add initial suffixes: + for (Word> suffix : initialSuffixes) { + if (suffixSet.add(suffix)) { + suffixes.add(suffix); + } + } + + // 1. Create initial location: + RowImpl> newLoc = this.addInitialLocation(); + this.initLocation(newLoc, oracle); + this.queryAllSuffixes(Collections.singleton(newLoc), oracle); + + // 2. Identify unclosed transitions: + return this.findUnclosedTransitions(); + } + + private void queryAllSuffixes(Collection>> rows, TimedQueryOracle timedOracle) { + + int numSuffixes = this.suffixes.size(); + List, Word>>> queries = new ArrayList<>(rows.size() * numSuffixes); + + for (RowImpl> row : rows) { + Word> prefix = row.getLabel(); + + for (Word> suffix : this.suffixes) { + queries.add(new DefaultQuery<>(prefix, suffix)); + } + } + + timedOracle.processQueries(queries); + Iterator, Word>>> iter = queries.iterator(); + + for (RowImpl> row : rows) { + List>> outputs = new ArrayList<>(numSuffixes); + fetchResults(iter, outputs, numSuffixes); + + this.processSuffixOutputs(row, outputs); + } + } + + private void processSuffixOutputs(RowImpl> row, List>> rowContents) { + if (rowContents.isEmpty()) { + row.setRowContentId(NO_CONTENT); + return; + } + + RowContent content = new RowContent<>(rowContents); + int contentId = content.hashCode(); + this.rowContentMap.putIfAbsent(contentId, content); + row.setRowContentId(contentId); + } + + private static void fetchResults(Iterator> queryIt, List output, int numSuffixes) { + for (int j = 0; j < numSuffixes; j++) { + DefaultQuery qry = queryIt.next(); + output.add(qry.getOutput()); + } + } + + List>>> addSuffixes(Collection>> newSuffixes, + TimedQueryOracle oracle) { + // 1. Extend current suffixes + identify new suffixes: + int numOld = this.suffixes.size(); + for (Word> suffix : newSuffixes) { + if (this.suffixSet.add(suffix)) { + LOGGER.debug("Adding new suffix '{}'", suffix); + this.suffixes.add(suffix); + } + } + int numNew = this.suffixes.size(); + + if (numOld == numNew) { + return Collections.emptyList(); + } + + // 2. Update row content: + int numNewSuffixes = numNew - numOld; + List, Word>>> queries = + new ArrayList<>(numNewSuffixes * numberOfRows()); + Iterable>> rows = + IterableUtil.concat(shortPrefixRowMap.values(), longPrefixRowMap.values()); + List>> newSuffixList = this.suffixes.subList(numOld, numNew); + + for (RowImpl> row : rows) { + for (Word> suffix : newSuffixList) { + queries.add(new DefaultQuery<>(row.getLabel(), suffix)); + } + } + + oracle.processQueries(queries); + Iterator, Word>>> iterator = queries.iterator(); + + for (RowImpl> row : rows) { + List>> updatedOutputs = new ArrayList<>(numNew); + if (row.getRowContentId() != NO_CONTENT) { + // Add existing suffix outputs: + updatedOutputs.addAll(rowContents(row)); + } + + fetchResults(iterator, updatedOutputs, numNewSuffixes); + this.processSuffixOutputs(row, updatedOutputs); + } + + return this.findUnclosedTransitions(); + } + + List>>> toShortPrefixes(List>> lpRows, TimedQueryOracle oracle) { + for (Row> row : lpRows) { + LOGGER.debug("Adding new location with prefix '{}'", row.getLabel()); + + final RowImpl> lpRow = (RowImpl>) row; + + // Delete from LP rows: + this.longPrefixRowMap.remove(row.getLabel()); + + // Add to SP rows: + this.shortPrefixRowMap.put(row.getLabel(), lpRow); + + lpRow.makeShort(alphabet.size()); + + this.initLocation(lpRow, oracle); + } + return this.findUnclosedTransitions(); + } + + @Override + public Alphabet> getInputAlphabet() { + return this.alphabet; + } + + @Override + public Collection>> getShortPrefixRows() { + return Collections.unmodifiableCollection(this.shortPrefixRowMap.values()); + } + + @Override + public Collection>> getLongPrefixRows() { + return Collections.unmodifiableCollection(this.longPrefixRowMap.values()); + } + + @Override + public Row> getRow(int idx) { + throw new UnsupportedOperationException("Not supported. Use prefix to access rows instead."); + } + + @Override + public @Nullable Row> getRow(Word> prefix) { + if (this.shortPrefixRowMap.containsKey(prefix)) { + return this.shortPrefixRowMap.get(prefix); + } + if (this.longPrefixRowMap.containsKey(prefix)) { + return this.longPrefixRowMap.get(prefix); + } + return null; + } + + @Override + public int numberOfDistinctRows() { + return this.rowContentMap.size(); + } + + @Override + public List>> getSuffixes() { + return this.suffixes; + } + + @Override + public List>> rowContents(Row> row) { + if (this.rowContentMap.isEmpty()) { + // OT may be empty if only single location with timers: + assert this.suffixes.isEmpty(); + return Collections.emptyList(); + } + + return this.rowContentMap.get(row.getRowContentId()).outputs(); + } + + @Override + public Word> transformAccessSequence(Word> word) { + throw new UnsupportedOperationException("Not implemented."); + } + + @Nullable TimerInfo getTimerInfo(Word> prefix, long initial) { + LocationTimerInfo info = this.timerInfoMap.get(prefix); + if (info != null) { + return info.getTimerInfo(initial); + } + return null; + } + + @Nullable LocationTimerInfo getLocationTimerInfo(Row> sp) { + return this.timerInfoMap.getOrDefault(sp.getLabel(), null); + } + + /** + * Adds an outgoing transition for the given symbol to the given location and subsequently tests for unclosed + * transitions. + *

+ * Raises an error if this transition already exists. + * + * @param spRow + * Source location + * @param symbol + * Input symbol + * @param timeOracle + * Oracle + * + * @return List of unclosed rows. Empty, if none. + */ + List>>> addOutgoingTransition(Row> spRow, + TimedInput symbol, + TimedQueryOracle timeOracle) { + if (!this.alphabet.containsSymbol(symbol)) { + throw new IllegalArgumentException("Unknown symbol."); + } + + Word> transitionPrefix = spRow.getLabel().append(symbol); + + // Add long-prefix row: + assert this.getRow(transitionPrefix) == null : + "Location already has an outgoing transition for the provided symbol"; + + RowImpl> succRow = this.createLpRow(transitionPrefix); + + // Set as successor: + int symIdx = this.alphabet.getSymbolIndex(symbol); + ((RowImpl>) spRow).setSuccessor(symIdx, succRow); + + // Update suffixes: + this.queryAllSuffixes(Collections.singleton(succRow), timeOracle); + + return this.findUnclosedTransitions(); + } + + List>>> addTimerTransition(Row> spRow, + TimerInfo timeout, + TimedQueryOracle timeOracle) { + return this.addOutgoingTransition(spRow, new TimeStepSequence<>(timeout.initial()), timeOracle); + } + + /** + * Removes a long prefix row. Should only be used when removing a transition of a former one-shot timer. When + * turning a long into a short prefix, use toShortPrefix instead, + * + * @param prefix + * Row prefix + */ + void removeLpRow(Word> prefix) { + if (!this.longPrefixRowMap.containsKey(prefix)) { + throw new IllegalArgumentException("Attempting to remove lp row that does not exist."); + } + + // Remove lp row: + this.longPrefixRowMap.remove(prefix); + + // Unset as successor: + int symIdx = this.alphabet.getSymbolIndex(prefix.lastSymbol()); + RowImpl> spRow = this.shortPrefixRowMap.get(prefix.prefix(-1)); + assert spRow != null; + + spRow.setSuccessor(symIdx, null); + } + + // ============================= + + private record RowContent(List>> outputs) {} + +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/ExtendedDecomposition.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/ExtendedDecomposition.java new file mode 100644 index 0000000000..ff354d0160 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/ExtendedDecomposition.java @@ -0,0 +1,58 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex; + +import net.automatalib.automaton.mmlt.State; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * An extended decomposition represents a transition with an incorrect target or output in the expanded form of a + * hypothesis MMLT. + * + * @param state + * source state in expanded form of hypothesis + * @param input + * input of some transition with incorrect output or target source state + * @param discriminator + * discriminator for identifying an incorrect target state (may be {@code null} if the decomposition identifies + * an incorrect output only) + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +record ExtendedDecomposition(State state, TimedInput input, + @Nullable Word> discriminator) { + + ExtendedDecomposition(State state, TimedInput input) { + this(state, input, null); + } + + boolean isForIncorrectOutput() { + return this.discriminator == null; + } + + @Override + public String toString() { + if (this.isForIncorrectOutput()) { + return String.format("Incorrect output (%s|%s)", state, input); + } else { + return String.format("Incorrect target (%s|%s|%s)", state, input, discriminator); + } + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTCounterexampleDecomposer.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTCounterexampleDecomposer.java new file mode 100644 index 0000000000..80bcbbcafb --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTCounterexampleDecomposer.java @@ -0,0 +1,172 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex; + +import de.learnlib.acex.AcexAnalyzer; +import de.learnlib.algorithm.lstar.mmlt.MMLTHypothesis; +import de.learnlib.oracle.TimedQueryOracle; +import net.automatalib.automaton.mmlt.State; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements the search for an extended decomposition of a truncated counterexample and the post-processing of an + * extended decomposition. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +class MMLTCounterexampleDecomposer { + + private static final Logger LOGGER = LoggerFactory.getLogger(MMLTCounterexampleDecomposer.class); + + private final TimedQueryOracle timeOracle; + private final AcexAnalyzer acexAnalyzer; + + MMLTCounterexampleDecomposer(TimedQueryOracle timeOracle, AcexAnalyzer acexAnalyzer) { + this.timeOracle = timeOracle; + this.acexAnalyzer = acexAnalyzer; + } + + ExtendedDecomposition findExtendedDecomposition(MMLTOutputInconsistency outIncons, + MMLTHypothesis hypothesis) { + + if (outIncons.suffix().length() == 1) { + // Incorrect output: + State prefixState = hypothesis.getSemantics().getState(outIncons.prefix()); + return new ExtendedDecomposition<>(prefixState, outIncons.suffix().firstSymbol()); + } + + // Verify breakpoint condition: + MMLTInconsPrefixTransformAcex acex = new MMLTInconsPrefixTransformAcex<>(outIncons.suffix(), + timeOracle, + w -> hypothesis.getPrefix( + outIncons.prefix() + .concat(w))); + + if (acex.testEffects(0, acex.getLength() - 1)) { + // Breakpoint condition not met -> must be incorrect output: + Word> lastStatePrefix = + outIncons.prefix().concat(outIncons.suffix().prefix(outIncons.suffix().length() - 1)); + State lastState = hypothesis.getSemantics().getState(lastStatePrefix); + + return new ExtendedDecomposition<>(lastState, outIncons.suffix().lastSymbol()); + } + + // Breakpoint condition met -> find decomposition: + int breakpoint = this.acexAnalyzer.analyzeAbstractCounterexample(acex); + + assert !acex.testEffects(breakpoint, breakpoint + 1) : "Failed to find valid decomposition."; + + // Get components: + Word> prefix = outIncons.prefix().concat(outIncons.suffix().prefix(breakpoint)); + TimedInput sym = outIncons.suffix().getSymbol(breakpoint); + Word> discriminator = outIncons.suffix().subWord(breakpoint + 1); + + State prefixState = hypothesis.getSemantics().getState(prefix); + + LOGGER.debug(""" + Decomposing to {}|{}|{} + Output at {}: {}. + Output at {}: {} + """, + prefixState, + sym, + discriminator, + breakpoint, + acex.computeEffect(breakpoint), + breakpoint + 1, + acex.computeEffect(breakpoint + 1)); + + return new ExtendedDecomposition<>(prefixState, sym, discriminator); + } + + /** + * Post-processes an extended decomposition: if the decomposition corresponds to a transition with an incorrect + * target or output at a timeout symbol, transforms the decomposition so that the input is either a non-delaying + * input or a single time step. + * + * @param decomposition + * extended decomposition + * + * @return post-processed decomposition + */ + ExtendedDecomposition postProcessExtendedDecomposition(ExtendedDecomposition decomposition, + MMLTHypothesis hypothesis) { + if (!(decomposition.input() instanceof TimeoutSymbol)) { + return decomposition; + } + + Word> statePrefix = hypothesis.getPrefix(decomposition.state()); + Word> hypOutput = + hypothesis.getSemantics().computeSuffixOutput(statePrefix, Word.fromLetter(decomposition.input())); + Word> sulOutput = timeOracle.answerQuery(statePrefix, Word.fromLetter(decomposition.input())); + + if (decomposition.isForIncorrectOutput()) { + assert hypOutput.firstSymbol().delay() != 0 || sulOutput.firstSymbol().delay() == 0; + + // Incorrect output at tout: + long minWaitTime; + if (hypOutput.firstSymbol().delay() != 0 && sulOutput.firstSymbol().delay() == 0) { + // If there is no timeout in either hyp or sul, need to trigger next observable timeout: + minWaitTime = hypOutput.firstSymbol().delay(); + } else { + // If there is a timeout in hyp and sul, go to next timeout: + minWaitTime = Math.min(hypOutput.firstSymbol().delay(), sulOutput.firstSymbol().delay()); + } + + // if minimum time is zero (= no timeout) or one, need to append empty word to prefix: + State newPrefixState; + if (minWaitTime <= 1) { + newPrefixState = decomposition.state(); + } else { + newPrefixState = + hypothesis.getSemantics().getState(statePrefix.append(TimedInput.step(minWaitTime - 1))); + } + + LOGGER.debug("Updated incorrect output at tout during post-processing."); + return new ExtendedDecomposition<>(newPrefixState, TimedInput.step()); + } else { + if (decomposition.state().isStableConfig() || hypOutput.equals(sulOutput)) { + // stable-configuration or same output at tout -> same wait time for output in hyp and SUL: + assert hypOutput.firstSymbol().delay() == sulOutput.firstSymbol().delay(); + + long waitTime = hypOutput.firstSymbol().delay(); + State newPrefixState; + if (waitTime <= 1) { + newPrefixState = decomposition.state(); + } else { + newPrefixState = + hypothesis.getSemantics().getState(statePrefix.append(TimedInput.step(waitTime - 1))); + } + + LOGGER.debug("Updated incorrect target at tout during post-processing."); + return new ExtendedDecomposition<>(newPrefixState, TimedInput.step(), decomposition.discriminator()); + } else { + // different output at tout -> found incorrect output: + LOGGER.debug("Found incorrect output through post-processing."); + return postProcessExtendedDecomposition(new ExtendedDecomposition<>(decomposition.state(), + decomposition.input()), hypothesis); + } + } + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTCounterexampleHandler.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTCounterexampleHandler.java new file mode 100644 index 0000000000..c8a641c8db --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTCounterexampleHandler.java @@ -0,0 +1,214 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex; + +import java.util.List; + +import de.learnlib.acex.AcexAnalyzer; +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLT; +import de.learnlib.algorithm.lstar.mmlt.MMLTHypothesis; +import de.learnlib.algorithm.lstar.mmlt.cex.results.CexAnalysisResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.FalseIgnoreResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.MissingDiscriminatorResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.MissingOneShotResult; +import de.learnlib.algorithm.lstar.mmlt.cex.results.MissingResetResult; +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.SymbolFilter; +import de.learnlib.oracle.TimedQueryOracle; +import net.automatalib.automaton.impl.CompactTransition; +import net.automatalib.automaton.mmlt.State; +import net.automatalib.automaton.mmlt.TimerInfo; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Processes a truncated counterexample for a hypothesis MMLT: searches for an extended decomposition, post-processes + * it, and infers an inaccuracy from it. + * + * @param input + * symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MMLTCounterexampleHandler { + + private static final Logger LOGGER = LoggerFactory.getLogger(MMLTCounterexampleHandler.class); + + protected final TimedQueryOracle timeOracle; + private final MMLTCounterexampleDecomposer decomposer; + private final SymbolFilter, InputSymbol> symbolFilter; + + public MMLTCounterexampleHandler(TimedQueryOracle timeOracle, + AcexAnalyzer acexAnalyzer, + SymbolFilter, InputSymbol> symbolFilter) { + this.timeOracle = timeOracle; + this.decomposer = new MMLTCounterexampleDecomposer<>(timeOracle, acexAnalyzer); + this.symbolFilter = symbolFilter; + } + + public CexAnalysisResult analyzeInconsistency(MMLTOutputInconsistency outIncons, + MMLTHypothesis hypothesis) { + + // Search for an extended decomposition: + ExtendedDecomposition decomposition = decomposer.findExtendedDecomposition(outIncons, hypothesis); + LOGGER.debug("Found an extended decomposition: {}", decomposition); + + // Post-process the decomposition: + decomposition = decomposer.postProcessExtendedDecomposition(decomposition, hypothesis); + LOGGER.debug("Post-processed decomposition: {}", decomposition); + + if (decomposition.isForIncorrectOutput()) { + return handleIncorrectOutput(decomposition, hypothesis); + } else { + return handleIncorrectTarget(decomposition, hypothesis); + } + } + + private CexAnalysisResult handleIncorrectOutput(ExtendedDecomposition decomposition, + MMLTHypothesis hypothesis) { + // Transition with incorrect output always implies missing one-shot timer: + LOGGER.debug("Found missing one-shot via incorrect output."); + return this.selectOneShotTimer(decomposition, hypothesis, decomposition.state().getEntryDistance()); + } + + private CexAnalysisResult handleIncorrectTarget(ExtendedDecomposition decomposition, + MMLTHypothesis hypothesis) { + if (decomposition.input() instanceof InputSymbol ndi) { + // If decomposition at non-delaying input + considered as self-loop, treat as false ignore: + if (symbolFilter.query(hypothesis.getLocationPrefix(decomposition.state()), ndi) == FilterResponse.IGNORE) { + return new FalseIgnoreResult<>(decomposition.state().getLocation(), ndi); + } + + return this.handleIncorrectTargetNonDelaying(decomposition, hypothesis); + + } else if (decomposition.input() instanceof TimeStepSequence) { + return this.handleIncorrectTargetTimeStep(decomposition, hypothesis); + } else { + throw new AssertionError("Unexpected symbol type."); + } + } + + private CexAnalysisResult selectOneShotTimer(ExtendedDecomposition decomposition, + MMLTHypothesis hypothesis, + long maxInitialValue) { + List> timers = hypothesis.getSortedTimers(decomposition.state().getLocation()); + int newOneShot = ExtensibleLStarMMLT.selectOneShotTimer(timers, maxInitialValue); + TimerInfo timer = timers.get(newOneShot); + LOGGER.debug("Missing one-shot: setting ({}|{}) to one-shot.", + hypothesis.getLocationPrefix(decomposition.state()), + timer); + return new MissingOneShotResult<>(decomposition.state().getLocation(), timer); + } + + private CexAnalysisResult handleIncorrectTargetTimeStep(ExtendedDecomposition decomposition, + MMLTHypothesis hypothesis) { + // Check if there is a one-shot timer expiring at the next time step: + List> localTimers = hypothesis.getSortedTimers(decomposition.state().getLocation()); + assert !localTimers.isEmpty(); + + // If location has a one-shot timer, this is the one with the highest initial value: + TimerInfo lastTimer = localTimers.get(localTimers.size() - 1); + if (!lastTimer.periodic()) { + assert lastTimer.initial() - 1 == decomposition.state().getEntryDistance() : + "Incorrect target must be at timeout of non-periodic timer."; + LOGGER.debug("Inferred missing discriminator at timeout."); + return new MissingDiscriminatorResult<>(decomposition.state().getLocation(), + decomposition.input(), + decomposition.discriminator()); + } else if (!decomposition.state().isStableConfig()) { + LOGGER.debug("Found missing one-shot via incorrect target in non-stable config."); + return this.selectOneShotTimer(decomposition, hypothesis, decomposition.state().getEntryDistance()); + } else { + LOGGER.debug("Found missing one-shot via incorrect target in stable config."); + return new MissingOneShotResult<>(decomposition.state().getLocation(), + localTimers.get(0)); // lowest initial value + } + } + + private CexAnalysisResult handleIncorrectTargetNonDelaying(ExtendedDecomposition decomposition, + MMLTHypothesis hypothesis) { + // 1: can be a missing discriminator? + + // Check if correct target in entry w.r.t. discriminator: + Word> transPrefix = + hypothesis.getLocationPrefix(decomposition.state()).append(decomposition.input()); + State succState = hypothesis.getSemantics().getState(transPrefix); // successor state in hypothesis + Word> discriminator = decomposition.discriminator(); + + assert succState != null && discriminator != null; + + Word> actualSuffixOutput = this.timeOracle.answerQuery(transPrefix, discriminator); + Word> expSuffixOutput = + this.timeOracle.answerQuery(hypothesis.getPrefix(succState), discriminator); + + if (!actualSuffixOutput.equals(expSuffixOutput)) { + LOGGER.debug("Inferred missing discriminator at non-delaying input."); + return new MissingDiscriminatorResult<>(decomposition.state().getLocation(), + decomposition.input(), + discriminator); + } + + // 2: can be a local reset? + if (decomposition.state().isStableConfig()) { + LOGGER.debug("Inferred missing reset in stable config."); + return new MissingResetResult<>(decomposition.state().getLocation(), + (InputSymbol) decomposition.input()); + } + + // Non-stable -> explicitly test for missing reset: + boolean isLocalReset = hypothesis.isLocalReset(decomposition.state().getLocation(), + ((InputSymbol) decomposition.input()).symbol()); + CompactTransition trans = hypothesis.getTransition(decomposition.state().getLocation(), + ((InputSymbol) decomposition.input()).symbol()); + assert trans != null; + Integer successor = hypothesis.getSuccessor(trans); + + // Must loop without reset: + if (successor.equals(decomposition.state().getLocation()) && !isLocalReset) { + // Must have at least two stable configs: + TimerInfo firstTimer = hypothesis.getSortedTimers(decomposition.state().getLocation()).get(0); + if (firstTimer.initial() > 1) { + // Must not self-loop in at least one non-entry stable config: + Word> resetTransPrefix = hypothesis.getPrefix(decomposition.state()) + .append(TimedInput.step()) // prefix of first stable config that is not entry config + .append(decomposition.input()); // successor at $i$ in that config + + Word> suffix = Word.fromLetter(new TimeoutSymbol<>()); + Word> transSuffixOutput = this.timeOracle.answerQuery(resetTransPrefix, suffix); + Word> entryConfigSuffixOutput = + this.timeOracle.answerQuery(hypothesis.getLocationPrefix(decomposition.state()), suffix); + + if (transSuffixOutput.equals(entryConfigSuffixOutput)) { + LOGGER.debug("Inferred missing reset in non-stable config."); + return new MissingResetResult<>(decomposition.state().getLocation(), + (InputSymbol) decomposition.input()); + } + } + + } + + // 3: add missing local reset + LOGGER.debug("Inferred missing one-shot timer from incorrect target at non-delaying input."); + return this.selectOneShotTimer(decomposition, hypothesis, decomposition.state().getEntryDistance()); + } + +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTInconsPrefixTransformAcex.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTInconsPrefixTransformAcex.java new file mode 100644 index 0000000000..8673765c69 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTInconsPrefixTransformAcex.java @@ -0,0 +1,83 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex; + +import java.util.function.Function; + +import de.learnlib.acex.AbstractBaseCounterexample; +import de.learnlib.oracle.TimedQueryOracle; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An abstract counterexample used by the MMLT learner. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MMLTInconsPrefixTransformAcex extends AbstractBaseCounterexample>> { + + private static final Logger LOGGER = LoggerFactory.getLogger(MMLTInconsPrefixTransformAcex.class); + + private final TimedQueryOracle timeOracle; + private final Word> suffix; + + private final Function>, Word>> asTransform; + + /** + * Constructor. + * + * @param suffix + * suffix of the counterexample, i.e., the word that we analyze + * @param timeOracle + * membership oracle + * @param asTransform + * retrieves the prefix of the system state in the hypothesis addressed by a word + */ + public MMLTInconsPrefixTransformAcex(Word> suffix, + TimedQueryOracle timeOracle, + Function>, Word>> asTransform) { + super(suffix.length()); + this.timeOracle = timeOracle; + this.suffix = suffix; + this.asTransform = asTransform; + } + + @Override + public Word> computeEffect(int index) { + // Split the word at our index: + Word> prefix = this.suffix.prefix(index); // everything up to *index* (exclusive) + Word> suffix = this.suffix.subWord(index); // everything from *index* (inclusive) + + // Identify access sequence of system state for prefix: + Word> accessSequence = this.asTransform.apply(prefix); + + // Query *hypothesis state* + *suffix*: + return this.timeOracle.answerQuery(accessSequence, suffix); + } + + @Override + public boolean checkEffects(Word> eff1, Word> eff2) { + // Same behavior at different indices? + LOGGER.debug("Comparing ({}) AND ({}): {}", eff1, eff2, eff2.isSuffixOf(eff1)); + return eff2.isSuffixOf(eff1); + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTOutputInconsistency.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTOutputInconsistency.java new file mode 100644 index 0000000000..c462a3d576 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/MMLTOutputInconsistency.java @@ -0,0 +1,39 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex; + +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; + +/** + * Represents an output inconsistency used by the MMLT learner. + * + * @param prefix + * prefix + * @param suffix + * suffix input + * @param targetOut + * suffix output in SUL + * @param hypOut + * suffix output in hypothesis + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public record MMLTOutputInconsistency(Word> prefix, Word> suffix, + Word> targetOut, Word> hypOut) {} diff --git a/api/src/main/java/de/learnlib/statistic/StatisticData.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/CexAnalysisResult.java similarity index 72% rename from api/src/main/java/de/learnlib/statistic/StatisticData.java rename to algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/CexAnalysisResult.java index aa0f80c0ca..8ce8585706 100644 --- a/api/src/main/java/de/learnlib/statistic/StatisticData.java +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/CexAnalysisResult.java @@ -13,19 +13,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package de.learnlib.statistic; +package de.learnlib.algorithm.lstar.mmlt.cex.results; /** - * Common interface for statistical data. + * Outcome of a counterexample analysis. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type */ -public interface StatisticData { - - String getName(); - - String getUnit(); - - String getSummary(); - - String getDetails(); -} - +public interface CexAnalysisResult {} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/FalseIgnoreResult.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/FalseIgnoreResult.java new file mode 100644 index 0000000000..15afbbd42c --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/FalseIgnoreResult.java @@ -0,0 +1,45 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex.results; + +import net.automatalib.symbol.time.InputSymbol; + +/** + * The specified symbol is considered to be falsely ignored by the symbol filter. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class FalseIgnoreResult implements CexAnalysisResult { + + private final Integer location; + private final InputSymbol symbol; + + public FalseIgnoreResult(Integer location, InputSymbol symbol) { + this.location = location; + this.symbol = symbol; + } + + public Integer getLocation() { + return location; + } + + public InputSymbol getSymbol() { + return symbol; + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingDiscriminatorResult.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingDiscriminatorResult.java new file mode 100644 index 0000000000..265999d673 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingDiscriminatorResult.java @@ -0,0 +1,53 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex.results; + +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; + +/** + * The target at the identified transition is incorrect due to a missing discriminator. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MissingDiscriminatorResult implements CexAnalysisResult { + + private final Integer location; + private final TimedInput input; + private final Word> discriminator; + + public MissingDiscriminatorResult(Integer location, TimedInput input, Word> discriminator) { + this.location = location; + this.input = input; + this.discriminator = discriminator; + } + + public Integer getLocation() { + return location; + } + + public TimedInput getInput() { + return input; + } + + public Word> getDiscriminator() { + return discriminator; + } + +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingOneShotResult.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingOneShotResult.java new file mode 100644 index 0000000000..0a73803ec8 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingOneShotResult.java @@ -0,0 +1,45 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex.results; + +import net.automatalib.automaton.mmlt.TimerInfo; + +/** + * The provided timer should become one-shot. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MissingOneShotResult implements CexAnalysisResult { + + private final Integer location; + private final TimerInfo timeout; + + public MissingOneShotResult(Integer location, TimerInfo timeout) { + this.location = location; + this.timeout = timeout; + } + + public Integer getLocation() { + return location; + } + + public TimerInfo getTimeout() { + return timeout; + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingResetResult.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingResetResult.java new file mode 100644 index 0000000000..5eb8da01c2 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/cex/results/MissingResetResult.java @@ -0,0 +1,46 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.cex.results; + +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; + +/** + * There should be a local reset at the specified transition. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MissingResetResult implements CexAnalysisResult { + + private final Integer location; + private final InputSymbol input; + + public MissingResetResult(Integer location, InputSymbol input) { + this.location = location; + this.input = input; + } + + public Integer getLocation() { + return location; + } + + public TimedInput getInput() { + return input; + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTPerfectSymbolFilter.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTPerfectSymbolFilter.java new file mode 100644 index 0000000000..5fd71ff054 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTPerfectSymbolFilter.java @@ -0,0 +1,46 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.filter; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.symbol.AbstractPerfectSymbolFilter; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; + +/** + * A symbol filter for MMLTs that correctly accepts and ignores all transitions that silently self-loop. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MMLTPerfectSymbolFilter extends AbstractPerfectSymbolFilter, InputSymbol> { + + private final MMLT automaton; + + public MMLTPerfectSymbolFilter(MMLT automaton) { + this.automaton = automaton; + } + + @Override + protected FilterResponse isIgnorable(Word> prefix, InputSymbol symbol) { + return MMLTSymbolFilterUtil.isIgnorable(this.automaton, prefix, symbol); + } + +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTRandomSymbolFilter.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTRandomSymbolFilter.java new file mode 100644 index 0000000000..cfd9ab4f6a --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTRandomSymbolFilter.java @@ -0,0 +1,48 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.filter; + +import java.util.Random; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.symbol.AbstractRandomSymbolFilter; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; + +/** + * A symbol filter that falsely answers a query with a specified probability. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class MMLTRandomSymbolFilter extends AbstractRandomSymbolFilter, InputSymbol> { + + private final MMLT automaton; + + public MMLTRandomSymbolFilter(MMLT automaton, double inaccurateProb, Random random) { + super(inaccurateProb, random); + this.automaton = automaton; + } + + @Override + protected FilterResponse isIgnorable(Word> prefix, InputSymbol symbol) { + return MMLTSymbolFilterUtil.isIgnorable(this.automaton, prefix, symbol); + } +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTStatisticsSymbolFilter.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTStatisticsSymbolFilter.java new file mode 100644 index 0000000000..69be8d097c --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTStatisticsSymbolFilter.java @@ -0,0 +1,41 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.filter; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.SymbolFilter; +import de.learnlib.filter.symbol.AbstractStatisticsSymbolFilter; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; + +public class MMLTStatisticsSymbolFilter extends AbstractStatisticsSymbolFilter, InputSymbol> { + + private final MMLT automaton; + + public MMLTStatisticsSymbolFilter(MMLT automaton, + SymbolFilter, InputSymbol> delegate) { + super(delegate); + this.automaton = automaton; + } + + @Override + protected FilterResponse isIgnorable(Word> prefix, InputSymbol symbol) { + return MMLTSymbolFilterUtil.isIgnorable(this.automaton, prefix, symbol); + } + +} diff --git a/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTSymbolFilterUtil.java b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTSymbolFilterUtil.java new file mode 100644 index 0000000000..37c038d3c9 --- /dev/null +++ b/algorithms/active/lstar/src/main/java/de/learnlib/algorithm/lstar/mmlt/filter/MMLTSymbolFilterUtil.java @@ -0,0 +1,53 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.mmlt.filter; + +import java.util.Objects; + +import de.learnlib.filter.FilterResponse; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.MMLTSemantics; +import net.automatalib.automaton.mmlt.State; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; + +final class MMLTSymbolFilterUtil { + + private MMLTSymbolFilterUtil() { + // prevent instantiation + } + + static FilterResponse isIgnorable(MMLT automaton, + Word> prefix, + InputSymbol symbol) { + return isIgnorable(automaton.getSemantics(), prefix, symbol); + } + + static FilterResponse isIgnorable(MMLTSemantics semantics, + Word> prefix, + InputSymbol symbol) { + State targetConfig = semantics.getState(prefix); + T trans = semantics.getTransition(targetConfig, symbol); + State target = semantics.getSuccessor(trans); + TimedOutput output = semantics.getTransitionOutput(trans); + + boolean ignorable = Objects.equals(output, semantics.getSilentOutput()) && Objects.equals(targetConfig, target); + + return ignorable ? FilterResponse.IGNORE : FilterResponse.ACCEPT; + } +} diff --git a/algorithms/active/lstar/src/main/java/module-info.java b/algorithms/active/lstar/src/main/java/module-info.java index 5ec25fb05b..ed5c9e9b54 100644 --- a/algorithms/active/lstar/src/main/java/module-info.java +++ b/algorithms/active/lstar/src/main/java/module-info.java @@ -37,6 +37,7 @@ requires de.learnlib.common.counterexample; requires de.learnlib.common.util; requires de.learnlib.datastructure; + requires de.learnlib.filter.symbol; requires net.automatalib.api; requires net.automatalib.common.util; requires net.automatalib.core; @@ -51,6 +52,10 @@ exports de.learnlib.algorithm.lstar.closing; exports de.learnlib.algorithm.lstar.dfa; exports de.learnlib.algorithm.lstar.mealy; + exports de.learnlib.algorithm.lstar.mmlt; + exports de.learnlib.algorithm.lstar.mmlt.cex; + exports de.learnlib.algorithm.lstar.mmlt.cex.results; + exports de.learnlib.algorithm.lstar.mmlt.filter; exports de.learnlib.algorithm.lstar.moore; exports de.learnlib.algorithm.malerpnueli; exports de.learnlib.algorithm.rivestschapire; diff --git a/algorithms/active/lstar/src/test/java/de/learnlib/algorithm/lstar/ExtensibleLStarMMLTCounterexampleTests.java b/algorithms/active/lstar/src/test/java/de/learnlib/algorithm/lstar/ExtensibleLStarMMLTCounterexampleTests.java new file mode 100644 index 0000000000..88e8f70574 --- /dev/null +++ b/algorithms/active/lstar/src/test/java/de/learnlib/algorithm/lstar/ExtensibleLStarMMLTCounterexampleTests.java @@ -0,0 +1,194 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar; + +import java.util.List; + +import de.learnlib.algorithm.lstar.it.ExtensibleLStarMMLTIT.Example; +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLT; +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.oracle.equivalence.mmlt.SimulatorEQOracle; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.query.DefaultQuery; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; +import org.testng.Assert; +import org.testng.annotations.Test; + +/** + * Tests several different cases of counterexamples. + */ +@Test +public class ExtensibleLStarMMLTCounterexampleTests { + + private static void learnModel(MMLT example, + MMLTModelParams params, + List>> counterexamples) { + + var sul = new MMLTSimulatorSUL<>(example); + var timeOracle = new TimedSULOracle<>(sul, params); + + var learner = new ExtensibleLStarMMLT<>(example.getInputAlphabet(), params, timeOracle); + + learner.startLearning(); + + for (var cex : counterexamples) { + var output = timeOracle.answerQuery(cex); + learner.refineHypothesis(new DefaultQuery<>(cex, output)); + } + + // Now continue until arriving at an accurate model: + SimulatorEQOracle simOracle = new SimulatorEQOracle<>(example); + + DefaultQuery, Word>> cex; + MMLT hyp = learner.getHypothesisModel(); + + while ((cex = simOracle.findCounterExample(hyp, hyp.getSemantics().getInputAlphabet())) != null) { + learner.refineHypothesis(cex); + hyp = learner.getHypothesisModel(); + } + + Assert.assertEquals(learner.getObservationTable().numberOfDistinctRows(), hyp.size()); + } + + @Test + public void testOverApproxReset() { + // Infers a missing local reset instead of a missing discriminator first. + var model = new Example("over_approx_reset.dot"); + + // Missing discriminator at non-del in stable config: + List>> cex1 = + List.of(Word.fromSymbols(TimedInput.step(), new InputSymbol<>("i"), new TimeoutSymbol<>())); + + learnModel(model.getReferenceAutomaton(), model.getParams(), cex1); + } + + @Test + public void testRecursiveDecomp() { + // Triggers recursive decomposition + var model = new Example("recursive_decomp.dot", 3); + + // Missing discriminator at non-del in stable config: + List>> cex1 = List.of(Word.upcast(TimedInput.inputs("p", "f")), + Word.fromWords(TimedInput.inputs("u"), + TimedInput.timeouts(4), + TimedInput.inputs("f")), + Word.fromWords(TimedInput.inputs("u"), TimedInput.timeouts(5))); + + learnModel(model.getReferenceAutomaton(), model.getParams(), cex1); + } + + @Test + public void testMissingDiscriminators() { + var model = MMLTExamples.sensorCollector(); + + // Missing discriminator at non-del in stable config: + List>> cex1 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromSymbols(TimedInput.input("p2"), + TimedInput.step(), + TimedInput.input("abort"), + TimedInput.timeout())); + + // Missing discriminator at one-shot: + List>> cex2 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromWords(TimedInput.inputs("p2"), TimedInput.timeouts(2))); + + learnModel(model.getReferenceAutomaton(), model.getParams(), cex1); + learnModel(model.getReferenceAutomaton(), model.getParams(), cex2); + } + + @Test + public void testMissingResets() { + var model = MMLTExamples.sensorCollector(); + var p = model.getParams(); + var params = new MMLTModelParams<>(p.silentOutput(), p.outputCombiner(), p.maxTimeoutWaitingTime(), 40); + + // Missing reset in stable config: + List>> cex1 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromSymbols(TimedInput.input("p1"), + TimedInput.step(), + TimedInput.input("abort"), + TimedInput.timeout())); + + // Missing reset in non-stable config: + List>> cex2 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromWords(TimedInput.inputs("p1"), + TimedInput.steps(3), + TimedInput.inputs("abort"), + TimedInput.timeouts(1))); + + learnModel(model.getReferenceAutomaton(), params, cex1); + learnModel(model.getReferenceAutomaton(), params, cex2); + + } + + @Test + public void testMissingOneShotModelB() { + // Setting max waiting = 6 -> all inferred timers are periodic: + var model = MMLTExamples.sensorCollector(); + var p = model.getParams(); + var params = new MMLTModelParams<>(p.silentOutput(), p.outputCombiner(), p.maxTimeoutWaitingTime(), 6); + + // Missing one-shot via bad return to entry: + List>> cex1 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromWords(TimedInput.inputs("p1"), TimedInput.timeouts(14))); + + // Missing one-shot in location with single timer: + List>> cex2 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromWords(TimedInput.inputs("p2"), TimedInput.timeouts(2))); + + learnModel(model.getReferenceAutomaton(), params, cex1); + learnModel(model.getReferenceAutomaton(), params, cex2); + } + + @Test + public void testMissingOneShotModelA() { + var model = MMLTExamples.sensorCollector(); + var p = model.getParams(); + var params = new MMLTModelParams<>(p.silentOutput(), p.outputCombiner(), p.maxTimeoutWaitingTime(), 40); + + // Missing one-shot via bad output: + List>> cex1 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromWords(TimedInput.inputs("p1"), + TimedInput.steps(40), + TimedInput.timeouts(1)) + ); + + // Missing one-shot via bad target: + List>> cex2 = List.of(Word.upcast(TimedInput.inputs("p1", "p1")), + Word.upcast(TimedInput.inputs("p2", "abort")), + Word.fromWords(TimedInput.inputs("p1"), + TimedInput.timeouts(14), + TimedInput.inputs("collect", "p1"))); + + learnModel(model.getReferenceAutomaton(), params, cex1); + learnModel(model.getReferenceAutomaton(), params, cex2); + } + +} diff --git a/algorithms/active/lstar/src/test/java/de/learnlib/algorithm/lstar/it/ExtensibleLStarMMLTIT.java b/algorithms/active/lstar/src/test/java/de/learnlib/algorithm/lstar/it/ExtensibleLStarMMLTIT.java new file mode 100644 index 0000000000..dbb4fc6b8f --- /dev/null +++ b/algorithms/active/lstar/src/test/java/de/learnlib/algorithm/lstar/it/ExtensibleLStarMMLTIT.java @@ -0,0 +1,185 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.algorithm.lstar.it; + +import java.io.IOException; +import java.io.InputStream; +import java.net.URISyntaxException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.stream.Stream; + +import de.learnlib.algorithm.lstar.closing.ClosingStrategies; +import de.learnlib.algorithm.lstar.closing.ClosingStrategy; +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLTBuilder; +import de.learnlib.algorithm.lstar.mmlt.filter.MMLTPerfectSymbolFilter; +import de.learnlib.algorithm.lstar.mmlt.filter.MMLTRandomSymbolFilter; +import de.learnlib.filter.SymbolFilter; +import de.learnlib.filter.symbol.AcceptAllSymbolFilter; +import de.learnlib.filter.symbol.CachedSymbolFilter; +import de.learnlib.filter.symbol.IgnoreAllSymbolFilter; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.testsupport.example.LearningExample.MMLTLearningExample; +import de.learnlib.testsupport.it.learner.AbstractMMLTLearnerIT; +import de.learnlib.testsupport.it.learner.LearnerVariantList.MMLTLearnerVariantList; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.alphabet.Alphabet; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.impl.StringSymbolCombiner; +import net.automatalib.exception.FormatException; +import net.automatalib.serialization.dot.DOTParsers; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.util.automaton.mmlt.MMLTs; +import net.automatalib.word.Word; +import org.testng.annotations.Test; + +@Test +public class ExtensibleLStarMMLTIT extends AbstractMMLTLearnerIT { + + @Override + protected void addLearnerVariants(Alphabet alphabet, + TimedQueryOracle mqOracle, + MMLTLearningExample example, + MMLTLearnerVariantList variants) { + + var mmlt = example.getReferenceAutomaton(); + var counters = countTimers(mmlt); + + List>> suffixes = new ArrayList<>(); + alphabet.forEach(s -> suffixes.add(Word.fromLetter(TimedInput.input(s)))); + // Do not include TimeoutSymbol because we want to check analyzing and handling counterexamples with it + // suffixes.add(Word.fromLetter(new TimeoutSymbol<>())); + + var builder = new ExtensibleLStarMMLTBuilder().withAlphabet(alphabet) + .withModelParams(example.getParams()) + .withTimeOracle(mqOracle) + .withInitialSuffixes(suffixes); + + var filters = Arrays.asList(new MMLTPerfectSymbolFilter<>(mmlt), + new MMLTRandomSymbolFilter<>(mmlt, 0.1, new Random(42)), + new IgnoreAllSymbolFilter, InputSymbol>(), + new AcceptAllSymbolFilter, InputSymbol>()); + + for (ClosingStrategy, ? super Word>> strategy : ClosingStrategies.values()) { + builder.setClosingStrategy(strategy); + for (SymbolFilter, InputSymbol> filter : filters) { + + var cachedFilter = new CachedSymbolFilter<>(filter); // need to wrap to enable updates to responses + var learner = builder.withSymbolFilter(cachedFilter).create(); + variants.addLearnerVariant("system=" + example + ",strategy=" + strategy + ",filter=" + filter, + learner, + counters + mmlt.size()); + } + } + } + + @Override + protected List> getAdditionalLearningExamples() { + var modelFiles = listModelFiles(); + var result = new ArrayList>(modelFiles.size()); + + for (String modelFile : modelFiles) { + result.add(new Example(modelFile)); + } + + return result; + } + + private static int countTimers(MMLT mmlt) { + int cntr = 0; + + for (S s : mmlt) { + cntr += mmlt.getSortedTimers(s).size(); + } + + return cntr; + } + + private static List listModelFiles() { + var models = new ArrayList(); + try { + var modelFiles = ExtensibleLStarMMLTIT.class.getResource("/mmlt"); + if (modelFiles != null) { + try (Stream paths = Files.list(Paths.get(modelFiles.toURI()))) { + paths.filter(p -> p.toString().endsWith(".dot")) + .map(p -> p.getFileName().toString()) + .forEach(models::add); + } + } + } catch (IOException | URISyntaxException e) { + throw new RuntimeException("Failed to list model files", e); + } + return models; + } + + public static class Example implements MMLTLearningExample { + + private final String name; + private final MMLT mmlt; + private final MMLTModelParams params; + + public Example(String name) { + this(name, -1); + } + + public Example(String name, int maxTimerQueryWaiting) { + this.name = name; + + var silentOutput = "void"; + var outputCombiner = StringSymbolCombiner.getInstance(); + var parser = DOTParsers.mmlt(silentOutput, outputCombiner); + + try (InputStream is = ExtensibleLStarMMLTIT.class.getResourceAsStream("/mmlt/" + name)) { + var model = parser.readModel(is); + var automaton = model.model; + + long maxTimeoutDelay = MMLTs.getMaximumTimeoutDelay(automaton); + long maxTimerQueryWaitingFinal = (maxTimerQueryWaiting > 0) ? + maxTimerQueryWaiting : + MMLTs.getMaximumInitialTimerValue(automaton) * 2; + + this.mmlt = automaton; + this.params = + new MMLTModelParams<>(silentOutput, outputCombiner, maxTimeoutDelay, maxTimerQueryWaitingFinal); + } catch (IOException | FormatException e) { + throw new RuntimeException("Unable to load model " + name, e); + } + } + + @Override + public MMLTModelParams getParams() { + return this.params; + } + + @Override + public MMLT getReferenceAutomaton() { + return this.mmlt; + } + + @Override + public String toString() { + return this.name; + } + } + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/ambiguous_minimal.dot b/algorithms/active/lstar/src/test/resources/mmlt/ambiguous_minimal.dot new file mode 100644 index 0000000000..115f68259a --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/ambiguous_minimal.dot @@ -0,0 +1,21 @@ +// In this example, the learner infers an equivalent MMLT +// that uses more timers per location but still has the same number of locations. +digraph g { + + s0 [shape="circle"]; + s1 [shape="circle" timers="a=2"]; + s2 [shape="circle" timers="b=1"]; + s3 [shape="circle"]; + s4 [shape="circle"]; + + s1 -> s2 [label="to[a] / A"]; + s2 -> s3 [label="to[b] / B"]; + + s0 -> s1 [label="y / X"]; + s0 -> s4 [label="x / Y"]; + s4 -> s2 [label="x / Z"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/duplicate_timer_outputs.dot b/algorithms/active/lstar/src/test/resources/mmlt/duplicate_timer_outputs.dot new file mode 100644 index 0000000000..fc0ca63e7c --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/duplicate_timer_outputs.dot @@ -0,0 +1,14 @@ +// In this example, the same output is produced multiple times at the same timeouts: +digraph g { + + s0 [timers="a=3,b=3,c=6,d=10" shape="circle"]; + + s0 -> s0 [label="to[a] / A"]; + s0 -> s0 [label="to[b] / A"]; + s0 -> s0 [label="to[c] / B"]; + s0 -> s0 [label="to[d] / A|A"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/greedy_fail.dot b/algorithms/active/lstar/src/test/resources/mmlt/greedy_fail.dot new file mode 100644 index 0000000000..21d0cf0eaf --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/greedy_fail.dot @@ -0,0 +1,14 @@ +// This is an example of a model where the greedy timer inference leads to an unnecessarily large model. +digraph g { + + s0 [timers="a=3" shape="circle"]; + s1 [timers="a=2,b=5" shape="circle"]; + + s0 -> s1 [label="to[a] / a"]; + s1 -> s1 [label="to[a] / b"]; + s1 -> s1 [label="to[b] / c"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/initial_value_one.dot b/algorithms/active/lstar/src/test/resources/mmlt/initial_value_one.dot new file mode 100644 index 0000000000..3e3a799324 --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/initial_value_one.dot @@ -0,0 +1,15 @@ +// The location s0 of this MMLT has a timer with the initial value one and a local reset. +// We allow local resets only in locations with at least two stable configurations. +// Hence, the smallest accurate hypothesis MMLT for this model must have two locations. +digraph g { + + s0 [timers="x=1,y=2" shape="circle"]; + + s0 -> s0 [label="a / void" resets="x,y"]; + s0 -> s0 [label="to[x] / X"]; + s0 -> s0 [label="to[y] / Y"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/isolated_loc_permanent.dot b/algorithms/active/lstar/src/test/resources/mmlt/isolated_loc_permanent.dot new file mode 100644 index 0000000000..1a6e3e0821 --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/isolated_loc_permanent.dot @@ -0,0 +1,19 @@ +// Example of location that becomes permanently isolated +digraph g { + + s0 [timers="x=3" shape="circle"]; + s1 [timers="y=2, z=3" shape="circle"]; + s2 [shape="circle"]; + + s0 -> s0 [label="a / A"]; + s1 -> s1 [label="a / B"]; + s2 -> s2 [label="a / C"]; + + s0 -> s1 [label="to[x] / X"]; + s1 -> s1 [label="to[y] / Y"]; + s1 -> s2 [label="to[z] / Z"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/isolated_loc_temp.dot b/algorithms/active/lstar/src/test/resources/mmlt/isolated_loc_temp.dot new file mode 100644 index 0000000000..0b6e851b2e --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/isolated_loc_temp.dot @@ -0,0 +1,27 @@ +// Example of location that becomes temporarily isolated +// Tested with a whitebox oracle only. +digraph g { + + s0 [timers="x=3" shape="circle"]; + s1 [timers="y=2, z=3" shape="circle"]; + s2 [shape="circle"]; + + s4 [shape="circle"]; + s5 [timers="v=1" shape="circle"]; + + s0 -> s0 [label="a / A"]; + s1 -> s1 [label="a / B"]; + s2 -> s4 [label="a / C"]; + s4 -> s5 [label="a / C"]; + + s0 -> s1 [label="to[x] / X"]; + s1 -> s1 [label="to[y] / Y"]; + s1 -> s2 [label="to[z] / Z"]; + + s5 -> s5 [label="a / B"] + s5 -> s2 [label="to[v] / Z"] + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/missing_oneshot_high_waiting.dot b/algorithms/active/lstar/src/test/resources/mmlt/missing_oneshot_high_waiting.dot new file mode 100644 index 0000000000..bde3ff6111 --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/missing_oneshot_high_waiting.dot @@ -0,0 +1,17 @@ +// This is an example of a model where we infer a periodic timer b that cannot be periodic, even with a high maximum waiting time.l +digraph g { + + s0 [timers="a=3" shape="circle"]; + s1 [timers="b=1" shape="circle"]; + s2 [timers="c=2" shape="circle"]; + + s0 -> s1 [label="to[a] / A"]; + s1 -> s2 [label="to[b] / B"]; + s2 -> s2 [label="to[c] / C"]; + + s1 -> s1 [label="x / X"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/over_approx_reset.dot b/algorithms/active/lstar/src/test/resources/mmlt/over_approx_reset.dot new file mode 100644 index 0000000000..3e08b3fbe3 --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/over_approx_reset.dot @@ -0,0 +1,17 @@ +// Make sure to not initialize the suffixes with the input alphabet in this example +// In this example, the learner will infer a missing local reset, although there is actually a missing discriminator. +digraph g { + + s0 [timers="a=3" shape="circle"]; + s1 [timers="b=3" shape="circle"]; + + s0 -> s0 [label="to[a] / A"]; + s0 -> s1 [label="i / void"]; + + s1 -> s1 [label="to[b] / A"]; + s1 -> s1 [label="x / X"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/recursive_decomp.dot b/algorithms/active/lstar/src/test/resources/mmlt/recursive_decomp.dot new file mode 100644 index 0000000000..67120a67d8 --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/recursive_decomp.dot @@ -0,0 +1,37 @@ +// This is an example of a model where the post-processing discovers an incorrect output while processing an incorrect target. +// You need to set maximum the query time to three. +// Use "TestDissExample" to send counterexamples that trigger the expected behavior. +digraph g { + + s0 [shape="circle"]; + s1 [timers="a=2,b=3" shape="circle"]; + s2 [shape="circle"]; + s3 [timers="c=2" shape="circle"]; + s4 [timers="d=1" shape="circle"]; + s5 [timers="e=1" shape="circle"]; + s6 [timers="f=2" shape="circle"]; + + s0 -> s0 [label="f / F"] + + s0 -> s1 [label="p / P"]; + s1 -> s1 [label="to[a] / A"]; + s1 -> s2 [label="to[b] / B"]; + + s1 -> s1 [label="f / G"] + s2 -> s2 [label="f / H"] + + // ---- + + s0 -> s3 [label="u / U"]; + s3 -> s4 [label="to[c] / A"]; + s4 -> s5 [label="to[d] / B"]; + s5 -> s6 [label="to[e] / A"]; + s6 -> s7 [label="to[f] / A|B"]; + + s3 -> s3 [label="f / G"] + + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/same_initial_value.dot b/algorithms/active/lstar/src/test/resources/mmlt/same_initial_value.dot new file mode 100644 index 0000000000..17ec7ede6f --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/same_initial_value.dot @@ -0,0 +1,13 @@ +// In this example, the learner will infer a timer with a combined output. +digraph g { + + s0 [timers="a=3,b=3,c=6" shape="circle"]; + + s0 -> s0 [label="to[a] / A"]; + s0 -> s0 [label="to[b] / C"]; + s0 -> s0 [label="to[c] / B"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/algorithms/active/lstar/src/test/resources/mmlt/syntax_demo.dot b/algorithms/active/lstar/src/test/resources/mmlt/syntax_demo.dot new file mode 100644 index 0000000000..f04f42d59f --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/syntax_demo.dot @@ -0,0 +1,35 @@ +// This file demonstrates the syntax for defining a custom MMLT. +// It is identical to "mmlt_example.dot" in the "examples" module. +digraph g { + s0 [label="L0" timers="a=2"] + s1 [label="L1" timers="b=4,c=6"] + s2 [label="L2" timers="d=2,e=3"] + + // one-shot with location change: + s0 -> s1 [label="to[a] / A"] + + // periodic with multiple outputs, + // assuming a {net.automatalib.automaton.mmlt.impl.StringSymbolCombiner} to combine and separate outputs: + s1 -> s1 [label="to[b] / B|Z"] + + // one-shot with loop: + s1 -> s1 [label="to[c] / C" resets="b,c"] + + // periodic with explicit resets: + s2 -> s2 [label="to[d] / D" resets="d"] + + // periodic: + s2 -> s2 [label="to[e] / E"] + + // normal transition with silent output: + s1 -> s2 [label="x / void"] + + // loop with reset: + s1 -> s1 [label="y / Y" resets="b,c"] + + // loop without reset: + s2 -> s2 [label="y / D"] + + __start0 [label="" shape="none" width="0" height="0"]; + __start0 -> s0; +} \ No newline at end of file diff --git a/algorithms/active/lstar/src/test/resources/mmlt/unique_minimal.dot b/algorithms/active/lstar/src/test/resources/mmlt/unique_minimal.dot new file mode 100644 index 0000000000..20749808c4 --- /dev/null +++ b/algorithms/active/lstar/src/test/resources/mmlt/unique_minimal.dot @@ -0,0 +1,18 @@ +// This example demonstrates that there is no unique minimal form for an MMLT. +// Learning this model with maxQueryTime=5 yields a model where s1 has two timers and s0 has one. +// Setting maxQueryTime to 10 instead yields a model where the initial location has two timers and the following has one. +// The total number of locations, timers, max. timers per location, and average timers per location are identical. +digraph g { + + s0 [shape="circle" timers="a=5"]; + s1 [shape="circle" timers="b=3"]; + s2 [shape="circle" timers="c=2"]; + + s0 -> s1 [label="to[a] / A"]; + s1 -> s2 [label="to[b] / B"]; + s2 -> s3 [label="to[c] / C"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s0; + +} diff --git a/api/pom.xml b/api/pom.xml index 680ec2a555..368515408b 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -52,31 +52,11 @@ limitations under the License. - - - ch.qos.logback - logback-core - test - - - ch.qos.logback - logback-classic - test - - org.testng testng test - - org.mockito - mockito-core - test - diff --git a/api/src/main/java/de/learnlib/statistic/StatisticCollector.java b/api/src/main/java/de/learnlib/filter/FilterResponse.java similarity index 61% rename from api/src/main/java/de/learnlib/statistic/StatisticCollector.java rename to api/src/main/java/de/learnlib/filter/FilterResponse.java index 9919b876ab..a811a76959 100644 --- a/api/src/main/java/de/learnlib/statistic/StatisticCollector.java +++ b/api/src/main/java/de/learnlib/filter/FilterResponse.java @@ -13,20 +13,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package de.learnlib.statistic; +package de.learnlib.filter; /** - * A utility interface to indicate that the implementing class collects statistical information that may be obtained via - * its {@link #getStatisticalData()} method. + * Describes the possible responses of a {@link SymbolFilter}. */ -@FunctionalInterface -public interface StatisticCollector { - +public enum FilterResponse { /** - * Returns this statistical data gathered by this collector. - * - * @return the statistical data gathered by this collector + * Indicates that a transition is relevant and should be included in, e.g., hypothesis construction. */ - StatisticData getStatisticalData(); - + ACCEPT, + /** + * Indicates that a transition can be ignored for, e.g., hypothesis construction. + */ + IGNORE } diff --git a/api/src/main/java/de/learnlib/filter/MutableSymbolFilter.java b/api/src/main/java/de/learnlib/filter/MutableSymbolFilter.java new file mode 100644 index 0000000000..15e6316989 --- /dev/null +++ b/api/src/main/java/de/learnlib/filter/MutableSymbolFilter.java @@ -0,0 +1,40 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter; + +import net.automatalib.word.Word; + +/** + * A mutable extension of a {@link SymbolFilter} that allows one to override the query behavior. In particular, + * previously ignored transitions can be marked as non-ignorable. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label + */ +public interface MutableSymbolFilter extends SymbolFilter { + + /** + * Sets the response of the filter for the given transition to the provided response. + * + * @param prefix + * State prefix. + * @param symbol + * Input of the transition that should be updated. + */ + void accept(Word prefix, V symbol); +} diff --git a/api/src/main/java/de/learnlib/filter/SymbolFilter.java b/api/src/main/java/de/learnlib/filter/SymbolFilter.java new file mode 100644 index 0000000000..47f7e9770c --- /dev/null +++ b/api/src/main/java/de/learnlib/filter/SymbolFilter.java @@ -0,0 +1,50 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter; + +import net.automatalib.word.Word; + +/** + * A symbol filter allows one to incorporate additional external knowledge by predicting whether a given transition + * (identified by an input symbol) is ignorable in a given state (identified by an access sequence). Ignorable + * typically means that the symbol triggers a (silent) self-loop in the considered state. This information can be used + * by, e.g., learning algorithms to avoid posing redundant queries. + *

+ * Note that a symbol filter is not required to answer queries correctly. In particular, an initially ignored transition + * may turn out to be relevant to the system behavior. As a result, learners that want to support these kinds of + * semantics need to be able to handle the potentially resulting nondeterministic query behavior. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label + */ +@FunctionalInterface +public interface SymbolFilter { + + /** + * Predicts whether the provided symbol is ignorable in the state that is addressed by the given prefix. + * + * @param prefix + * the prefix identifying the state + * @param symbol + * the input symbol identifying the transition + * + * @return {@link FilterResponse#IGNORE} if the symbol is considered ignorable, {@link FilterResponse#ACCEPT} + * otherwise + */ + FilterResponse query(Word prefix, V symbol); +} diff --git a/api/src/main/java/de/learnlib/oracle/EquivalenceOracle.java b/api/src/main/java/de/learnlib/oracle/EquivalenceOracle.java index ee8b75e88d..dc7928116b 100644 --- a/api/src/main/java/de/learnlib/oracle/EquivalenceOracle.java +++ b/api/src/main/java/de/learnlib/oracle/EquivalenceOracle.java @@ -19,8 +19,11 @@ import de.learnlib.query.DefaultQuery; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; import net.automatalib.word.Word; import org.checkerframework.checker.nullness.qual.Nullable; @@ -92,4 +95,15 @@ interface MealyEquivalenceOracle extends EquivalenceOracle extends EquivalenceOracle, I, Word> {} + /** + * A specialization of the {@link EquivalenceOracle} interface for an {@link MMLT} learning scenario. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ + @FunctionalInterface + interface MMLTEquivalenceOracle + extends EquivalenceOracle, TimedInput, Word>> {} } diff --git a/api/src/main/java/de/learnlib/statistic/StatisticOracle.java b/api/src/main/java/de/learnlib/oracle/ParallelTimedQueryOracle.java similarity index 70% rename from api/src/main/java/de/learnlib/statistic/StatisticOracle.java rename to api/src/main/java/de/learnlib/oracle/ParallelTimedQueryOracle.java index 6a9a6cab00..6f32e3df03 100644 --- a/api/src/main/java/de/learnlib/statistic/StatisticOracle.java +++ b/api/src/main/java/de/learnlib/oracle/ParallelTimedQueryOracle.java @@ -13,17 +13,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -package de.learnlib.statistic; - -import de.learnlib.oracle.MembershipOracle; +package de.learnlib.oracle; /** - * Common interface for oracles keeping statistics. + * {@link ParallelOracle} equivalent for {@link OmegaMembershipOracle}s. * + * @param + * oracle state type * @param - * input symbol class + * input symbol type * @param - * output domain class + * output domain type */ -public interface StatisticOracle extends MembershipOracle, StatisticCollector {} +public interface ParallelTimedQueryOracle extends ThreadPool, TimedQueryOracle {} diff --git a/api/src/main/java/de/learnlib/oracle/SingleQueryOracle.java b/api/src/main/java/de/learnlib/oracle/SingleQueryOracle.java index ba940dccbb..a67973d077 100644 --- a/api/src/main/java/de/learnlib/oracle/SingleQueryOracle.java +++ b/api/src/main/java/de/learnlib/oracle/SingleQueryOracle.java @@ -18,6 +18,8 @@ import java.util.Collection; import de.learnlib.query.Query; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; import net.automatalib.word.Word; /** @@ -58,4 +60,7 @@ interface SingleQueryOracleMealy extends SingleQueryOracle>, Me @FunctionalInterface interface SingleQueryOracleMoore extends SingleQueryOracle>, MooreMembershipOracle {} + interface SingleQueryOracleMMLT + extends SingleQueryOracle, Word>>, TimedQueryOracle {} + } diff --git a/api/src/main/java/de/learnlib/oracle/TimedQueryOracle.java b/api/src/main/java/de/learnlib/oracle/TimedQueryOracle.java new file mode 100644 index 0000000000..ab93843d87 --- /dev/null +++ b/api/src/main/java/de/learnlib/oracle/TimedQueryOracle.java @@ -0,0 +1,61 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle; + +import java.util.List; + +import de.learnlib.oracle.MembershipOracle.MealyMembershipOracle; +import net.automatalib.automaton.mmlt.TimerInfo; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; + +/** + * An oracle for querying {@link TimedInput timed inputs} and timers by observing timeouts. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public interface TimedQueryOracle extends MealyMembershipOracle, TimedOutput> { + + /** + * Observes and aggregates any timeouts that occur after providing the given input to the SUL. Stops when observing + * inconsistent behavior. + * + * @param prefix + * input to give to the SUL. + * @param maxTotalWaitingTime + * maximum total time that is waited for timeouts. + * + * @return observed timeouts (may be empty) + */ + TimerQueryResult queryTimers(Word> prefix, long maxTotalWaitingTime); + + /** + * Response for a timer query. + * + * @param aborted + * {@code true} if query was aborted due to missing timeout, {@code false} otherwise. + * @param timers + * identified timers + * @param + * output symbol type + */ + record TimerQueryResult(boolean aborted, List> timers) {} + +} diff --git a/api/src/main/java/de/learnlib/query/Query.java b/api/src/main/java/de/learnlib/query/Query.java index 0eab8d69ce..50561ccd76 100644 --- a/api/src/main/java/de/learnlib/query/Query.java +++ b/api/src/main/java/de/learnlib/query/Query.java @@ -25,8 +25,8 @@ import org.checkerframework.checker.nullness.qual.Nullable; /** - * A query is the basic form of interaction between a {@link LearningAlgorithm learner} and a {@link MembershipOracle - * (membership) oracle}, or teacher. + * A query is the basic form of interaction between a {@link LearningAlgorithm learner} and a + * {@link MembershipOracle (membership) oracle}, or teacher. *

* In LearnLib, queries are performed in a callback-like fashion: an oracle does not return the responses to the * queries, but rather invokes the {@link #answer(Object)} method on the query objects it was provided with. This allows @@ -35,9 +35,9 @@ * is no need for maintaining a common (synchronized) result data structure such as a map. However, this means that a * learner cannot rely on the {@link #answer(Object)} method of a query being called from the same thread which invoked * {@link MembershipOracle#processQueries(java.util.Collection)}. If this causes concurrency issues, a safe choice is to - * use queries of class {@link DefaultQuery}, which simply store the response and make it accessible via {@link - * DefaultQuery#getOutput()} for processing after the {@link MembershipOracle#processQueries(java.util.Collection)} call - * returns, guaranteeing thread-safety. + * use queries of class {@link DefaultQuery}, which simply store the response and make it accessible via + * {@link DefaultQuery#getOutput()} for processing after the + * {@link MembershipOracle#processQueries(java.util.Collection)} call returns, guaranteeing thread-safety. *

* Conceptually, a query is divided into a {@link #getPrefix() prefix} and a {@link #getSuffix()} suffix. The prefix * part of a query identifies a state in the (unknown) target system, whereas the suffix is the "experiment" which is @@ -73,8 +73,8 @@ public abstract class Query { * throwing an exception. * * @param output - * the output, i.e., the directly observable response to the query's suffix (cf. {@link Query main - * documentation}) + * the output, i.e., the directly observable response to the query's suffix (cf. + * {@link Query main documentation}) */ public abstract void answer(D output); @@ -104,6 +104,16 @@ public Word getInput() { */ public abstract Word getSuffix(); + /** + * Returns the number of symbols of this query by adding the {@link Word#length() lengths} of the + * {@link #getPrefix() prefix} and {@link #getSuffix() suffix}. + * + * @return the length of this query + */ + public int length() { + return getPrefix().length() + getSuffix().length(); + } + @Override public final int hashCode() { if (hashCode != 0) { @@ -125,8 +135,8 @@ public final boolean equals(@Nullable Object o) { /** * Returns the string representation of this query. * - * @return A string of the form {@code "Query[|]"} for queries not containing an answer or {@code - * "Query[| / ]"} if an answer may be specified. + * @return A string of the form {@code "Query[|]"} for queries not containing an answer or + * {@code "Query[| / ]"} if an answer may be specified. */ @Override public String toString() { diff --git a/api/src/main/java/de/learnlib/statistic/NoopCollector.java b/api/src/main/java/de/learnlib/statistic/NoopCollector.java new file mode 100644 index 0000000000..3a81a8c8b7 --- /dev/null +++ b/api/src/main/java/de/learnlib/statistic/NoopCollector.java @@ -0,0 +1,96 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.statistic; + +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A no-op implementation of a {@link StatisticsCollector} that does nothing. + */ +class NoopCollector implements StatisticsCollector { + + @Override + public Collection getKeys() { + return Collections.emptyList(); + } + + @Override + public void clear() {} + + @Override + public void addText(String id, @Nullable String description, String text) {} + + @Override + public Optional getText(String id) { + return Optional.empty(); + } + + @Override + public void setFlag(String id, @Nullable String description, boolean value) {} + + @Override + public Optional getFlag(String id) { + return Optional.empty(); + } + + @Override + public void startOrResumeClock(String id, @Nullable String description) {} + + @Override + public void pauseClock(String id) {} + + @Override + public Optional getClock(String id) { + return Optional.empty(); + } + + @Override + public void increaseCounter(String id, @Nullable String description, long increment) {} + + @Override + public void setCounter(String id, @Nullable String description, long count) {} + + @Override + public Optional getCount(String id) { + return Optional.empty(); + } + + @Override + public String printStats() { + return """ + ################################################ + This is a no-op collector. If you plan on + collecting statistics, make sure to provide a + StatisticsProvider service on the classpath. + + A default implementation can be found in the + statistics module of LearnLib which can be + included with the following Maven dependency: + + + de.learnlib + learnlib-statistics + ${version} + + ################################################ + """; + } +} diff --git a/api/src/main/java/de/learnlib/statistic/StatisticLearner.java b/api/src/main/java/de/learnlib/statistic/StatisticLearner.java deleted file mode 100644 index d4530511b2..0000000000 --- a/api/src/main/java/de/learnlib/statistic/StatisticLearner.java +++ /dev/null @@ -1,41 +0,0 @@ -/* Copyright (C) 2013-2025 TU Dortmund University - * This file is part of LearnLib . - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package de.learnlib.statistic; - -import de.learnlib.algorithm.LearningAlgorithm; -import net.automatalib.automaton.fsa.DFA; -import net.automatalib.automaton.transducer.MealyMachine; -import net.automatalib.automaton.transducer.MooreMachine; -import net.automatalib.word.Word; - -/** - * Common interface for learners keeping statistics. - * - * @param - * the automaton type - * @param - * input symbol class - * @param - * output symbol class - */ -public interface StatisticLearner extends LearningAlgorithm, StatisticCollector { - - interface DFAStatisticLearner extends StatisticLearner, I, Boolean> {} - - interface MealyStatisticLearner extends StatisticLearner, I, Word> {} - - interface MooreStatisticLearner extends StatisticLearner, I, Word> {} -} diff --git a/api/src/main/java/de/learnlib/statistic/Statistics.java b/api/src/main/java/de/learnlib/statistic/Statistics.java new file mode 100644 index 0000000000..228d02647a --- /dev/null +++ b/api/src/main/java/de/learnlib/statistic/Statistics.java @@ -0,0 +1,70 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.statistic; + +import java.util.ServiceLoader; + +/** + * Factory for obtaining {@link StatisticsCollector}s. + */ +public final class Statistics { + + private static final StatisticsProvider PROVIDER; + + static { + final ServiceLoader loader = ServiceLoader.load(StatisticsProvider.class); + + StatisticsProvider bestProvider = new NoopProvider(); + for (StatisticsProvider sp : loader) { + if (sp.getPriority() > bestProvider.getPriority()) { + bestProvider = sp; + } + } + + PROVIDER = bestProvider; + } + + private Statistics() { + // prevent instantiation + } + + /** + * Returns a {@link StatisticsCollector} for collecting statistics. Note that the returned instances should behave + * as "per-thread-singletons", i.e., within a thread, the same instance should be returned as to enable client-code + * to collect statistics over various invocations across different components. However, in a multi-threaded + * benchmark scenario, each thread should obtain its own copy. + * + * @return the collector + */ + public static StatisticsCollector getCollector() { + return PROVIDER.getCollector(); + } + + private static final class NoopProvider implements StatisticsProvider { + + private static final StatisticsCollector COLLECTOR = new NoopCollector(); + + @Override + public int getPriority() { + return Integer.MIN_VALUE; + } + + @Override + public StatisticsCollector getCollector() { + return COLLECTOR; + } + } +} diff --git a/api/src/main/java/de/learnlib/statistic/StatisticsCollector.java b/api/src/main/java/de/learnlib/statistic/StatisticsCollector.java new file mode 100644 index 0000000000..1c2bf9f6d5 --- /dev/null +++ b/api/src/main/java/de/learnlib/statistic/StatisticsCollector.java @@ -0,0 +1,181 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.statistic; + +import java.time.Duration; +import java.util.Collection; +import java.util.Optional; + +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A container that collects various statistics of different types. Individual measurements are identified via an + * {@code id} which can be optionally enhanced by a description for the purpose of pretty-printing. Implementations may + * allow for key collisions as a means to merge statistics from different components. + *

+ * Note that implementations of this interface should be thread-safe as instances of the same collector may be passed + * across multiple threads. + */ +public interface StatisticsCollector { + + // General + + /** + * Returns all registered keys of this collector. May be used when exporting data. + * + * @return the keys for which any data as been collected. + */ + Collection getKeys(); + + /** + * Clears the data of this collector. May be used when wanting to start a fresh data collection. + */ + void clear(); + + // Generic text + + /** + * Stores the provided text for the given id. + * + * @param id + * the id of the text + * @param description + * description of the data, e.g., "configuration" + * @param text + * the text to be stored + */ + void addText(String id, @Nullable String description, String text); + + /** + * Retrieves the text with the given id. + * + * @param id + * the id of the text + * + * @return the stored text, or {@link Optional#empty()} if no text for this id exists + */ + Optional getText(String id); + + // Boolean flags + + /** + * Stores the provided boolean for the given id. + * + * @param id + * the id of the flag + * @param description + * description of the boolean, e.g., "accurate" + * @param value + * the boolean value to be stored + */ + void setFlag(String id, @Nullable String description, boolean value); + + /** + * Retrieves the flag with the given id. + * + * @param id + * the id of the flag + * + * @return the stored boolean, or {@link Optional#empty()} if no boolean for this id exists + */ + Optional getFlag(String id); + + // Time + + /** + * Starts the clock with the given id. If there is already a clock with this id, it is resumed. + * + * @param id + * the id of the clock + * @param description + * description of the clock, e.g., "learning time" + */ + void startOrResumeClock(String id, @Nullable String description); + + /** + * Pauses the clock with the given id. If there is no clock with this id, nothing happens. + * + * @param id + * the id of the clock + */ + void pauseClock(String id); + + /** + * Returns the current value of the clock with the given id. + * + * @param id + * the id of the clock + * + * @return the current value of the clock, or {@link Optional#empty()} if no clock for this id exists + */ + Optional getClock(String id); + + // Counter + + /** + * Increases the counter with the given id. If no counter with this id exists, it is created. + * + * @param id + * the id of the counter + * @param description + * description of the counter, e.g., "number of rounds" + */ + default void increaseCounter(String id, @Nullable String description) { + increaseCounter(id, description, 1); + } + + /** + * Increases the counter with the given id by the provided increment. If no counter with this id exists, it is + * created and initialized with the provided increment. + * + * @param id + * the id of the counter + * @param description + * description of the counter, e.g., "number of rounds" + * @param increment + * the amount to increase the counter by (must be greater than zero) + */ + void increaseCounter(String id, @Nullable String description, long increment); + + /** + * Sets the counter with the given id to the provided value. If no counter with this id exists, it is created. + * + * @param id + * the id of the counter + * @param description + * description of the counter, e.g., "number of rounds" + * @param count + * New value for the counter (must be greater than zero) + */ + void setCounter(String id, @Nullable String description, long count); + + /** + * Gets the value of the counter with the given id. + * + * @param id + * the id of the counter + * + * @return The value of the counter, or {@link Optional#empty()} if no counter for this id exists + */ + Optional getCount(String id); + + /** + * Returns a string-based representation of the collected statistics. + * + * @return a string-based representation of the collected statistics + */ + String printStats(); +} diff --git a/api/src/main/java/de/learnlib/statistic/StatisticsProvider.java b/api/src/main/java/de/learnlib/statistic/StatisticsProvider.java new file mode 100644 index 0000000000..1b57da1938 --- /dev/null +++ b/api/src/main/java/de/learnlib/statistic/StatisticsProvider.java @@ -0,0 +1,43 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.statistic; + +import java.util.ServiceLoader; + +/** + * A statistics provider is means to register different implementations of {@link StatisticsCollector}s via + * {@link ServiceLoader service loading}. + */ +public interface StatisticsProvider { + + /** + * Returns the priority of this provider. In general, providers with a high priority should be preferred over + * providers with a low priority. + * + * @return the priority of this provider + */ + int getPriority(); + + /** + * Returns the instance of the {@link StatisticsCollector}. Note that the returned instances should behave as + * "per-thread-singletons", i.e., within a thread, the same instance should be returned as to enable client-code to + * collect statistics over various invocations across different components. However, in a multi-threaded benchmark + * scenario, each thread should obtain its own copy. + * + * @return the statistics collector + */ + StatisticsCollector getCollector(); +} diff --git a/api/src/main/java/de/learnlib/sul/TimedSUL.java b/api/src/main/java/de/learnlib/sul/TimedSUL.java new file mode 100644 index 0000000000..6721697881 --- /dev/null +++ b/api/src/main/java/de/learnlib/sul/TimedSUL.java @@ -0,0 +1,131 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.sul; + +import java.util.ArrayList; +import java.util.List; + +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Interface for a SUL with MMLT semantics. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public interface TimedSUL extends SUL, TimedOutput> { + + /** + * Follows the provided input word, starting at the current system state. The input word must not contain timeout + * symbols. Otherwise, an error occurs. + * + * @param input + * the input word + */ + default void follow(Word> input) { + this.follow(input, -1); + } + + /** + * Follows the provided input word, starting at the current configuration. + * + * @param input + * the input word + * @param maxTimeout + * maximum waiting time to use for {@link TimeoutSymbol}s. + */ + default void follow(Word> input, long maxTimeout) { + for (TimedInput i : input) { + if (i instanceof InputSymbol ndi) { + this.step(ndi); + } else if (i instanceof TimeStepSequence tss) { + this.collectTimeouts(tss); + } else if (i instanceof TimeoutSymbol) { + if (maxTimeout <= 0) { + throw new IllegalArgumentException("Must supply timeout when using timeout symbols."); + } + this.timeoutStep(maxTimeout); + } else { + throw new IllegalArgumentException("Unknown suffix type."); + } + } + } + + /** + * Waits until a timeout occurs or the provided time is reached. May observe no timeout if either the waiting time + * is too small or if the active location has no timers. + * + * @param maxTime + * maximum waiting time. + * + * @return observed timer output with waiting time, or {@code null} if no timeout was observed. + */ + @Nullable TimedOutput timeoutStep(long maxTime); + + /** + * Waits for one time unit and returns the observed output. + * + * @return {@code null} if no output occurred, or a timer output if at least one timer expired. The delay of this + * output is set to zero. + */ + + default @Nullable TimedOutput timeStep() { + TimedOutput res = this.timeoutStep(1); + if (res != null) { + return new TimedOutput<>(res.symbol()); + } + return null; + } + + /** + * Waits for the duration of the given time step and returns all observed timeouts. + * + * @param input + * the time step to wait. + * + * @return a list of observed timeouts (may be empty if no time outs occurred in the given time) + */ + default List> collectTimeouts(TimeStepSequence input) { + List> timeouts = new ArrayList<>(); + + long remainingTime = input.timeSteps(); + while (remainingTime > 0) { + TimedOutput nextTimeout = this.timeoutStep(remainingTime); + if (nextTimeout == null) { + // No timer will expire during remaining waiting time: + break; + } else { + timeouts.add(nextTimeout); + remainingTime -= nextTimeout.delay(); + } + } + + return timeouts; + } + + @Override + default TimedSUL fork() { + throw new UnsupportedOperationException(); + } +} diff --git a/api/src/main/java/de/learnlib/time/MMLTModelParams.java b/api/src/main/java/de/learnlib/time/MMLTModelParams.java new file mode 100644 index 0000000000..2946a817ea --- /dev/null +++ b/api/src/main/java/de/learnlib/time/MMLTModelParams.java @@ -0,0 +1,42 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.time; + +import net.automatalib.automaton.mmlt.SymbolCombiner; + +/** + * Model-specific parameters for MMLT-based learners. These are used by various filters, oracles, and the MMLT + * simulator. + * + * @param silentOutput + * Silent output symbol + * @param outputCombiner + * Function for combining simultaneously occurring outputs of timers + * @param maxTimeoutWaitingTime + * Maximum time to wait for a timeout in any configuration. If no timeout is observed after this time, the + * learner assumes that no timers are active. Hence, if this value is set too low, the learner will miss + * timeouts. This usually results in an incomplete model but can also trigger exceptions due to unsatisfied + * assumptions. + * @param maxTimerQueryWaitingTime + * Maximum waiting time to wait when inferring timers for a location. This must be at least the max. time for a + * timeout. We recommend setting this value to at least twice the highest value of any timer in the SUL, if + * these values are known or can be estimated. This increases the likelihood of detecting non-periodic behavior + * during timer inference, and thus reduces the need for equivalence queries. + * @param + * Output symbol type + */ +public record MMLTModelParams(O silentOutput, SymbolCombiner outputCombiner, long maxTimeoutWaitingTime, + long maxTimerQueryWaitingTime) {} diff --git a/api/src/main/java/module-info.java b/api/src/main/java/module-info.java index e62d48409f..73411f18bb 100644 --- a/api/src/main/java/module-info.java +++ b/api/src/main/java/module-info.java @@ -14,6 +14,8 @@ * limitations under the License. */ +import de.learnlib.statistic.StatisticsProvider; + /** * This module provides the core interfaces of LearnLib. *

@@ -38,9 +40,13 @@ exports de.learnlib; exports de.learnlib.algorithm; exports de.learnlib.exception; + exports de.learnlib.filter; exports de.learnlib.logging; exports de.learnlib.oracle; exports de.learnlib.query; exports de.learnlib.statistic; exports de.learnlib.sul; + exports de.learnlib.time; + + uses StatisticsProvider; } diff --git a/api/src/test/java/de/learnlib/statistic/StatisticTest.java b/api/src/test/java/de/learnlib/statistic/StatisticTest.java new file mode 100644 index 0000000000..3c887eacd1 --- /dev/null +++ b/api/src/test/java/de/learnlib/statistic/StatisticTest.java @@ -0,0 +1,59 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.statistic; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class StatisticTest { + + @Test + public void testSingleton() { + StatisticsCollector col1 = Statistics.getCollector(); + StatisticsCollector col2 = Statistics.getCollector(); + + Assert.assertSame(col1, col2); + } + + @Test + public void testNoop() { + StatisticsCollector collector = Statistics.getCollector(); + + String id1 = "id1"; + collector.addText(id1, null, "text"); + Assert.assertTrue(collector.getText(id1).isEmpty()); + + String id2 = "id2"; + collector.setFlag(id2, null, true); + Assert.assertTrue(collector.getFlag(id2).isEmpty()); + + String id3 = "id3"; + collector.startOrResumeClock(id3, null); + collector.pauseClock(id3); + Assert.assertTrue(collector.getClock(id3).isEmpty()); + + String id4 = "id4"; + collector.increaseCounter(id4, null); + Assert.assertTrue(collector.getCount(id4).isEmpty()); + + Assert.assertTrue(collector.getKeys().isEmpty()); + + // assert no throws + collector.clear(); + + Assert.assertNotNull(collector.printStats()); + } +} diff --git a/archetypes/basic/src/main/resources/archetype-resources/pom.xml b/archetypes/basic/src/main/resources/archetype-resources/pom.xml index fca03662dc..de5e1db1c5 100644 --- a/archetypes/basic/src/main/resources/archetype-resources/pom.xml +++ b/archetypes/basic/src/main/resources/archetype-resources/pom.xml @@ -48,6 +48,10 @@ de.learnlib learnlib-lstar + + de.learnlib + learnlib-statistics + de.learnlib learnlib-util diff --git a/archetypes/basic/src/main/resources/archetype-resources/src/main/java/Example.java b/archetypes/basic/src/main/resources/archetype-resources/src/main/java/Example.java index 4b108515b3..c1b71bae79 100644 --- a/archetypes/basic/src/main/resources/archetype-resources/src/main/java/Example.java +++ b/archetypes/basic/src/main/resources/archetype-resources/src/main/java/Example.java @@ -9,8 +9,8 @@ import de.learnlib.oracle.MembershipOracle.DFAMembershipOracle; import de.learnlib.oracle.equivalence.DFAWMethodEQOracle; import de.learnlib.oracle.membership.DFASimulatorOracle; +import de.learnlib.statistic.Statistics; import de.learnlib.util.Experiment.DFAExperiment; -import de.learnlib.util.statistic.SimpleProfiler; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.Alphabets; import net.automatalib.automaton.fsa.DFA; @@ -60,12 +60,6 @@ public static void main(String[] args) throws IOException { // active learning DFAExperiment experiment = new DFAExperiment<>(lstar, wMethod, inputs); - // turn on time profiling - experiment.setProfile(true); - - // enable logging of models - experiment.setLogModels(true); - // run experiment experiment.run(); @@ -75,12 +69,8 @@ public static void main(String[] args) throws IOException { // report results System.out.println("-------------------------------------------------------"); - // profiling - SimpleProfiler.logResults(); - // learning statistics - System.out.println(experiment.getRounds().getSummary()); - System.out.println(mqOracle.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); // model statistics System.out.println("States: " + result.size()); diff --git a/archetypes/complete/src/main/resources/archetype-resources/src/main/java/Example.java b/archetypes/complete/src/main/resources/archetype-resources/src/main/java/Example.java index 4b108515b3..c1b71bae79 100644 --- a/archetypes/complete/src/main/resources/archetype-resources/src/main/java/Example.java +++ b/archetypes/complete/src/main/resources/archetype-resources/src/main/java/Example.java @@ -9,8 +9,8 @@ import de.learnlib.oracle.MembershipOracle.DFAMembershipOracle; import de.learnlib.oracle.equivalence.DFAWMethodEQOracle; import de.learnlib.oracle.membership.DFASimulatorOracle; +import de.learnlib.statistic.Statistics; import de.learnlib.util.Experiment.DFAExperiment; -import de.learnlib.util.statistic.SimpleProfiler; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.Alphabets; import net.automatalib.automaton.fsa.DFA; @@ -60,12 +60,6 @@ public static void main(String[] args) throws IOException { // active learning DFAExperiment experiment = new DFAExperiment<>(lstar, wMethod, inputs); - // turn on time profiling - experiment.setProfile(true); - - // enable logging of models - experiment.setLogModels(true); - // run experiment experiment.run(); @@ -75,12 +69,8 @@ public static void main(String[] args) throws IOException { // report results System.out.println("-------------------------------------------------------"); - // profiling - SimpleProfiler.logResults(); - // learning statistics - System.out.println(experiment.getRounds().getSummary()); - System.out.println(mqOracle.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); // model statistics System.out.println("States: " + result.size()); diff --git a/commons/datastructures/src/main/java/de/learnlib/datastructure/observationtable/RowImpl.java b/commons/datastructures/src/main/java/de/learnlib/datastructure/observationtable/RowImpl.java index cdee25aa0c..b75ec76e75 100644 --- a/commons/datastructures/src/main/java/de/learnlib/datastructure/observationtable/RowImpl.java +++ b/commons/datastructures/src/main/java/de/learnlib/datastructure/observationtable/RowImpl.java @@ -18,7 +18,7 @@ import net.automatalib.common.util.array.ArrayStorage; import net.automatalib.word.Word; -final class RowImpl implements Row { +public final class RowImpl implements Row { private final Word label; private final int rowId; @@ -37,7 +37,7 @@ final class RowImpl implements Row { * @param alphabetSize * the size of the alphabet, used for initializing the successor array */ - RowImpl(Word label, int rowId, int alphabetSize) { + public RowImpl(Word label, int rowId, int alphabetSize) { this(label, rowId); makeShort(alphabetSize); @@ -51,7 +51,7 @@ final class RowImpl implements Row { * @param rowId * the unique row identifier */ - RowImpl(Word label, int rowId) { + public RowImpl(Word label, int rowId) { this.label = label; this.rowId = rowId; } @@ -63,7 +63,7 @@ final class RowImpl implements Row { * @param initialAlphabetSize * the size of the input alphabet. */ - void makeShort(int initialAlphabetSize) { + public void makeShort(int initialAlphabetSize) { if (lpIndex == -1) { return; } @@ -85,7 +85,7 @@ public RowImpl getSuccessor(int inputIdx) { * @param succ * the successor row */ - void setSuccessor(int inputIdx, RowImpl succ) { + public void setSuccessor(int inputIdx, RowImpl succ) { successors.set(inputIdx, succ); } @@ -110,7 +110,7 @@ public int getRowContentId() { * @param id * the contents id */ - void setRowContentId(int id) { + public void setRowContentId(int id) { this.rowContentId = id; } @@ -127,11 +127,11 @@ int getLpIndex() { return lpIndex; } - void setLpIndex(int lpIndex) { + public void setLpIndex(int lpIndex) { this.lpIndex = lpIndex; } - void ensureInputCapacity(int capacity) { + public void ensureInputCapacity(int capacity) { this.successors.ensureCapacity(capacity); } } diff --git a/commons/datastructures/src/main/java/module-info.java b/commons/datastructures/src/main/java/module-info.java index e9ccbfa9b9..5b1d9590d5 100644 --- a/commons/datastructures/src/main/java/module-info.java +++ b/commons/datastructures/src/main/java/module-info.java @@ -36,6 +36,7 @@ // annotations are 'provided'-scoped and do not need to be loaded at runtime requires static org.checkerframework.checker.qual; + requires org.slf4j; exports de.learnlib.datastructure.discriminationtree; exports de.learnlib.datastructure.discriminationtree.iterators; diff --git a/commons/util/pom.xml b/commons/util/pom.xml index 5ac79e0514..e25cd3d981 100644 --- a/commons/util/pom.xml +++ b/commons/util/pom.xml @@ -36,10 +36,6 @@ limitations under the License. de.learnlib learnlib-api - - de.learnlib - learnlib-statistics - @@ -66,6 +62,11 @@ limitations under the License. + + org.mockito + mockito-core + + org.testng testng diff --git a/commons/util/src/main/java/de/learnlib/util/Experiment.java b/commons/util/src/main/java/de/learnlib/util/Experiment.java index 305d4f877a..dc240b09d5 100644 --- a/commons/util/src/main/java/de/learnlib/util/Experiment.java +++ b/commons/util/src/main/java/de/learnlib/util/Experiment.java @@ -16,12 +16,13 @@ package de.learnlib.util; import de.learnlib.algorithm.LearningAlgorithm; -import de.learnlib.filter.statistic.Counter; import de.learnlib.logging.Category; import de.learnlib.oracle.EquivalenceOracle; import de.learnlib.query.DefaultQuery; -import de.learnlib.util.statistic.SimpleProfiler; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import net.automatalib.alphabet.Alphabet; +import net.automatalib.automaton.concept.FiniteRepresentation; import net.automatalib.automaton.fsa.DFA; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; @@ -36,22 +37,23 @@ * @param * the automaton type */ -public class Experiment { +public class Experiment { - public static final String LEARNING_PROFILE_KEY = "Learning"; - public static final String COUNTEREXAMPLE_PROFILE_KEY = "Searching for counterexample"; + public static final String LEARNING_PROFILE_KEY = "exp-expl-dur"; + public static final String COUNTEREXAMPLE_PROFILE_KEY = "exp-ce-dur"; + public static final String LEARNING_ROUNDS_KEY = "exp-rnd"; + public static final String FINAL_SIZE_KEY = "hyp-size"; private static final Logger LOGGER = LoggerFactory.getLogger(Experiment.class); private final ExperimentImpl impl; - private boolean logModels; - private boolean profile; - private final Counter rounds = new Counter("Learning rounds", "#"); + private final StatisticsCollector statisticsCollector; private @Nullable A finalHypothesis; public Experiment(LearningAlgorithm learningAlgorithm, EquivalenceOracle equivalenceAlgorithm, Alphabet inputs) { this.impl = new ExperimentImpl<>(learningAlgorithm, equivalenceAlgorithm, inputs); + this.statisticsCollector = Statistics.getCollector(); } /** @@ -87,52 +89,12 @@ public A getFinalHypothesis() { return finalHypothesis; } - private void profileStart(String taskname) { - if (profile) { - SimpleProfiler.start(taskname); - } - } - - private void profileStop(String taskname) { - if (profile) { - SimpleProfiler.stop(taskname); - } - } - - /** - * Decides whether intermediate hypothesis models should be logged. - * - * @param logModels - * flag whether models should be logged - */ - public void setLogModels(boolean logModels) { - this.logModels = logModels; - } - - /** - * Decides whether the experiment runtime should be profiled. - * - * @param profile - * flag whether learning process should be profiled - */ - public void setProfile(boolean profile) { - this.profile = profile; - } - - /** - * Returns the counter for the number of refinement rounds the experiment took. - * - * @return the rounds - */ - public Counter getRounds() { - return rounds; - } - private final class ExperimentImpl { private final LearningAlgorithm learningAlgorithm; private final EquivalenceOracle equivalenceAlgorithm; private final Alphabet inputs; + private int rounds; ExperimentImpl(LearningAlgorithm learningAlgorithm, EquivalenceOracle equivalenceAlgorithm, @@ -143,41 +105,40 @@ private final class ExperimentImpl { } public A run() { - rounds.increment(); - LOGGER.info(Category.PHASE, "Starting round {}", rounds.getCount()); + rounds++; + statisticsCollector.increaseCounter(LEARNING_ROUNDS_KEY, "Number of learning rounds"); + LOGGER.info(Category.PHASE, "Starting round {}", rounds); LOGGER.info(Category.PHASE, "Learning"); - profileStart(LEARNING_PROFILE_KEY); + statisticsCollector.startOrResumeClock(LEARNING_PROFILE_KEY, "Duration of exploration"); learningAlgorithm.startLearning(); - profileStop(LEARNING_PROFILE_KEY); + statisticsCollector.pauseClock(LEARNING_PROFILE_KEY); while (true) { final A hyp = learningAlgorithm.getHypothesisModel(); - if (logModels) { - LOGGER.info(Category.MODEL, hyp.toString()); - } - LOGGER.info(Category.PHASE, "Searching for counterexample"); - profileStart(COUNTEREXAMPLE_PROFILE_KEY); + statisticsCollector.startOrResumeClock(COUNTEREXAMPLE_PROFILE_KEY, "Duration of counterexample search"); DefaultQuery ce = equivalenceAlgorithm.findCounterExample(hyp, inputs); - profileStop(COUNTEREXAMPLE_PROFILE_KEY); + statisticsCollector.pauseClock(COUNTEREXAMPLE_PROFILE_KEY); if (ce == null) { + statisticsCollector.setCounter(FINAL_SIZE_KEY, "Final hypothesis size", hyp.size()); return hyp; } LOGGER.info(Category.COUNTEREXAMPLE, ce.getInput().toString()); // next round ... - rounds.increment(); - LOGGER.info(Category.PHASE, "Starting round {}", rounds.getCount()); + rounds++; + statisticsCollector.increaseCounter(LEARNING_ROUNDS_KEY, "Number of learning rounds"); + LOGGER.info(Category.PHASE, "Starting round {}", rounds); LOGGER.info(Category.PHASE, "Learning"); - profileStart(LEARNING_PROFILE_KEY); + statisticsCollector.startOrResumeClock(LEARNING_PROFILE_KEY, "Duration of exploration"); final boolean refined = learningAlgorithm.refineHypothesis(ce); - profileStop(LEARNING_PROFILE_KEY); + statisticsCollector.pauseClock(LEARNING_PROFILE_KEY); assert refined; } diff --git a/commons/util/src/main/java/de/learnlib/util/mealy/MealyUtil.java b/commons/util/src/main/java/de/learnlib/util/mealy/MealyUtil.java index 56b80adb4a..36dcdcbdfa 100644 --- a/commons/util/src/main/java/de/learnlib/util/mealy/MealyUtil.java +++ b/commons/util/src/main/java/de/learnlib/util/mealy/MealyUtil.java @@ -22,6 +22,7 @@ import de.learnlib.algorithm.LearningAlgorithm.MealyLearner; import de.learnlib.oracle.MembershipOracle; import de.learnlib.query.DefaultQuery; +import net.automatalib.automaton.concept.SuffixOutput; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.word.Word; import org.checkerframework.checker.nullness.qual.Nullable; @@ -85,7 +86,7 @@ private static int doFindMismatch(MealyMachine hypothes return NO_MISMATCH; } - public static @Nullable DefaultQuery> shortenCounterExample(MealyMachine hypothesis, + public static @Nullable DefaultQuery> shortenCounterExample(SuffixOutput> hypothesis, DefaultQuery> ceQuery) { Word cePrefix = ceQuery.getPrefix(), ceSuffix = ceQuery.getSuffix(); Word hypOut = hypothesis.computeSuffixOutput(cePrefix, ceSuffix); diff --git a/commons/util/src/main/java/de/learnlib/util/statistic/SimpleProfiler.java b/commons/util/src/main/java/de/learnlib/util/statistic/SimpleProfiler.java deleted file mode 100644 index f45c28fb39..0000000000 --- a/commons/util/src/main/java/de/learnlib/util/statistic/SimpleProfiler.java +++ /dev/null @@ -1,97 +0,0 @@ -/* Copyright (C) 2013-2025 TU Dortmund University - * This file is part of LearnLib . - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package de.learnlib.util.statistic; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import de.learnlib.filter.statistic.Counter; -import de.learnlib.logging.Category; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Very rudimentary profiler. - */ -public final class SimpleProfiler { - - private static final Map CUMULATED = new ConcurrentHashMap<>(); - private static final Map PENDING = new ConcurrentHashMap<>(); - private static final Logger LOGGER = LoggerFactory.getLogger(SimpleProfiler.class.getName()); - private static final double MILLISECONDS_PER_SECOND = 1000.0; - - private SimpleProfiler() { - // prevent initialization - } - - /** - * Reset internal data. - */ - public static void reset() { - CUMULATED.clear(); - PENDING.clear(); - } - - /** - * Start the timer identified by the given key. - * - * @param name - * The name of the timer to be started. - */ - public static void start(String name) { - PENDING.put(name, System.currentTimeMillis()); - } - - /** - * Stop the timer identified by the given key. After stopping a timer, the time passed from its - * {@link #start(String) initialization} will be added to the cumulated time of the specific timer. - * - * @param name - * The name of the timer to be stopped. - */ - public static void stop(String name) { - Long start = PENDING.remove(name); - if (start == null) { - return; - } - long duration = System.currentTimeMillis() - start; - Counter sum = CUMULATED.computeIfAbsent(name, k -> new Counter(k, "ms")); - sum.increment(duration); - } - - /** - * Return the counter for the cumulated (passed) time of the given timer. - * - * @param name - * The name of the timer to be returned. - * - * @return The counter for tracking the passed milliseconds of the timer - */ - public static @Nullable Counter cumulated(String name) { - return CUMULATED.get(name); - } - - /** - * Log results in category PROFILING. - */ - public static void logResults() { - for (Counter c : CUMULATED.values()) { - LOGGER.info(Category.PROFILING, "{}, ({} s)", c.getSummary(), c.getCount() / MILLISECONDS_PER_SECOND); - } - } - -} diff --git a/commons/util/src/main/java/module-info.java b/commons/util/src/main/java/module-info.java index e4b6a888c4..255f653e64 100644 --- a/commons/util/src/main/java/module-info.java +++ b/commons/util/src/main/java/module-info.java @@ -29,7 +29,6 @@ open module de.learnlib.common.util { requires de.learnlib.api; - requires de.learnlib.filter.statistic; requires net.automatalib.api; requires net.automatalib.core; requires net.automatalib.util; @@ -42,5 +41,4 @@ exports de.learnlib.util.mealy; exports de.learnlib.util.moore; exports de.learnlib.util.nfa; - exports de.learnlib.util.statistic; } diff --git a/commons/util/src/test/java/de/learnlib/util/ExperimentTest.java b/commons/util/src/test/java/de/learnlib/util/ExperimentTest.java index 80b1c19031..ad26e25f70 100644 --- a/commons/util/src/test/java/de/learnlib/util/ExperimentTest.java +++ b/commons/util/src/test/java/de/learnlib/util/ExperimentTest.java @@ -21,8 +21,9 @@ import de.learnlib.algorithm.LearningAlgorithm.DFALearner; import de.learnlib.oracle.EquivalenceOracle.DFAEquivalenceOracle; import de.learnlib.query.DefaultQuery; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import de.learnlib.util.Experiment.DFAExperiment; -import de.learnlib.util.statistic.SimpleProfiler; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.Alphabets; import net.automatalib.automaton.fsa.DFA; @@ -30,6 +31,9 @@ import net.automatalib.util.automaton.random.RandomAutomata; import net.automatalib.word.Word; import org.checkerframework.checker.nullness.qual.Nullable; +import org.mockito.ArgumentMatchers; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -48,25 +52,40 @@ public void testExperiment() { final MockUpLearner learner = new MockUpLearner<>(target, intermediateTarget); final DFAEquivalenceOracle eq = new MockUpOracle<>(intermediateTarget); - DFAExperiment experiment = new DFAExperiment<>(learner, eq, alphabet); - experiment.setProfile(true); + final StatisticsCollector statMock = Mockito.mock(StatisticsCollector.class); - Assert.assertThrows(experiment::getFinalHypothesis); + try (MockedStatic statistics = Mockito.mockStatic(Statistics.class)) { + statistics.when(Statistics::getCollector).thenReturn(statMock); - experiment.run(); + DFAExperiment experiment = new DFAExperiment<>(learner, eq, alphabet); - Assert.assertThrows(experiment::run); + Assert.assertThrows(experiment::getFinalHypothesis); - DFA finalModel = experiment.getFinalHypothesis(); + experiment.run(); - Assert.assertNotNull(experiment.getFinalHypothesis()); - Assert.assertSame(finalModel, target); + Assert.assertThrows(experiment::run); - Assert.assertTrue(learner.startLearningCalled); - Assert.assertEquals(learner.refinementSteps, REFINEMENT_STEPS); + DFA finalModel = experiment.getFinalHypothesis(); - Assert.assertNotNull(SimpleProfiler.cumulated(Experiment.LEARNING_PROFILE_KEY)); - Assert.assertNotNull(SimpleProfiler.cumulated(Experiment.COUNTEREXAMPLE_PROFILE_KEY)); + Assert.assertNotNull(experiment.getFinalHypothesis()); + Assert.assertSame(finalModel, target); + + Assert.assertTrue(learner.startLearningCalled); + Assert.assertEquals(learner.refinementSteps, REFINEMENT_STEPS); + + Mockito.verify(statMock, Mockito.atLeastOnce()) + .startOrResumeClock(ArgumentMatchers.eq(Experiment.LEARNING_PROFILE_KEY), + ArgumentMatchers.anyString()); + Mockito.verify(statMock, Mockito.atLeastOnce()) + .pauseClock(ArgumentMatchers.eq(Experiment.LEARNING_PROFILE_KEY)); + Mockito.verify(statMock, Mockito.atLeastOnce()) + .startOrResumeClock(ArgumentMatchers.eq(Experiment.COUNTEREXAMPLE_PROFILE_KEY), + ArgumentMatchers.anyString()); + Mockito.verify(statMock, Mockito.atLeastOnce()) + .pauseClock(ArgumentMatchers.eq(Experiment.COUNTEREXAMPLE_PROFILE_KEY)); + Mockito.verify(statMock, Mockito.atLeastOnce()) + .increaseCounter(ArgumentMatchers.eq(Experiment.LEARNING_ROUNDS_KEY), ArgumentMatchers.anyString()); + } } private static final class MockUpLearner implements DFALearner { diff --git a/drivers/simulator/pom.xml b/drivers/simulator/pom.xml index d9d6fa3e9e..cf2f8aa044 100644 --- a/drivers/simulator/pom.xml +++ b/drivers/simulator/pom.xml @@ -43,9 +43,31 @@ limitations under the License. automata-api + org.checkerframework checker-qual + + + + de.learnlib.testsupport + learnlib-learning-examples + + + + net.automatalib + automata-core + + + + org.mockito + mockito-core + + + org.testng + testng + + diff --git a/drivers/simulator/src/main/java/de/learnlib/driver/simulator/MMLTSimulatorSUL.java b/drivers/simulator/src/main/java/de/learnlib/driver/simulator/MMLTSimulatorSUL.java new file mode 100644 index 0000000000..4b755d9e13 --- /dev/null +++ b/drivers/simulator/src/main/java/de/learnlib/driver/simulator/MMLTSimulatorSUL.java @@ -0,0 +1,89 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.driver.simulator; + +import de.learnlib.sul.TimedSUL; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.MMLTSemantics; +import net.automatalib.automaton.mmlt.State; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Simulates the semantics of an {@link MMLT}. + * + * @param + * input symbol type (of non-delaying inputs). + * @param + * output symbol type. + */ +public class MMLTSimulatorSUL extends MealySimulatorSUL, TimedOutput> + implements TimedSUL { + + private final MMLTSimulatorSULImpl impl; + + public MMLTSimulatorSUL(MMLT semantics) { + this(new MMLTSimulatorSULImpl<>(semantics.getSemantics())); + } + + private MMLTSimulatorSUL(MMLTSimulatorSULImpl impl) { + super(impl); + this.impl = impl; + } + + @Override + public @Nullable TimedOutput timeoutStep(long maxTime) { + return this.impl.timeoutStep(maxTime); + } + + @Override + public TimedSUL fork() { + return new MMLTSimulatorSUL<>(this.impl.fork()); + } + + private static final class MMLTSimulatorSULImpl + extends MealySimulatorSULImpl, InputSymbol, T, TimedOutput> implements TimedSUL { + + private final MMLTSemantics semantics; + + MMLTSimulatorSULImpl(MMLTSemantics semantics) { + super(semantics, semantics.getSilentOutput()); + this.semantics = semantics; + } + + @Override + public @Nullable TimedOutput timeoutStep(long maxTime) { + final State curr = getCurr(); + final T trans = this.semantics.getTransition(curr, TimedInput.timeout(), maxTime); + setCurr(this.semantics.getSuccessor(trans)); + TimedOutput output = this.semantics.getTransitionOutput(trans); + + if (output.equals(semantics.getSilentOutput())) { + // No timeout observed: + return null; + } else { + return output; + } + } + + @Override + public MMLTSimulatorSULImpl fork() { + return new MMLTSimulatorSULImpl<>(semantics); + } + } +} diff --git a/drivers/simulator/src/main/java/de/learnlib/driver/simulator/MealySimulatorSUL.java b/drivers/simulator/src/main/java/de/learnlib/driver/simulator/MealySimulatorSUL.java index dfbef28cd2..9fa41628ce 100644 --- a/drivers/simulator/src/main/java/de/learnlib/driver/simulator/MealySimulatorSUL.java +++ b/drivers/simulator/src/main/java/de/learnlib/driver/simulator/MealySimulatorSUL.java @@ -17,6 +17,7 @@ import de.learnlib.sul.SUL; import net.automatalib.automaton.transducer.MealyMachine; +import net.automatalib.ts.output.MealyTransitionSystem; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -105,11 +106,11 @@ public SUL fork() { */ static class MealySimulatorSULImpl implements SUL { - private final MealyMachine mealy; + private final MealyTransitionSystem mealy; private final O noTransOut; private @Nullable S curr; - MealySimulatorSULImpl(MealyMachine mealy, O noTransOut) { + MealySimulatorSULImpl(MealyTransitionSystem mealy, O noTransOut) { this.mealy = mealy; this.noTransOut = noTransOut; } @@ -155,6 +156,10 @@ S getCurr() { } return curr; } + + void setCurr(S curr) { + this.curr = curr; + } } } diff --git a/drivers/simulator/src/test/java/de/learnlib/driver/simulator/ForkTest.java b/drivers/simulator/src/test/java/de/learnlib/driver/simulator/ForkTest.java new file mode 100644 index 0000000000..6688bbf944 --- /dev/null +++ b/drivers/simulator/src/test/java/de/learnlib/driver/simulator/ForkTest.java @@ -0,0 +1,174 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.driver.simulator; + +import de.learnlib.testsupport.example.mealy.ExampleCoffeeMachine; +import de.learnlib.testsupport.example.mealy.ExampleCoffeeMachine.Input; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class ForkTest { + + @Test + public void testMealy() { + var sul = new MealySimulatorSUL<>(ExampleCoffeeMachine.constructMachine()); + + Assert.assertTrue(sul.canFork()); + + // check delegation + var spy = Mockito.spy(sul); + var fork = spy.fork(); + + Assert.assertNotNull(fork); + + fork.pre(); + fork.step(Input.CLEAN); + fork.post(); + fork.canFork(); + fork.fork(); + + Mockito.verify(spy, Mockito.only()).fork(); + + // check independency + spy.pre(); + spy.step(Input.WATER); + spy.step(Input.POD); + + fork.pre(); + fork.post(); + + var out = spy.step(Input.BUTTON); + Assert.assertEquals(out, ExampleCoffeeMachine.OUT_COFFEE); + + spy.post(); + } + + @Test + public void testMMLT() { + var mmlt = MMLTExamples.sensorCollector().getReferenceAutomaton(); + var alphabet = mmlt.getInputAlphabet(); + var sul = new MMLTSimulatorSUL<>(mmlt); + var input = TimedInput.input(alphabet.getSymbol(0)); + + Assert.assertTrue(sul.canFork()); + + // check delegation + var spy = Mockito.spy(sul); + var fork = spy.fork(); + + Assert.assertNotNull(fork); + + fork.pre(); + fork.step(input); + fork.follow(Word.fromLetter(input)); + fork.timeoutStep(2); + fork.post(); + fork.canFork(); + fork.fork(); + + Mockito.verify(spy, Mockito.only()).fork(); + + // check independency + spy.pre(); + spy.step(input); + + fork.pre(); + fork.post(); + + var out = spy.timeoutStep(3); + Assert.assertEquals(out, new TimedOutput<>("part", 3)); + + spy.post(); + } + + @Test + public void testObservable() { + var sul = new ObservableMealySimulatorSUL<>(ExampleCoffeeMachine.constructMachine()); + + Assert.assertTrue(sul.canFork()); + + // check delegation + var spy = Mockito.spy(sul); + var fork = spy.fork(); + + Assert.assertNotNull(fork); + + fork.pre(); + fork.step(Input.CLEAN); + fork.deepCopies(); + fork.getState(); + fork.post(); + fork.canFork(); + fork.fork(); + + Mockito.verify(spy, Mockito.only()).fork(); + + // check independency + spy.pre(); + spy.step(Input.WATER); + spy.step(Input.POD); + + fork.pre(); + fork.post(); + + var out = spy.step(Input.BUTTON); + Assert.assertEquals(out, ExampleCoffeeMachine.OUT_COFFEE); + + spy.post(); + } + + @Test + public void testSLI() { + var sul = new StateLocalInputMealySimulatorSUL<>(ExampleCoffeeMachine.constructMachine()); + + Assert.assertTrue(sul.canFork()); + + // check delegation + var spy = Mockito.spy(sul); + var fork = spy.fork(); + + Assert.assertNotNull(fork); + + fork.pre(); + fork.step(Input.CLEAN); + fork.currentlyEnabledInputs(); + fork.post(); + fork.canFork(); + fork.fork(); + + Mockito.verify(spy, Mockito.only()).fork(); + + // check independency + spy.pre(); + spy.step(Input.WATER); + spy.step(Input.POD); + + fork.pre(); + fork.post(); + + var out = spy.step(Input.BUTTON); + Assert.assertEquals(out, ExampleCoffeeMachine.OUT_COFFEE); + + spy.post(); + } +} + + diff --git a/examples/pom.xml b/examples/pom.xml index 1afeacbc7c..7edf1f1f26 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -111,6 +111,10 @@ limitations under the License. de.learnlib learnlib-statistics + + de.learnlib + learnlib-symbol-filters + de.learnlib learnlib-ttt diff --git a/examples/src/main/java/de/learnlib/example/Example1.java b/examples/src/main/java/de/learnlib/example/Example1.java index 4f445a05a4..dfe55cb3f8 100644 --- a/examples/src/main/java/de/learnlib/example/Example1.java +++ b/examples/src/main/java/de/learnlib/example/Example1.java @@ -26,8 +26,8 @@ import de.learnlib.oracle.MembershipOracle.DFAMembershipOracle; import de.learnlib.oracle.equivalence.DFAWMethodEQOracle; import de.learnlib.oracle.membership.DFASimulatorOracle; +import de.learnlib.statistic.Statistics; import de.learnlib.util.Experiment.DFAExperiment; -import de.learnlib.util.statistic.SimpleProfiler; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.Alphabets; import net.automatalib.automaton.fsa.DFA; @@ -79,12 +79,6 @@ public static void main(String[] args) throws IOException { // active learning DFAExperiment experiment = new DFAExperiment<>(lstar, wMethod, inputs); - // turn on time profiling - experiment.setProfile(true); - - // enable logging of models - experiment.setLogModels(true); - // run experiment experiment.run(); @@ -94,12 +88,8 @@ public static void main(String[] args) throws IOException { // report results System.out.println("-------------------------------------------------------"); - // profiling - SimpleProfiler.logResults(); - // learning statistics - System.out.println(experiment.getRounds().getSummary()); - System.out.println(mqOracle.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); // model statistics System.out.println("States: " + result.size()); diff --git a/examples/src/main/java/de/learnlib/example/Example2.java b/examples/src/main/java/de/learnlib/example/Example2.java index fb76febdf6..a665382be8 100644 --- a/examples/src/main/java/de/learnlib/example/Example2.java +++ b/examples/src/main/java/de/learnlib/example/Example2.java @@ -33,10 +33,9 @@ import de.learnlib.oracle.EquivalenceOracle.MealyEquivalenceOracle; import de.learnlib.oracle.equivalence.mealy.RandomWalkEQOracle; import de.learnlib.oracle.membership.SULOracle; -import de.learnlib.statistic.StatisticSUL; +import de.learnlib.statistic.Statistics; import de.learnlib.sul.SUL; import de.learnlib.util.Experiment.MealyExperiment; -import de.learnlib.util.statistic.SimpleProfiler; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.serialization.dot.GraphDOT; import net.automatalib.visualization.Visualization; @@ -75,9 +74,8 @@ public static void main(String[] args) throws NoSuchMethodException, IOException MethodInput poll = driver.addInput("poll", mPoll); // oracle for counting queries wraps sul - StatisticSUL statisticSul = new CounterSUL<>(driver); + SUL effectiveSul = new CounterSUL<>(driver); - SUL effectiveSul = statisticSul; // use caching in order to avoid duplicate queries effectiveSul = SULCaches.createCache(driver.getInputs(), effectiveSul); @@ -114,12 +112,6 @@ public static void main(String[] args) throws NoSuchMethodException, IOException MealyExperiment experiment = new MealyExperiment<>(lstar, randomWalks, driver.getInputs()); - // turn on time profiling - experiment.setProfile(true); - - // enable logging of models - experiment.setLogModels(true); - // run experiment experiment.run(); @@ -129,12 +121,8 @@ public static void main(String[] args) throws NoSuchMethodException, IOException // report results System.out.println("-------------------------------------------------------"); - // profiling - SimpleProfiler.logResults(); - // learning statistics - System.out.println(experiment.getRounds().getSummary()); - System.out.println(statisticSul.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); // model statistics System.out.println("States: " + result.size()); diff --git a/examples/src/main/java/de/learnlib/example/Example3.java b/examples/src/main/java/de/learnlib/example/Example3.java index c73da80a8e..17627ea984 100644 --- a/examples/src/main/java/de/learnlib/example/Example3.java +++ b/examples/src/main/java/de/learnlib/example/Example3.java @@ -202,7 +202,7 @@ class FullMembershipQueryOracle implements MealyMembershipOracle>> queries) { for (Query> query : queries) { resets++; - symbols += query.getInput().size(); + symbols += query.length(); BoundedStringQueue s = new BoundedStringQueue(); diff --git a/examples/src/main/java/de/learnlib/example/mmlt/Example1.java b/examples/src/main/java/de/learnlib/example/mmlt/Example1.java new file mode 100644 index 0000000000..99cab63ba4 --- /dev/null +++ b/examples/src/main/java/de/learnlib/example/mmlt/Example1.java @@ -0,0 +1,136 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.example.mmlt; + +import java.util.ArrayList; +import java.util.List; + +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLTBuilder; +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.filter.cache.mmlt.TimedSULTreeCache; +import de.learnlib.filter.cache.mmlt.TimeoutReducerSUL; +import de.learnlib.filter.statistic.sul.CounterTimedSUL; +import de.learnlib.oracle.equivalence.mmlt.SimulatorEQOracle; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.statistic.Statistics; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; + +/** + * This example shows a basic learning setup for Mealy machine with local timers (MMLT), an automaton model for + * real-time systems. + *

+ * Mealy Machines with Local Timers (MMLTs) are an extension of Mealy machines for real-time behavior. + * They extend Mealy machines with multiple timers. A timer in an MMLT counts down as time progresses. When + * reaching zero, it stops and triggers an action.

+ * + *
    + *
  • A timer in an MMLT is bound to a specific location. It can only time out in its associated location and only be reset + * at transitions that target this location.
  • + *
  • The timeout-action of a timer $x$ is modeled with a transition that uses the internal input {@code to[x]}. These inputs + * cannot be provided to the model directly. Instead, they are internally triggered after sufficient time has passed. All + * other input symbols are called non-delaying inputs.
  • + *
  • The output of a timer at timeout must not be silent.
  • + *
  • There are two types of timers: + *
      + *
    • A periodic timer automatically resets itself on timeout. It cannot cause a change to a different location.
    • + *
    • A one-shot timer may cause a change to a different location on timeout. Regardless of that, it resets all timers + * of the targeted location.
    • + *
    + *
  • + *
  • All timers of a location reset to their initial value when this location is entered from a different location. If + * the initial location has timers, they are reset when the system is activated.
  • + *
  • A self-loop with a non-delaying input does not reset timers by default. However, it might optionally reset all timers + * of its source location. This behavior is called a local reset.
  • + *
  • A location can have multiple timers: + *
      + *
    • A location can also have multiple periodic timers. These can even time out simultaneously. Then, their outputs are + * combined to a single output through concatenation.
    • + *
    • A periodic and a one-shot timer must never time out simultaneously.
    • + *
    + *
  • + *
+ * + *

As for many other real-time systems, the semantics of an MMLT are defined with an associated transition system. + * For MMLTs, this system is a Mealy machine. When inferring the behavior of the unknown system, the learner + * conceptually interacts with this Mealy machine. + * The inputs of this machine are the non-delaying inputs of the MMLT, discrete time steps, + * and a symbolic timeout symbol. The latter prompts a delay until the next timeout.

+ * + *

More information about MMLTs can be found here: + * Learning Mealy Machines with Local Timers.

+ */ +@SuppressWarnings("PMD.UseExplicitTypes") // allow vars in examples +public final class Example1 { + + private Example1() { + // prevent instantiation + } + + public static void main(String[] args) { + // We use the included sensor collector model as reference automaton: + var model = MMLTExamples.sensorCollector(); + var mmlt = model.getReferenceAutomaton(); + var alphabet = mmlt.getInputAlphabet(); + + // We first create a statistics container. + // This container will store various statistical data during learning: + var stats = Statistics.getCollector(); + stats.addText("model", null, model.toString()); + stats.setCounter("original_locs", "Locations in original", mmlt.getStates().size()); + stats.setCounter("original_inputs", "Untimed alphabet size in original", alphabet.size()); + + // ====================== + // Set up the pipeline: + // We use a simulator SUL to simulate our automaton: + var sul = new MMLTSimulatorSUL<>(mmlt); + + // We count all operations that are performed on the SUL with a stats-SUL: + var statsAfterCache = new CounterTimedSUL<>(sul); + + // We use a cache to avoid redundant operations: + var cacheSUL = new TimedSULTreeCache<>(statsAfterCache, model.getParams()); + var toReducerSul = new TimeoutReducerSUL<>(cacheSUL, model.getParams().maxTimeoutWaitingTime()); + + // We use a query oracle to answer queries from the learner: + var timeOracle = new TimedSULOracle<>(toReducerSul, model.getParams()); + + // In the basic set-up, we use a simulator oracle to answer equivalence queries. + // This oracle has perfect knowledge of the reference automaton. + var eqOracle = new SimulatorEQOracle<>(mmlt); + + // Set up our L* learner: + + // We provide the learner with an initial set of suffixes. + // We include all untimed inputs and the symbolic timeout symbol, which causes the learner to wait + // until the next timeout (but no longer than model.getParams().maxTimeoutWaitingTime()). + List>> suffixes = new ArrayList<>(); + alphabet.forEach(s -> suffixes.add(Word.fromLetter(TimedInput.input(s)))); + suffixes.add(Word.fromLetter(new TimeoutSymbol<>())); + + var learner = new ExtensibleLStarMMLTBuilder().withAlphabet(alphabet) + .withModelParams(model.getParams()) + .withTimeOracle(timeOracle) + .withInitialSuffixes(suffixes) + .create(); + + // Start learning: + ExampleRunner.runExperiment(learner, eqOracle, mmlt.getSemantics().getInputAlphabet(), stats); + } + +} diff --git a/examples/src/main/java/de/learnlib/example/mmlt/Example2.java b/examples/src/main/java/de/learnlib/example/mmlt/Example2.java new file mode 100644 index 0000000000..d2dff2ab2c --- /dev/null +++ b/examples/src/main/java/de/learnlib/example/mmlt/Example2.java @@ -0,0 +1,132 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.example.mmlt; + +import java.util.ArrayList; +import java.util.List; + +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLTBuilder; +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.filter.cache.mmlt.TimedSULTreeCache; +import de.learnlib.filter.cache.mmlt.TimeoutReducerSUL; +import de.learnlib.filter.statistic.oracle.CounterEQOracle; +import de.learnlib.filter.statistic.sul.CounterTimedSUL; +import de.learnlib.oracle.equivalence.MMLTEQOracleChain; +import de.learnlib.oracle.equivalence.mmlt.RandomWpMethodEQOracle; +import de.learnlib.oracle.equivalence.mmlt.ResetSearchEQOracle; +import de.learnlib.oracle.equivalence.mmlt.SimulatorEQOracle; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.statistic.Statistics; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; + +/** + * This example shows a basic set-up of the MMLT-learner for a black-box setting. + *

+ * For this, we use a chain of different equivalence oracles that can be applied if the reference automaton is not + * known. + */ +@SuppressWarnings("PMD.UseExplicitTypes") // allow magic numbers and vars in examples +public final class Example2 { + + private static final int BOUND = 100; + private static final int MIN_SIZE = 16; + private static final double PERCENTAGE = 1.0; + private static final int SEED = 100; + + private Example2() { + // prevent instantiation + } + + public static void main(String[] args) { + var model = MMLTExamples.sensorCollector(); + var mmlt = model.getReferenceAutomaton(); + var alphabet = mmlt.getInputAlphabet(); + + // We first create a statistics container. + // This container will store various statistical data during learning: + var stats = Statistics.getCollector(); + stats.addText("model", null, model.toString()); + stats.setCounter("original_locs", "Locations in original", mmlt.getStates().size()); + stats.setCounter("original_inputs", "Untimed alphabet size in original", alphabet.size()); + + // ====================== + // Set up the pipeline: + // We use a simulator SUL to simulate our automaton: + var sul = new MMLTSimulatorSUL<>(mmlt); + + // We count all operations that are performed on the SUL with a stats-SUL: + var statsAfterCache = new CounterTimedSUL<>(sul); + + // We use a cache to avoid redundant operations: + var cacheSUL = new TimedSULTreeCache<>(statsAfterCache, model.getParams()); + var toReducerSul = new TimeoutReducerSUL<>(cacheSUL, model.getParams().maxTimeoutWaitingTime()); + + // We use a query oracle to answer queries from the learner: + var timeOracle = new TimedSULOracle<>(toReducerSul, model.getParams()); + + // We use a chain of different equivalence oracles to find counterexamples more efficiently: + var chainOracle = new MMLTEQOracleChain(); + // A cache oracle tests if the current hypothesis and the reference automaton give the same outputs + // for all words that have already been queried. As the words have already been queried, this + // executes no additional queries on the SUL: + chainOracle.addOracle(new CounterEQOracle<>(cacheSUL.createCacheConsistencyTest(), "cache")); + + // A ResetSearchOracle tests for missing local resets, which often require many and/or long test words + // when using random-based testing. We configure the tester to consider all transitions that might cause a reset: + chainOracle.addOracle(new CounterEQOracle<>(new ResetSearchEQOracle<>(timeOracle, SEED, PERCENTAGE, PERCENTAGE), + "reset")); + + // Finally, we add an MMLT-specific RandomWp oracle: + chainOracle.addOracle(new CounterEQOracle<>(new RandomWpMethodEQOracle<>(timeOracle, SEED, MIN_SIZE, 0, BOUND), + "wp")); + + // Set up our L* learner: + List>> suffixes = new ArrayList<>(); + alphabet.forEach(s -> suffixes.add(Word.fromLetter(TimedInput.input(s)))); + suffixes.add(Word.fromLetter(new TimeoutSymbol<>())); + + var learner = new ExtensibleLStarMMLTBuilder().withAlphabet(alphabet) + .withModelParams(model.getParams()) + .withTimeOracle(timeOracle) + .withInitialSuffixes(suffixes) + .create(); + + // Start learning: + var finalModel = + ExampleRunner.runExperiment(learner, chainOracle, mmlt.getSemantics().getInputAlphabet(), stats); + + // In this set-up, we actually know the reference automaton. + // This allows us to check that we learned an accurate model: + var simOracle = new SimulatorEQOracle<>(mmlt); + if (simOracle.findCounterExample(finalModel, finalModel.getSemantics().getInputAlphabet()) != null) { + throw new IllegalStateException("Incorrect model learned."); + } + + // Troubleshooting + // If you attempt to learn a model of some application and the learner + // throws assertion errors or illegal state exceptions, + // your SUL likely has no MMLT semantics. + // In this case, you can try to learn a partial model by excluding TimeStepSymbol + // from the input alphabet for the counterexample search: + // In your learn-loop (see ExampleUtil), replace + // tester.findCounterExample(hyp, hyp.getSemantics().getInputAlphabet()); + // with: tester.findCounterExample(hyp, hyp.getSemantics().getInputAlphabet().stream().filter(s -> !(s instanceof TimeStepSymbol)).toList()); + } + +} diff --git a/examples/src/main/java/de/learnlib/example/mmlt/Example3.java b/examples/src/main/java/de/learnlib/example/mmlt/Example3.java new file mode 100644 index 0000000000..6783989fbe --- /dev/null +++ b/examples/src/main/java/de/learnlib/example/mmlt/Example3.java @@ -0,0 +1,150 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.example.mmlt; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLTBuilder; +import de.learnlib.algorithm.lstar.mmlt.filter.MMLTRandomSymbolFilter; +import de.learnlib.algorithm.lstar.mmlt.filter.MMLTStatisticsSymbolFilter; +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.filter.SymbolFilter; +import de.learnlib.filter.cache.mmlt.TimedSULTreeCache; +import de.learnlib.filter.cache.mmlt.TimeoutReducerSUL; +import de.learnlib.filter.statistic.oracle.CounterEQOracle; +import de.learnlib.filter.statistic.sul.CounterTimedSUL; +import de.learnlib.filter.symbol.CachedSymbolFilter; +import de.learnlib.oracle.equivalence.MMLTEQOracleChain; +import de.learnlib.oracle.equivalence.mmlt.RandomWpMethodEQOracle; +import de.learnlib.oracle.equivalence.mmlt.ResetSearchEQOracle; +import de.learnlib.oracle.equivalence.mmlt.SimulatorEQOracle; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.statistic.Statistics; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; + +/** + * This example illustrates how to learn MMLTs with symbol filtering. + *

+ * A symbol filter is a component that provides information about transitions that might silently self-loop. The learner + * exploits this information to avoid redundant queries on the SUL. The symbol filter might incorrectly classify a + * transition as silent self-loop. The MMLT-learner detects and corrects such errors. + *

+ * LearnLib includes four types of symbol filter: + *

    + *
  • AcceptAllSymbolFilter: no transition is considered as silent self-loop. + * This is the default behavior if no filter is provided.
  • + *
  • PerfectSymbolFilter: simulates perfect knowledge of silent self-loops. + * Perfect knowledge is useful for benchmarking but rarely the case in practice.
  • + *
  • IgnoreAllSymbolFilter: considers all transitions to be silent self-loops. + * If no knowledge of the SUL is available, this filter still often yields strong performance improvements.
  • + *
  • RandomSymbolFilter: simulates incorrect responses with a certain percentage.
  • + *
+ *

+ * When you apply MMLT-learning in practice, you usually want to implement your own symbol filter that exploits specific + * domain knowledge. + */ +@SuppressWarnings({"checkstyle:magicnumber", "PMD.UseExplicitTypes"}) // allow magic numbers and vars in examples +public final class Example3 { + + private static final int BOUND = 100; + private static final double INACC_PROB = 0.1; + private static final int MIN_SIZE = 16; + private static final double PERCENTAGE = 1.0; + private static final int SEED = 100; + + private Example3() { + // prevent instantiation + } + + public static void main(String[] args) { + var model = MMLTExamples.sensorCollector(); + var mmlt = model.getReferenceAutomaton(); + var alphabet = mmlt.getInputAlphabet(); + + // We first create a statistics container. + // This container will store various statistical data during learning: + var stats = Statistics.getCollector(); + stats.addText("model", null, model.toString()); + stats.setCounter("original_locs", "Locations in original", mmlt.getStates().size()); + stats.setCounter("original_inputs", "Untimed alphabet size in original", alphabet.size()); + + // ====================== + // Set up the pipeline: + // We use a simulator SUL to simulate our automaton: + var sul = new MMLTSimulatorSUL<>(mmlt); + + // We count all operations that are performed on the SUL with a stats-SUL: + var statsAfterCache = new CounterTimedSUL<>(sul); + + // We use a cache to avoid redundant operations: + var cacheSUL = new TimedSULTreeCache<>(statsAfterCache, model.getParams()); + var toReducerSul = new TimeoutReducerSUL<>(cacheSUL, model.getParams().maxTimeoutWaitingTime()); + + // We use a query oracle to answer queries from the learner: + var timeOracle = new TimedSULOracle<>(toReducerSul, model.getParams()); + + // We use a chain of different equivalence oracles (see Example2): + MMLTEQOracleChain chainOracle = new MMLTEQOracleChain<>(); + chainOracle.addOracle(new CounterEQOracle<>(cacheSUL.createCacheConsistencyTest(), "cache")); + chainOracle.addOracle(new CounterEQOracle<>(new ResetSearchEQOracle<>(timeOracle, SEED, PERCENTAGE, PERCENTAGE), + "reset")); + chainOracle.addOracle(new CounterEQOracle<>(new RandomWpMethodEQOracle<>(timeOracle, SEED, MIN_SIZE, 0, BOUND), + "wp")); + + // Set up our L* learner: + List>> suffixes = new ArrayList<>(); + alphabet.forEach(s -> suffixes.add(Word.fromLetter(TimedInput.input(s)))); + suffixes.add(Word.fromLetter(new TimeoutSymbol<>())); + + // A symbol filter allows us to reduce queries by exploiting prior knowledge. + // For this example, we use a AbstractRandomSymbolFilter. This filter correctly predicts + // whether a transition silently self-loops with an accuracy of 90%: + SymbolFilter, InputSymbol> filter = + new MMLTRandomSymbolFilter<>(mmlt, INACC_PROB, new Random(SEED)); + + // We wrap our filter with a StatisticsFilter to collect useful statistics about the filter: + filter = new MMLTStatisticsSymbolFilter<>(mmlt, filter); + + // The learner may need to update incorrect responses of the filter. + // To facilitate this, we wrap our filter with a CachedFilter: + var cachedFilter = new CachedSymbolFilter<>(filter); + + var learner = new ExtensibleLStarMMLTBuilder().withAlphabet(alphabet) + .withModelParams(model.getParams()) + .withTimeOracle(timeOracle) + .withInitialSuffixes(suffixes) + .withSymbolFilter(cachedFilter) + .create(); + + // Start learning: + var finalModel = + ExampleRunner.runExperiment(learner, chainOracle, mmlt.getSemantics().getInputAlphabet(), stats); + + // In this set-up, we actually know the reference automaton. + // This allows us to check that we learned an accurate model: + var simOracle = new SimulatorEQOracle<>(mmlt); + if (simOracle.findCounterExample(finalModel, finalModel.getSemantics().getInputAlphabet()) != null) { + throw new IllegalStateException("Incorrect model learned."); + } + } + +} diff --git a/examples/src/main/java/de/learnlib/example/mmlt/Example4.java b/examples/src/main/java/de/learnlib/example/mmlt/Example4.java new file mode 100644 index 0000000000..8a971948f4 --- /dev/null +++ b/examples/src/main/java/de/learnlib/example/mmlt/Example4.java @@ -0,0 +1,135 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.example.mmlt; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; + +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLTBuilder; +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.filter.cache.mmlt.TimedSULTreeCache; +import de.learnlib.filter.cache.mmlt.TimeoutReducerSUL; +import de.learnlib.filter.statistic.sul.CounterTimedSUL; +import de.learnlib.oracle.equivalence.mmlt.SimulatorEQOracle; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.statistic.Statistics; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.impl.StringSymbolCombiner; +import net.automatalib.exception.FormatException; +import net.automatalib.serialization.dot.DOTMMLTParser; +import net.automatalib.serialization.dot.DOTParsers; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.util.automaton.mmlt.MMLTs; +import net.automatalib.word.Word; + +/** + * This example demonstrates how to load an MMLT from a dot-file and learn it using the L* algorithm. + *

+ * A description of the dot-file syntax for MMLTs can be found in AutomataLib (see {@link DOTMMLTParser}). + */ +@SuppressWarnings("PMD.UseExplicitTypes") // allow vars in examples +public final class Example4 { + + private Example4() { + // prevent instantiation + } + + public static void main(String[] args) { + // First, we load the file "mmlt_example.dot" from the "resources" folder: + MMLT targetModel; + MMLTModelParams params; + + // We define the output that represents silence: + var silentOutput = "void"; + + // In an MMLT, a timeout may yield multiple outputs. + // We use a symbol combiner to combine them into a single output. + // Here, we use a StringSymbolCombiner that sorts symbols and then concatenates them with a pipe: + var outputCombiner = StringSymbolCombiner.getInstance(); + var parser = DOTParsers.mmlt(silentOutput, outputCombiner); + + try (InputStream is = Example4.class.getResourceAsStream("/mmlt_example.dot")) { + var parsedModel = parser.readModel(is); + targetModel = parsedModel.model; + + // During learning, we use a symbolic "timeout" symbol to indicate that the + // teacher should wait for the next timeout. To avoid an infinite runtime, + // we set a maximum waiting time for these symbols. + // This time should be at least the maximum time to the next timeout in any + // state of the system. We configure this as follows: + long maxTimeoutDelay = MMLTs.getMaximumTimeoutDelay(targetModel); + + // After adding a new location, the learner infers timers for it by watching the SUL for timeouts. + // To learn an accurate model, the maximum time to watch for these timeouts must be at + // least the value of "maxTimeoutDelay". + // If the maximum initial value of timers in the SUL is known or can be reasonably estimated, + // setting the watch time to twice that value usually yields good results: + long maxTimerQueryWaitingFinal = MMLTs.getMaximumInitialTimerValue(targetModel) * 2; + + params = new MMLTModelParams<>(silentOutput, outputCombiner, maxTimeoutDelay, maxTimerQueryWaitingFinal); + } catch (IOException | FormatException e) { + throw new IllegalStateException("Unable to load model from file.", e); + } + + // Proceed as in Example1: + + var stats = Statistics.getCollector(); + stats.addText("model", null, "mmlt_example.dot"); + stats.setCounter("original_locs", "Locations in original", targetModel.getStates().size()); + stats.setCounter("original_inputs", "Untimed alphabet size in original", targetModel.getInputAlphabet().size()); + + // Set up the pipeline: + // We use a simulator SUL to simulate our automaton: + var sul = new MMLTSimulatorSUL<>(targetModel); + + // We count all operations that are performed on the SUL with a stats-SUL: + var statsAfterCache = new CounterTimedSUL<>(sul); + + // We use a cache to avoid redundant operations: + var cacheSUL = new TimedSULTreeCache<>(statsAfterCache, params); + var toReducerSul = new TimeoutReducerSUL<>(cacheSUL, params.maxTimeoutWaitingTime()); + + // We use a query oracle to answer queries from the learner: + var timeOracle = new TimedSULOracle<>(toReducerSul, params); + + // In the basic set-up, we use a simulator oracle to answer equivalence queries. + // This oracle has perfect knowledge of the reference automaton. + var eqOracle = new SimulatorEQOracle<>(targetModel); + + // Set up our L* learner: + + // We provide the learner with an initial set of suffixes. + // We include all untimed inputs and the symbolic timeout symbol, which causes the learner to wait + // until the next timeout (but no longer than params.maxTimeoutWaitingTime()). + List>> suffixes = new ArrayList<>(); + targetModel.getInputAlphabet().forEach(s -> suffixes.add(Word.fromLetter(TimedInput.input(s)))); + suffixes.add(Word.fromLetter(new TimeoutSymbol<>())); + + var learner = new ExtensibleLStarMMLTBuilder().withAlphabet(targetModel.getInputAlphabet()) + .withModelParams(params) + .withTimeOracle(timeOracle) + .withInitialSuffixes(suffixes) + .create(); + + // Start learning: + ExampleRunner.runExperiment(learner, eqOracle, targetModel.getSemantics().getInputAlphabet(), stats); + } + +} diff --git a/examples/src/main/java/de/learnlib/example/mmlt/ExampleRunner.java b/examples/src/main/java/de/learnlib/example/mmlt/ExampleRunner.java new file mode 100644 index 0000000000..05381d2bba --- /dev/null +++ b/examples/src/main/java/de/learnlib/example/mmlt/ExampleRunner.java @@ -0,0 +1,57 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.example.mmlt; + +import de.learnlib.algorithm.lstar.mmlt.ExtensibleLStarMMLT; +import de.learnlib.datastructure.observationtable.writer.ObservationTableASCIIWriter; +import de.learnlib.oracle.EquivalenceOracle; +import de.learnlib.statistic.StatisticsCollector; +import de.learnlib.util.Experiment; +import net.automatalib.alphabet.Alphabet; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.visualization.MMLTVisualizationHelper; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.visualization.Visualization; + +@SuppressWarnings({"PMD.SystemPrintln", "PMD.UseExplicitTypes"}) // allow for sysouts and vars in examples +final class ExampleRunner { + + private ExampleRunner() { + // prevent instantiation + } + + static MMLT runExperiment(ExtensibleLStarMMLT learner, + EquivalenceOracle.MMLTEquivalenceOracle tester, + Alphabet> alphabet, + StatisticsCollector statisticsCollector) { + // Start learning: + final var experiment = new Experiment<>(learner, tester, alphabet); + experiment.run(); + + final var finalHypothesis = experiment.getFinalHypothesis(); + + // Print final result + statistics: + System.out.println(statisticsCollector.printStats()); + + new ObservationTableASCIIWriter<>().write(learner.getObservationTable(), System.out); + + System.out.println("Final hypothesis:"); + Visualization.visualize(finalHypothesis.graphView(), + new MMLTVisualizationHelper<>(finalHypothesis, true, true)); + + return finalHypothesis; + } +} diff --git a/examples/src/main/java/de/learnlib/example/parallelism/ParallelismExample2.java b/examples/src/main/java/de/learnlib/example/parallelism/ParallelismExample2.java index 86ded3983c..01888cd422 100644 --- a/examples/src/main/java/de/learnlib/example/parallelism/ParallelismExample2.java +++ b/examples/src/main/java/de/learnlib/example/parallelism/ParallelismExample2.java @@ -30,6 +30,7 @@ import de.learnlib.oracle.ParallelOracle; import de.learnlib.oracle.parallelism.ParallelOracleBuilders; import de.learnlib.query.DefaultQuery; +import de.learnlib.statistic.Statistics; import de.learnlib.sul.SUL; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.Alphabets; @@ -110,9 +111,9 @@ private void runSingleCache() { final MealyMembershipOracle cache = MealyCaches.createCache(alphabet, parallelOracle); // print results - System.out.println("Single-threaded cache performance: "); + System.out.println("Single-threaded cache performance:"); answerQueries(cache); - System.out.println(" " + counter.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); parallelOracle.shutdownNow(); } @@ -142,9 +143,9 @@ private void runThreadSafeCache() { .create(); // print results - System.out.println("Shared cache performance: "); + System.out.println("Shared cache performance:"); answerQueries(parallelOracle); - System.out.println(" " + counter.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); parallelOracle.shutdownNow(); } diff --git a/examples/src/main/java/de/learnlib/example/resumable/ResumableExample.java b/examples/src/main/java/de/learnlib/example/resumable/ResumableExample.java index 7679367373..f70b7ff3b2 100644 --- a/examples/src/main/java/de/learnlib/example/resumable/ResumableExample.java +++ b/examples/src/main/java/de/learnlib/example/resumable/ResumableExample.java @@ -27,6 +27,7 @@ import de.learnlib.oracle.equivalence.DFASimulatorEQOracle; import de.learnlib.oracle.membership.DFASimulatorOracle; import de.learnlib.query.DefaultQuery; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.Alphabets; import net.automatalib.alphabet.impl.GrowingMapAlphabet; @@ -121,7 +122,7 @@ private static T fromBytes(byte[] bytes) { private static void printStats(Setup setup) { System.out.println("Hypothesis size: " + setup.learner.getHypothesisModel().size()); - System.out.println(setup.counter.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); System.out.println(); } diff --git a/examples/src/main/java/de/learnlib/example/sli/Example2.java b/examples/src/main/java/de/learnlib/example/sli/Example2.java index 18fa718fd8..e3fcb5f71f 100644 --- a/examples/src/main/java/de/learnlib/example/sli/Example2.java +++ b/examples/src/main/java/de/learnlib/example/sli/Example2.java @@ -34,6 +34,7 @@ import de.learnlib.oracle.equivalence.mealy.StateLocalInputMealySimulatorEQOracle; import de.learnlib.oracle.membership.SULOracle; import de.learnlib.oracle.membership.StateLocalInputSULOracle; +import de.learnlib.statistic.Statistics; import de.learnlib.sul.SUL; import de.learnlib.sul.StateLocalInputSUL; import de.learnlib.testsupport.example.mealy.ExampleRandomStateLocalInputMealy; @@ -82,6 +83,8 @@ public static void main(String[] args) { */ static void runSLILearner(boolean withCache) { + Statistics.getCollector().clear(); + // setup SULs and counters final StateLocalInputSUL target = new StateLocalInputMealySimulatorSUL<>(TARGET); final CounterStateLocalInputSUL counterSUL = new CounterStateLocalInputSUL<>(target); @@ -127,7 +130,7 @@ static void runSLILearner(boolean withCache) { System.out.println("State Local Input SUL" + (withCache ? ", with cache" : "")); System.out.println("-------------------------------------------------------"); - System.out.println(counterSUL.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); System.out.println("-------------------------------------------------------"); } @@ -137,6 +140,8 @@ static void runSLILearner(boolean withCache) { */ static void runNormalLearner(boolean withCache) { + Statistics.getCollector().clear(); + // setup SULs and counters final SUL target = new MealySimulatorSUL<>(TARGET, UNDEFINED); final CounterSUL counterSUL = new CounterSUL<>(target); @@ -181,7 +186,7 @@ static void runNormalLearner(boolean withCache) { System.out.println("Regular SUL" + (withCache ? ", with cache" : "")); System.out.println("-------------------------------------------------------"); - System.out.println(counterSUL.getStatisticalData().getSummary()); + System.out.println(Statistics.getCollector().printStats()); System.out.println("-------------------------------------------------------"); } diff --git a/examples/src/main/java/module-info.java b/examples/src/main/java/module-info.java index 30cc747eb9..22f860017f 100644 --- a/examples/src/main/java/module-info.java +++ b/examples/src/main/java/module-info.java @@ -36,6 +36,7 @@ requires de.learnlib.filter.cache; requires de.learnlib.filter.reuse; requires de.learnlib.filter.statistic; + requires de.learnlib.filter.symbol; requires de.learnlib.oracle.emptiness; requires de.learnlib.oracle.equivalence; requires de.learnlib.oracle.membership; diff --git a/examples/src/main/resources/mmlt_example.dot b/examples/src/main/resources/mmlt_example.dot new file mode 100644 index 0000000000..645f39e275 --- /dev/null +++ b/examples/src/main/resources/mmlt_example.dot @@ -0,0 +1,34 @@ +// This file demonstrates the syntax for defining a custom MMLT. +digraph g { + s0 [label="L0" timers="a=2"] + s1 [label="L1" timers="b=4,c=6"] + s2 [label="L2" timers="d=2,e=3"] + + // one-shot with location change: + s0 -> s1 [label="to[a] / A"] + + // periodic with multiple outputs, + // assuming a {net.automatalib.automaton.mmlt.impl.StringSymbolCombiner} to combine and separate outputs: + s1 -> s1 [label="to[b] / B|Z"] + + // one-shot with loop: + s1 -> s1 [label="to[c] / C" resets="b,c"] + + // periodic with explicit resets: + s2 -> s2 [label="to[d] / D" resets="d"] + + // periodic: + s2 -> s2 [label="to[e] / E"] + + // normal transition with silent output: + s1 -> s2 [label="x / void"] + + // loop with reset: + s1 -> s1 [label="y / Y" resets="b,c"] + + // loop without reset: + s2 -> s2 [label="y / D"] + + __start0 [label="" shape="none" width="0" height="0"]; + __start0 -> s0; +} \ No newline at end of file diff --git a/examples/src/test/java/de/learnlib/example/ExamplesTest.java b/examples/src/test/java/de/learnlib/example/ExamplesTest.java index f1491d23d7..e02ad99682 100644 --- a/examples/src/test/java/de/learnlib/example/ExamplesTest.java +++ b/examples/src/test/java/de/learnlib/example/ExamplesTest.java @@ -106,6 +106,30 @@ public void testBBCExample4() { de.learnlib.example.bbc.Example4.main(new String[0]); } + @Test + public void testMMLTExample1() { + requireJVMCompatibility(); + de.learnlib.example.mmlt.Example1.main(new String[0]); + } + + @Test + public void testMMLTExample2() { + requireJVMCompatibility(); + de.learnlib.example.mmlt.Example2.main(new String[0]); + } + + @Test + public void testMMLTExample3() { + requireJVMCompatibility(); + de.learnlib.example.mmlt.Example3.main(new String[0]); + } + + @Test + public void testMMLTExample4() { + requireJVMCompatibility(); + de.learnlib.example.mmlt.Example4.main(new String[0]); + } + @Test public void testParallelismExample1() { de.learnlib.example.parallelism.ParallelismExample1.main(new String[0]); diff --git a/filters/cache/pom.xml b/filters/cache/pom.xml index 55327c4491..1dcf9a2678 100644 --- a/filters/cache/pom.xml +++ b/filters/cache/pom.xml @@ -109,6 +109,10 @@ limitations under the License. test + + org.mockito + mockito-core + org.testng testng diff --git a/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCache.java b/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCache.java index 9f6b132444..5c03a934bb 100644 --- a/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCache.java +++ b/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCache.java @@ -17,8 +17,11 @@ import de.learnlib.oracle.EquivalenceOracle; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; import net.automatalib.word.Word; /** @@ -84,4 +87,15 @@ interface MealyLearningCache extends LearningCache extends LearningCache, I, Word> {} + + /** + * Specialization of the {@link LearningCache} interface for MMLT learning. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ + @FunctionalInterface + interface MMLTLearningCache extends LearningCache, TimedInput, Word>> {} } diff --git a/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCacheOracle.java b/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCacheOracle.java index fbc158cbaf..1d0e3bfbeb 100644 --- a/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCacheOracle.java +++ b/filters/cache/src/main/java/de/learnlib/filter/cache/LearningCacheOracle.java @@ -16,9 +16,13 @@ package de.learnlib.filter.cache; import de.learnlib.oracle.MembershipOracle; +import de.learnlib.oracle.TimedQueryOracle; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; import net.automatalib.word.Word; /** @@ -65,4 +69,17 @@ interface MealyLearningCacheOracle extends LearningCacheOracle extends LearningCacheOracle, I, Word>, MooreLearningCache, MooreMembershipOracle {} + + /** + * Specialization of the {@link LearningCacheOracle} interface for MMLT learning. + * + * @param + * input symbol type + * @param + * output symbol type + */ + interface MMMLTLearningCacheOracle + extends LearningCacheOracle, TimedInput, Word>>, + MMLTLearningCache, + TimedQueryOracle {} } diff --git a/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/CacheTreeNode.java b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/CacheTreeNode.java new file mode 100644 index 0000000000..770522affb --- /dev/null +++ b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/CacheTreeNode.java @@ -0,0 +1,151 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.cache.mmlt; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import org.checkerframework.checker.nullness.qual.EnsuresNonNullIf; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A node in the {@link TimedSULTreeCache}. A node has a parent and children for an arbitrary number of transitions with + * a non-delaying input. There is at most one timed transition. This transition has a sequence of time steps as input. + * The output is the output at the last time step in the sequence. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +class CacheTreeNode { + + private @Nullable CacheTreeNode parent; + private @Nullable TimedInput parentInput; + private long timeout; + private @Nullable CacheTreeTransition timeTransition; + private final Map, CacheTreeTransition> untimedChildren; + + CacheTreeNode(@Nullable CacheTreeNode parent, @Nullable TimedInput parentInput) { + this.parent = parent; + this.parentInput = parentInput; + + this.timeTransition = null; + this.timeout = -1; + + this.untimedChildren = new HashMap<>(); + } + + CacheTreeNode addTimeChild(long timeout, TimedOutput output) { + assert !this.hasTimeChild() : "State already has time child."; + CacheTreeNode newChild = new CacheTreeNode<>(this, new TimeStepSequence<>(timeout)); + this.timeout = timeout; + this.timeTransition = new CacheTreeTransition<>(output, newChild); + return newChild; + } + + // ------------------------------------------------------- + + @EnsuresNonNullIf(result = true, expression = "this.timeTransition") + boolean hasTimeChild() { + return this.timeTransition != null; + } + + long getTimeout() { + assert this.hasTimeChild(); + return timeout; + } + + TimedOutput getTimeoutOutput() { + assert this.hasTimeChild(); + return this.timeTransition.output(); + } + + CacheTreeNode getTimeoutChild() { + assert this.hasTimeChild(); + return this.timeTransition.target(); + } + + /** + * Breaks the time sequence: introduces a new child cx after the given number of time steps and adds the former + * child as child to cx. + * + * @param newTimeout + * Time at which the timeout sequence is split + * @param output + * Output at the end of the new time sequence + * + * @return New child node + */ + CacheTreeNode splitTimeout(long newTimeout, TimedOutput output) { + assert this.hasTimeChild() && newTimeout < this.getTimeout() : "Must split at lower timeout."; + + CacheTreeNode newChild = new CacheTreeNode<>(this, new TimeStepSequence<>(newTimeout)); + newChild.timeout = this.timeout - newTimeout; + newChild.timeTransition = this.timeTransition; // keep output + target + this.timeTransition.target().setParent(newChild, new TimeStepSequence<>(this.timeout - newTimeout)); + + this.timeout = newTimeout; + this.timeTransition = new CacheTreeTransition<>(output, newChild); + + return newChild; + } + + // ------------------------------------------------------- + CacheTreeNode getParent() { + return parent; + } + + TimedInput getParentInput() { + return parentInput; + } + + void setParent(CacheTreeNode parent, TimedInput parentInput) { + this.parent = parent; + this.parentInput = parentInput; + } + + // ------------------------------------------------------- + boolean hasChild(InputSymbol input) { + return this.untimedChildren.containsKey(input); + } + + TimedOutput getOutput(InputSymbol input) { + return this.untimedChildren.get(input).output(); + } + + CacheTreeNode getChild(InputSymbol input) { + return this.untimedChildren.get(input).target(); + } + + CacheTreeNode addUntimedChild(InputSymbol input, TimedOutput output) { + assert !untimedChildren.containsKey(input) : "State already has an child for this input."; + CacheTreeNode child = new CacheTreeNode<>(this, input); + this.untimedChildren.put(input, new CacheTreeTransition<>(output, child)); + return child; + } + + Map, CacheTreeTransition> getUntimedChildren() { + return Collections.unmodifiableMap(this.untimedChildren); + } + + record CacheTreeTransition(TimedOutput output, CacheTreeNode target) {} +} diff --git a/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/MMLTCacheConsistencyTest.java b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/MMLTCacheConsistencyTest.java new file mode 100644 index 0000000000..e5ab6cc683 --- /dev/null +++ b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/MMLTCacheConsistencyTest.java @@ -0,0 +1,214 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.cache.mmlt; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; + +import de.learnlib.oracle.EquivalenceOracle.MMLTEquivalenceOracle; +import de.learnlib.query.DefaultQuery; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; +import net.automatalib.word.WordBuilder; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Searches for counterexamples by comparing the behavior of the hypothesis and the query cache. If there are multiple + * counterexamples, the shortest one is returned. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +@SuppressWarnings("PMD.TestClassWithoutTestCases") // not a traditional test class +public class MMLTCacheConsistencyTest implements MMLTEquivalenceOracle { + + private static final Logger LOGGER = LoggerFactory.getLogger(MMLTCacheConsistencyTest.class); + + private final TimedSULTreeCache sulCache; + private final MMLTModelParams modelParams; + + MMLTCacheConsistencyTest(TimedSULTreeCache sulCache, MMLTModelParams modelParams) { + this.sulCache = sulCache; + this.modelParams = modelParams; + } + + private DefaultQuery, Word>> queryCache(Word> word) { + WordBuilder> wbInput = new WordBuilder<>(); + WordBuilder> wbOutput = new WordBuilder<>(); + + this.sulCache.pre(); + for (TimedInput sym : word) { + if (sym instanceof InputSymbol ndi) { + TimedOutput res = this.sulCache.step(ndi); + wbInput.append(ndi); + wbOutput.append(res); + } else if (sym instanceof TimeStepSequence ws) { + TimedOutput res = this.sulCache.timeoutStep(ws.timeSteps()); + wbInput.append(ws); + + if (res == null) { + wbOutput.append(new TimedOutput<>(this.modelParams.silentOutput())); + } else { + wbOutput.append(res); + } + } else { + throw new IllegalArgumentException("Symbol type must not be used in cache."); + } + } + this.sulCache.post(); + + return new DefaultQuery<>(wbInput.toWord(), wbOutput.toWord()); + } + + /** + * The cache does not use timeout symbols. Using these instead of time-step-sequences has several performance + * benefits. This function converts a query with a time-step-sequence to one that uses timeout symbols where + * possible. + * + * @param originalQuery + * Original query + * + * @return Converted query + */ + private DefaultQuery, Word>> convertTimeSequences(DefaultQuery, Word>> originalQuery) { + WordBuilder> wbInput = new WordBuilder<>(); + WordBuilder> wbOutput = new WordBuilder<>(); + + int symIdx = 0; + Word> queryInput = originalQuery.getInput(); + Word> queryOutput = originalQuery.getOutput(); + + while (symIdx < queryInput.length()) { + TimedInput inputSym = queryInput.getSymbol(symIdx); + TimedOutput outputSym = queryOutput.getSymbol(symIdx); + symIdx++; + + if (inputSym instanceof InputSymbol ds) { + wbInput.append(ds); + wbOutput.append(outputSym); + } else if (inputSym instanceof TimeStepSequence ws) { + if (!Objects.equals(outputSym.symbol(), this.modelParams.silentOutput()) || + ws.timeSteps() == this.modelParams.maxTimeoutWaitingTime()) { + // Found a timeout OR no timeout after max_delay: + wbInput.append(new TimeoutSymbol<>()); + wbOutput.append(outputSym); + continue; + } + + assert ws.timeSteps() < this.modelParams.maxTimeoutWaitingTime() : + "Wait time that exceeds max_delay in cache."; + + // Special case: silent output before max delay + // Cannot replace with "timeout", as this implies wait until max_delay. + // Hence: skip subsequent waits until reaching wait with output OR max_delay OR end of word: + long combinedWaitTime = ws.timeSteps(); + TimedOutput combinedOutput = outputSym; + + while (Objects.equals(combinedOutput.symbol(), this.modelParams.silentOutput()) && + combinedWaitTime < this.modelParams.maxTimeoutWaitingTime() && symIdx < queryInput.length() && + queryInput.getSymbol(symIdx) instanceof TimeStepSequence nextWs) { + combinedWaitTime += nextWs.timeSteps(); + combinedOutput = queryOutput.getSymbol(symIdx); + symIdx++; + } + + if (combinedWaitTime >= this.modelParams.maxTimeoutWaitingTime() || + !Objects.equals(combinedOutput.symbol(), this.modelParams.silentOutput())) { + wbInput.append(new TimeoutSymbol<>()); + + if (Objects.equals(combinedOutput.symbol(), this.modelParams.silentOutput())) { + // Reached max delay -> waiting for any time will now produce no more timeouts: + wbOutput.append(new TimedOutput<>(this.modelParams.silentOutput())); + } else { + // Found non-silent output: + wbOutput.append(new TimedOutput<>(combinedOutput.symbol(), combinedWaitTime)); + } + } else { + // Reached end of word before max_delay OR non-wait symbol -> ignore rest of this word: + if (symIdx < queryInput.length() - 1) { + LOGGER.debug("Ignoring at least one symbol during cache comparison."); + } + break; + } + } + } + return new DefaultQuery<>(wbInput.toWord(), wbOutput.toWord()); + } + + private DefaultQuery, Word>> reduceToAllowedInputs(Set> allowedInputs, + DefaultQuery, Word>> query) { + // Find the longest prefix with allowed inputs: + int prefixLength = 0; + while (prefixLength < query.length() && allowedInputs.contains(query.getInput().getSymbol(prefixLength))) { + prefixLength++; + } + + if (prefixLength == query.length()) { + return query; // maximum length -> no need to reduce + } else { + return new DefaultQuery<>(query.getInput().subWord(0, prefixLength), + query.getOutput().subWord(0, prefixLength)); + } + } + + @Override + public @Nullable DefaultQuery, Word>> findCounterExample(MMLT hypothesis, + Collection> inputs) { + Set> allowedInputs = new HashSet<>(inputs); + boolean allInputsConsidered = allowedInputs.containsAll(hypothesis.getSemantics().getInputAlphabet()); + + // Iterator over all cached words: + Iterator>> iter = this.sulCache.allWordsIterator(); + + while (iter.hasNext()) { + Word> word = iter.next(); + + // First, query word as-is (may include wait-symbols in input): + DefaultQuery, Word>> rawCacheQuery = this.queryCache(word); + + // Next, convert query that includes wait-symbols to query with timeout-symbols: + DefaultQuery, Word>> convertedQuery = this.convertTimeSequences(rawCacheQuery); + + // The counterexample may only use a subset of the allowed inputs. + // If so, cut the query to the prefix of the word that is allowed: + DefaultQuery, Word>> reducedQuery = + allInputsConsidered ? convertedQuery : this.reduceToAllowedInputs(allowedInputs, convertedQuery); + + // Finally, query hypothesis using the converted query: + Word> hypOutput = hypothesis.getSemantics().computeOutput(reducedQuery.getInput()); + + if (!hypOutput.equals(reducedQuery.getOutput())) { + // Hyp gives different output than cache (= SUL): + return reducedQuery; + } + } + + return null; + } +} diff --git a/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/TimedSULTreeCache.java b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/TimedSULTreeCache.java new file mode 100644 index 0000000000..de88d99814 --- /dev/null +++ b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/TimedSULTreeCache.java @@ -0,0 +1,242 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.cache.mmlt; + +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.Iterator; + +import de.learnlib.filter.cache.LearningCache.MMLTLearningCache; +import de.learnlib.filter.cache.mmlt.CacheTreeNode.CacheTreeTransition; +import de.learnlib.oracle.EquivalenceOracle.MMLTEquivalenceOracle; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; +import de.learnlib.sul.TimedSUL; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.common.util.collection.AbstractSimplifiedIterator; +import net.automatalib.common.util.collection.IteratorUtil; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import net.automatalib.word.WordBuilder; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Caches queries sent to a {@link TimedSUL}. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class TimedSULTreeCache implements TimedSUL, MMLTLearningCache { + + private final TimedSUL delegate; + + private final CacheTreeNode cacheRoot; + private @Nullable CacheTreeNode currentState; + + private final MMLTModelParams modelParams; + private final TimedOutput silentOutput; + private boolean cacheMiss; + + private final StatisticsCollector statisticsCollector; + + public TimedSULTreeCache(TimedSUL delegate, MMLTModelParams modelParams) { + this.delegate = delegate; + this.modelParams = modelParams; + this.silentOutput = new TimedOutput<>(modelParams.silentOutput()); + this.statisticsCollector = Statistics.getCollector(); + + // Init cache: + this.cacheRoot = new CacheTreeNode<>(null, null); + this.currentState = null; + } + + private void followCurrentPrefix() { + this.delegate.pre(); + + WordBuilder> wbPrefix = new WordBuilder<>(); + + CacheTreeNode current = this.currentState; + while (current.getParent() != null) { + wbPrefix.append(current.getParentInput()); + current = current.getParent(); + } + + Word> prefix = wbPrefix.reverse().toWord(); + this.delegate.follow(prefix); + } + + @Override + public TimedOutput step(InputSymbol input) { + if (this.currentState == null) { + throw new IllegalStateException(); + } + + if (!cacheMiss) { + if (this.currentState.hasChild(input)) { + TimedOutput output = this.currentState.getOutput(input); + this.currentState = this.currentState.getChild(input); + return output; + } + this.followCurrentPrefix(); + this.cacheMiss = true; + } + + // Cache miss -> query + insert: + TimedOutput output = this.delegate.step(input); + this.currentState = this.currentState.addUntimedChild(input, output); + return output; + } + + @Override + public @Nullable TimedOutput timeoutStep(long maxTime) { + if (currentState == null) { + throw new IllegalStateException(); + } + + long remaining = maxTime; + if (!this.cacheMiss) { + // Move to closest state in cache: + while (remaining > 0) { + if (!currentState.hasTimeChild()) { + break; // cache miss + } + + if (currentState.getTimeout() > remaining) { + // Split current timeout: + this.currentState = this.currentState.splitTimeout(remaining, this.silentOutput); + return null; // no timer in this state + } + + TimedOutput currentOutput = currentState.getTimeoutOutput(); + remaining -= currentState.getTimeout(); + this.currentState = this.currentState.getTimeoutChild(); + + if (!currentOutput.equals(this.silentOutput)) { + // Found valid timeout: + return new TimedOutput<>(currentOutput.symbol(), maxTime - remaining); + } + } + + if (remaining == 0) { + return null; // no timer in this state + } + + this.followCurrentPrefix(); + this.cacheMiss = true; + } + + TimedOutput timeoutStepResult = this.delegate.timeoutStep(remaining); + if (timeoutStepResult == null) { // no timers here + this.currentState = this.currentState.addTimeChild(remaining, this.silentOutput); + return null; + } else { + this.currentState = this.currentState.addTimeChild(timeoutStepResult.delay(), + new TimedOutput<>(timeoutStepResult.symbol())); + return new TimedOutput<>(timeoutStepResult.symbol(), maxTime - remaining + timeoutStepResult.delay()); + } + + } + + @Override + public void pre() { + this.currentState = this.cacheRoot; + this.cacheMiss = false; + } + + @Override + public void post() { + this.currentState = null; + + if (this.cacheMiss) { + this.delegate.post(); + statisticsCollector.increaseCounter("Cache_Missed_Count", "Cache misses"); + } else { + statisticsCollector.increaseCounter("Cache_Hit_Count", "Cache hits"); + } + } + + @Override + public MMLTEquivalenceOracle createCacheConsistencyTest() { + return new MMLTCacheConsistencyTest<>(this, this.modelParams); + } + + /** + * Returns an iterator that traverses all words (leaves of this tree) in a BFS-style fashion. + * + * @return iterator over all words of this tree + */ + public Iterator>> allWordsIterator() { + return IteratorUtil.map(new LeavesIterator<>(this.cacheRoot), this::extractWord); + } + + private Word> extractWord(CacheTreeNode leaf) { + final WordBuilder> wb = new WordBuilder<>(); + + // Move towards the root: + CacheTreeNode current = leaf; + while (current.getParent() != null) { + wb.append(current.getParentInput()); + current = current.getParent(); + } + + // Start at root -> flip buffer: + wb.reverse(); + return wb.toWord(); + } + + private static final class LeavesIterator extends AbstractSimplifiedIterator> { + + private final Deque> queue; + + private LeavesIterator(CacheTreeNode root) { + this.queue = new ArrayDeque<>(); + this.queue.add(root); + } + + @Override + protected boolean calculateNext() { + + while (!queue.isEmpty()) { + @SuppressWarnings("nullness") //false positive https://github.com/typetools/checker-framework/issues/399 + final @NonNull CacheTreeNode node = queue.poll(); + + boolean hasChildren = false; + + if (node.hasTimeChild()) { + queue.add(node.getTimeoutChild()); + hasChildren = true; + } + + for (CacheTreeTransition t : node.getUntimedChildren().values()) { + queue.add(t.target()); + hasChildren = true; + } + + if (!hasChildren) { + super.nextValue = node; + return true; + } + } + + return false; + } + } +} diff --git a/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/TimeoutReducerSUL.java b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/TimeoutReducerSUL.java new file mode 100644 index 0000000000..33b3256b1c --- /dev/null +++ b/filters/cache/src/main/java/de/learnlib/filter/cache/mmlt/TimeoutReducerSUL.java @@ -0,0 +1,84 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.cache.mmlt; + +import de.learnlib.sul.TimedSUL; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedOutput; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Avoids redundant queries for timeouts. + *

+ * Assume we waited maxDelay for a timeout and observed no expiration. Then any consecutive timeout-input must also show + * no timer (assuming sufficient maxDelay). Hence, we do not need to query the SUL for these. + *

+ * We may observe a timeout again after any non-delaying input, as this may trigger a change of location. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class TimeoutReducerSUL implements TimedSUL { + + private final TimedSUL delegate; + private final long maxDelay; + + /** + * Delay since the last non-delaying input OR timer expiration. + */ + private long noTimeoutWaitingTime; + + public TimeoutReducerSUL(TimedSUL delegate, long maxDelay) { + this.delegate = delegate; + this.maxDelay = maxDelay; + } + + @Override + public TimedOutput step(InputSymbol input) { + this.noTimeoutWaitingTime = 0; // might observe expirations again + return delegate.step(input); + } + + @Override + public @Nullable TimedOutput timeoutStep(long maxTime) { + if (this.noTimeoutWaitingTime >= this.maxDelay) { + return null; // cannot observe expiration until non-delaying input + } + + TimedOutput result = delegate.timeoutStep(maxTime); + + if (result == null) { + this.noTimeoutWaitingTime += maxTime; + } else { + this.noTimeoutWaitingTime = 0; + } + + return result; + } + + @Override + public void pre() { + delegate.pre(); + this.noTimeoutWaitingTime = 0; + } + + @Override + public void post() { + delegate.post(); + } +} diff --git a/filters/cache/src/main/java/de/learnlib/filter/cache/sul/SULCaches.java b/filters/cache/src/main/java/de/learnlib/filter/cache/sul/SULCaches.java index aff177cb97..ac3e1ed4cd 100644 --- a/filters/cache/src/main/java/de/learnlib/filter/cache/sul/SULCaches.java +++ b/filters/cache/src/main/java/de/learnlib/filter/cache/sul/SULCaches.java @@ -15,8 +15,11 @@ */ package de.learnlib.filter.cache.sul; +import de.learnlib.filter.cache.mmlt.TimedSULTreeCache; import de.learnlib.sul.SUL; import de.learnlib.sul.StateLocalInputSUL; +import de.learnlib.sul.TimedSUL; +import de.learnlib.time.MMLTModelParams; import net.automatalib.alphabet.Alphabet; import net.automatalib.incremental.mealy.dag.IncrementalMealyDAGBuilder; import net.automatalib.incremental.mealy.tree.IncrementalMealyTreeBuilder; @@ -33,8 +36,8 @@ private SULCaches() { /** * Creates a {@link SULCache} for a given {@link SUL}. *

- * Note that this method does not specify the implementation to use for the cache. Currently, a DAG ({@link - * #createDAGCache}) is used; however, this may change in the future. + * Note that this method does not specify the implementation to use for the cache. Currently, a DAG + * ({@link #createDAGCache}) is used; however, this may change in the future. * * @param alphabet * the input alphabet @@ -94,8 +97,8 @@ public static SULCache createTreeCache(Alphabet alphabet, SUL - * Note that this method does not specify the implementation to use for the cache. Currently, a tree ({@link - * #createStateLocalInputTreeCache}) is used; however, this may change in the future. + * Note that this method does not specify the implementation to use for the cache. Currently, a tree + * ({@link #createStateLocalInputTreeCache}) is used; however, this may change in the future. * * @param alphabet * the input alphabet @@ -134,4 +137,42 @@ public static StateLocalInputSULCache createStateLocalInputTreeCach StateLocalInputSUL sul) { return new StateLocalInputSULCache<>(new IncrementalMealyTreeBuilder<>(alphabet), sul); } + + /** + * Creates a {@link TimedSULTreeCache} for a given {@link TimedSUL}. + * + * @param sul + * the sul + * @param params + * the specific parameter for time related queries + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return the cache + * + */ + public static TimedSULTreeCache createTimedCache(TimedSUL sul, MMLTModelParams params) { + return createTimedTreeCache(sul, params); + } + + /** + * Creates a {@link TimedSULTreeCache} for a given {@link TimedSUL}, using a tree for internal cache organization. + * + * @param sul + * the sul + * @param params + * the specific parameter for time related queries + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return the cache + * + */ + public static TimedSULTreeCache createTimedTreeCache(TimedSUL sul, MMLTModelParams params) { + return new TimedSULTreeCache<>(sul, params); + } } diff --git a/filters/cache/src/main/java/module-info.java b/filters/cache/src/main/java/module-info.java index d75b963446..edd994d448 100644 --- a/filters/cache/src/main/java/module-info.java +++ b/filters/cache/src/main/java/module-info.java @@ -47,4 +47,5 @@ exports de.learnlib.filter.cache.mealy; exports de.learnlib.filter.cache.moore; exports de.learnlib.filter.cache.sul; + exports de.learnlib.filter.cache.mmlt; } diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractCacheTest.java index 4f26db5f61..9a6daf510d 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractCacheTest.java @@ -24,10 +24,10 @@ import de.learnlib.oracle.EquivalenceOracle; import de.learnlib.query.DefaultQuery; import de.learnlib.query.Query; +import de.learnlib.statistic.Statistics; import de.learnlib.testsupport.ResumeUtils; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.SupportsGrowingAlphabet; -import net.automatalib.automaton.concept.Output; import net.automatalib.word.Word; import net.automatalib.word.WordBuilder; import org.testng.Assert; @@ -37,7 +37,7 @@ /** * A simple test against various cache implementations. */ -public abstract class AbstractCacheTest, A extends Output, I, D> { +public abstract class AbstractCacheTest, A, I, D> { protected static final int LENGTH = 5; private final Random random = new Random(42); @@ -50,6 +50,7 @@ public void setup() { alphabet = getAlphabet(); oracle = getCachedOracle(); queries = new ArrayList<>(); + Statistics.getCollector().clear(); } @Test @@ -123,8 +124,8 @@ public void testCacheConsistency() { Assert.assertNull(targetCE); Assert.assertNotNull(invalidTargetCE); - Assert.assertNotEquals(invalidTarget.computeOutput(invalidTargetCE.getInput()), - target.computeOutput(invalidTargetCE.getInput())); + Assert.assertNotEquals(computeOutput(invalidTarget, invalidTargetCE.getInput()), + computeOutput(target, invalidTargetCE.getInput())); } @Test(dependsOnMethods = "testCacheConsistency") @@ -248,6 +249,8 @@ protected Query getQuery(int i) { protected abstract OR getResumedOracle(OR original); + protected abstract D computeOutput(A model, Word input); + protected abstract long getNumberOfPosedQueries(); protected abstract boolean supportsPrefixes(); diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractParallelCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractParallelCacheTest.java index 8377ff6edd..b881299500 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractParallelCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/AbstractParallelCacheTest.java @@ -22,6 +22,7 @@ import de.learnlib.oracle.EquivalenceOracle; import de.learnlib.oracle.ParallelOracle; import de.learnlib.query.DefaultQuery; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.common.util.collection.IterableUtil; import net.automatalib.word.Word; @@ -64,6 +65,7 @@ public void setUp() { this.targetModel = getTargetModel(); this.cache = getCacheRepresentative(); this.parallelOracle = getParallelOracle(); + Statistics.getCollector().clear(); } @AfterClass @@ -71,7 +73,7 @@ public void teardown() { this.parallelOracle.shutdownNow(); } - @Test(timeOut = 20000) + @Test public void testConcurrentMembershipQueries() { Assert.assertEquals(getNumberOfQueries(), 0); @@ -103,7 +105,7 @@ public void testConcurrentMembershipQueries() { Assert.assertEquals(numOfQueriesAfter, numOfQueriesBefore); } - @Test(dependsOnMethods = "testConcurrentMembershipQueries", timeOut = 20000) + @Test(dependsOnMethods = "testConcurrentMembershipQueries") public void testConcurrentEquivalenceQueries() { final long previousCount = getNumberOfQueries(); final EquivalenceOracle eqOracle = cache.createCacheConsistencyTest(); diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/CacheTestUtils.java b/filters/cache/src/test/java/de/learnlib/filter/cache/CacheTestUtils.java index ea47d5017f..e1405f8b19 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/CacheTestUtils.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/CacheTestUtils.java @@ -29,9 +29,12 @@ import de.learnlib.oracle.membership.MooreSimulatorOracle; import de.learnlib.sul.SUL; import de.learnlib.sul.StateLocalInputSUL; +import de.learnlib.time.MMLTModelParams; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.Alphabets; import net.automatalib.automaton.fsa.impl.CompactDFA; +import net.automatalib.automaton.mmlt.impl.CompactMMLT; +import net.automatalib.automaton.mmlt.impl.StringSymbolCombiner; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; import net.automatalib.automaton.transducer.impl.CompactMealy; @@ -50,6 +53,9 @@ public final class CacheTestUtils { public static final CompactMealy MEALY_INVALID; public static final CompactMoore MOORE; public static final CompactMoore MOORE_INVALID; + public static final CompactMMLT MMLT; + public static final MMLTModelParams MMLT_PARAMS; + public static final CompactMMLT MMLT_INVALID; public static final SUL SUL; public static final StateLocalInputSUL SLI_SUL; @@ -73,12 +79,44 @@ public final class CacheTestUtils { MEALY_INVALID = RandomAutomata.randomMealy(random, size, combinedAlphabet, OUTPUT_ALPHABET); MOORE_INVALID = RandomAutomata.randomMoore(random, size, combinedAlphabet, OUTPUT_ALPHABET); + MMLT = buildMMLT(); + MMLT_PARAMS = new MMLTModelParams<>("void", StringSymbolCombiner.getInstance(), 4, 80); + MMLT_INVALID = buildMMLT(); + MMLT_INVALID.removeTimer(2, "d"); + MMLT_INVALID.addPeriodicTimer(2, "d", 4, "done"); + SUL = new MealySimulatorSUL<>(MEALY); SLI_SUL = new StateLocalInputMealySimulatorSUL<>(MEALY); } private CacheTestUtils() {} + private static CompactMMLT buildMMLT() { + var alphabet = Alphabets.fromArray("p1", "p2", "abort", "collect"); + var model = new CompactMMLT<>(alphabet, "void", StringSymbolCombiner.getInstance()); + + var s0 = model.addInitialState(); + var s1 = model.addState(); + var s2 = model.addState(); + var s3 = model.addState(); + + model.addTransition(s0, "p1", s1, "go"); + model.addTransition(s1, "abort", s1, "ok"); + model.addLocalReset(s1, "abort"); + + model.addPeriodicTimer(s1, "a", 3, "part"); + model.addPeriodicTimer(s1, "b", 6, "noise"); + model.addOneShotTimer(s1, "c", 40, "done", s3); + + model.addTransition(s0, "p2", s2, "go"); + model.addTransition(s2, "abort", s3, "void"); + model.addOneShotTimer(s2, "d", 4, "done", s3); + + model.addTransition(s3, "collect", s0, "void"); + + return model; + } + public static DFACounterOracle getCounter(net.automatalib.automaton.fsa.DFA delegate) { return new DFACounterOracle<>(new DFASimulatorOracle<>(delegate)); } diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/TimedSULLearningCacheOracle.java b/filters/cache/src/test/java/de/learnlib/filter/cache/TimedSULLearningCacheOracle.java new file mode 100644 index 0000000000..d6d70e5b5a --- /dev/null +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/TimedSULLearningCacheOracle.java @@ -0,0 +1,72 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.cache; + +import java.util.Collection; + +import de.learnlib.filter.cache.LearningCache.MMLTLearningCache; +import de.learnlib.filter.cache.LearningCacheOracle.MMMLTLearningCacheOracle; +import de.learnlib.filter.cache.mmlt.TimedSULTreeCache; +import de.learnlib.oracle.EquivalenceOracle; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.query.Query; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; + +public class TimedSULLearningCacheOracle> + implements MMMLTLearningCacheOracle { + + private final C cache; + private final TimedQueryOracle oracle; + + public TimedSULLearningCacheOracle(C cache, TimedQueryOracle oracle) { + this.cache = cache; + this.oracle = oracle; + } + + @Override + public void processQueries(Collection, Word>>> queries) { + oracle.processQueries(queries); + } + + @Override + public EquivalenceOracle, TimedInput, Word>> createCacheConsistencyTest() { + return cache.createCacheConsistencyTest(); + } + + public C getCache() { + return cache; + } + + public TimedQueryOracle getOracle() { + return oracle; + } + + @Override + public TimerQueryResult queryTimers(Word> prefix, long maxTotalWaitingTime) { + return oracle.queryTimers(prefix, maxTotalWaitingTime); + } + + public static TimedSULLearningCacheOracle> fromTimedSULCache(TimedSULTreeCache cache, + MMLTModelParams params) { + return new TimedSULLearningCacheOracle<>(cache, new TimedSULOracle<>(cache, params)); + } + +} diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/AbstractDFACacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/AbstractDFACacheTest.java index 9b1fb889d3..ba496cb0fa 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/AbstractDFACacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/AbstractDFACacheTest.java @@ -20,9 +20,11 @@ import de.learnlib.filter.statistic.oracle.DFACounterOracle; import de.learnlib.oracle.MembershipOracle.DFAMembershipOracle; import de.learnlib.oracle.membership.DFASimulatorOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.GrowingMapAlphabet; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.word.Word; public abstract class AbstractDFACacheTest extends AbstractCacheTest, DFA, Character, Boolean> { @@ -55,9 +57,14 @@ protected DFACacheOracle getResumedOracle(DFACacheOracle o return fresh; } + @Override + protected Boolean computeOutput(DFA model, Word input) { + return model.computeOutput(input); + } + @Override protected long getNumberOfPosedQueries() { - return counter.getQueryCounter().getCount(); + return Statistics.getCollector().getCount(DFACounterOracle.QUERY_KEY).orElse(0L); } @Override diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAHashCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAHashCacheTest.java index fb6c1735ef..4ee38c62fc 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAHashCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAHashCacheTest.java @@ -19,8 +19,10 @@ import de.learnlib.filter.cache.CacheTestUtils; import de.learnlib.filter.statistic.oracle.DFACounterOracle; import de.learnlib.oracle.membership.DFASimulatorOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.word.Word; public class DFAHashCacheTest extends AbstractCacheTest, DFA, Character, Boolean> { @@ -53,9 +55,14 @@ protected DFAHashCacheOracle getResumedOracle(DFAHashCacheOracle model, Word input) { + return model.computeOutput(input); + } + @Override protected long getNumberOfPosedQueries() { - return counter.getQueryCounter().getCount(); + return Statistics.getCollector().getCount(DFACounterOracle.QUERY_KEY).orElse(0L); } @Override diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAParallelCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAParallelCacheTest.java index 9d7acab787..093613fec9 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAParallelCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/dfa/DFAParallelCacheTest.java @@ -21,6 +21,7 @@ import de.learnlib.filter.cache.CacheTestUtils; import de.learnlib.filter.statistic.oracle.DFACounterOracle; import de.learnlib.oracle.ParallelOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.fsa.DFA; import org.testng.annotations.DataProvider; @@ -28,16 +29,15 @@ public class DFAParallelCacheTest extends AbstractParallelCacheTest, Character, Boolean> { - private final DFACounterOracle sul; private final ThreadSafeDFACacheOracle cacheRepresentative; private final ParallelOracle parallelOracle; @Factory(dataProvider = "caches") public DFAParallelCacheTest(DFACacheCreator> creator) { - this.sul = CacheTestUtils.getCounter(CacheTestUtils.DFA); + DFACounterOracle sul = CacheTestUtils.getCounter(CacheTestUtils.DFA); final CacheConfig> config = - creator.apply(CacheTestUtils.INPUT_ALPHABET, this.sul); + creator.apply(CacheTestUtils.INPUT_ALPHABET, sul); this.cacheRepresentative = config.getRepresentative(); this.parallelOracle = config.getParallelOracle(); @@ -75,6 +75,6 @@ protected ParallelOracle getParallelOracle() { @Override protected long getNumberOfQueries() { - return this.sul.getQueryCounter().getCount(); + return Statistics.getCollector().getCount(DFACounterOracle.QUERY_KEY).orElse(0L); } } diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AbstractMealyCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AbstractMealyCacheTest.java index f51b681f3f..f8ea36dc53 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AbstractMealyCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AbstractMealyCacheTest.java @@ -20,6 +20,7 @@ import de.learnlib.filter.statistic.oracle.MealyCounterOracle; import de.learnlib.oracle.MembershipOracle.MealyMembershipOracle; import de.learnlib.oracle.membership.MealySimulatorOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.GrowingMapAlphabet; import net.automatalib.automaton.transducer.MealyMachine; @@ -60,9 +61,14 @@ protected MealyCacheOracle getResumedOracle(MealyCacheOracle return fresh; } + @Override + protected Word computeOutput(MealyMachine model, Word input) { + return model.computeOutput(input); + } + @Override protected long getNumberOfPosedQueries() { - return counter.getQueryCounter().getCount(); + return Statistics.getCollector().getCount(MealyCounterOracle.QUERY_KEY).orElse(0L); } @Override diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AdaptiveQueryCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AdaptiveQueryCacheTest.java index 1ccf91b629..b9bb6de80a 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AdaptiveQueryCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/AdaptiveQueryCacheTest.java @@ -29,6 +29,7 @@ import de.learnlib.oracle.membership.SULAdaptiveOracle; import de.learnlib.query.AdaptiveQuery; import de.learnlib.query.Query; +import de.learnlib.statistic.Statistics; import de.learnlib.util.mealy.PresetAdaptiveQuery; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.SupportsGrowingAlphabet; @@ -81,9 +82,14 @@ protected Wrapper getResumedOracle(Wrapper(fresh); } + @Override + protected Word computeOutput(MealyMachine model, Word input) { + return model.computeOutput(input); + } + @Override protected long getNumberOfPosedQueries() { - return counter.getResetCounter().getCount(); + return Statistics.getCollector().getCount(CounterAdaptiveQueryOracle.RESET_KEY).orElse(0L); } @Override diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/MealyParallelCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/MealyParallelCacheTest.java index 18ce4fb91b..13c3cdd692 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/MealyParallelCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/mealy/MealyParallelCacheTest.java @@ -26,6 +26,7 @@ import de.learnlib.filter.statistic.oracle.MealyCounterOracle; import de.learnlib.oracle.MembershipOracle; import de.learnlib.oracle.ParallelOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.word.Word; @@ -35,16 +36,15 @@ public class MealyParallelCacheTest extends AbstractParallelCacheTest, Character, Word> { - private final MealyCounterOracle sul; private final ThreadSafeMealyCacheOracle cacheRepresentative; private final ParallelOracle> parallelOracle; @Factory(dataProvider = "caches") public MealyParallelCacheTest(MealyCacheCreator> creator) { - this.sul = CacheTestUtils.getCounter(CacheTestUtils.MEALY); + MealyCounterOracle sul = CacheTestUtils.getCounter(CacheTestUtils.MEALY); final CacheConfig, ThreadSafeMealyCacheOracle> config = - creator.apply(CacheTestUtils.INPUT_ALPHABET, this.sul); + creator.apply(CacheTestUtils.INPUT_ALPHABET, sul); this.cacheRepresentative = config.getRepresentative(); this.parallelOracle = config.getParallelOracle(); @@ -82,6 +82,6 @@ protected ParallelOracle> getParallelOracle() { @Override protected long getNumberOfQueries() { - return this.sul.getQueryCounter().getCount(); + return Statistics.getCollector().getCount(MealyCounterOracle.QUERY_KEY).orElse(0L); } } diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/mmlt/MMLTCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/mmlt/MMLTCacheTest.java new file mode 100644 index 0000000000..05c65c7ad0 --- /dev/null +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/mmlt/MMLTCacheTest.java @@ -0,0 +1,124 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.cache.mmlt; + +import java.util.List; + +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.filter.cache.AbstractCacheTest; +import de.learnlib.filter.cache.CacheTestUtils; +import de.learnlib.filter.cache.TimedSULLearningCacheOracle; +import de.learnlib.filter.cache.sul.SULCaches; +import de.learnlib.filter.statistic.sul.CounterTimedSUL; +import de.learnlib.statistic.Statistics; +import net.automatalib.alphabet.Alphabet; +import net.automatalib.alphabet.impl.Alphabets; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; +import org.testng.Assert; +import org.testng.annotations.Test; + +@Test +public class MMLTCacheTest + extends AbstractCacheTest>, MMLT, TimedInput, Word>> { + + private final CounterTimedSUL counter; + + public MMLTCacheTest() { + counter = new CounterTimedSUL<>(new MMLTSimulatorSUL<>(CacheTestUtils.MMLT)); + } + + @Override + protected Alphabet> getAlphabet() { + return CacheTestUtils.MMLT.getSemantics().getInputAlphabet(); + } + + @Override + protected Alphabet> getExtensionAlphabet() { + return Alphabets.fromArray(); + } + + @Override + protected MMLT getTargetModel() { + return CacheTestUtils.MMLT; + } + + @Override + protected MMLT getInvalidTargetModel() { + return CacheTestUtils.MMLT_INVALID; + } + + @Override + protected TimedSULLearningCacheOracle> getCachedOracle() { + return TimedSULLearningCacheOracle.fromTimedSULCache(SULCaches.createTimedCache(counter, + CacheTestUtils.MMLT_PARAMS), + CacheTestUtils.MMLT_PARAMS); + } + + @Override + protected TimedSULLearningCacheOracle> getResumedOracle( + TimedSULLearningCacheOracle> original) { + return original; + } + + @Override + protected Word> computeOutput(MMLT model, + Word> input) { + return model.getSemantics().computeOutput(input); + } + + @Override + protected long getNumberOfPosedQueries() { + return Statistics.getCollector().getCount(CounterTimedSUL.KEY_RESETS).orElse(0L); + } + + @Override + protected boolean supportsPrefixes() { + return true; + } + + @Override + protected boolean supportsGrowing() { + return false; + } + + @Override + @Test(dependsOnMethods = "testPrefix") + public void testCacheConsistency() { + // Add word to cache to ensure counter example + Word> testWord = + Word.fromSymbols(TimedInput.input("p2"), TimedInput.timeout(), TimedInput.step(), TimedInput.timeout()); + super.oracle.getOracle().answerQuery(testWord); + + super.testCacheConsistency(); + } + + @Test(dependsOnMethods = "testCacheConsistency") + public void testReducedAlphabet() { + // Now test with a reduced alphabet: + var symbols = List.of("p1", "abort", "collect"); // not p1 + var reducedAlphabet = symbols.stream().>map(InputSymbol::new).collect(Alphabets.collector()); + reducedAlphabet.add(new TimeoutSymbol<>()); + + // The only counterexample in the cache has the prefix p2, which is now omitted: + Assert.assertNull(super.oracle.createCacheConsistencyTest() + .findCounterExample(CacheTestUtils.MMLT_INVALID, reducedAlphabet)); + } +} diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/mmlt/TimeoutReducerSULTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/mmlt/TimeoutReducerSULTest.java new file mode 100644 index 0000000000..a947b2ee78 --- /dev/null +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/mmlt/TimeoutReducerSULTest.java @@ -0,0 +1,57 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.cache.mmlt; + +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.filter.cache.CacheTestUtils; +import net.automatalib.symbol.time.TimedInput; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TimeoutReducerSULTest { + + @Test + public void testCaching() { + + var mmlt = CacheTestUtils.MMLT; + var sul = new MMLTSimulatorSUL<>(mmlt); + + var mock = Mockito.spy(sul); + var toSUL = new TimeoutReducerSUL<>(mock, 1); + + toSUL.pre(); + toSUL.step(TimedInput.input("p1")); + var output = toSUL.timeStep(); + + Assert.assertNull(output); + Mockito.verify(mock, Mockito.times(1)).timeoutStep(ArgumentMatchers.anyLong()); + + output = toSUL.timeStep(); + + Assert.assertNull(output); + Mockito.verify(mock, Mockito.times(1)).timeoutStep(ArgumentMatchers.anyLong()); + + toSUL.step(TimedInput.input("p2")); + output = toSUL.timeoutStep(4); + + Assert.assertNotNull(output); + Mockito.verify(mock, Mockito.times(2)).timeoutStep(ArgumentMatchers.anyLong()); + + toSUL.post(); + } +} diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/moore/AbstractMooreCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/moore/AbstractMooreCacheTest.java index 007011aa3e..21450745a9 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/moore/AbstractMooreCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/moore/AbstractMooreCacheTest.java @@ -20,6 +20,7 @@ import de.learnlib.filter.statistic.oracle.MooreCounterOracle; import de.learnlib.oracle.MembershipOracle.MooreMembershipOracle; import de.learnlib.oracle.membership.MooreSimulatorOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.GrowingMapAlphabet; import net.automatalib.automaton.transducer.MooreMachine; @@ -60,9 +61,14 @@ protected MooreCacheOracle getResumedOracle(MooreCacheOracle return fresh; } + @Override + protected Word computeOutput(MooreMachine model, Word input) { + return model.computeOutput(input); + } + @Override protected long getNumberOfPosedQueries() { - return counter.getQueryCounter().getCount(); + return Statistics.getCollector().getCount(MooreCounterOracle.QUERY_KEY).orElse(0L); } @Override diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/moore/MooreParallelCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/moore/MooreParallelCacheTest.java index 369b1c51c8..b4f432079f 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/moore/MooreParallelCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/moore/MooreParallelCacheTest.java @@ -21,6 +21,7 @@ import de.learnlib.filter.cache.CacheTestUtils; import de.learnlib.filter.statistic.oracle.MooreCounterOracle; import de.learnlib.oracle.ParallelOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.transducer.MooreMachine; import net.automatalib.word.Word; @@ -30,16 +31,15 @@ public class MooreParallelCacheTest extends AbstractParallelCacheTest, Character, Word> { - private final MooreCounterOracle sul; private final ThreadSafeMooreCacheOracle cacheRepresentative; private final ParallelOracle> parallelOracle; @Factory(dataProvider = "caches") public MooreParallelCacheTest(MooreCacheCreator> creator) { - this.sul = CacheTestUtils.getCounter(CacheTestUtils.MOORE); + MooreCounterOracle sul = CacheTestUtils.getCounter(CacheTestUtils.MOORE); final CacheConfig, ThreadSafeMooreCacheOracle> config = - creator.apply(CacheTestUtils.INPUT_ALPHABET, this.sul); + creator.apply(CacheTestUtils.INPUT_ALPHABET, sul); this.cacheRepresentative = config.getRepresentative(); this.parallelOracle = config.getParallelOracle(); @@ -75,6 +75,6 @@ protected ParallelOracle> getParallelOracle() { @Override protected long getNumberOfQueries() { - return this.sul.getQueryCounter().getCount(); + return Statistics.getCollector().getCount(MooreCounterOracle.QUERY_KEY).orElse(0L); } } diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/AbstractSULCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/AbstractSULCacheTest.java index cb73609213..ca771496b9 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/AbstractSULCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/AbstractSULCacheTest.java @@ -20,6 +20,7 @@ import de.learnlib.filter.cache.CacheTestUtils; import de.learnlib.filter.cache.SULLearningCacheOracle; import de.learnlib.filter.statistic.sul.CounterSUL; +import de.learnlib.statistic.Statistics; import de.learnlib.sul.SUL; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.impl.GrowingMapAlphabet; @@ -58,9 +59,14 @@ protected SULLearningCacheOracle computeOutput(MealyMachine model, Word input) { + return model.computeOutput(input); + } + @Override protected long getNumberOfPosedQueries() { - return counter.getResetCounter().getCount(); + return Statistics.getCollector().getCount(CounterSUL.RESET_KEY).orElse(0L); } @Override diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SLISULParallelCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SLISULParallelCacheTest.java index c340a7f9c5..60e3bbc74e 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SLISULParallelCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SLISULParallelCacheTest.java @@ -19,8 +19,10 @@ import de.learnlib.filter.cache.CacheConfig; import de.learnlib.filter.cache.CacheCreator.SLISULCacheCreator; import de.learnlib.filter.cache.CacheTestUtils; +import de.learnlib.filter.statistic.sul.CounterSUL; import de.learnlib.filter.statistic.sul.CounterStateLocalInputSUL; import de.learnlib.oracle.ParallelOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.word.Word; @@ -30,13 +32,12 @@ public class SLISULParallelCacheTest extends AbstractParallelCacheTest, Character, Word> { - private final CounterStateLocalInputSUL sul; private final ThreadSafeStateLocalInputSULCache cacheRepresentative; private final ParallelOracle> parallelOracle; @Factory(dataProvider = "caches") public SLISULParallelCacheTest(SLISULCacheCreator> creator) { - this.sul = CacheTestUtils.getCounter(CacheTestUtils.SLI_SUL); + CounterStateLocalInputSUL sul = CacheTestUtils.getCounter(CacheTestUtils.SLI_SUL); final CacheConfig, ThreadSafeStateLocalInputSULCache> config = creator.apply(CacheTestUtils.INPUT_ALPHABET, sul); @@ -73,6 +74,6 @@ protected ParallelOracle> getParallelOracle() { @Override protected long getNumberOfQueries() { - return this.sul.getResetCounter().getCount(); + return Statistics.getCollector().getCount(CounterSUL.RESET_KEY).orElse(0L); } } diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SULParallelCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SULParallelCacheTest.java index e3f0bec389..4ee05dc3b3 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SULParallelCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/SULParallelCacheTest.java @@ -21,6 +21,7 @@ import de.learnlib.filter.cache.CacheTestUtils; import de.learnlib.filter.statistic.sul.CounterSUL; import de.learnlib.oracle.ParallelOracle; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.word.Word; @@ -30,13 +31,12 @@ public class SULParallelCacheTest extends AbstractParallelCacheTest, Character, Word> { - private final CounterSUL sul; private final ThreadSafeSULCache cacheRepresentative; private final ParallelOracle> parallelOracle; @Factory(dataProvider = "caches") public SULParallelCacheTest(SULCacheCreator> creator) { - this.sul = CacheTestUtils.getCounter(CacheTestUtils.SUL); + CounterSUL sul = CacheTestUtils.getCounter(CacheTestUtils.SUL); final CacheConfig, ThreadSafeSULCache> config = creator.apply(CacheTestUtils.INPUT_ALPHABET, sul); @@ -74,6 +74,6 @@ protected ParallelOracle> getParallelOracle() { @Override protected long getNumberOfQueries() { - return this.sul.getResetCounter().getCount(); + return Statistics.getCollector().getCount(CounterSUL.RESET_KEY).orElse(0L); } } diff --git a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/StateLocalInputSULTreeCacheTest.java b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/StateLocalInputSULTreeCacheTest.java index 10062f5677..6c30014bdf 100644 --- a/filters/cache/src/test/java/de/learnlib/filter/cache/sul/StateLocalInputSULTreeCacheTest.java +++ b/filters/cache/src/test/java/de/learnlib/filter/cache/sul/StateLocalInputSULTreeCacheTest.java @@ -20,6 +20,7 @@ import de.learnlib.filter.cache.CacheTestUtils; import de.learnlib.filter.cache.SULLearningCacheOracle; import de.learnlib.filter.statistic.sul.CounterStateLocalInputSUL; +import de.learnlib.statistic.Statistics; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.word.Word; @@ -41,42 +42,47 @@ public StateLocalInputSULTreeCacheTest() { @Override public void testNoQueriesReceived() { super.testNoQueriesReceived(); - Assert.assertEquals(counter.getInputCounter().getCount(), 0); + Assert.assertEquals(Statistics.getCollector().getCount(CounterStateLocalInputSUL.INPUT_KEY).orElse(0L), 0); } @Test(dependsOnMethods = "testNoQueriesReceived") @Override public void testFirstQuery() { super.testFirstQuery(); - Assert.assertEquals(counter.getInputCounter().getCount(), oracle.getCache().size()); + Assert.assertEquals(Statistics.getCollector().getCount(CounterStateLocalInputSUL.INPUT_KEY).orElse(0L), + oracle.getCache().size()); } @Test(dependsOnMethods = "testFirstQuery") @Override public void testFirstDuplicate() { super.testFirstDuplicate(); - Assert.assertEquals(counter.getInputCounter().getCount(), oracle.getCache().size()); + Assert.assertEquals(Statistics.getCollector().getCount(CounterStateLocalInputSUL.INPUT_KEY).orElse(0L), + oracle.getCache().size()); } @Test(dependsOnMethods = "testFirstDuplicate") @Override public void testTwoQueriesOneDuplicate() { super.testTwoQueriesOneDuplicate(); - Assert.assertEquals(counter.getInputCounter().getCount(), oracle.getCache().size()); + Assert.assertEquals(Statistics.getCollector().getCount(CounterStateLocalInputSUL.INPUT_KEY).orElse(0L), + oracle.getCache().size()); } @Test(dependsOnMethods = "testTwoQueriesOneDuplicate") @Override public void testOneNewQuery() { super.testOneNewQuery(); - Assert.assertEquals(counter.getInputCounter().getCount(), oracle.getCache().size()); + Assert.assertEquals(Statistics.getCollector().getCount(CounterStateLocalInputSUL.INPUT_KEY).orElse(0L), + oracle.getCache().size()); } @Test(dependsOnMethods = "testOneNewQuery") @Override public void testPrefix() { super.testPrefix(); - Assert.assertEquals(counter.getInputCounter().getCount(), oracle.getCache().size()); + Assert.assertEquals(Statistics.getCollector().getCount(CounterStateLocalInputSUL.INPUT_KEY).orElse(0L), + oracle.getCache().size()); } @Test(dependsOnMethods = "testPrefix") @@ -140,9 +146,14 @@ protected SULLearningCacheOracle computeOutput(MealyMachine model, Word input) { + return model.computeOutput(input); + } + @Override protected long getNumberOfPosedQueries() { - return counter.getResetCounter().getCount(); + return Statistics.getCollector().getCount(CounterStateLocalInputSUL.RESET_KEY).orElse(0L); } @Override diff --git a/filters/pom.xml b/filters/pom.xml index ce77c4bb69..6bb856a924 100644 --- a/filters/pom.xml +++ b/filters/pom.xml @@ -35,5 +35,6 @@ limitations under the License. cache reuse statistics + symbol-filters diff --git a/filters/statistics/pom.xml b/filters/statistics/pom.xml index 83ab5b76ec..11351b016e 100644 --- a/filters/statistics/pom.xml +++ b/filters/statistics/pom.xml @@ -52,6 +52,10 @@ limitations under the License. de.learnlib.tooling annotations + + org.kohsuke.metainf-services + metainf-services + @@ -76,6 +80,11 @@ limitations under the License. mockito-core + + org.checkerframework + checker-qual + + org.testng testng diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/CounterCollection.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/CounterCollection.java deleted file mode 100644 index b475a3dfcb..0000000000 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/CounterCollection.java +++ /dev/null @@ -1,63 +0,0 @@ -/* Copyright (C) 2013-2025 TU Dortmund University - * This file is part of LearnLib . - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package de.learnlib.filter.statistic; - -import java.util.StringJoiner; -import java.util.function.Function; - -import de.learnlib.statistic.StatisticData; - -/** - * A collection of counters. - */ -public class CounterCollection implements StatisticData { - - private final Counter[] counters; - - public CounterCollection(Counter... counters) { - this.counters = counters; - } - - @Override - public String getName() { - return collect(Counter::getName); - } - - @Override - public String getUnit() { - return collect(Counter::getUnit); - } - - @Override - public String getSummary() { - return collect(Counter::getSummary); - } - - @Override - public String getDetails() { - return collect(Counter::getDetails); - } - - private String collect(Function extractor) { - final StringJoiner sj = new StringJoiner("\n"); - - for (Counter c : counters) { - sj.add(extractor.apply(c)); - } - - return sj.toString(); - } -} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/HistogramDataSet.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/HistogramDataSet.java deleted file mode 100644 index 159f132c47..0000000000 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/HistogramDataSet.java +++ /dev/null @@ -1,99 +0,0 @@ -/* Copyright (C) 2013-2025 TU Dortmund University - * This file is part of LearnLib . - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package de.learnlib.filter.statistic; - -import java.util.Map.Entry; -import java.util.SortedMap; -import java.util.TreeMap; - -/** - * A simple histogram data set. - */ -public class HistogramDataSet extends AbstractStatisticData { - - private final SortedMap histogram = new TreeMap<>(); - - private long size; - - private long sum; - - private double mean; - - public HistogramDataSet(String name, String unit) { - super(name, unit); - } - - public void addDataPoint(Long value) { - Integer i = histogram.get(value); - if (i == null) { - i = 0; - } - histogram.put(value, i + 1); - sum += value; - size++; - mean = mean + ((value - mean) / size); - } - - public SortedMap getHistogram() { - return histogram; - } - - public double getMean() { - return mean; - } - - public long getSize() { - return size; - } - - public long getSum() { - return sum; - } - - public double getMedian() { - long idx = 0; - for (Entry e : histogram.entrySet()) { - int count = e.getValue(); - idx += count; - if (idx >= size / 2) { - return e.getKey(); - } - } - return 0.0; - } - - @Override - public String getSummary() { - return getName() + " [" + getUnit() + "]: " + size + " (count), " + sum + " (sum), " + mean + " (mean), " + - getMedian() + " (median)"; - } - - @Override - public String getDetails() { - StringBuilder sb = new StringBuilder(); - sb.append(getSummary()).append(System.lineSeparator()); - for (Entry e : histogram.entrySet()) { - sb.append('\t') - .append(e.getKey()) - .append(", ") - .append(e.getValue()) - .append(System.lineSeparator()); - } - return sb.toString(); - } - -} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/AbstractStatistic.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/AbstractStatistic.java new file mode 100644 index 0000000000..37ed7de7c2 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/AbstractStatistic.java @@ -0,0 +1,64 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.container; + +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Various types of statistical data to be stored in a StatisticsCollector. + */ +abstract class AbstractStatistic { + + private final String id; + private final @Nullable String description; + + /** + * Default constructor. + * + * @param id + * id of the statistic. Must be unique within the StatisticsCollector. + * @param description + * Optional description of the statistic. If no description is provided, the id is used. + */ + AbstractStatistic(String id, @Nullable String description) { + this.id = id; + + if (description == null) { + this.description = id; + } else { + this.description = description; + } + } + + public String getId() { + return id; + } + + public @Nullable String getDescription() { + return description; + } + + protected abstract String renderValue(); + + @Override + public String toString() { + if (description == null) { + return id + ": " + renderValue(); + } else { + return description + ": " + renderValue(); + } + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/CounterStatistic.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/CounterStatistic.java new file mode 100644 index 0000000000..1f2cb03042 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/CounterStatistic.java @@ -0,0 +1,55 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.container; + +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A counter that can be increased and set to a particular positive number. + */ +class CounterStatistic extends AbstractStatistic { + + private long count; + + CounterStatistic(String id, @Nullable String description) { + this(id, description, 0); + } + + CounterStatistic(String id, @Nullable String description, long count) { + super(id, description); + this.count = count; + } + + public void setCount(long count) { + if (count < 0) { + throw new IllegalArgumentException(); + } + this.count = count; + } + + public void increase(long increment) { + this.count += increment; + } + + public long getCount() { + return count; + } + + @Override + public String renderValue() { + return Long.toString(count); + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/FlagStatistic.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/FlagStatistic.java new file mode 100644 index 0000000000..9f70a85849 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/FlagStatistic.java @@ -0,0 +1,44 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.container; + +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A boolean flag that is unset by default and can be set. + */ +class FlagStatistic extends AbstractStatistic { + + private boolean flagged; + + FlagStatistic(String id, @Nullable String description, boolean value) { + super(id, description); + this.flagged = value; + } + + public void setFlag(boolean value) { + this.flagged = value; + } + + public boolean isFlagged() { + return flagged; + } + + @Override + public String renderValue() { + return Boolean.toString(flagged); + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/MapStatisticsCollector.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/MapStatisticsCollector.java new file mode 100644 index 0000000000..3d9d3d20b7 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/MapStatisticsCollector.java @@ -0,0 +1,157 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.container; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import de.learnlib.statistic.StatisticsCollector; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A {@link StatisticsCollector} that stores all statistics in a {@link Map}. + */ +@SuppressWarnings("PMD.AvoidSynchronizedAtMethodLevel") // quick'n'dirty for now +public class MapStatisticsCollector implements StatisticsCollector { + + private final Map statistics; + + public MapStatisticsCollector() { + this.statistics = new HashMap<>(); + } + + @Override + public Collection getKeys() { + return new HashSet<>(this.statistics.keySet()); + } + + @Override + public synchronized void clear() { + statistics.clear(); + } + + @Override + public synchronized void addText(String id, @Nullable String description, String text) { + statistics.put(id, new TextStatistic(id, description, text)); + } + + @Override + public synchronized Optional getText(String id) { + AbstractStatistic value = statistics.get(id); + if (value instanceof TextStatistic textStatistic) { + return Optional.of(textStatistic.getText()); + } + return Optional.empty(); + } + + @Override + public synchronized void setFlag(String id, @Nullable String description, boolean value) { + statistics.put(id, new FlagStatistic(id, description, value)); + } + + @Override + public synchronized Optional getFlag(String id) { + AbstractStatistic value = statistics.get(id); + if (value instanceof FlagStatistic flagStatistic) { + return Optional.of(flagStatistic.isFlagged()); + } + return Optional.empty(); + } + + @Override + public synchronized void startOrResumeClock(String id, @Nullable String description) { + AbstractStatistic value = statistics.get(id); + if (value instanceof StopClockStatistic clockStatistic) { + clockStatistic.resume(); + } else { + // Create and start a new clock: + StopClockStatistic newClock = new StopClockStatistic(id, description); + statistics.put(id, newClock); + newClock.resume(); + } + } + + @Override + public synchronized void pauseClock(String id) { + AbstractStatistic value = statistics.get(id); + if (value instanceof StopClockStatistic clockStatistic) { + clockStatistic.pause(); + } + } + + @Override + public synchronized Optional getClock(String id) { + AbstractStatistic value = statistics.get(id); + if (value instanceof StopClockStatistic clockStatistic) { + return Optional.of(clockStatistic.getElapsed()); + } + return Optional.empty(); + } + + @Override + public synchronized void increaseCounter(String id, @Nullable String description, long increment) { + AbstractStatistic value = statistics.get(id); + if (value instanceof CounterStatistic counterStatistic) { + counterStatistic.increase(increment); + } else { + // Create a new counter: + setCounter(id, description, increment); + } + } + + @Override + public synchronized void setCounter(String id, @Nullable String description, long count) { + statistics.put(id, new CounterStatistic(id, description, count)); + } + + @Override + public synchronized Optional getCount(String id) { + AbstractStatistic value = statistics.get(id); + if (value instanceof CounterStatistic counterStatistic) { + return Optional.of(counterStatistic.getCount()); + } + return Optional.empty(); + } + + // ================== + + @Override + public synchronized String printStats() { + + List stats = new ArrayList<>(statistics.values()); + stats.sort(Comparator.comparing(AbstractStatistic::getDescription).thenComparing(AbstractStatistic::getId)); + + final StringBuilder sb = new StringBuilder(125); + + sb.append("Statistics:\n============================================\n"); + + for (AbstractStatistic stat : stats) { + sb.append("* ").append(stat).append('\n'); + } + + sb.append("============================================\n"); + + return sb.toString(); + } + +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/MapStatsProvider.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/MapStatsProvider.java new file mode 100644 index 0000000000..62b85e3b75 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/MapStatsProvider.java @@ -0,0 +1,36 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.container; + +import de.learnlib.statistic.StatisticsCollector; +import de.learnlib.statistic.StatisticsProvider; +import org.kohsuke.MetaInfServices; + +@MetaInfServices(StatisticsProvider.class) +public class MapStatsProvider implements StatisticsProvider { + + final ThreadLocal threadLocal = ThreadLocal.withInitial(MapStatisticsCollector::new); + + @Override + public int getPriority() { + return 0; + } + + @Override + public StatisticsCollector getCollector() { + return threadLocal.get(); + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/StopClockStatistic.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/StopClockStatistic.java new file mode 100644 index 0000000000..877bac88b2 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/StopClockStatistic.java @@ -0,0 +1,57 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.container; + +import java.time.Duration; +import java.time.Instant; + +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A stop clock that can be paused and resumed. + */ +class StopClockStatistic extends AbstractStatistic { + + private @Nullable Instant started; + private Duration elapsed; + + StopClockStatistic(String id, @Nullable String description) { + super(id, description); + this.elapsed = Duration.ZERO; + this.started = null; + } + + public void resume() { + this.started = Instant.now(); + } + + public void pause() { + if (started == null) { + return; + } + this.elapsed = this.elapsed.plus(Duration.between(started, Instant.now())); + this.started = null; + } + + public Duration getElapsed() { + return this.elapsed; + } + + @Override + public String renderValue() { + return elapsed.toMillis() + " ms"; + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/TextStatistic.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/TextStatistic.java new file mode 100644 index 0000000000..9767a793b2 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/container/TextStatistic.java @@ -0,0 +1,37 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.container; + +import org.checkerframework.checker.nullness.qual.Nullable; + +class TextStatistic extends AbstractStatistic { + + private final String text; + + TextStatistic(String id, @Nullable String description, String text) { + super(id, description); + this.text = text; + } + + public String getText() { + return text; + } + + @Override + public String renderValue() { + return text; + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/learner/RefinementCounterLearner.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/learner/RefinementCounterLearner.java index ec64d4595b..a498270880 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/learner/RefinementCounterLearner.java +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/learner/RefinementCounterLearner.java @@ -19,15 +19,11 @@ import de.learnlib.algorithm.LearningAlgorithm.DFALearner; import de.learnlib.algorithm.LearningAlgorithm.MealyLearner; import de.learnlib.algorithm.LearningAlgorithm.MooreLearner; -import de.learnlib.filter.statistic.Counter; import de.learnlib.query.DefaultQuery; -import de.learnlib.statistic.StatisticLearner; -import de.learnlib.statistic.StatisticLearner.DFAStatisticLearner; -import de.learnlib.statistic.StatisticLearner.MealyStatisticLearner; -import de.learnlib.statistic.StatisticLearner.MooreStatisticLearner; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import de.learnlib.tooling.annotation.refinement.GenerateRefinement; import de.learnlib.tooling.annotation.refinement.Generic; -import de.learnlib.tooling.annotation.refinement.Interface; import de.learnlib.tooling.annotation.refinement.Mapping; import net.automatalib.automaton.fsa.DFA; import net.automatalib.automaton.transducer.MealyMachine; @@ -37,8 +33,6 @@ /** * Counts the number of hypothesis refinements. *

- * The value of the {@link Counter} returned by {@link #getStatisticalData()} returns the same value as - * Experiment.getRounds(). * * @param * automaton type @@ -54,8 +48,7 @@ @Generic(clazz = Boolean.class)}, typeMappings = @Mapping(from = LearningAlgorithm.class, to = DFALearner.class, - generics = @Generic("I")), - interfaces = @Interface(clazz = DFAStatisticLearner.class, generics = @Generic("I"))) + generics = @Generic("I"))) @GenerateRefinement(name = "MealyRefinementCounterLearner", generics = {@Generic(value = "I", desc = "input symbol type"), @Generic(value = "O", desc = "output symbol type")}, @@ -64,8 +57,6 @@ @Generic(clazz = Word.class, generics = "O")}, typeMappings = @Mapping(from = LearningAlgorithm.class, to = MealyLearner.class, - generics = {@Generic("I"), @Generic("O")}), - interfaces = @Interface(clazz = MealyStatisticLearner.class, generics = {@Generic("I"), @Generic("O")})) @GenerateRefinement(name = "MooreRefinementCounterLearner", generics = {@Generic(value = "I", desc = "input symbol type"), @@ -75,18 +66,23 @@ @Generic(clazz = Word.class, generics = "O")}, typeMappings = @Mapping(from = LearningAlgorithm.class, to = MooreLearner.class, - generics = {@Generic("I"), @Generic("O")}), - interfaces = @Interface(clazz = MooreStatisticLearner.class, generics = {@Generic("I"), @Generic("O")})) -public class RefinementCounterLearner implements StatisticLearner { +public class RefinementCounterLearner implements LearningAlgorithm { - private final LearningAlgorithm learningAlgorithm; + public static final String KEY_CNT = "ref-cnt"; - private final Counter counter; + private final LearningAlgorithm learningAlgorithm; + private final StatisticsCollector statisticsCollector; + private final String id; public RefinementCounterLearner(LearningAlgorithm learningAlgorithm) { + this(learningAlgorithm, ""); + } + + public RefinementCounterLearner(LearningAlgorithm learningAlgorithm, String id) { this.learningAlgorithm = learningAlgorithm; - this.counter = new Counter("Refinements", "#"); + this.id = id; + this.statisticsCollector = Statistics.getCollector(); } @Override @@ -98,7 +94,7 @@ public void startLearning() { public boolean refineHypothesis(DefaultQuery ceQuery) { final boolean refined = learningAlgorithm.refineHypothesis(ceQuery); if (refined) { - counter.increment(); + statisticsCollector.increaseCounter(KEY_CNT + id, "Number of refinements"); } return refined; } @@ -107,9 +103,4 @@ public boolean refineHypothesis(DefaultQuery ceQuery) { public M getHypothesisModel() { return learningAlgorithm.getHypothesisModel(); } - - @Override - public Counter getStatisticalData() { - return counter; - } } diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveQueryOracle.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveQueryOracle.java index f6e04cd07e..e1e2ec7617 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveQueryOracle.java +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveQueryOracle.java @@ -19,13 +19,11 @@ import java.util.Collection; import java.util.List; -import de.learnlib.filter.statistic.Counter; -import de.learnlib.filter.statistic.CounterCollection; import de.learnlib.oracle.AdaptiveMembershipOracle; import de.learnlib.query.AdaptiveQuery; import de.learnlib.query.AdaptiveQuery.Response; -import de.learnlib.statistic.StatisticCollector; -import de.learnlib.statistic.StatisticData; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; /** * A simple wrapper for counting the number of {@link Response#RESET resets} and {@link Response#SYMBOL symbols} of an @@ -36,45 +34,48 @@ * @param * output symbol type */ -public class CounterAdaptiveQueryOracle implements AdaptiveMembershipOracle, StatisticCollector { +public class CounterAdaptiveQueryOracle implements AdaptiveMembershipOracle { + + public static final String DUR_KEY = "amq-qry-dur"; + public static final String RESET_KEY = "amq-reset-cnt"; + public static final String SYMBOL_KEY = "amq-sym-cnt"; private final AdaptiveMembershipOracle delegate; - private final Counter resetCounter; - private final Counter symbolCounter; + private final StatisticsCollector statisticsCollector; + private final String id; public CounterAdaptiveQueryOracle(AdaptiveMembershipOracle delegate) { - this.delegate = delegate; - this.resetCounter = new Counter("Resets", "#"); - this.symbolCounter = new Counter("Symbols", "#"); + this(delegate, ""); } - public Counter getResetCounter() { - return resetCounter; - } - - public Counter getSymbolCounter() { - return symbolCounter; + public CounterAdaptiveQueryOracle(AdaptiveMembershipOracle delegate, String id) { + this.delegate = delegate; + this.id = id; + this.statisticsCollector = Statistics.getCollector(); } @Override public void processQueries(Collection> queries) { - final List wrappers = new ArrayList<>(queries.size()); + final List> wrappers = new ArrayList<>(queries.size()); for (AdaptiveQuery q : queries) { - wrappers.add(new CountingQuery(q)); + wrappers.add(new CountingQuery<>(q)); } + statisticsCollector.startOrResumeClock(DUR_KEY + id, "Duration of queries"); this.delegate.processQueries(wrappers); + statisticsCollector.pauseClock(DUR_KEY + id); + // statContainer is not thread-safe so we need to count in post-processing + for (CountingQuery wrapper : wrappers) { + this.statisticsCollector.increaseCounter(RESET_KEY + id, "Number of resets", wrapper.resets); + this.statisticsCollector.increaseCounter(SYMBOL_KEY + id, "Number of symbols", wrapper.symbols); + } } - @Override - public StatisticData getStatisticalData() { - return new CounterCollection(this.resetCounter, this.symbolCounter); - } - - private class CountingQuery implements AdaptiveQuery { + private static class CountingQuery implements AdaptiveQuery { private final AdaptiveQuery delegate; + private int symbols, resets; CountingQuery(AdaptiveQuery delegate) { this.delegate = delegate; @@ -87,12 +88,12 @@ public I getInput() { @Override public Response processOutput(O out) { - symbolCounter.increment(); + symbols++; final Response response = delegate.processOutput(out); if (response != Response.SYMBOL) { - resetCounter.increment(); + resets++; } return response; diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterEQOracle.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterEQOracle.java new file mode 100644 index 0000000000..d9020ec096 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterEQOracle.java @@ -0,0 +1,57 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.oracle; + +import java.util.Collection; + +import de.learnlib.oracle.EquivalenceOracle; +import de.learnlib.query.DefaultQuery; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class CounterEQOracle implements EquivalenceOracle { + + public static final String KEY_CEX_CNT = "cex-cnt"; + public static final String KEY_CEX_DUR = "cex-dur"; + + private final EquivalenceOracle delegate; + private final StatisticsCollector statisticsCollector; + private final String id; + + public CounterEQOracle(EquivalenceOracle delegate) { + this(delegate, ""); + } + + public CounterEQOracle(EquivalenceOracle delegate, String id) { + this.delegate = delegate; + this.id = id; + this.statisticsCollector = Statistics.getCollector(); + } + + @Override + public @Nullable DefaultQuery findCounterExample(A hypothesis, Collection inputs) { + final String suffix = id.isEmpty() ? "" : " from '" + id + '\''; + + statisticsCollector.startOrResumeClock(KEY_CEX_DUR + id, "Duration of CEX search" + suffix); + final DefaultQuery cex = this.delegate.findCounterExample(hypothesis, inputs); + statisticsCollector.pauseClock(KEY_CEX_DUR + id); + if (cex != null) { + statisticsCollector.increaseCounter(KEY_CEX_CNT + id, "Found CEX" + suffix); + } + return cex; + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterOracle.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterOracle.java index 130a9f2cbc..94c396b6fc 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterOracle.java +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/CounterOracle.java @@ -17,15 +17,13 @@ import java.util.Collection; -import de.learnlib.filter.statistic.Counter; -import de.learnlib.filter.statistic.CounterCollection; import de.learnlib.oracle.MembershipOracle; import de.learnlib.oracle.MembershipOracle.DFAMembershipOracle; import de.learnlib.oracle.MembershipOracle.MealyMembershipOracle; import de.learnlib.oracle.MembershipOracle.MooreMembershipOracle; import de.learnlib.query.Query; -import de.learnlib.statistic.StatisticData; -import de.learnlib.statistic.StatisticOracle; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import de.learnlib.tooling.annotation.refinement.GenerateRefinement; import de.learnlib.tooling.annotation.refinement.Generic; import de.learnlib.tooling.annotation.refinement.Interface; @@ -66,47 +64,34 @@ generics = {@Generic("I"), @Generic("O")}), interfaces = @Interface(clazz = MooreMembershipOracle.class, generics = {@Generic("I"), @Generic("O")})) -public class CounterOracle implements StatisticOracle { +public class CounterOracle implements MembershipOracle { + + public static final String DUR_KEY = "mq-qry-dur"; + public static final String QUERY_KEY = "mq-qry-cnt"; + public static final String SYMBOL_KEY = "mq-sym-cnt"; private final MembershipOracle delegate; - private final Counter queryCounter; - private final Counter symbolCounter; + private final StatisticsCollector statisticsCollector; + private final String id; public CounterOracle(MembershipOracle delegate) { + this(delegate, ""); + } + + public CounterOracle(MembershipOracle delegate, String id) { this.delegate = delegate; - this.queryCounter = new Counter("Queries", "#"); - this.symbolCounter = new Counter("Symbols", "#"); + this.id = id; + this.statisticsCollector = Statistics.getCollector(); } @Override public void processQueries(Collection> queries) { - queryCounter.increment(queries.size()); + statisticsCollector.increaseCounter(QUERY_KEY + id, "Number of queries", queries.size()); for (Query qry : queries) { - symbolCounter.increment(qry.getPrefix().length() + qry.getSuffix().length()); + statisticsCollector.increaseCounter(SYMBOL_KEY + id, "Number of symbols", qry.length()); } + statisticsCollector.startOrResumeClock(DUR_KEY + id, "Duration of queries"); delegate.processQueries(queries); - } - - /** - * Retrieves {@link Counter} for the number of queries posed to this oracle. - * - * @return the counter of queries - */ - public Counter getQueryCounter() { - return queryCounter; - } - - /** - * Retrieves the {@link Counter} for the number of symbols in all queries posed to this oracle. - * - * @return the counter of symbols - */ - public Counter getSymbolCounter() { - return symbolCounter; - } - - @Override - public StatisticData getStatisticalData() { - return new CounterCollection(queryCounter, symbolCounter); + statisticsCollector.pauseClock(DUR_KEY + id); } } diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/HistogramOracle.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/HistogramOracle.java deleted file mode 100644 index 5cf1ee1c79..0000000000 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/oracle/HistogramOracle.java +++ /dev/null @@ -1,103 +0,0 @@ -/* Copyright (C) 2013-2025 TU Dortmund University - * This file is part of LearnLib . - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package de.learnlib.filter.statistic.oracle; - -import java.util.Collection; - -import de.learnlib.filter.statistic.HistogramDataSet; -import de.learnlib.oracle.MembershipOracle; -import de.learnlib.oracle.MembershipOracle.DFAMembershipOracle; -import de.learnlib.oracle.MembershipOracle.MealyMembershipOracle; -import de.learnlib.oracle.MembershipOracle.MooreMembershipOracle; -import de.learnlib.query.Query; -import de.learnlib.statistic.StatisticOracle; -import de.learnlib.tooling.annotation.refinement.GenerateRefinement; -import de.learnlib.tooling.annotation.refinement.Generic; -import de.learnlib.tooling.annotation.refinement.Interface; -import de.learnlib.tooling.annotation.refinement.Mapping; -import net.automatalib.word.Word; - -/** - * Collects a histogram of passed query lengths. - * - * @param - * input symbol type - * @param - * output symbol type - */ -@GenerateRefinement(name = "DFAHistogramOracle", - generics = @Generic(value = "I", desc = "input symbol type"), - parentGenerics = {@Generic("I"), @Generic(clazz = Boolean.class)}, - typeMappings = @Mapping(from = MembershipOracle.class, - to = DFAMembershipOracle.class, - generics = @Generic("I")), - interfaces = @Interface(clazz = DFAMembershipOracle.class, generics = @Generic("I"))) -@GenerateRefinement(name = "MealyHistogramOracle", - generics = {@Generic(value = "I", desc = "input symbol type"), - @Generic(value = "O", desc = "output symbol type")}, - parentGenerics = {@Generic("I"), @Generic(clazz = Word.class, generics = "O")}, - typeMappings = @Mapping(from = MembershipOracle.class, - to = MealyMembershipOracle.class, - generics = {@Generic("I"), @Generic("O")}), - interfaces = @Interface(clazz = MealyMembershipOracle.class, - generics = {@Generic("I"), @Generic("O")})) -@GenerateRefinement(name = "MooreHistogramOracle", - generics = {@Generic(value = "I", desc = "input symbol type"), - @Generic(value = "O", desc = "output symbol type")}, - parentGenerics = {@Generic("I"), @Generic(clazz = Word.class, generics = "O")}, - typeMappings = @Mapping(from = MembershipOracle.class, - to = MooreMembershipOracle.class, - generics = {@Generic("I"), @Generic("O")}), - interfaces = @Interface(clazz = MooreMembershipOracle.class, - generics = {@Generic("I"), @Generic("O")})) -public class HistogramOracle implements StatisticOracle { - - /** - * dataset to be collected. - */ - private final HistogramDataSet dataSet; - - /** - * oracle used to answer queries. - */ - private final MembershipOracle delegate; - - /** - * Default constructor. - * - * @param next - * real oracle - * @param name - * name of the collected data set - */ - public HistogramOracle(MembershipOracle next, String name) { - this.delegate = next; - this.dataSet = new HistogramDataSet(name, "query length"); - } - - @Override - public final void processQueries(Collection> queries) { - for (Query q : queries) { - this.dataSet.addDataPoint((long) q.getPrefix().size() + q.getSuffix().size()); - } - this.delegate.processQueries(queries); - } - - @Override - public final HistogramDataSet getStatisticalData() { - return this.dataSet; - } -} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterObservableSUL.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterObservableSUL.java index ce1dfc3b2f..dc1629c2f1 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterObservableSUL.java +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterObservableSUL.java @@ -15,7 +15,8 @@ */ package de.learnlib.filter.statistic.sul; -import de.learnlib.filter.statistic.Counter; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import de.learnlib.sul.ObservableSUL; public class CounterObservableSUL extends CounterSUL implements ObservableSUL { @@ -23,18 +24,21 @@ public class CounterObservableSUL extends CounterSUL implements O private final ObservableSUL sul; public CounterObservableSUL(ObservableSUL sul) { - super(sul); - this.sul = sul; + this(sul, ""); + } + + public CounterObservableSUL(ObservableSUL sul, String id) { + this(sul, id, Statistics.getCollector()); } - private CounterObservableSUL(ObservableSUL sul, Counter resetCounter, Counter symbolCounter) { - super(sul, resetCounter, symbolCounter); + protected CounterObservableSUL(ObservableSUL sul, String id, StatisticsCollector statistics) { + super(sul, id, statistics); this.sul = sul; } @Override public ObservableSUL fork() { - return new CounterObservableSUL<>(this.sul.fork(), super.resetCounter, super.symbolCounter); + return new CounterObservableSUL<>(this.sul.fork(), super.id, super.statisticsCollector); } @Override diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterSUL.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterSUL.java index 85b1e8c596..675e9524fd 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterSUL.java +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterSUL.java @@ -15,31 +15,36 @@ */ package de.learnlib.filter.statistic.sul; -import de.learnlib.filter.statistic.Counter; -import de.learnlib.filter.statistic.CounterCollection; -import de.learnlib.statistic.StatisticData; -import de.learnlib.statistic.StatisticSUL; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import de.learnlib.sul.SUL; -public class CounterSUL implements StatisticSUL { +public class CounterSUL implements SUL { + + public static final String RESET_KEY = "sul-reset-cnt"; + public static final String SYMBOL_KEY = "sul-step-cnt"; private final SUL sul; - protected final Counter resetCounter; - protected final Counter symbolCounter; + protected final StatisticsCollector statisticsCollector; + protected final String id; public CounterSUL(SUL sul) { - this(sul, new Counter("Resets", "#"), new Counter("Symbols", "#")); + this(sul, ""); + } + + public CounterSUL(SUL sul, String id) { + this(sul, id, Statistics.getCollector()); } - protected CounterSUL(SUL sul, Counter resetCounter, Counter symbolCounter) { + protected CounterSUL(SUL sul, String id, StatisticsCollector statisticsCollector) { this.sul = sul; - this.resetCounter = resetCounter; - this.symbolCounter = symbolCounter; + this.id = id; + this.statisticsCollector = statisticsCollector; } @Override public void pre() { - this.resetCounter.increment(); + this.statisticsCollector.increaseCounter(RESET_KEY + id, "Number of SUL resets"); this.sul.pre(); } @@ -50,7 +55,7 @@ public void post() { @Override public O step(I in) { - this.symbolCounter.increment(); + this.statisticsCollector.increaseCounter(SYMBOL_KEY + id, "Number of SUL steps"); return sul.step(in); } @@ -61,19 +66,6 @@ public boolean canFork() { @Override public SUL fork() { - return new CounterSUL<>(this.sul.fork(), this.resetCounter, this.symbolCounter); - } - - @Override - public StatisticData getStatisticalData() { - return new CounterCollection(this.resetCounter, this.symbolCounter); - } - - public Counter getResetCounter() { - return this.resetCounter; - } - - public Counter getSymbolCounter() { - return this.symbolCounter; + return new CounterSUL<>(this.sul.fork(), this.id, this.statisticsCollector); } } diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterStateLocalInputSUL.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterStateLocalInputSUL.java index 976f2233be..e249becea1 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterStateLocalInputSUL.java +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterStateLocalInputSUL.java @@ -17,51 +17,38 @@ import java.util.Collection; -import de.learnlib.filter.statistic.Counter; -import de.learnlib.filter.statistic.CounterCollection; -import de.learnlib.statistic.StatisticData; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import de.learnlib.sul.StateLocalInputSUL; public class CounterStateLocalInputSUL extends CounterSUL implements StateLocalInputSUL { + public static final String INPUT_KEY = "sul-input-cnt"; + private final StateLocalInputSUL sul; - private final Counter inputCounter; public CounterStateLocalInputSUL(StateLocalInputSUL sul) { - super(sul); - this.sul = sul; - this.inputCounter = new Counter("Input Checks", "#"); + this(sul, ""); + } + + private CounterStateLocalInputSUL(StateLocalInputSUL sul, String id) { + this(sul, id, Statistics.getCollector()); } - private CounterStateLocalInputSUL(StateLocalInputSUL sul, - Counter resetCounter, - Counter symbolCounter, - Counter inputCounter) { - super(sul, resetCounter, symbolCounter); + protected CounterStateLocalInputSUL(StateLocalInputSUL sul, String id, StatisticsCollector statistics) { + super(sul, id, statistics); this.sul = sul; - this.inputCounter = inputCounter; } @Override public Collection currentlyEnabledInputs() { - this.inputCounter.increment(); + super.statisticsCollector.increaseCounter(INPUT_KEY + super.id, "Number of enabled input checks"); return this.sul.currentlyEnabledInputs(); } @Override public StateLocalInputSUL fork() { - return new CounterStateLocalInputSUL<>(this.sul.fork(), - super.resetCounter, - super.symbolCounter, - this.inputCounter); + return new CounterStateLocalInputSUL<>(this.sul.fork(), super.id, super.statisticsCollector); } - @Override - public StatisticData getStatisticalData() { - return new CounterCollection(super.resetCounter, super.symbolCounter, this.inputCounter); - } - - public Counter getInputCounter() { - return this.inputCounter; - } } diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterTimedSUL.java b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterTimedSUL.java new file mode 100644 index 0000000000..95d826b882 --- /dev/null +++ b/filters/statistics/src/main/java/de/learnlib/filter/statistic/sul/CounterTimedSUL.java @@ -0,0 +1,112 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.statistic.sul; + +import java.util.List; + +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; +import de.learnlib.sul.TimedSUL; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedOutput; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Wrapper for a {@link TimedSUL} that gathers various statistics on queries sent to this SUL. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class CounterTimedSUL implements TimedSUL { + + public static final String KEY_RESETS = "sul_resets_counter"; + + private final TimedSUL delegate; + private final StatisticsCollector stats; + + private final @Nullable String name; + + public CounterTimedSUL(TimedSUL delegate) { + this(delegate, null); + } + + public CounterTimedSUL(TimedSUL delegate, @Nullable String name) { + this(delegate, name, Statistics.getCollector()); + } + + protected CounterTimedSUL(TimedSUL delegate, @Nullable String name, StatisticsCollector statistics) { + this.delegate = delegate; + this.name = name; + this.stats = statistics; + } + + private String withPrefix(String label) { + if (this.name == null) { + return label; + } + return this.name + ":" + label; + } + + @Override + public TimedOutput step(InputSymbol input) { + stats.increaseCounter(withPrefix("sul_untimed_syms_counter"), withPrefix("Total untimed symbols")); + return this.delegate.step(input); + } + + @Override + public @Nullable TimedOutput timeoutStep(long maxTime) { + TimedOutput res = this.delegate.timeoutStep(maxTime); + if (res == null) { + // Waited until maxTime, no timeout occurred: + stats.increaseCounter(withPrefix("sul_total_time"), withPrefix("Total query time"), maxTime); + } else { + stats.increaseCounter(withPrefix("sul_total_time"), withPrefix("Total query time"), res.delay()); + } + + return res; + } + + @Override + public List> collectTimeouts(TimeStepSequence input) { + stats.increaseCounter(withPrefix("sul_total_time"), withPrefix("Total query time"), input.timeSteps()); + return this.delegate.collectTimeouts(input); + } + + @Override + public void pre() { + this.delegate.pre(); + stats.increaseCounter(withPrefix(KEY_RESETS), withPrefix("SUL resets")); + } + + @Override + public void post() { + this.delegate.post(); + } + + @Override + public boolean canFork() { + return this.delegate.canFork(); + } + + @Override + public TimedSUL fork() { + return new CounterTimedSUL<>(this.delegate.fork(), this.name, this.stats); + } + +} diff --git a/filters/statistics/src/main/java/module-info.java b/filters/statistics/src/main/java/module-info.java index f262e24f56..f9fbdf818b 100644 --- a/filters/statistics/src/main/java/module-info.java +++ b/filters/statistics/src/main/java/module-info.java @@ -14,6 +14,9 @@ * limitations under the License. */ +import de.learnlib.filter.statistic.container.MapStatsProvider; +import de.learnlib.statistic.StatisticsProvider; + /** * This module provides filters for collecting statistical data. *

@@ -34,9 +37,12 @@ // annotations are 'provided'-scoped and do not need to be loaded at runtime requires static de.learnlib.tooling.annotation; + requires static org.checkerframework.checker.qual; + requires static org.kohsuke.metainf_services; - exports de.learnlib.filter.statistic; exports de.learnlib.filter.statistic.learner; exports de.learnlib.filter.statistic.oracle; exports de.learnlib.filter.statistic.sul; + + provides StatisticsProvider with MapStatsProvider; } diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveOracleTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveOracleTest.java index 178c175dbf..ba3c699f8e 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveOracleTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterAdaptiveOracleTest.java @@ -25,10 +25,12 @@ import de.learnlib.query.AdaptiveQuery; import de.learnlib.query.AdaptiveQuery.Response; import de.learnlib.query.Query; -import de.learnlib.statistic.StatisticData; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import net.automatalib.word.Word; import net.automatalib.word.WordBuilder; import org.testng.Assert; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; public class CounterAdaptiveOracleTest { @@ -39,6 +41,11 @@ public CounterAdaptiveOracleTest() { this.oracle = new CounterAdaptiveQueryOracle<>(new DummyOracle()); } + @BeforeClass + public void setUp() { + Statistics.getCollector().clear(); + } + @Test public void testInitialState() { verifyCounts(0, 0); @@ -71,18 +78,16 @@ public void testSecondQueryBatch() { verifyCounts(3, 11); } - @Test + @Test(dependsOnMethods = "testSecondQueryBatch") public void testStatistics() { - final StatisticData statisticalData = oracle.getStatisticalData(); - Assert.assertTrue(statisticalData.getName().contains("\n")); - Assert.assertTrue(statisticalData.getUnit().contains("\n")); - Assert.assertTrue(statisticalData.getSummary().contains("\n")); - Assert.assertTrue(statisticalData.getDetails().contains("\n")); + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + Assert.assertFalse(statisticsCollector.getKeys().isEmpty()); } private void verifyCounts(long queries, long symbols) { - Assert.assertEquals(oracle.getResetCounter().getCount(), queries); - Assert.assertEquals(oracle.getSymbolCounter().getCount(), symbols); + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + Assert.assertEquals(statisticsCollector.getCount(CounterAdaptiveQueryOracle.RESET_KEY).orElse(0L), queries); + Assert.assertEquals(statisticsCollector.getCount(CounterAdaptiveQueryOracle.SYMBOL_KEY).orElse(0L), symbols); } private Collection>> generateQueries(int numQueries, diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterOracleTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterOracleTest.java index 81fb3d4bc6..790bfbe555 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterOracleTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/CounterOracleTest.java @@ -21,10 +21,12 @@ import de.learnlib.filter.statistic.TestQueries; import de.learnlib.oracle.MembershipOracle; import de.learnlib.query.Query; -import de.learnlib.statistic.StatisticData; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; import net.automatalib.word.Word; import org.mockito.Mockito; import org.testng.Assert; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; public class CounterOracleTest { @@ -36,6 +38,11 @@ public CounterOracleTest() { this.oracle = new CounterOracle>(Mockito.mock(MembershipOracle.class)); } + @BeforeClass + public void setUp() { + Statistics.getCollector().clear(); + } + @Test public void testInitialState() { verifyCounts(0, 0); @@ -62,18 +69,16 @@ public void testSecondQueryBatch() { verifyCounts(4, 10); } - @Test + @Test(dependsOnMethods = "testSecondQueryBatch") public void testStatistics() { - final StatisticData statisticalData = oracle.getStatisticalData(); - Assert.assertTrue(statisticalData.getName().contains("\n")); - Assert.assertTrue(statisticalData.getUnit().contains("\n")); - Assert.assertTrue(statisticalData.getSummary().contains("\n")); - Assert.assertTrue(statisticalData.getDetails().contains("\n")); + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + Assert.assertFalse(statisticsCollector.getKeys().isEmpty()); } private void verifyCounts(long queries, long symbols) { - Assert.assertEquals(oracle.getQueryCounter().getCount(), queries); - Assert.assertEquals(oracle.getSymbolCounter().getCount(), symbols); + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + Assert.assertEquals(statisticsCollector.getCount(CounterOracle.QUERY_KEY).orElse(0L), queries); + Assert.assertEquals(statisticsCollector.getCount(CounterOracle.SYMBOL_KEY).orElse(0L), symbols); } } diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/HistogramOracleTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/HistogramOracleTest.java deleted file mode 100644 index 632f067f1a..0000000000 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/oracle/HistogramOracleTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* Copyright (C) 2013-2025 TU Dortmund University - * This file is part of LearnLib . - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package de.learnlib.filter.statistic.oracle; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Collections; - -import de.learnlib.filter.statistic.TestQueries; -import de.learnlib.oracle.MembershipOracle; -import de.learnlib.query.Query; -import net.automatalib.common.util.IOUtil; -import net.automatalib.word.Word; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.Test; - -public class HistogramOracleTest { - - private static final String COUNTER_NAME = "testCounter"; - - private final HistogramOracle> oracle; - - @SuppressWarnings("unchecked") - public HistogramOracleTest() { - this.oracle = new HistogramOracle>(Mockito.mock(MembershipOracle.class), COUNTER_NAME); - } - - @Test - public void testInitialState() { - verifyCounts(0, 0, 0, 0); - } - - @Test(dependsOnMethods = "testInitialState") - public void testFirstQueryBatch() { - Collection>> queries = TestQueries.createNoopQueries(2); - oracle.processQueries(queries); - verifyCounts(2, 0, 0, 0); - } - - @Test(dependsOnMethods = "testFirstQueryBatch") - public void testEmptyQueryBatch() { - Collection>> noQueries = Collections.emptySet(); - oracle.processQueries(noQueries); - verifyCounts(2, 0, 0, 0); - } - - @Test(dependsOnMethods = "testEmptyQueryBatch") - public void testSecondQueryBatch() { - Collection>> queries = TestQueries.createNoopQueries(2, 5, TestQueries.INPUTS); - oracle.processQueries(queries); - verifyCounts(4, 10, 2.5, 0); - } - - @Test(dependsOnMethods = "testSecondQueryBatch") - public void testSummary() throws IOException { - - final String details = oracle.getStatisticalData().getDetails(); - final String summary = oracle.getStatisticalData().getSummary(); - - try (InputStream detailStream = HistogramOracleTest.class.getResourceAsStream("/histogram_details.txt"); - InputStream summaryStream = HistogramOracleTest.class.getResourceAsStream("/histogram_summary.txt")) { - - final String expectedDetail = IOUtil.toString(IOUtil.asBufferedUTF8Reader(detailStream)); - final String expectedSummary = IOUtil.toString(IOUtil.asBufferedUTF8Reader(summaryStream)); - - Assert.assertEquals(details, expectedDetail); - Assert.assertEquals(summary, expectedSummary); - } - } - - @Test - public void testGetName() { - Assert.assertEquals(oracle.getStatisticalData().getName(), COUNTER_NAME); - } - - private void verifyCounts(long size, long sum, double mean, long median) { - Assert.assertEquals(oracle.getStatisticalData().getSize(), size); - Assert.assertEquals(oracle.getStatisticalData().getSum(), sum); - Assert.assertEquals(oracle.getStatisticalData().getMean(), mean); - Assert.assertEquals(oracle.getStatisticalData().getMedian(), median); - } -} diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractCounterSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractCounterSULTest.java index 37ab6bdd05..bfe648adc2 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractCounterSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractCounterSULTest.java @@ -17,35 +17,40 @@ import java.util.Collection; import java.util.Collections; +import java.util.Optional; -import de.learnlib.filter.statistic.Counter; import de.learnlib.oracle.MembershipOracle.MealyMembershipOracle; import de.learnlib.oracle.SingleQueryOracle.SingleQueryOracleMealy; import de.learnlib.query.Query; -import de.learnlib.statistic.StatisticSUL; +import de.learnlib.statistic.Statistics; import de.learnlib.sul.SUL; import net.automatalib.word.Word; import org.testng.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -public abstract class AbstractCounterSULTest> { +public abstract class AbstractCounterSULTest> { private S statisticSUL; private MealyMembershipOracle asOracle; protected abstract S getStatisticSUL(); - protected abstract Counter getCounter(S sul); - protected abstract int getCountIncreasePerQuery(); protected abstract Collection>> createQueries(int num); + protected abstract Optional getCount(S sul); + + private long getCount() { + return getCount(statisticSUL).orElse(0L); + } + @BeforeClass public void setUp() { this.statisticSUL = getStatisticSUL(); this.asOracle = getSimulator(this.statisticSUL); + Statistics.getCollector().clear(); } @Test @@ -99,10 +104,6 @@ public void testSharedForkCounter() { Assert.assertEquals(getCount(), oldCount + 2L * 3 * getCountIncreasePerQuery()); } - private long getCount() { - return getCounter(statisticSUL).getCount(); - } - // use custom class to prevent cyclic dependency on learnlib-membership-oracles private static SingleQueryOracleMealy getSimulator(SUL sul) { return (prefix, suffix) -> { diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractResetCounterSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractResetCounterSULTest.java index 6642b6b295..6f4a56a192 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractResetCounterSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractResetCounterSULTest.java @@ -19,10 +19,10 @@ import de.learnlib.filter.statistic.TestQueries; import de.learnlib.query.Query; -import de.learnlib.statistic.StatisticSUL; +import de.learnlib.sul.SUL; import net.automatalib.word.Word; -public abstract class AbstractResetCounterSULTest> +public abstract class AbstractResetCounterSULTest> extends AbstractCounterSULTest { @Override diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractSymbolCounterSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractSymbolCounterSULTest.java index f60e73db66..25a6e5e223 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractSymbolCounterSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/AbstractSymbolCounterSULTest.java @@ -19,10 +19,10 @@ import de.learnlib.filter.statistic.TestQueries; import de.learnlib.query.Query; -import de.learnlib.statistic.StatisticSUL; +import de.learnlib.sul.SUL; import net.automatalib.word.Word; -public abstract class AbstractSymbolCounterSULTest> +public abstract class AbstractSymbolCounterSULTest> extends AbstractCounterSULTest { private static final int QUERY_LENGTH = 5; diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterObservableSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterObservableSULTest.java index 86c22f70ab..c116a7dba0 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterObservableSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterObservableSULTest.java @@ -15,9 +15,12 @@ */ package de.learnlib.filter.statistic.sul; +import java.util.Optional; + import de.learnlib.driver.simulator.ObservableMealySimulatorSUL; -import de.learnlib.filter.statistic.Counter; import de.learnlib.filter.statistic.TestQueries; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; public class ResetCounterObservableSULTest extends AbstractResetCounterSULTest> { @@ -28,7 +31,8 @@ protected CounterObservableSUL getStatisticSUL() { } @Override - protected Counter getCounter(CounterObservableSUL sul) { - return sul.getResetCounter(); + protected Optional getCount(CounterObservableSUL sul) { + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + return statisticsCollector.getCount(CounterObservableSUL.RESET_KEY); } } diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterSULTest.java index c5005aac02..5aa26d731c 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterSULTest.java @@ -15,9 +15,12 @@ */ package de.learnlib.filter.statistic.sul; +import java.util.Optional; + import de.learnlib.driver.simulator.MealySimulatorSUL; -import de.learnlib.filter.statistic.Counter; import de.learnlib.filter.statistic.TestQueries; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; public class ResetCounterSULTest extends AbstractResetCounterSULTest> { @@ -27,7 +30,8 @@ protected CounterSUL getStatisticSUL() { } @Override - protected Counter getCounter(CounterSUL sul) { - return sul.getResetCounter(); + protected Optional getCount(CounterSUL sul) { + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + return statisticsCollector.getCount(CounterSUL.RESET_KEY); } } diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterStateLocalInputSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterStateLocalInputSULTest.java index d96ffe930d..99723bf6cb 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterStateLocalInputSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/ResetCounterStateLocalInputSULTest.java @@ -15,9 +15,12 @@ */ package de.learnlib.filter.statistic.sul; +import java.util.Optional; + import de.learnlib.driver.simulator.StateLocalInputMealySimulatorSUL; -import de.learnlib.filter.statistic.Counter; import de.learnlib.filter.statistic.TestQueries; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; public class ResetCounterStateLocalInputSULTest extends AbstractResetCounterSULTest> { @@ -28,7 +31,8 @@ protected CounterStateLocalInputSUL getStatisticSUL() { } @Override - protected Counter getCounter(CounterStateLocalInputSUL sul) { - return sul.getResetCounter(); + protected Optional getCount(CounterStateLocalInputSUL sul) { + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + return statisticsCollector.getCount(CounterStateLocalInputSUL.RESET_KEY); } } diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterObservableSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterObservableSULTest.java index f6958d146b..8172027e55 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterObservableSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterObservableSULTest.java @@ -15,9 +15,12 @@ */ package de.learnlib.filter.statistic.sul; +import java.util.Optional; + import de.learnlib.driver.simulator.ObservableMealySimulatorSUL; -import de.learnlib.filter.statistic.Counter; import de.learnlib.filter.statistic.TestQueries; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; public class SymbolCounterObservableSULTest extends AbstractSymbolCounterSULTest> { @@ -28,7 +31,8 @@ protected CounterObservableSUL getStatisticSUL() { } @Override - protected Counter getCounter(CounterObservableSUL sul) { - return sul.getSymbolCounter(); + protected Optional getCount(CounterObservableSUL sul) { + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + return statisticsCollector.getCount(CounterObservableSUL.SYMBOL_KEY); } } diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterSULTest.java index 3ac59bb13f..34d5395677 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterSULTest.java @@ -15,9 +15,12 @@ */ package de.learnlib.filter.statistic.sul; +import java.util.Optional; + import de.learnlib.driver.simulator.MealySimulatorSUL; -import de.learnlib.filter.statistic.Counter; import de.learnlib.filter.statistic.TestQueries; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; public class SymbolCounterSULTest extends AbstractSymbolCounterSULTest> { @@ -27,8 +30,9 @@ protected CounterSUL getStatisticSUL() { } @Override - protected Counter getCounter(CounterSUL sul) { - return sul.getSymbolCounter(); + protected Optional getCount(CounterSUL sul) { + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + return statisticsCollector.getCount(CounterSUL.SYMBOL_KEY); } } diff --git a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterStateLocalInputSULTest.java b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterStateLocalInputSULTest.java index 8b543ffbf6..6fa73d2c9a 100644 --- a/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterStateLocalInputSULTest.java +++ b/filters/statistics/src/test/java/de/learnlib/filter/statistic/sul/SymbolCounterStateLocalInputSULTest.java @@ -15,9 +15,12 @@ */ package de.learnlib.filter.statistic.sul; +import java.util.Optional; + import de.learnlib.driver.simulator.StateLocalInputMealySimulatorSUL; -import de.learnlib.filter.statistic.Counter; import de.learnlib.filter.statistic.TestQueries; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; public class SymbolCounterStateLocalInputSULTest extends AbstractSymbolCounterSULTest> { @@ -28,7 +31,8 @@ protected CounterStateLocalInputSUL getStatisticSUL() { } @Override - protected Counter getCounter(CounterStateLocalInputSUL sul) { - return sul.getSymbolCounter(); + protected Optional getCount(CounterStateLocalInputSUL sul) { + final StatisticsCollector statisticsCollector = Statistics.getCollector(); + return statisticsCollector.getCount(CounterStateLocalInputSUL.SYMBOL_KEY); } } diff --git a/filters/statistics/src/test/resources/histogram_details.txt b/filters/statistics/src/test/resources/histogram_details.txt deleted file mode 100644 index 11944aacc7..0000000000 --- a/filters/statistics/src/test/resources/histogram_details.txt +++ /dev/null @@ -1,3 +0,0 @@ -testCounter [query length]: 4 (count), 10 (sum), 2.5 (mean), 0.0 (median) - 0, 2 - 5, 2 diff --git a/filters/statistics/src/test/resources/histogram_summary.txt b/filters/statistics/src/test/resources/histogram_summary.txt deleted file mode 100644 index a034eecfb2..0000000000 --- a/filters/statistics/src/test/resources/histogram_summary.txt +++ /dev/null @@ -1 +0,0 @@ -testCounter [query length]: 4 (count), 10 (sum), 2.5 (mean), 0.0 (median) \ No newline at end of file diff --git a/filters/symbol-filters/pom.xml b/filters/symbol-filters/pom.xml new file mode 100644 index 0000000000..cc79558e52 --- /dev/null +++ b/filters/symbol-filters/pom.xml @@ -0,0 +1,47 @@ + + + 4.0.0 + + + de.learnlib + learnlib-filters-parent + 0.19.0-SNAPSHOT + ../pom.xml + + + learnlib-symbol-filters + + LearnLib :: Filters :: Symbol Filters + A collection of symbol filters + + + + + de.learnlib + learnlib-api + + + + + net.automatalib + automata-api + + + + + de.learnlib.tooling + annotations + + + + org.checkerframework + checker-qual + + + + + org.testng + testng + + + diff --git a/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractPerfectSymbolFilter.java b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractPerfectSymbolFilter.java new file mode 100644 index 0000000000..d071d841f0 --- /dev/null +++ b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractPerfectSymbolFilter.java @@ -0,0 +1,41 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.symbol; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.SymbolFilter; +import net.automatalib.word.Word; + +/** + * A symbol filter that answers all queries correctly. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label + */ +public abstract class AbstractPerfectSymbolFilter extends AbstractTruthfulSymbolFilter + implements SymbolFilter { + + @Override + public FilterResponse query(Word prefix, V symbol) { + if (isIgnorable(prefix, symbol) == FilterResponse.IGNORE) { + return FilterResponse.IGNORE; + } else { + return FilterResponse.ACCEPT; + } + } +} diff --git a/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractRandomSymbolFilter.java b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractRandomSymbolFilter.java new file mode 100644 index 0000000000..c999d4cd46 --- /dev/null +++ b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractRandomSymbolFilter.java @@ -0,0 +1,70 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.symbol; + +import java.util.Random; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.SymbolFilter; +import net.automatalib.word.Word; + +/** + * A symbol filter that falsely answers a query with a specified probability. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label + */ +public abstract class AbstractRandomSymbolFilter extends AbstractTruthfulSymbolFilter implements SymbolFilter { + + private final double inaccurateProb; + private final Random random; + + public AbstractRandomSymbolFilter(double inaccurateProb, Random random) { + this(inaccurateProb, random, valideProbability(inaccurateProb)); + } + + // utility constructor to prevent finalizer attacks, see SEI CERT Rule OBJ-11 + @SuppressWarnings("PMD.UnusedFormalParameter") + private AbstractRandomSymbolFilter(double inaccurateProb, Random random, boolean validated) { + this.inaccurateProb = inaccurateProb; + this.random = random; + } + + private static boolean valideProbability(double inaccurateProb) { + if (inaccurateProb > 1 || inaccurateProb < 0) { + throw new IllegalArgumentException("Ratios must be between zero and 1 (inclusive)."); + } + return true; + } + + @Override + public FilterResponse query(Word prefix, V symbol) { + boolean ignorable = isIgnorable(prefix, symbol) == FilterResponse.IGNORE; + + // Randomly misclassify: + if (this.random.nextDouble() <= this.inaccurateProb) { + ignorable = !ignorable; + } + + if (ignorable) { + return FilterResponse.IGNORE; + } else { + return FilterResponse.ACCEPT; + } + } +} diff --git a/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractStatisticsSymbolFilter.java b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractStatisticsSymbolFilter.java new file mode 100644 index 0000000000..4db8657971 --- /dev/null +++ b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractStatisticsSymbolFilter.java @@ -0,0 +1,83 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.symbol; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.MutableSymbolFilter; +import de.learnlib.filter.SymbolFilter; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; +import net.automatalib.word.Word; + +/** + * Collects various statistics on symbol filtering, including false accepts + false ignores. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label + */ +public abstract class AbstractStatisticsSymbolFilter extends AbstractTruthfulSymbolFilter + implements MutableSymbolFilter { + + public static final String KEY_QUERIES = "sf-qry-cnt"; + public static final String KEY_TRUE_POSITIVES = "sf-tp-cnt"; + public static final String KEY_FALSE_POSITIVES = "sf-fp-cnt"; + public static final String KEY_TRUE_NEGATIVES = "sf-tn-cnt"; + public static final String KEY_FALSE_NEGATIVES = "sf-fn-cnt"; + + private final SymbolFilter delegate; + private final StatisticsCollector statisticsCollector; + + public AbstractStatisticsSymbolFilter(SymbolFilter delegate) { + this.delegate = delegate; + this.statisticsCollector = Statistics.getCollector(); + } + + @Override + public FilterResponse query(Word prefix, V symbol) { + statisticsCollector.increaseCounter(KEY_QUERIES, "Filter: queries"); + + FilterResponse filterResponse = this.delegate.query(prefix, symbol); + FilterResponse expectedResponse = this.isIgnorable(prefix, symbol); + + // Count false ignores, rejects + correct predictions: + if (filterResponse == FilterResponse.ACCEPT) { + if (filterResponse.equals(expectedResponse)) { + statisticsCollector.increaseCounter(KEY_TRUE_POSITIVES, "Filter: correct accepts"); + } else { + statisticsCollector.increaseCounter(KEY_FALSE_POSITIVES, "Filter: false accepts"); + } + } else { + if (filterResponse == expectedResponse) { + statisticsCollector.increaseCounter(KEY_TRUE_NEGATIVES, "Filter: correct ignores"); + } else { + statisticsCollector.increaseCounter(KEY_FALSE_NEGATIVES, "Filter: false ignores"); + } + } + + return filterResponse; + } + + @Override + public void accept(Word prefix, V symbol) { + if (delegate instanceof MutableSymbolFilter mut) { + mut.accept(prefix, symbol); + } else { + throw new UnsupportedOperationException("delegate filter does not support updates"); + } + } +} diff --git a/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractTruthfulSymbolFilter.java b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractTruthfulSymbolFilter.java new file mode 100644 index 0000000000..b027cbc4e2 --- /dev/null +++ b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AbstractTruthfulSymbolFilter.java @@ -0,0 +1,46 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.symbol; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.SymbolFilter; +import net.automatalib.word.Word; + +/** + * A utility class that requires to answer the filter question truthfully. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label + */ +public abstract class AbstractTruthfulSymbolFilter implements SymbolFilter { + + /** + * See {@link SymbolFilter#query(Word, Object)}. The only difference is that this method is not allowed to return + * false responses. + * + * @param prefix + * the prefix identifying the state + * @param symbol + * the input symbol identifying the transition + * + * @return {@link FilterResponse#IGNORE} if the symbol is ignorable, {@link FilterResponse#ACCEPT} otherwise + * + * @see SymbolFilter#query(Word, Object) + */ + protected abstract FilterResponse isIgnorable(Word prefix, V symbol); +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/Counter.java b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AcceptAllSymbolFilter.java similarity index 51% rename from filters/statistics/src/main/java/de/learnlib/filter/statistic/Counter.java rename to filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AcceptAllSymbolFilter.java index eae4a3f680..59c901089e 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/Counter.java +++ b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/AcceptAllSymbolFilter.java @@ -13,42 +13,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package de.learnlib.filter.statistic; +package de.learnlib.filter.symbol; -import java.util.concurrent.atomic.AtomicLong; +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.MutableSymbolFilter; +import net.automatalib.word.Word; /** - * A simple counter. + * A pass-through filter that accepts all inputs. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label */ -public class Counter extends AbstractStatisticData { - - private final AtomicLong count; - - public Counter(String name, String unit) { - super(name, unit); - this.count = new AtomicLong(0L); - } - - public void increment(long inc) { - count.addAndGet(inc); - } - - public void increment() { - count.incrementAndGet(); - } - - public long getCount() { - return count.get(); - } +public class AcceptAllSymbolFilter implements MutableSymbolFilter { @Override - public String getSummary() { - return getName() + " [" + getUnit() + "]: " + count; + public FilterResponse query(Word prefix, V symbol) { + return FilterResponse.ACCEPT; } @Override - public String getDetails() { - return getSummary(); + public void accept(Word prefix, V symbol) { + // we don't need to do anything because we always return ACCEPT anyway } - } diff --git a/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/CachedSymbolFilter.java b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/CachedSymbolFilter.java new file mode 100644 index 0000000000..1a25009f82 --- /dev/null +++ b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/CachedSymbolFilter.java @@ -0,0 +1,66 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.filter.symbol; + +import java.util.HashMap; +import java.util.Map; + +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.MutableSymbolFilter; +import de.learnlib.filter.SymbolFilter; +import net.automatalib.word.Word; + +/** + * Wrapper for a symbol filter that caches previous responses + allows caller to update these. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label + */ +public class CachedSymbolFilter implements MutableSymbolFilter { + + private final Map, Map> previousResponses; // prefix -> (input -> legal/ignore) + private final SymbolFilter delegate; + + public CachedSymbolFilter(SymbolFilter delegate) { + this.delegate = delegate; + this.previousResponses = new HashMap<>(); + } + + @Override + public FilterResponse query(Word prefix, V symbol) { + this.previousResponses.putIfAbsent(prefix, new HashMap<>()); + Boolean oldResponse = this.previousResponses.get(prefix).get(symbol); + if (oldResponse != null) { + return oldResponse ? FilterResponse.ACCEPT : FilterResponse.IGNORE; + } + + FilterResponse res = delegate.query(prefix, symbol); + this.update(prefix, symbol, res); + return res; + } + + @Override + public void accept(Word prefix, V symbol) { + this.update(prefix, symbol, FilterResponse.ACCEPT); + } + + private void update(Word prefix, V symbol, FilterResponse response) { + this.previousResponses.putIfAbsent(prefix, new HashMap<>()); + this.previousResponses.get(prefix).put(symbol, response == FilterResponse.ACCEPT); + } +} diff --git a/filters/statistics/src/main/java/de/learnlib/filter/statistic/AbstractStatisticData.java b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/IgnoreAllSymbolFilter.java similarity index 58% rename from filters/statistics/src/main/java/de/learnlib/filter/statistic/AbstractStatisticData.java rename to filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/IgnoreAllSymbolFilter.java index 03dec7b594..0100d89960 100644 --- a/filters/statistics/src/main/java/de/learnlib/filter/statistic/AbstractStatisticData.java +++ b/filters/symbol-filters/src/main/java/de/learnlib/filter/symbol/IgnoreAllSymbolFilter.java @@ -13,32 +13,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package de.learnlib.filter.symbol; -package de.learnlib.filter.statistic; - -import de.learnlib.statistic.StatisticData; +import de.learnlib.filter.FilterResponse; +import de.learnlib.filter.SymbolFilter; +import net.automatalib.word.Word; /** - * Common interface for statistical data. + * A pass-through filter that ignores all inputs. + * + * @param + * input symbol type of the prefix + * @param + * input symbol type of the transition label */ -public abstract class AbstractStatisticData implements StatisticData { - - private final String name; - private final String unit; - - protected AbstractStatisticData(String name, String unit) { - this.name = name; - this.unit = unit; - } +public class IgnoreAllSymbolFilter implements SymbolFilter { @Override - public String getName() { - return name; + public FilterResponse query(Word prefix, V symbol) { + return FilterResponse.IGNORE; } - - @Override - public String getUnit() { - return unit; - } - } diff --git a/filters/symbol-filters/src/main/java/module-info.java b/filters/symbol-filters/src/main/java/module-info.java new file mode 100644 index 0000000000..c96f2fc027 --- /dev/null +++ b/filters/symbol-filters/src/main/java/module-info.java @@ -0,0 +1,39 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * This module provides a collection of symbol filters. + *

+ * This module is provided by the following Maven dependency: + *

+ * <dependency>
+ *   <groupId>de.learnlib</groupId>
+ *   <artifactId>learnlib-symbol-filters</artifactId>
+ *   <version>${version}</version>
+ * </dependency>
+ * 
+ */ +open module de.learnlib.filter.symbol { + + requires de.learnlib.api; + requires net.automatalib.api; + + // annotations are 'provided'-scoped and do not need to be loaded at runtime + requires static de.learnlib.tooling.annotation; + requires static org.checkerframework.checker.qual; + + exports de.learnlib.filter.symbol; +} diff --git a/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/EQOracleChain.java b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/EQOracleChain.java index 50b8e9d26f..cf0fed1679 100644 --- a/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/EQOracleChain.java +++ b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/EQOracleChain.java @@ -22,6 +22,7 @@ import de.learnlib.oracle.EquivalenceOracle; import de.learnlib.oracle.EquivalenceOracle.DFAEquivalenceOracle; +import de.learnlib.oracle.EquivalenceOracle.MMLTEquivalenceOracle; import de.learnlib.oracle.EquivalenceOracle.MealyEquivalenceOracle; import de.learnlib.oracle.EquivalenceOracle.MooreEquivalenceOracle; import de.learnlib.query.DefaultQuery; @@ -30,8 +31,10 @@ import de.learnlib.tooling.annotation.refinement.Interface; import de.learnlib.tooling.annotation.refinement.Mapping; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; +import net.automatalib.symbol.time.TimedInput; import net.automatalib.word.Word; import org.checkerframework.checker.nullness.qual.Nullable; @@ -66,6 +69,17 @@ generics = {@Generic("I"), @Generic("O")}), interfaces = @Interface(clazz = MooreEquivalenceOracle.class, generics = {@Generic("I"), @Generic("O")})) +@GenerateRefinement(name = "MMLTEQOracleChain", + generics = {@Generic(value = "I", desc = "input symbol type"), + @Generic(value = "O", desc = "output symbol type")}, + parentGenerics = {@Generic(clazz = MMLT.class, generics = {"?", "I", "?", "O"}), + @Generic(clazz = TimedInput.class, generics = "I"), + @Generic(clazz = Word.class, generics = "net.automatalib.symbol.time.TimedOutput")}, + typeMappings = @Mapping(from = EquivalenceOracle.class, + to = MMLTEquivalenceOracle.class, + generics = {@Generic("I"), @Generic("O")}), + interfaces = @Interface(clazz = MMLTEquivalenceOracle.class, + generics = {@Generic("I"), @Generic("O")})) public class EQOracleChain implements EquivalenceOracle { private final List> oracles; diff --git a/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/RandomWpMethodEQOracle.java b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/RandomWpMethodEQOracle.java new file mode 100644 index 0000000000..4acdad8eba --- /dev/null +++ b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/RandomWpMethodEQOracle.java @@ -0,0 +1,164 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.equivalence.mmlt; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import de.learnlib.oracle.EquivalenceOracle.MMLTEquivalenceOracle; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.DefaultQuery; +import de.learnlib.statistic.Statistics; +import de.learnlib.statistic.StatisticsCollector; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.State; +import net.automatalib.automaton.mmlt.impl.ReducedMMLTSemantics; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.util.automaton.Automata; +import net.automatalib.util.automaton.cover.MMLTCover; +import net.automatalib.word.Word; +import net.automatalib.word.WordBuilder; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Implements the partial W-method for MMLT learning. The key modification compared to + * {@link de.learnlib.oracle.equivalence.RandomWpMethodEQOracle} is that prefixes are sampled from entry prefixes only + * instead of all state prefixes. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class RandomWpMethodEQOracle implements MMLTEquivalenceOracle { + + public static final String KEY_TESTED_WORDS = "WP_TESTED_WORD"; + + private final TimedQueryOracle timeOracle; + private final StatisticsCollector statisticsCollector; + + private final Random random; + private final int minSize; + private final int rndLen; + private final int bound; + + public RandomWpMethodEQOracle(TimedQueryOracle timeOracle, + long randomSeed, + int minSize, + int rndAddLength, + int bound) { + + this.timeOracle = timeOracle; + this.statisticsCollector = Statistics.getCollector(); + + this.random = new Random(randomSeed); + + this.minSize = minSize; + this.rndLen = rndAddLength; + this.bound = bound; + } + + @Override + public @Nullable DefaultQuery, Word>> findCounterExample(MMLT hypothesis, + Collection> inputs) { + return findCounterExampleInternal(hypothesis, inputs); + } + + private DefaultQuery, Word>> findCounterExampleInternal(MMLT hypothesis, + Collection> inputs) { + // Make expanded form of hypothesis: + ReducedMMLTSemantics hypSemModel = ReducedMMLTSemantics.forMMLT(hypothesis); + + // Create a list of symbols (for faster access): + List> listAlphabet = new ArrayList<>(inputs); + + // Identify global suffixes: + List>> globalSuffixes = Automata.characterizingSet(hypSemModel, inputs); + + // Get list of prefixes in deterministic order (so we can reproduce experiments easily): + Map>> locationCover = MMLTCover.getMMLTLocationCover(hypothesis, listAlphabet); + List>> prefixList = new ArrayList<>(locationCover.values()); + + // Generate test words: + for (int i = 0; i < this.bound; i++) { + statisticsCollector.increaseCounter(KEY_TESTED_WORDS, "RandomWpOracle: tested words"); + + Word> testword = + this.generateTestword(prefixList, globalSuffixes, hypothesis, hypSemModel, listAlphabet); + + Word> sulAnswer = timeOracle.answerQuery(testword); + Word> hypAnswer = hypothesis.getSemantics().computeOutput(testword); + + // Found inconsistency if outputs do no match: + if (!sulAnswer.equals(hypAnswer)) { + return new DefaultQuery<>(testword, sulAnswer); + } + } + + return null; + } + + private Word> generateTestword(List>> prefixes, + List>> globalSuffixes, + MMLT hypothesis, + ReducedMMLTSemantics hypSemModel, + List> alphabet) { + + WordBuilder> wb = new WordBuilder<>(); + + // 1. Pick a random entry config prefix: + Word> prefix = prefixes.get(this.random.nextInt(prefixes.size())); + wb.append(prefix); + + // 2. Add random middle part: + int size = minSize; + while (size > 0 || this.random.nextDouble() > 1 / (this.rndLen + 1.0)) { + TimedInput nextSymbol = alphabet.get(this.random.nextInt(alphabet.size())); + wb.append(nextSymbol); + + if (size > 0) { + size--; + } + } + + // 3. Pick a random suffix for this state: + // 50% chance for state testing, 50% chance for transition testing + Word> suffix = Word.epsilon(); + if (this.random.nextBoolean()) { + if (!globalSuffixes.isEmpty()) { + suffix = globalSuffixes.get(random.nextInt(globalSuffixes.size())); + } + } else { + // Identify configuration reached by prefix: + State currentConfig = hypothesis.getSemantics().getState(wb); + assert currentConfig != null; + Integer state = hypSemModel.getStateForConfiguration(currentConfig, true); + List>> localSuffixes = Automata.stateCharacterizingSet(hypSemModel, alphabet, state); + + if (!localSuffixes.isEmpty()) { + suffix = localSuffixes.get(random.nextInt(localSuffixes.size())); + } + } + + wb.append(suffix); + + return wb.toWord(); + } +} diff --git a/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/ResetSearchEQOracle.java b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/ResetSearchEQOracle.java new file mode 100644 index 0000000000..912baf0b5c --- /dev/null +++ b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/ResetSearchEQOracle.java @@ -0,0 +1,204 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.equivalence.mmlt; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Random; + +import de.learnlib.oracle.EquivalenceOracle.MMLTEquivalenceOracle; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.DefaultQuery; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.State; +import net.automatalib.automaton.mmlt.TimerInfo; +import net.automatalib.common.util.random.RandomUtil; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.util.automaton.cover.MMLTCover; +import net.automatalib.word.Word; +import net.automatalib.word.WordBuilder; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Searches for counterexamples that reveal local resets by + *
    + *
  • taking any prefix from a known location
  • + *
  • appending a single time step
  • + *
  • appending inputs of all non-delaying inputs that self-loop in that location
  • + *
  • appending timeout
  • + *
. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class ResetSearchEQOracle implements MMLTEquivalenceOracle { + + private static final Logger LOGGER = LoggerFactory.getLogger(ResetSearchEQOracle.class); + + private final TimedQueryOracle timeOracle; + private final Random locPrefixRandom; + + private final double loopInsertPercentage; + private final double testedLocPercentage; + + private final long loopingInputSelectionSeed; + + /** + * Constructor. + * + * @param timeOracle + * the oracle to execute tests on + * @param seed + * the seed for sampling prefixes + * @param loopInsertPercentage + * the percentage of looping symbols that should be used for sampling the random infix (should be between 0 + * and 1) + * @param testedLocPercentage + * the percentage of locations for which prefixes should be included (should be between 0 and 1) + */ + public ResetSearchEQOracle(TimedQueryOracle timeOracle, + long seed, + double loopInsertPercentage, + double testedLocPercentage) { + this.timeOracle = timeOracle; + this.locPrefixRandom = new Random(seed); + + this.loopInsertPercentage = Math.max(0, Math.min(loopInsertPercentage, 1)); + this.testedLocPercentage = Math.max(0, Math.min(testedLocPercentage, 1)); + + this.loopingInputSelectionSeed = seed; + } + + @Override + public @Nullable DefaultQuery, Word>> findCounterExample(MMLT hypothesis, + Collection> inputs) { + if (loopInsertPercentage == 0) { + return null; // oracle is disabled + } + + if (!containsTimeoutAndTimeStep(inputs)) { + LOGGER.warn("Inputs must contain TimeoutSymbol and TimeStepSymbol. Will not find counterexample."); + return null; + } + + return this.findCexInternal(hypothesis, inputs); + } + + private boolean containsTimeoutAndTimeStep(Collection> inputs) { + boolean timeout = false; + boolean timestep = false; + + for (TimedInput i : inputs) { + timeout |= i instanceof TimeoutSymbol; + timestep |= i instanceof TimeStepSequence; + } + + return timeout && timestep; + } + + private @Nullable DefaultQuery, Word>> findCexInternal(MMLT hypothesis, + Collection> inputs) { + + // Retrieve prefixes from state cover, to establish some separation between learner and teacher: + Map>> stateCover = MMLTCover.getMMLTLocationCover(hypothesis, inputs); + + // Only keep locations that have at least two stable configs (only these can have local resets): + List>> prefixes = new ArrayList<>(); + for (Entry>> e : stateCover.entrySet()) { + List> timers = hypothesis.getSortedTimers(e.getKey()); + if (!timers.isEmpty() && timers.get(0).initial() > 1) { + prefixes.add(e.getValue()); + } + } + + // Determine number of tested locations: + int randPrefixes = (int) Math.round(testedLocPercentage * prefixes.size()); + if (randPrefixes == 0) { + LOGGER.warn("No prefixes tested. Need higher percentage?"); + return null; + } + + List>> chosenPrefixes = RandomUtil.sampleUnique(locPrefixRandom, prefixes, randPrefixes); + + for (Word> prefix : chosenPrefixes) { + // Retrieve looping symbols: + State state = hypothesis.getSemantics().getState(prefix); + assert state != null; + S sourceLoc = state.getLocation(); + List> loopingInputs = getLoopingSymbols(sourceLoc, inputs, hypothesis); + if (loopingInputs.isEmpty()) { + continue; // no loops + } + + // Determine number of looping symbols we want to append: + int randElements = (int) Math.round(loopInsertPercentage * loopingInputs.size()); + randElements = Math.min(loopingInputs.size(), randElements); + + List> chosenLoopingInputs = + RandomUtil.sampleUnique(new Random(loopingInputSelectionSeed), loopingInputs, randElements); + + // Create test word: + WordBuilder> wbTestWord = new WordBuilder<>(); + wbTestWord.append(prefix); + wbTestWord.append(TimedInput.step()); + wbTestWord.append(Word.fromList(chosenLoopingInputs)); + wbTestWord.append(TimedInput.timeout()); + + // Check if counterexample: + Word> testWord = wbTestWord.toWord(); + + Word> hypOutput = hypothesis.getSemantics().computeOutput(testWord); + Word> sulOutput = timeOracle.answerQuery(testWord); + if (!hypOutput.equals(sulOutput)) { + return new DefaultQuery<>(testWord, sulOutput); + } + } + return null; + } + + private List> getLoopingSymbols(S sourceLoc, + Collection> inputs, + MMLT hypothesis) { + final List> loopingInputs = new ArrayList<>(); + + for (TimedInput sym : inputs) { + // only consider non-delaying inputs, as only these can perform local resets + if (sym instanceof InputSymbol ndi) { + final T trans = hypothesis.getTransition(sourceLoc, ndi.symbol()); + + // Collect self-loops: + if (trans != null && Objects.equals(hypothesis.getSuccessor(trans), sourceLoc)) { + loopingInputs.add(sym); + } + } + } + + return loopingInputs; + } + +} diff --git a/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/SimulatorEQOracle.java b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/SimulatorEQOracle.java new file mode 100644 index 0000000000..eb9bd6d15c --- /dev/null +++ b/oracles/equivalence-oracles/src/main/java/de/learnlib/oracle/equivalence/mmlt/SimulatorEQOracle.java @@ -0,0 +1,57 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.equivalence.mmlt; + +import java.util.Collection; + +import de.learnlib.oracle.EquivalenceOracle.MMLTEquivalenceOracle; +import de.learnlib.query.DefaultQuery; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.util.automaton.mmlt.MMLTs; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A simulator oracle for MMLTs. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class SimulatorEQOracle implements MMLTEquivalenceOracle { + + private final MMLT refModel; + + public SimulatorEQOracle(MMLT refModel) { + this.refModel = refModel; + } + + @Override + public @Nullable DefaultQuery, Word>> findCounterExample(MMLT hypothesis, + Collection> inputs) { + final Word> separatingWord = MMLTs.findSeparatingWord(refModel, hypothesis, inputs); + + if (separatingWord != null) { + final Word> sulOutput = refModel.getSemantics().computeOutput(separatingWord); + return new DefaultQuery<>(separatingWord, sulOutput); + } else { + return null; + } + } +} diff --git a/oracles/equivalence-oracles/src/main/java/module-info.java b/oracles/equivalence-oracles/src/main/java/module-info.java index 9c5d3a390b..c7d83f3289 100644 --- a/oracles/equivalence-oracles/src/main/java/module-info.java +++ b/oracles/equivalence-oracles/src/main/java/module-info.java @@ -46,4 +46,5 @@ exports de.learnlib.oracle.equivalence.spa; exports de.learnlib.oracle.equivalence.spmm; exports de.learnlib.oracle.equivalence.vpa; + exports de.learnlib.oracle.equivalence.mmlt; } diff --git a/oracles/equivalence-oracles/src/test/java/de/learnlib/oracle/equivalence/mmlt/RandomWpMethodEQOracleTest.java b/oracles/equivalence-oracles/src/test/java/de/learnlib/oracle/equivalence/mmlt/RandomWpMethodEQOracleTest.java new file mode 100644 index 0000000000..c2bd7f4558 --- /dev/null +++ b/oracles/equivalence-oracles/src/test/java/de/learnlib/oracle/equivalence/mmlt/RandomWpMethodEQOracleTest.java @@ -0,0 +1,88 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.equivalence.mmlt; + +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.alphabet.impl.Alphabets; +import net.automatalib.automaton.mmlt.impl.CompactMMLT; +import net.automatalib.automaton.mmlt.impl.StringSymbolCombiner; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class RandomWpMethodEQOracleTest { + + @Test + public void testEquivalence() { + var example = MMLTExamples.sensorCollector(); + var mmlt = example.getReferenceAutomaton(); + var alphabet = example.getAlphabet(); + + var mqo = new TimedSULOracle<>(new MMLTSimulatorSUL<>(mmlt), example.getParams()); + var eqo = new RandomWpMethodEQOracle<>(mqo, 123, 10, 0, 100); + var cex = eqo.findCounterExample(mmlt, alphabet); + + Assert.assertNull(cex); + } + + @Test + public void testInequivalence() { + var mmlt = buildMMLT(); + var alphabet = mmlt.getSemantics().getInputAlphabet(); + var params = new MMLTModelParams<>("void", StringSymbolCombiner.getInstance(), 4, 80); + + var hyp = buildMMLT(); + var t = hyp.getTransition(1, "abort"); + hyp.setTransitionOutput(t, "part"); + + var mqo = new TimedSULOracle<>(new MMLTSimulatorSUL<>(mmlt), params); + var eqo = new RandomWpMethodEQOracle<>(mqo, 42, 0, 2, 100); + var cex = eqo.findCounterExample(hyp, alphabet); + + Assert.assertNotNull(cex); + Assert.assertEquals(cex.getOutput(), mmlt.getSemantics().computeSuffixOutput(cex.getPrefix(), cex.getSuffix())); + Assert.assertNotEquals(mmlt.getSemantics().computeSuffixOutput(cex.getPrefix(), cex.getSuffix()), + hyp.getSemantics().computeSuffixOutput(cex.getPrefix(), cex.getSuffix())); + } + + private static CompactMMLT buildMMLT() { + var alphabet = Alphabets.fromArray("p1", "p2", "abort", "collect"); + var model = new CompactMMLT<>(alphabet, "void", StringSymbolCombiner.getInstance()); + + var s0 = model.addInitialState(); + var s1 = model.addState(); + var s2 = model.addState(); + var s3 = model.addState(); + + model.addTransition(s0, "p1", s1, "go"); + model.addTransition(s1, "abort", s1, "ok"); + model.addLocalReset(s1, "abort"); + + model.addPeriodicTimer(s1, "a", 3, "part"); + model.addPeriodicTimer(s1, "b", 6, "noise"); + model.addOneShotTimer(s1, "c", 40, "done", s3); + + model.addTransition(s0, "p2", s2, "go"); + model.addTransition(s2, "abort", s3, "void"); + model.addOneShotTimer(s2, "d", 4, "done", s3); + + model.addTransition(s3, "collect", s0, "void"); + + return model; + } +} diff --git a/oracles/equivalence-oracles/src/test/java/de/learnlib/oracle/equivalence/mmlt/ResetSearchEQOracleTest.java b/oracles/equivalence-oracles/src/test/java/de/learnlib/oracle/equivalence/mmlt/ResetSearchEQOracleTest.java new file mode 100644 index 0000000000..0e87e9b42b --- /dev/null +++ b/oracles/equivalence-oracles/src/test/java/de/learnlib/oracle/equivalence/mmlt/ResetSearchEQOracleTest.java @@ -0,0 +1,87 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.equivalence.mmlt; + +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import net.automatalib.alphabet.impl.GrowingMapAlphabet; +import net.automatalib.symbol.time.TimedInput; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class ResetSearchEQOracleTest { + + @Test + public void testInsertPercentages() { + @SuppressWarnings("unchecked") + TimedQueryOracle mock = Mockito.mock(TimedQueryOracle.class); + + var example = MMLTExamples.sensorCollector(); + var mmlt = example.getReferenceAutomaton(); + var alphabet = example.getAlphabet(); + + var eqo = new ResetSearchEQOracle<>(mock, 123, 0, 1); + var cex = eqo.findCounterExample(mmlt, alphabet); + + Assert.assertNull(cex); + Mockito.verifyNoInteractions(mock); + + eqo = new ResetSearchEQOracle<>(mock, 123, 1, 0); + cex = eqo.findCounterExample(mmlt, alphabet); + + Assert.assertNull(cex); + Mockito.verifyNoInteractions(mock); + } + + @Test + public void testAlphabetFilter() { + @SuppressWarnings("unchecked") + TimedQueryOracle mock = Mockito.spy(TimedQueryOracle.class); + + var example = MMLTExamples.sensorCollector(); + var mmlt = example.getReferenceAutomaton(); + var alphabet = example.getUntimedAlphabet().stream().>map(TimedInput::input).toList(); + + var eqo = new ResetSearchEQOracle<>(mock, 123, 1, 1); + var cex = eqo.findCounterExample(mmlt, alphabet); + + Assert.assertNull(cex); + Mockito.verifyNoInteractions(mock); + + var alphabetWithTimeOut = new GrowingMapAlphabet<>(alphabet); + alphabetWithTimeOut.add(TimedInput.timeout()); + cex = eqo.findCounterExample(mmlt, alphabetWithTimeOut); + + Assert.assertNull(cex); + Mockito.verifyNoInteractions(mock); + + var alphabetWithTimestep = new GrowingMapAlphabet<>(alphabet); + alphabetWithTimestep.add(TimedInput.step()); + cex = eqo.findCounterExample(mmlt, alphabetWithTimestep); + + Assert.assertNull(cex); + Mockito.verifyNoInteractions(mock); + + var validAlphabe = example.getAlphabet(); + cex = eqo.findCounterExample(mmlt, validAlphabe); + + // mock always returns null which differs from any non-null hypothesis output + Assert.assertNotNull(cex); + Mockito.verify(mock, Mockito.atLeastOnce()).processQueries(ArgumentMatchers.anyCollection()); + } +} diff --git a/oracles/membership-oracles/src/main/java/de/learnlib/oracle/membership/TimedSULOracle.java b/oracles/membership-oracles/src/main/java/de/learnlib/oracle/membership/TimedSULOracle.java new file mode 100644 index 0000000000..b3b5d47025 --- /dev/null +++ b/oracles/membership-oracles/src/main/java/de/learnlib/oracle/membership/TimedSULOracle.java @@ -0,0 +1,290 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.membership; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import de.learnlib.oracle.SingleQueryOracle.SingleQueryOracleMMLT; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.sul.TimedSUL; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.automaton.mmlt.TimerInfo; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.symbol.time.TimeoutSymbol; +import net.automatalib.word.Word; +import net.automatalib.word.WordBuilder; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements a {@link TimedQueryOracle} given a {@link TimedSUL}. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + */ +public class TimedSULOracle implements SingleQueryOracleMMLT { + + private static final Logger LOGGER = LoggerFactory.getLogger(TimedSULOracle.class); + + /** + * To ensure globally unique timer names, we index them according to this counter. + */ + private int timerCounter; + + private final TimedSUL sul; + private final MMLTModelParams modelParams; + + public TimedSULOracle(TimedSUL sul, MMLTModelParams modelParams) { + this.sul = sul; + this.modelParams = modelParams; + this.timerCounter = 0; + } + + @Override + public Word> answerQuery(Word> prefix, Word> suffix) { + sul.pre(); + sul.follow(prefix, this.modelParams.maxTimeoutWaitingTime()); + + // Query the SUL, one symbol at a time: + WordBuilder> wbOutput = new WordBuilder<>(); + for (TimedInput s : suffix) { + if (s instanceof TimeoutSymbol) { + TimedOutput output = sul.timeoutStep(this.modelParams.maxTimeoutWaitingTime()); + if (output != null) { + wbOutput.append(output); + } else { + wbOutput.append(new TimedOutput<>(this.modelParams.silentOutput())); // no output in time -> silent + } + } else if (s instanceof InputSymbol ndi) { + TimedOutput output = sul.step(ndi); + wbOutput.append(output); + } else if (s instanceof TimeStepSequence ws) { + if (ws.timeSteps() > 1) { + throw new IllegalArgumentException("Only single wait step allowed in suffix."); + } + + // Wait for a single time step: + TimedOutput output = sul.timeStep(); + if (output != null) { + wbOutput.append(output); + } else { + wbOutput.append(new TimedOutput<>(this.modelParams.silentOutput())); // no output in time -> silent + } + + } else { + throw new IllegalArgumentException("Only timeout or untimed symbols allowed in suffix."); + } + } + + sul.post(); + return wbOutput.toWord(); + } + + @Override + public TimerQueryResult queryTimers(Word> prefix, long maxTotalWaitingTime) { + this.sul.pre(); + + // Go to location: + this.sul.follow(prefix); + + // Collect timeouts: + TimerQueryResult timers = this.collectTimeouts(maxTotalWaitingTime); + + this.sul.post(); + return timers; + } + + /** + * Identifies the time at which the next known timeout(s) are expected. + * + * @param timeouts + * known timeouts + * @param currentTime + * current time + * + * @return next timeout time + */ + private long calcNextExpectedTimeout(List> timeouts, long currentTime) { + assert !timeouts.isEmpty(); + + long minNext = Long.MAX_VALUE; + for (TimerInfo to : timeouts) { + long occurrences = currentTime / to.initial(); + long nextOcc = (occurrences + 1) * to.initial(); // time of next occ + + if (nextOcc < minNext) { + minNext = nextOcc; + } + } + + assert minNext != Long.MAX_VALUE; + + return minNext; + } + + private String newUniqueTimerName() { + return "t_" + (++this.timerCounter); + } + + /** + * Identifies timeouts in the current location by waiting at most {@code maxTotalWaitingTime}. + *

+ * All inferred timers are initially considered periodic. Stops when reaching {@code maxTotalWaitingTime} or when an + * expected timeout does not occur. In the latter case, the {@link TimerQueryResult#aborted()}} flag is set. + * + * @param maxTotalWaitingTime + * maximum time until timeouts are collected + * + * @return list of periodic timeouts or {@code null} if none observed + */ + private TimerQueryResult collectTimeouts(long maxTotalWaitingTime) { + if (maxTotalWaitingTime < this.modelParams.maxTimeoutWaitingTime()) { + throw new IllegalArgumentException( + "Timer query waiting time must be at least max. waiting time for a single timeout."); + } + + List> knownTimers = new ArrayList<>(); + + // Wait for the first timeout: + TimedOutput firstTimeout = this.sul.timeoutStep(this.modelParams.maxTimeoutWaitingTime()); + if (firstTimeout == null) { + return new TimerQueryResult<>(false, Collections.emptyList()); // no timeouts found + } + + List firstTimeoutOutputs = this.modelParams.outputCombiner().separateSymbols(firstTimeout.symbol()); + if (firstTimeoutOutputs.size() > 1) { + LOGGER.warn("Multiple timers expiring at first timeout, automaton may not be minimal."); + } + + knownTimers.add(new TimerInfo<>(newUniqueTimerName(), firstTimeout.delay(), firstTimeoutOutputs, null, true)); + + // Wait for further timeouts: + long currentTimeStep = firstTimeout.delay(); // already waited for first timeout + + boolean inconsistent = false; + while (currentTimeStep < maxTotalWaitingTime) { + // Identify time of next expected timeout: + long nextExpectedTime = this.calcNextExpectedTimeout(knownTimers, currentTimeStep); + + // Wait either until next timeout OR until maximum waiting time reached: + long nextWaiting = Math.min(nextExpectedTime, maxTotalWaitingTime) - currentTimeStep; + + // Wait until next timeout: + TimedOutput nextOutput = this.sul.timeoutStep(nextWaiting); + if (nextOutput == null) { + if (nextExpectedTime <= maxTotalWaitingTime) { + // Expected a timeout within max. waiting time but nothing happened: + inconsistent = true; + } + + break; // either max time exceeded OR missing timeout (-> inconsistent) + } + + // Compare observed timeout with expectation: + long nextActualTime = nextOutput.delay() + currentTimeStep; + + TimerCheckResult evalResult = + evaluateNextTimer(nextActualTime, nextExpectedTime, nextOutput, knownTimers); + if (evalResult.newTimer() != null) { + knownTimers.add(evalResult.newTimer()); + } else if (evalResult.inconsistent()) { + inconsistent = true; + break; + } + + currentTimeStep = nextActualTime; + } + + knownTimers.sort(Comparator.comparingLong(TimerInfo::initial)); + return new TimerQueryResult<>(inconsistent, knownTimers); + } + + private TimerCheckResult evaluateNextTimer(long nextActualTime, + long nextExpectedTime, + TimedOutput nextOutput, + List> knownTimers) { + + List nextOutputSymbols = this.modelParams.outputCombiner().separateSymbols(nextOutput.symbol()); + + if (nextActualTime < nextExpectedTime) { + // A timeout occurred before we expected one -> new timer: + TimerInfo newTimer = + new TimerInfo<>(newUniqueTimerName(), nextActualTime, nextOutputSymbols, null, true); + return new TimerCheckResult<>(newTimer, false); + } else { + assert nextActualTime == nextExpectedTime; + // Timeout occurred at expected time -> check if matching expected output: + Map expectedOutputs = new HashMap<>(); + for (TimerInfo t : knownTimers) { + if (nextExpectedTime % t.initial() == 0) { + for (O o : t.outputs()) { + expectedOutputs.merge(o, 1L, Long::sum); + } + } + } + + Map actualOutputs = new HashMap<>(); + for (O o : nextOutputSymbols) { + actualOutputs.merge(o, 1L, Long::sum); + } + + // Any outputs that were expected but are not present? + for (Entry e : expectedOutputs.entrySet()) { + if (actualOutputs.getOrDefault(e.getKey(), 0L) < e.getValue()) { + // Same time but missing output -> missed location change: + return new TimerCheckResult<>(null, true); + } + } + + // At least all expected outputs are present. + // Check for additional outputs: + List newOutputs = new ArrayList<>(); + for (Entry e : actualOutputs.entrySet()) { + O output = e.getKey(); + long expectedCount = expectedOutputs.getOrDefault(output, 0L); + long actualCount = e.getValue(); + + long additional = actualCount - expectedCount; + for (int i = 0; i < additional; i++) { + newOutputs.add(output); + } + } + + if (!newOutputs.isEmpty()) { + // Same time and more outputs -> add new timer that uses the new outputs: + TimerInfo newTimer = + new TimerInfo<>(newUniqueTimerName(), nextActualTime, newOutputs, null, true); + return new TimerCheckResult<>(newTimer, false); + } + } + + return new TimerCheckResult<>(null, false); + } + + private record TimerCheckResult(@Nullable TimerInfo newTimer, boolean inconsistent) {} +} diff --git a/oracles/membership-oracles/src/test/java/de/learnlib/oracle/membership/SimulatorOracleTest.java b/oracles/membership-oracles/src/test/java/de/learnlib/oracle/membership/SimulatorOracleTest.java index ac68bfae78..48e99a2873 100644 --- a/oracles/membership-oracles/src/test/java/de/learnlib/oracle/membership/SimulatorOracleTest.java +++ b/oracles/membership-oracles/src/test/java/de/learnlib/oracle/membership/SimulatorOracleTest.java @@ -46,8 +46,8 @@ public void testDFASimulatorOracle() { queries.add(q1); queries.add(q2); - Assert.assertEquals(queries.get(0).getInput().size(), 3); - Assert.assertEquals(queries.get(1).getInput().size(), 3); + Assert.assertEquals(queries.get(0).length(), 3); + Assert.assertEquals(queries.get(1).length(), 3); oracle.processQueries(queries); diff --git a/oracles/membership-oracles/src/test/java/de/learnlib/oracle/membership/TimedSULOracleTest.java b/oracles/membership-oracles/src/test/java/de/learnlib/oracle/membership/TimedSULOracleTest.java new file mode 100644 index 0000000000..15510a9ed3 --- /dev/null +++ b/oracles/membership-oracles/src/test/java/de/learnlib/oracle/membership/TimedSULOracleTest.java @@ -0,0 +1,49 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.membership; + +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.word.Word; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class TimedSULOracleTest { + + @Test + public void testValidation() { + var example = MMLTExamples.sensorCollector(); + var mmlt = example.getReferenceAutomaton(); + var params = example.getParams(); + + var oracle = new TimedSULOracle<>(new MMLTSimulatorSUL<>(mmlt), params); + + Assert.assertThrows(IllegalArgumentException.class, + () -> oracle.queryTimers(Word.epsilon(), params.maxTimeoutWaitingTime() - 1)); + Assert.assertThrows(IllegalArgumentException.class, + () -> oracle.queryTimers(Word.fromLetter(TimedInput.timeout()), + params.maxTimeoutWaitingTime())); + Assert.assertThrows(IllegalArgumentException.class, + () -> oracle.answerQuery(Word.epsilon(), Word.fromLetter(TimedInput.step(2)))); + + // assert not throwing + Assert.assertEquals(oracle.answerQuery(Word.fromLetter(TimedInput.timeout()), + Word.fromLetter(TimedInput.timeout())), + Word.fromLetter(mmlt.getSemantics().getSilentOutput())); + + } +} diff --git a/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/DynamicParallelTimedOracleQueryBuilder.java b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/DynamicParallelTimedOracleQueryBuilder.java new file mode 100644 index 0000000000..9a0fdba89a --- /dev/null +++ b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/DynamicParallelTimedOracleQueryBuilder.java @@ -0,0 +1,53 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.Collection; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; + +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.Query; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; + +/** + * A specialized {@link AbstractDynamicBatchProcessorBuilder} for {@link TimedQueryOracle}s. + * + * @param + * input symbol type + * @param + * output symbol type + */ +public class DynamicParallelTimedOracleQueryBuilder + extends AbstractDynamicBatchProcessorBuilder, Word>>, TimedQueryOracle, DynamicParallelTimedQueryOracle> { + + public DynamicParallelTimedOracleQueryBuilder(Supplier> oracleSupplier) { + super(oracleSupplier); + } + + public DynamicParallelTimedOracleQueryBuilder(Collection> oracles) { + super(oracles); + } + + @Override + protected DynamicParallelTimedQueryOracle buildOracle(Supplier> supplier, + int minBatchSize, + ExecutorService executor) { + return new DynamicParallelTimedQueryOracle<>(supplier, minBatchSize, executor); + } +} diff --git a/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/DynamicParallelTimedQueryOracle.java b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/DynamicParallelTimedQueryOracle.java new file mode 100644 index 0000000000..14c5cd1c1c --- /dev/null +++ b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/DynamicParallelTimedQueryOracle.java @@ -0,0 +1,58 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.Collection; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; + +import de.learnlib.oracle.ParallelTimedQueryOracle; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.Query; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.checkerframework.checker.index.qual.NonNegative; + +/** + * A specialized {@link AbstractDynamicBatchProcessor} for {@link TimedQueryOracle}s that implements + * {@link ParallelTimedQueryOracle}. + * + * @param + * input symbol type + * @param + * output symbol type + */ +public class DynamicParallelTimedQueryOracle + extends AbstractDynamicBatchProcessor, Word>>, TimedQueryOracle> + implements ParallelTimedQueryOracle { + + public DynamicParallelTimedQueryOracle(Supplier> oracleSupplier, + @NonNegative int minBatchSize, + ExecutorService executorService) { + super(oracleSupplier, minBatchSize, executorService); + } + + @Override + public void processQueries(Collection, Word>>> queries) { + processBatch(queries); + } + + @Override + public TimerQueryResult queryTimers(Word> prefix, long maxTotalWaitingTime) { + return getProcessor().queryTimers(prefix, maxTotalWaitingTime); + } +} diff --git a/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/ParallelOracleBuilders.java b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/ParallelOracleBuilders.java index 328267304b..91eb4822a6 100644 --- a/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/ParallelOracleBuilders.java +++ b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/ParallelOracleBuilders.java @@ -22,13 +22,17 @@ import de.learnlib.oracle.MembershipOracle; import de.learnlib.oracle.OmegaMembershipOracle; import de.learnlib.oracle.ThreadPool.PoolPolicy; +import de.learnlib.oracle.TimedQueryOracle; import de.learnlib.oracle.membership.AbstractSULOmegaOracle; import de.learnlib.oracle.membership.SULAdaptiveOracle; import de.learnlib.oracle.membership.SULOracle; import de.learnlib.oracle.membership.StateLocalInputSULOracle; +import de.learnlib.oracle.membership.TimedSULOracle; import de.learnlib.sul.ObservableSUL; import de.learnlib.sul.SUL; import de.learnlib.sul.StateLocalInputSUL; +import de.learnlib.sul.TimedSUL; +import de.learnlib.time.MMLTModelParams; import net.automatalib.common.util.collection.CollectionUtil; import net.automatalib.word.Word; @@ -320,6 +324,83 @@ public static DynamicParallelAdaptiveOracleBuilder newDynamicParall return new DynamicParallelAdaptiveOracleBuilder<>(oracles); } + /** + * Creates a {@link DynamicParallelTimedOracleQueryBuilder} using the provided {@code sul} as a supplier. This + * requires that the sul is {@link SUL#canFork() forkable}. + * + * @param sul + * the sul instance for spawning new thread-specific oracle instances + * @param params + * additional parameters for answering queries + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return a preconfigured oracle builder + */ + public static DynamicParallelTimedOracleQueryBuilder newDynamicParallelTimedQueryOracle(TimedSUL sul, + MMLTModelParams params) { + checkFork(sul); + return newDynamicParallelTimedQueryOracle(toSupplier(sul, params)); + } + + /** + * Creates a {@link DynamicParallelTimedOracleQueryBuilder} using the provided supplier. + * + * @param oracleSupplier + * the supplier for spawning new thread-specific oracle instances + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return a preconfigured oracle builder + */ + public static DynamicParallelTimedOracleQueryBuilder newDynamicParallelTimedQueryOracle(Supplier> oracleSupplier) { + return new DynamicParallelTimedOracleQueryBuilder<>(oracleSupplier); + } + + /** + * Convenience method for {@link #newDynamicParallelTimedQueryOracle(Collection)}. + * + * @param firstOracle + * the first (mandatory) oracle + * @param otherOracles + * further (optional) oracles to be used by other threads + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return a preconfigured oracle builder + */ + @SafeVarargs + public static DynamicParallelTimedOracleQueryBuilder newDynamicParallelTimedQueryOracle( + TimedQueryOracle firstOracle, + TimedQueryOracle... otherOracles) { + return newDynamicParallelTimedQueryOracle(CollectionUtil.list(firstOracle, otherOracles)); + } + + /** + * Creates a {@link DynamicParallelTimedOracleQueryBuilder} using the provided collection of membership oracles. The + * resulting parallel oracle will always use a {@link PoolPolicy#FIXED} pool policy and spawn a separate thread for + * each of the provided oracles (so that the oracles do not need to care about synchronization if they don't share + * state). + * + * @param oracles + * the oracle instances to distribute the queries to + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return the preconfigured oracle builder + */ + public static DynamicParallelTimedOracleQueryBuilder newDynamicParallelTimedQueryOracle(Collection> oracles) { + return new DynamicParallelTimedOracleQueryBuilder<>(oracles); + } + /** * Creates a {@link StaticParallelOracleBuilder} using the provided {@code sul} as a supplier. This requires that * the sul is {@link SUL#canFork() forkable}. @@ -569,6 +650,82 @@ public static StaticParallelAdaptiveOracleBuilder newStaticParallel return new StaticParallelAdaptiveOracleBuilder<>(oracles); } + /** + * Creates a {@link StaticParallelTimedQueryOracleBuilder} using the provided {@code sul} as a supplier. This + * requires that the sul is {@link SUL#canFork() forkable}. + * + * @param sul + * the sul instance for spawning new thread-specific oracle instances + * @param params + * additional parameters for answering queries + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return a preconfigured oracle builder + */ + public static StaticParallelTimedQueryOracleBuilder newStaticParallelTimedQueryOracle(TimedSUL sul, + MMLTModelParams params) { + checkFork(sul); + return newStaticParallelTimedQueryOracle(toSupplier(sul, params)); + } + + /** + * Creates a {@link StaticParallelTimedQueryOracleBuilder} using the provided supplier. + * + * @param oracleSupplier + * the supplier for spawning new thread-specific oracle instances + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return a preconfigured oracle builder + */ + public static StaticParallelTimedQueryOracleBuilder newStaticParallelTimedQueryOracle(Supplier> oracleSupplier) { + return new StaticParallelTimedQueryOracleBuilder<>(oracleSupplier); + } + + /** + * Convenience method for {@link #newStaticParallelTimedQueryOracle(Collection)}. + * + * @param firstOracle + * the first (mandatory) oracle + * @param otherOracles + * further (optional) oracles to be used by other threads + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return a preconfigured oracle builder + */ + @SafeVarargs + public static StaticParallelTimedQueryOracleBuilder newStaticParallelTimedQueryOracle(TimedQueryOracle firstOracle, + TimedQueryOracle... otherOracles) { + return newStaticParallelTimedQueryOracle(CollectionUtil.list(firstOracle, otherOracles)); + } + + /** + * Creates a {@link StaticParallelTimedQueryOracleBuilder} using the provided collection of membership oracles. The + * resulting parallel oracle will always use a {@link PoolPolicy#FIXED} pool policy and spawn a separate thread for + * each of the provided oracles (so that the oracles do not need to care about synchronization if they don't share + * state). + * + * @param oracles + * the oracle instances to distribute the queries to + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * + * @return the preconfigured oracle builder + */ + public static StaticParallelTimedQueryOracleBuilder newStaticParallelTimedQueryOracle(Collection> oracles) { + return new StaticParallelTimedQueryOracleBuilder<>(oracles); + } + private static Supplier> toSupplier(SUL sul) { return () -> new SULOracle<>(sul.fork()); } @@ -582,6 +739,10 @@ private static Supplier>> toSuppli return () -> AbstractSULOmegaOracle.newOracle(sul.fork()); } + private static Supplier> toSupplier(TimedSUL sul, MMLTModelParams params) { + return () -> new TimedSULOracle<>(sul.fork(), params); + } + private static Supplier> toAdaptiveSupplier(SUL sul) { return () -> new SULAdaptiveOracle<>(sul.fork()); } diff --git a/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracle.java b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracle.java new file mode 100644 index 0000000000..ca465bb829 --- /dev/null +++ b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracle.java @@ -0,0 +1,57 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.Collection; +import java.util.concurrent.ExecutorService; + +import de.learnlib.oracle.ParallelTimedQueryOracle; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.Query; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.checkerframework.checker.index.qual.NonNegative; + +/** + * A specialized {@link AbstractStaticBatchProcessor} for {@link TimedQueryOracle}s that implements + * {@link ParallelTimedQueryOracle}. + * + * @param + * input symbol type + * @param + * output symbol type + */ +public class StaticParallelTimedQueryOracle + extends AbstractStaticBatchProcessor, Word>>, TimedQueryOracle> + implements ParallelTimedQueryOracle { + + public StaticParallelTimedQueryOracle(Collection> oracles, + @NonNegative int minBatchSize, + ExecutorService executorService) { + super(oracles, minBatchSize, executorService); + } + + @Override + public void processQueries(Collection, Word>>> queries) { + processBatch(queries); + } + + @Override + public TimerQueryResult queryTimers(Word> prefix, long maxTotalWaitingTime) { + return getProcessor().queryTimers(prefix, maxTotalWaitingTime); + } +} diff --git a/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracleBuilder.java b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracleBuilder.java new file mode 100644 index 0000000000..75fcfa7c05 --- /dev/null +++ b/oracles/parallelism/src/main/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracleBuilder.java @@ -0,0 +1,53 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.Collection; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; + +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.Query; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; + +/** + * A specialized {@link AbstractStaticBatchProcessorBuilder} for {@link TimedQueryOracle}s. + * + * @param + * input symbol type + * @param + * output symbol type + */ +public class StaticParallelTimedQueryOracleBuilder + extends AbstractStaticBatchProcessorBuilder, Word>>, TimedQueryOracle, StaticParallelTimedQueryOracle> { + + public StaticParallelTimedQueryOracleBuilder(Supplier> oracleSupplier) { + super(oracleSupplier); + } + + public StaticParallelTimedQueryOracleBuilder(Collection> oracles) { + super(oracles); + } + + @Override + protected StaticParallelTimedQueryOracle buildOracle(Collection> oracleInstances, + int minBatchSize, + ExecutorService executor) { + return new StaticParallelTimedQueryOracle<>(oracleInstances, minBatchSize, executor); + } +} diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/AbstractDynamicParallelTimedQueryOracleTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/AbstractDynamicParallelTimedQueryOracleTest.java new file mode 100644 index 0000000000..d5f7125c19 --- /dev/null +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/AbstractDynamicParallelTimedQueryOracleTest.java @@ -0,0 +1,163 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +import de.learnlib.oracle.ParallelTimedQueryOracle; +import de.learnlib.oracle.ThreadPool.PoolPolicy; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.query.Query; +import de.learnlib.sul.TimedSUL; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.testng.Assert; +import org.testng.annotations.Test; + +@Test +public abstract class AbstractDynamicParallelTimedQueryOracleTest { + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testEmpty(PoolPolicy poolPolicy) { + ParallelTimedQueryOracle oracle = getBuilder().withPoolPolicy(poolPolicy).create(); + + try { + oracle.processQueries(Collections.emptyList()); + } finally { + oracle.shutdownNow(); + } + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testDistinctQueries(PoolPolicy poolPolicy) { + ParallelTimedQueryOracle oracle = + getBuilder().withBatchSize(1).withPoolSize(4).withPoolPolicy(poolPolicy).create(); + + try { + List> queries = createQueries(100); + + oracle.processQueries(queries); + + for (AnswerOnceQuery query : queries) { + Assert.assertTrue(query.answered.get()); + } + } finally { + oracle.shutdown(); + } + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class, expectedExceptions = IllegalStateException.class) + public void testDuplicateQueries(PoolPolicy poolPolicy) { + ParallelTimedQueryOracle oracle = + getBuilder().withBatchSize(1).withPoolSize(4).withPoolPolicy(poolPolicy).create(); + try { + List> queries = new ArrayList<>(createQueries(100)); + queries.add(queries.get(0)); + + oracle.processQueries(queries); + } finally { + oracle.shutdown(); + } + } + + protected abstract DynamicParallelTimedOracleQueryBuilder getBuilder(); + + protected static List> createQueries(int numQueries) { + List> queries = new ArrayList<>(numQueries); + + for (int i = 0; i < numQueries; i++) { + queries.add(new AnswerOnceQuery<>()); + } + + return queries; + } + + static class NullSUL implements TimedSUL { + + @Override + public void pre() {} + + @Override + public void post() {} + + @Override + public TimedOutput step(InputSymbol in) { + return new TimedOutput<>(null); + } + + @Override + public @Nullable TimedOutput timeoutStep(long maxTime) { + return null; + } + + @Override + public boolean canFork() { + return true; + } + + @Override + public TimedSUL fork() { + return new NullSUL(); + } + } + + static class NullOracle implements TimedQueryOracle { + + @Override + public TimerQueryResult queryTimers(Word> prefix, long maxTotalWaitingTime) { + return new TimerQueryResult<>(false, Collections.emptyList()); + } + + @Override + public void processQueries(Collection, Word>>> queries) { + for (Query q : queries) { + q.answer(null); + } + + } + } + + static final class AnswerOnceQuery extends Query, Word>> { + + private final AtomicBoolean answered = new AtomicBoolean(false); + + @Override + public void answer(Word> output) { + boolean wasAnswered = answered.getAndSet(true); + if (wasAnswered) { + throw new IllegalStateException("Query was already answered"); + } + } + + @Override + public Word> getPrefix() { + return Word.epsilon(); + } + + @Override + public Word> getSuffix() { + return Word.epsilon(); + } + } + +} diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/AbstractStaticParallelTimedQueryOracleTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/AbstractStaticParallelTimedQueryOracleTest.java new file mode 100644 index 0000000000..63a72459a7 --- /dev/null +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/AbstractStaticParallelTimedQueryOracleTest.java @@ -0,0 +1,292 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import de.learnlib.oracle.ThreadPool.PoolPolicy; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.oracle.parallelism.Utils.Analysis; +import de.learnlib.query.DefaultQuery; +import de.learnlib.query.Query; +import de.learnlib.sul.TimedSUL; +import net.automatalib.symbol.time.InputSymbol; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import net.automatalib.word.WordBuilder; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.testng.Assert; +import org.testng.annotations.Test; + +public abstract class AbstractStaticParallelTimedQueryOracleTest { + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testZeroQueries(PoolPolicy policy) { + StaticParallelTimedQueryOracle oracle = getOracle(policy); + oracle.processQueries(Collections.emptyList()); + Analysis ana = analyze(Collections.emptyList()); + Utils.sanityCheck(ana); + Assert.assertEquals(ana.involvedOracles.size(), 0); + oracle.shutdownNow(); + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testLessThanMin(PoolPolicy policy) { + StaticParallelTimedQueryOracle oracle = getOracle(policy); + List, Word>>> queries = createQueries(Utils.MIN_BATCH_SIZE - 1); + oracle.processQueries(queries); + Analysis ana = analyze(queries); + Utils.sanityCheck(ana); + Assert.assertEquals(ana.involvedOracles.size(), 1); + oracle.shutdown(); + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testMin(PoolPolicy policy) { + StaticParallelTimedQueryOracle oracle = getOracle(policy); + List, Word>>> queries = createQueries(Utils.MIN_BATCH_SIZE); + oracle.processQueries(queries); + Analysis ana = analyze(queries); + Utils.sanityCheck(ana); + Assert.assertEquals(ana.involvedOracles.size(), 1); + oracle.shutdown(); + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testLessThanTwoBatches(PoolPolicy policy) { + StaticParallelTimedQueryOracle oracle = getOracle(policy); + List, Word>>> queries = + createQueries(2 * Utils.MIN_BATCH_SIZE - 1); + oracle.processQueries(queries); + Analysis ana = analyze(queries); + Utils.sanityCheck(ana); + Assert.assertEquals(ana.involvedOracles.size(), 1); + oracle.shutdown(); + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testLessThanSixBatches(PoolPolicy policy) { + StaticParallelTimedQueryOracle oracle = getOracle(policy); + List, Word>>> queries = + createQueries(5 * Utils.MIN_BATCH_SIZE + Utils.MIN_BATCH_SIZE / 2); + oracle.processQueries(queries); + Analysis ana = analyze(queries); + Utils.sanityCheck(ana); + Assert.assertEquals(ana.involvedOracles.size(), 5); + oracle.shutdown(); + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class) + public void testFullLoad(PoolPolicy policy) { + StaticParallelTimedQueryOracle oracle = getOracle(policy); + List, Word>>> queries = + createQueries(2 * Utils.NUM_ORACLES * Utils.MIN_BATCH_SIZE); + oracle.processQueries(queries); + Analysis ana = analyze(queries); + Utils.sanityCheck(ana); + Assert.assertEquals(ana.involvedOracles.size(), Utils.NUM_ORACLES); + oracle.shutdown(); + } + + protected abstract StaticParallelTimedQueryOracleBuilder getBuilder(); + + protected abstract TestOutput extractTestOutput(Word> output); + + protected TestMembershipOracle[] getOracles() { + TestMembershipOracle[] oracles = new TestMembershipOracle[Utils.NUM_ORACLES]; + for (int i = 0; i < Utils.NUM_ORACLES; i++) { + oracles[i] = new TestMembershipOracle(i); + } + + return oracles; + } + + private StaticParallelTimedQueryOracle getOracle(PoolPolicy poolPolicy) { + return getBuilder().withMinBatchSize(Utils.MIN_BATCH_SIZE) + .withNumInstances(Utils.NUM_ORACLES) + .withPoolPolicy(poolPolicy) + .create(); + } + + protected int getMinQueryLength() { + return 0; + } + + private List, Word>>> createQueries(int num) { + List, Word>>> result = new ArrayList<>(num); + for (int i = 0; i < num; i++) { + Word> prefix = + Utils.createWord(getMinQueryLength()).stream().map(TimedInput::input).collect(Word.collector()); + Word> suffix = + Utils.createWord(getMinQueryLength()).stream().map(TimedInput::input).collect(Word.collector()); + result.add(new DefaultQuery<>(prefix, suffix)); + } + return result; + } + + private Analysis analyze(Collection, Word>>> queries) { + List oracles = new ArrayList<>(); + Map> seqIds = new HashMap<>(); + Map incorrectAnswers = new HashMap<>(); + + for (DefaultQuery, Word>> qry : queries) { + TestOutput out = extractTestOutput(qry.getOutput()); + Assert.assertNotNull(out); + int oracleId = out.oracleId; + List seqIdList = seqIds.get(oracleId); + if (seqIdList == null) { + oracles.add(oracleId); + seqIdList = new ArrayList<>(); + seqIds.put(oracleId, seqIdList); + incorrectAnswers.put(oracleId, 0); + } + + int seqId = out.batchSeqId; + seqIdList.add(seqId); + + if (!qry.getPrefix().equals(out.prefix) || !qry.getSuffix().equals(out.suffix)) { + incorrectAnswers.put(oracleId, incorrectAnswers.get(oracleId) + 1); + } + } + + int minBatchSize = -1; + int maxBatchSize = -1; + for (List batch : seqIds.values()) { + if (minBatchSize == -1) { + maxBatchSize = batch.size(); + minBatchSize = maxBatchSize; + } else { + if (batch.size() < minBatchSize) { + minBatchSize = batch.size(); + } + if (batch.size() > maxBatchSize) { + maxBatchSize = batch.size(); + } + } + } + + return new Analysis(oracles, seqIds, incorrectAnswers, minBatchSize, maxBatchSize); + } + + static final class TestOutput { + + public final int oracleId; + public final int batchSeqId; + public final Word> prefix; + public final Word> suffix; + + TestOutput(int oracleId, int batchSeqId, Word> prefix, Word> suffix) { + this.oracleId = oracleId; + this.batchSeqId = batchSeqId; + this.prefix = prefix; + this.suffix = suffix; + } + } + + static final class TestSULOutput { + + final int oracleId; + final int batchSeqId; + final Word> word; + + TestSULOutput(int oracleId, int batchSeqId, Word> word) { + this.oracleId = oracleId; + this.batchSeqId = batchSeqId; + this.word = word; + } + } + + static final class TestMembershipOracle implements TimedQueryOracle { + + private final int oracleId; + + TestMembershipOracle(int oracleId) { + this.oracleId = oracleId; + } + + @Override + public void processQueries(Collection, Word>>> queries) { + int batchSeqId = 0; + for (Query, Word>> qry : queries) { + qry.answer(Word.fromLetter(new TimedOutput<>(new TestOutput(oracleId, + batchSeqId++, + qry.getPrefix(), + qry.getSuffix())))); + } + } + + @Override + public TimerQueryResult queryTimers(Word> prefix, long maxTotalWaitingTime) { + return new TimerQueryResult<>(false, Collections.emptyList()); + } + } + + static final class TestSUL implements TimedSUL { + + private final AtomicInteger atomicInteger; + private final int oracleId; + private int batchSeqId; + + private final WordBuilder> wb; + + TestSUL(AtomicInteger atomicInteger) { + this.atomicInteger = atomicInteger; + this.oracleId = atomicInteger.getAndIncrement(); + this.batchSeqId = -1; // so that our first query starts at 0 + + this.wb = new WordBuilder<>(); + } + + @Override + public void pre() { + batchSeqId++; + } + + @Override + public void post() { + this.wb.clear(); + } + + @Override + public TimedOutput step(InputSymbol in) { + this.wb.append(in); + return new TimedOutput<>(new TestSULOutput(oracleId, batchSeqId, wb.toWord())); + } + + @Override + public @Nullable TimedOutput timeoutStep(long maxTime) { + return null; + } + + @Override + public boolean canFork() { + return true; + } + + @Override + public TestSUL fork() { + return new TestSUL(this.atomicInteger); + } + } + +} diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedQueryOracleTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedQueryOracleTest.java new file mode 100644 index 0000000000..1a27adb06d --- /dev/null +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedQueryOracleTest.java @@ -0,0 +1,133 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CountDownLatch; + +import de.learnlib.oracle.ParallelTimedQueryOracle; +import de.learnlib.oracle.ThreadPool.PoolPolicy; +import de.learnlib.oracle.TimedQueryOracle.TimerQueryResult; +import de.learnlib.query.Query; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class DynamicParallelTimedQueryOracleTest extends AbstractDynamicParallelTimedQueryOracleTest { + + @Override + protected DynamicParallelTimedOracleQueryBuilder getBuilder() { + return ParallelOracleBuilders.newDynamicParallelTimedQueryOracle(Arrays.asList(new NullOracle(), + new NullOracle(), + new NullOracle())); + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class, timeOut = 2000) + public void testThreadCreation(PoolPolicy poolPolicy) { + + final List> queries = createQueries(10); + final int expectedThreads = queries.size(); + + final CountDownLatch latch = new CountDownLatch(expectedThreads); + final NullOracle[] oracles = new NullOracle[expectedThreads]; + + for (int i = 0; i < expectedThreads; i++) { + oracles[i] = new NullOracle() { + + @Override + public void processQueries(Collection, Word>>> queries) { + try { + latch.countDown(); + latch.await(); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + super.processQueries(queries); + } + }; + } + + final ParallelTimedQueryOracle oracle = ParallelOracleBuilders.newDynamicParallelTimedQueryOracle( + oracles[0], + Arrays.copyOfRange(oracles, 1, oracles.length)) + .withBatchSize(1) + .withPoolSize(oracles.length) + .withPoolPolicy(poolPolicy) + .create(); + + try { + // this method only returns, if 'expectedThreads' threads are spawned, which all decrease the shared latch + oracle.processQueries(queries); + } finally { + oracle.shutdown(); + } + } + + @Test(dataProvider = "policies", dataProviderClass = Utils.class, timeOut = 2000) + public void testThreadScheduling(PoolPolicy poolPolicy) { + + final List> queries = createQueries(10); + final CountDownLatch latch = new CountDownLatch(queries.size() - 1); + + final NullOracle awaitingOracle = new NullOracle() { + + @Override + public void processQueries(Collection, Word>>> queries) { + try { + latch.await(); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + super.processQueries(queries); + } + }; + + final NullOracle countDownOracle = new NullOracle() { + + @Override + public void processQueries(Collection, Word>>> queries) { + latch.countDown(); + super.processQueries(queries); + } + }; + + final ParallelTimedQueryOracle oracle = + ParallelOracleBuilders.newDynamicParallelTimedQueryOracle(awaitingOracle, countDownOracle) + .withPoolSize(2) + .withPoolPolicy(poolPolicy) + .create(); + + try { + // this method only returns, if the countDownOracle was scheduled 9 times to unblock the awaitingOracle + oracle.processQueries(queries); + } finally { + oracle.shutdown(); + } + } + + @Test + public void testSingleMethods() { + final ParallelTimedQueryOracle oracle = getBuilder().create(); + + TimerQueryResult timer = oracle.queryTimers(Word.epsilon(), 0); + Assert.assertFalse(timer.aborted()); + Assert.assertTrue(timer.timers().isEmpty()); + } +} diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedSULTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedSULTest.java new file mode 100644 index 0000000000..036b8798be --- /dev/null +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedSULTest.java @@ -0,0 +1,27 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import de.learnlib.time.MMLTModelParams; + +public class DynamicParallelTimedSULTest extends AbstractDynamicParallelTimedQueryOracleTest { + + @Override + protected DynamicParallelTimedOracleQueryBuilder getBuilder() { + return ParallelOracleBuilders.newDynamicParallelTimedQueryOracle(new NullSUL(), + new MMLTModelParams<>(null, null, 0, 0)); + } +} diff --git a/api/src/main/java/de/learnlib/statistic/StatisticSUL.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedSupplierTest.java similarity index 66% rename from api/src/main/java/de/learnlib/statistic/StatisticSUL.java rename to oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedSupplierTest.java index e44a87342d..882b8b78c1 100644 --- a/api/src/main/java/de/learnlib/statistic/StatisticSUL.java +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/DynamicParallelTimedSupplierTest.java @@ -13,8 +13,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package de.learnlib.statistic; +package de.learnlib.oracle.parallelism; -import de.learnlib.sul.SUL; +public class DynamicParallelTimedSupplierTest extends AbstractDynamicParallelTimedQueryOracleTest { -public interface StatisticSUL extends SUL, StatisticCollector {} + @Override + protected DynamicParallelTimedOracleQueryBuilder getBuilder() { + return ParallelOracleBuilders.newDynamicParallelTimedQueryOracle(NullOracle::new); + } +} diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelObservableSULTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelObservableSULTest.java index 34972d28e3..7734cd52fe 100644 --- a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelObservableSULTest.java +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelObservableSULTest.java @@ -19,6 +19,7 @@ import de.learnlib.oracle.parallelism.Utils.TestSULOutput; import net.automatalib.word.Word; +import org.testng.Assert; public class StaticParallelObservableSULTest extends AbstractStaticParallelOmegaOracleTest> { @@ -30,7 +31,7 @@ protected StaticParallelOmegaOracleBuilder> getB @Override protected TestOutput extractTestOutput(Word output) { - assert !output.isEmpty(); + Assert.assertFalse(output.isEmpty()); final TestSULOutput lastSym = output.lastSymbol(); final int oracleId = lastSym.oracleId; diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracleTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracleTest.java new file mode 100644 index 0000000000..55baf1a1ab --- /dev/null +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedQueryOracleTest.java @@ -0,0 +1,53 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.Arrays; + +import de.learnlib.oracle.ParallelTimedQueryOracle; +import de.learnlib.oracle.TimedQueryOracle.TimerQueryResult; +import de.learnlib.oracle.parallelism.AbstractStaticParallelTimedQueryOracleTest.TestOutput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class StaticParallelTimedQueryOracleTest extends AbstractStaticParallelTimedQueryOracleTest { + + @Override + protected StaticParallelTimedQueryOracleBuilder getBuilder() { + TestMembershipOracle[] oracles = getOracles(); + return ParallelOracleBuilders.newStaticParallelTimedQueryOracle(oracles[0], + Arrays.copyOfRange(oracles, 1, oracles.length)); + } + + @Override + protected TestOutput extractTestOutput(Word> output) { + Assert.assertFalse(output.isEmpty()); + + final TestOutput lastSym = output.lastSymbol().symbol(); + return new TestOutput(lastSym.oracleId, lastSym.batchSeqId, lastSym.prefix, lastSym.suffix); + } + + @Test + public void testSingleMethods() { + final ParallelTimedQueryOracle oracle = getBuilder().create(); + + TimerQueryResult timer = oracle.queryTimers(Word.epsilon(), 0); + Assert.assertFalse(timer.aborted()); + Assert.assertTrue(timer.timers().isEmpty()); + } +} diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedSULTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedSULTest.java new file mode 100644 index 0000000000..915cff221d --- /dev/null +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedSULTest.java @@ -0,0 +1,55 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import java.util.concurrent.atomic.AtomicInteger; + +import de.learnlib.oracle.parallelism.AbstractStaticParallelTimedQueryOracleTest.TestSULOutput; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.testng.Assert; + +public class StaticParallelTimedSULTest extends AbstractStaticParallelTimedQueryOracleTest { + + @Override + protected StaticParallelTimedQueryOracleBuilder getBuilder() { + // since we fork our initial SUL, start at -1 + return ParallelOracleBuilders.newStaticParallelTimedQueryOracle(new TestSUL(new AtomicInteger(-1)), + new MMLTModelParams<>(null, null, 0, 0)); + } + + @Override + protected TestOutput extractTestOutput(Word> output) { + Assert.assertFalse(output.isEmpty()); + + final TestSULOutput lastSym = output.lastSymbol().symbol(); + final int oracleId = lastSym.oracleId; + final int batchSeqId = lastSym.batchSeqId; + + final Word> word = lastSym.word; + final Word> prefix = word.prefix(word.size() - output.size()); + final Word> suffix = word.subWord(word.size() - output.size()); + + return new TestOutput(oracleId, batchSeqId, prefix, suffix); + } + + @Override + protected int getMinQueryLength() { + return 1; + } +} diff --git a/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedSupplierTest.java b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedSupplierTest.java new file mode 100644 index 0000000000..77fa871b46 --- /dev/null +++ b/oracles/parallelism/src/test/java/de/learnlib/oracle/parallelism/StaticParallelTimedSupplierTest.java @@ -0,0 +1,39 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.oracle.parallelism; + +import de.learnlib.oracle.parallelism.AbstractDynamicBatchProcessorBuilder.StaticOracleProvider; +import de.learnlib.oracle.parallelism.AbstractStaticParallelTimedQueryOracleTest.TestOutput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.word.Word; +import org.testng.Assert; + +public class StaticParallelTimedSupplierTest extends AbstractStaticParallelTimedQueryOracleTest { + + @Override + protected StaticParallelTimedQueryOracleBuilder getBuilder() { + TestMembershipOracle[] oracles = getOracles(); + return ParallelOracleBuilders.newStaticParallelTimedQueryOracle(new StaticOracleProvider<>(oracles)); + } + + @Override + protected TestOutput extractTestOutput(Word> output) { + Assert.assertFalse(output.isEmpty()); + + final TestOutput lastSym = output.lastSymbol().symbol(); + return new TestOutput(lastSym.oracleId, lastSym.batchSeqId, lastSym.prefix, lastSym.suffix); + } +} diff --git a/pom.xml b/pom.xml index b249b5ead5..9835d620e9 100644 --- a/pom.xml +++ b/pom.xml @@ -103,6 +103,15 @@ limitations under the License. Developer + + Paul Kogel + p.kogel@tu-berlin.de + TU Berlin, Software and Embedded Systems Engineering + https://www.tu.berlin/sese + + Developer + + Jeroen Meijer j.j.g.meijer@utwente.nl @@ -528,6 +537,11 @@ limitations under the License. learnlib-parallelism ${project.version} + + de.learnlib + learnlib-symbol-filters + ${project.version} + de.learnlib learnlib-property-oracles diff --git a/test-support/learner-it-support/pom.xml b/test-support/learner-it-support/pom.xml index 703a76c22b..bda6c74f30 100644 --- a/test-support/learner-it-support/pom.xml +++ b/test-support/learner-it-support/pom.xml @@ -71,10 +71,6 @@ limitations under the License. net.automatalib automata-api - - net.automatalib - automata-commons-util - net.automatalib automata-core diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/AbstractLearnerVariantITCase.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/AbstractLearnerVariantITCase.java index 1b96e719f2..be66a2e66c 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/AbstractLearnerVariantITCase.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/AbstractLearnerVariantITCase.java @@ -17,7 +17,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Random; import de.learnlib.algorithm.LearningAlgorithm; import de.learnlib.logging.Category; @@ -26,17 +25,13 @@ import de.learnlib.testsupport.example.LearningExample; import net.automatalib.alphabet.Alphabet; import net.automatalib.automaton.concept.FiniteRepresentation; -import net.automatalib.automaton.concept.Output; -import net.automatalib.common.util.random.RandomUtil; -import net.automatalib.word.Word; -import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.ITest; import org.testng.annotations.Test; -abstract class AbstractLearnerVariantITCase> implements ITest { +abstract class AbstractLearnerVariantITCase implements ITest { private static final Logger LOGGER = LoggerFactory.getLogger(AbstractLearnerVariantITCase.class); @@ -73,6 +68,7 @@ public void testLearning() { int roundCounter = 0; DefaultQuery ceQuery; + List> ceQueries = new ArrayList<>(); while ((ceQuery = eqOracle.findCounterExample(learner.getHypothesisModel(), alphabet)) != null) { roundCounter++; @@ -82,16 +78,21 @@ public void testLearning() { boolean refined = learner.refineHypothesis(ceQuery); Assert.assertTrue(refined, "Real counterexample " + ceQuery.getInput() + " did not refine hypothesis"); + ceQueries.add(ceQuery); } M hypothesis = learner.getHypothesisModel(); - Assert.assertEquals(hypothesis.size(), reference.size()); - Assert.assertNull(checkEquivalence(hypothesis), "Final hypothesis does not match reference automaton"); + Assert.assertTrue(testEquivalence(hypothesis), "Final hypothesis does not match reference automaton"); - final List trace = RandomUtil.sample(new Random(42), new ArrayList<>(alphabet), 5); - final D output = reference.computeOutput(trace); + if (hasCanonicalModel()) { + Assert.assertEquals(hypothesis.size(), reference.size(), "Final hypothesis is not canonical"); + } - Assert.assertFalse(learner.refineHypothesis(new DefaultQuery<>(Word.fromList(trace), output))); + if (!ceQueries.isEmpty()) { + DefaultQuery oldCe = ceQueries.get(0); + Assert.assertFalse(learner.refineHypothesis(oldCe), + "Learner should not report a hypothesis update on outdated counterexample"); + } long duration = (System.nanoTime() - start) / NANOS_PER_MILLISECOND; LOGGER.info(Category.EVENT, @@ -105,6 +106,10 @@ public String getTestName() { return variant.getLearnerName() + "[" + variant.getName() + "]/" + example.getClass().getSimpleName(); } - protected abstract @Nullable Word checkEquivalence(M hypothesis); + protected boolean hasCanonicalModel() { + return true; + } + + protected abstract boolean testEquivalence(M hypothesis); } diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/AbstractMMLTLearnerIT.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/AbstractMMLTLearnerIT.java new file mode 100644 index 0000000000..bb73477efc --- /dev/null +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/AbstractMMLTLearnerIT.java @@ -0,0 +1,89 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.testsupport.it.learner; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import de.learnlib.driver.simulator.MMLTSimulatorSUL; +import de.learnlib.oracle.TimedQueryOracle; +import de.learnlib.oracle.equivalence.mmlt.SimulatorEQOracle; +import de.learnlib.oracle.membership.TimedSULOracle; +import de.learnlib.testsupport.example.LearningExample.MMLTLearningExample; +import de.learnlib.testsupport.example.LearningExamples; +import de.learnlib.testsupport.it.learner.LearnerVariantList.MMLTLearnerVariantList; +import de.learnlib.testsupport.it.learner.LearnerVariantListImpl.MMLTLearnerVariantListImpl; +import net.automatalib.alphabet.Alphabet; +import org.testng.annotations.Factory; + +public abstract class AbstractMMLTLearnerIT { + + @Factory + public Object[] createExampleITCases() { + final List> examples = LearningExamples.createMMLTExamples(); + final List> extras = getAdditionalLearningExamples(); + final List> result = new ArrayList<>(); + + for (MMLTLearningExample example : examples) { + result.addAll(createAllVariantsITCase(example)); + } + for (MMLTLearningExample example : extras) { + result.addAll(createAllVariantsITCase(example)); + } + + return result.toArray(); + } + + private List> createAllVariantsITCase(MMLTLearningExample example) { + + final Alphabet alphabet = example.getUntimedAlphabet(); + final TimedQueryOracle mqOracle = + new TimedSULOracle<>(new MMLTSimulatorSUL<>(example.getReferenceAutomaton()), example.getParams()); + final MMLTLearnerVariantListImpl variants = new MMLTLearnerVariantListImpl<>(); + addLearnerVariants(alphabet, mqOracle, example, variants); + + return LearnerITUtil.createExampleITCases(example, + variants, + new SimulatorEQOracle<>(example.getReferenceAutomaton())); + } + + protected List> getAdditionalLearningExamples() { + return Collections.emptyList(); + } + + /** + * Adds, for a given setup, all the variants of the MMLT learner to be tested to the specified + * {@link LearnerVariantList variant list}. + * + * @param + * input symbol type (of non-delaying inputs) + * @param + * output symbol type + * @param alphabet + * the input alphabet + * @param mqOracle + * the membership oracle + * @param example + * the learning example to potentially extract additional information + * @param variants + * list to add the learner variants to + */ + protected abstract void addLearnerVariants(Alphabet alphabet, + TimedQueryOracle mqOracle, + MMLTLearningExample example, + MMLTLearnerVariantList variants); +} diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerITUtil.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerITUtil.java index d81d0b1774..da9fbef25d 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerITUtil.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerITUtil.java @@ -23,12 +23,14 @@ import de.learnlib.oracle.EquivalenceOracle; import de.learnlib.query.DefaultQuery; import de.learnlib.testsupport.example.LearningExample; +import de.learnlib.testsupport.example.LearningExample.MMLTLearningExample; import de.learnlib.testsupport.example.LearningExample.OneSEVPALearningExample; import de.learnlib.testsupport.example.LearningExample.SBALearningExample; import de.learnlib.testsupport.example.LearningExample.SPALearningExample; import de.learnlib.testsupport.example.LearningExample.SPMMLearningExample; import de.learnlib.testsupport.example.LearningExample.UniversalDeterministicLearningExample; import de.learnlib.testsupport.example.PassiveLearningExample; +import de.learnlib.testsupport.it.learner.LearnerVariantListImpl.MMLTLearnerVariantListImpl; import de.learnlib.testsupport.it.learner.LearnerVariantListImpl.OneSEVPALearnerVariantListImpl; import de.learnlib.testsupport.it.learner.LearnerVariantListImpl.SBALearnerVariantListImpl; import de.learnlib.testsupport.it.learner.LearnerVariantListImpl.SPALearnerVariantListImpl; @@ -39,10 +41,13 @@ import net.automatalib.automaton.concept.FiniteRepresentation; import net.automatalib.automaton.concept.Output; import net.automatalib.automaton.concept.SuffixOutput; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.procedural.SBA; import net.automatalib.automaton.procedural.SPA; import net.automatalib.automaton.procedural.SPMM; import net.automatalib.automaton.vpa.OneSEVPA; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; import net.automatalib.word.Word; import net.automatalib.word.WordBuilder; @@ -91,6 +96,33 @@ private LearnerITUtil() { UniversalDeterministicLearnerITCase::new); } + /** + * Creates a list of per-example test cases for all learner variants. + * + * @param example + * the example system + * @param variants + * the list containing the various learner variants + * @param eqOracle + * the equivalence oracle to use by the learning process + * @param + * input symbol type + * @param + * output symbol type + * + * @return the list of test cases, one for each example + */ + public static List> createExampleITCases(MMLTLearningExample example, + MMLTLearnerVariantListImpl variants, + EquivalenceOracle, TimedInput, Word>> eqOracle) { + // explicit generics are required for correct type-inference + return LearnerITUtil., Word>, MMLT, MMLTLearningExample, MMLTLearnerITCase>createExampleITCasesInternal( + example, + variants, + eqOracle, + MMLTLearnerITCase::new); + } + /** * Creates a list of per-example test cases for all learner variants. * @@ -193,7 +225,7 @@ public static List> createExampleITCases(OneSEVPALe OneSEVPALearnerITCase::new); } - private static , L extends LearningExample, C extends AbstractLearnerVariantITCase> List createExampleITCasesInternal( + private static , C extends AbstractLearnerVariantITCase> List createExampleITCasesInternal( L example, LearnerVariantListImpl variants, EquivalenceOracle eqOracle, @@ -266,7 +298,7 @@ public static > List, L extends LearningExample, C extends AbstractLearnerVariantITCase> { + private interface ITCaseBuilder, C extends AbstractLearnerVariantITCase> { C build(LearnerVariant variant, L example, EquivalenceOracle eqOracle); } diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantList.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantList.java index 02537613fb..b5404319aa 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantList.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantList.java @@ -17,12 +17,15 @@ import de.learnlib.algorithm.LearningAlgorithm; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.procedural.SBA; import net.automatalib.automaton.procedural.SPA; import net.automatalib.automaton.procedural.SPMM; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; import net.automatalib.automaton.vpa.OneSEVPA; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; import net.automatalib.word.Word; /** @@ -79,6 +82,8 @@ interface MooreLearnerVariantList extends LearnerVariantList extends LearnerVariantList, I, O> {} + interface MMLTLearnerVariantList extends LearnerVariantList, TimedInput, Word>> {} + interface SPALearnerVariantList extends LearnerVariantList, I, Boolean> {} interface SBALearnerVariantList extends LearnerVariantList, I, Boolean> {} diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantListImpl.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantListImpl.java index ccdcae879c..89149e98e6 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantListImpl.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/LearnerVariantListImpl.java @@ -22,12 +22,15 @@ import de.learnlib.util.mealy.MealyUtil; import de.learnlib.util.moore.MooreUtil; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.procedural.SBA; import net.automatalib.automaton.procedural.SPA; import net.automatalib.automaton.procedural.SPMM; import net.automatalib.automaton.transducer.MealyMachine; import net.automatalib.automaton.transducer.MooreMachine; import net.automatalib.automaton.vpa.OneSEVPA; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; import net.automatalib.word.Word; public class LearnerVariantListImpl implements LearnerVariantList { @@ -60,6 +63,9 @@ public static class MooreLearnerVariantListImpl extends LearnerVariantListImpl, I, Word> implements MooreLearnerVariantList {} + public static class MMLTLearnerVariantListImpl extends LearnerVariantListImpl, TimedInput, Word>> + implements MMLTLearnerVariantList {} + public static class OneSEVPALearnerVariantListImpl extends LearnerVariantListImpl, I, Boolean> implements OneSEVPALearnerVariantList {} diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/MMLTLearnerITCase.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/MMLTLearnerITCase.java new file mode 100644 index 0000000000..9f27ba8391 --- /dev/null +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/MMLTLearnerITCase.java @@ -0,0 +1,49 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.testsupport.it.learner; + +import de.learnlib.oracle.EquivalenceOracle; +import de.learnlib.testsupport.example.LearningExample.MMLTLearningExample; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimedOutput; +import net.automatalib.util.automaton.mmlt.MMLTs; +import net.automatalib.word.Word; + +public class MMLTLearnerITCase + extends AbstractLearnerVariantITCase, Word>, MMLT> { + + private final MMLTLearningExample example; + + MMLTLearnerITCase(LearnerVariant, TimedInput, Word>> variant, + MMLTLearningExample example, + EquivalenceOracle, TimedInput, Word>> eqOracle) { + super(variant, example, eqOracle); + this.example = example; + } + + @Override + protected boolean hasCanonicalModel() { + return false; + } + + @Override + protected boolean testEquivalence(MMLT hypothesis) { + return MMLTs.testEquivalence(this.example.getReferenceAutomaton(), + hypothesis, + this.example.getReferenceAutomaton().getSemantics().getInputAlphabet()); + } +} diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/OneSEVPALearnerITCase.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/OneSEVPALearnerITCase.java index 8f6b9e77f1..8d7fe1bbe0 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/OneSEVPALearnerITCase.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/OneSEVPALearnerITCase.java @@ -19,8 +19,6 @@ import de.learnlib.testsupport.example.LearningExample.OneSEVPALearningExample; import net.automatalib.automaton.vpa.OneSEVPA; import net.automatalib.util.automaton.vpa.OneSEVPAs; -import net.automatalib.word.Word; -import org.checkerframework.checker.nullness.qual.Nullable; public class OneSEVPALearnerITCase extends AbstractLearnerVariantITCase> { @@ -34,9 +32,7 @@ public class OneSEVPALearnerITCase extends AbstractLearnerVariantITCase checkEquivalence(OneSEVPA hypothesis) { - return OneSEVPAs.findSeparatingWord(this.example.getReferenceAutomaton(), - hypothesis, - this.example.getAlphabet()); + protected boolean testEquivalence(OneSEVPA hypothesis) { + return OneSEVPAs.testEquivalence(this.example.getReferenceAutomaton(), hypothesis, this.example.getAlphabet()); } } diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SBALearnerITCase.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SBALearnerITCase.java index 039ee9b789..68812b1b28 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SBALearnerITCase.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SBALearnerITCase.java @@ -19,8 +19,6 @@ import de.learnlib.testsupport.example.LearningExample.SBALearningExample; import net.automatalib.automaton.procedural.SBA; import net.automatalib.util.automaton.procedural.SBAs; -import net.automatalib.word.Word; -import org.checkerframework.checker.nullness.qual.Nullable; public class SBALearnerITCase extends AbstractLearnerVariantITCase> { @@ -34,9 +32,7 @@ public class SBALearnerITCase extends AbstractLearnerVariantITCase checkEquivalence(SBA hypothesis) { - return SBAs.findSeparatingWord(this.example.getReferenceAutomaton(), - hypothesis, - this.example.getAlphabet()); + protected boolean testEquivalence(SBA hypothesis) { + return SBAs.testEquivalence(this.example.getReferenceAutomaton(), hypothesis, this.example.getAlphabet()); } } diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPALearnerITCase.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPALearnerITCase.java index 94ef835bcf..315498a7a9 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPALearnerITCase.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPALearnerITCase.java @@ -19,8 +19,6 @@ import de.learnlib.testsupport.example.LearningExample.SPALearningExample; import net.automatalib.automaton.procedural.SPA; import net.automatalib.util.automaton.procedural.SPAs; -import net.automatalib.word.Word; -import org.checkerframework.checker.nullness.qual.Nullable; public class SPALearnerITCase extends AbstractLearnerVariantITCase> { @@ -34,9 +32,7 @@ public class SPALearnerITCase extends AbstractLearnerVariantITCase checkEquivalence(SPA hypothesis) { - return SPAs.findSeparatingWord(this.example.getReferenceAutomaton(), - hypothesis, - this.example.getAlphabet()); + protected boolean testEquivalence(SPA hypothesis) { + return SPAs.testEquivalence(this.example.getReferenceAutomaton(), hypothesis, this.example.getAlphabet()); } } diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPMMLearnerITCase.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPMMLearnerITCase.java index a5c0d2151d..79cc0c26fa 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPMMLearnerITCase.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/SPMMLearnerITCase.java @@ -20,7 +20,6 @@ import net.automatalib.automaton.procedural.SPMM; import net.automatalib.util.automaton.procedural.SPMMs; import net.automatalib.word.Word; -import org.checkerframework.checker.nullness.qual.Nullable; public class SPMMLearnerITCase extends AbstractLearnerVariantITCase, SPMM> { @@ -34,9 +33,7 @@ public class SPMMLearnerITCase extends AbstractLearnerVariantITCase checkEquivalence(SPMM hypothesis) { - return SPMMs.findSeparatingWord(this.example.getReferenceAutomaton(), - hypothesis, - this.example.getAlphabet()); + protected boolean testEquivalence(SPMM hypothesis) { + return SPMMs.testEquivalence(this.example.getReferenceAutomaton(), hypothesis, this.example.getAlphabet()); } } diff --git a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/UniversalDeterministicLearnerITCase.java b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/UniversalDeterministicLearnerITCase.java index defc7fdab8..c541bcc504 100644 --- a/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/UniversalDeterministicLearnerITCase.java +++ b/test-support/learner-it-support/src/main/java/de/learnlib/testsupport/it/learner/UniversalDeterministicLearnerITCase.java @@ -20,8 +20,6 @@ import net.automatalib.automaton.UniversalDeterministicAutomaton; import net.automatalib.automaton.concept.Output; import net.automatalib.util.automaton.Automata; -import net.automatalib.word.Word; -import org.checkerframework.checker.nullness.qual.Nullable; public class UniversalDeterministicLearnerITCase & Output> extends AbstractLearnerVariantITCase { @@ -36,9 +34,7 @@ public class UniversalDeterministicLearnerITCase checkEquivalence(M hypothesis) { - return Automata.findSeparatingWord(this.example.getReferenceAutomaton(), - hypothesis, - this.example.getAlphabet()); + protected boolean testEquivalence(M hypothesis) { + return Automata.testEquivalence(this.example.getReferenceAutomaton(), hypothesis, this.example.getAlphabet()); } } diff --git a/test-support/learner-it-support/src/main/java/module-info.java b/test-support/learner-it-support/src/main/java/module-info.java index 92bc5f47a5..f98b1411ff 100644 --- a/test-support/learner-it-support/src/main/java/module-info.java +++ b/test-support/learner-it-support/src/main/java/module-info.java @@ -37,7 +37,6 @@ requires de.learnlib.oracle.equivalence; requires de.learnlib.testsupport.example; requires net.automatalib.api; - requires net.automatalib.common.util; requires net.automatalib.util; requires org.slf4j; requires org.testng; diff --git a/test-support/learning-examples/pom.xml b/test-support/learning-examples/pom.xml index 87a2d91f6e..ac5160f302 100644 --- a/test-support/learning-examples/pom.xml +++ b/test-support/learning-examples/pom.xml @@ -51,6 +51,10 @@ limitations under the License. net.automatalib automata-core + + net.automatalib + automata-serialization-dot + net.automatalib automata-serialization-learnlibv2 diff --git a/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/DefaultLearningExample.java b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/DefaultLearningExample.java index 1d35b97235..d52da05396 100644 --- a/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/DefaultLearningExample.java +++ b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/DefaultLearningExample.java @@ -15,6 +15,7 @@ */ package de.learnlib.testsupport.example; +import de.learnlib.time.MMLTModelParams; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.ProceduralInputAlphabet; import net.automatalib.alphabet.VPAlphabet; @@ -22,6 +23,7 @@ import net.automatalib.automaton.concept.InputAlphabetHolder; import net.automatalib.automaton.concept.SuffixOutput; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.procedural.SBA; import net.automatalib.automaton.procedural.SPA; import net.automatalib.automaton.procedural.SPMM; @@ -112,6 +114,27 @@ public DefaultSSTLearningExample(Alphabet alphabet, SubsequentialTransducer implements MMLTLearningExample { + + private final MMLT mmlt; + private final MMLTModelParams params; + + public DefaultMMLTLearningExample(MMLT mmlt, MMLTModelParams params) { + this.mmlt = mmlt; + this.params = params; + } + + @Override + public MMLTModelParams getParams() { + return this.params; + } + + @Override + public MMLT getReferenceAutomaton() { + return this.mmlt; + } + } + public static class DefaultOneSEVPALearningExample implements OneSEVPALearningExample { private final VPAlphabet alphabet; diff --git a/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExample.java b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExample.java index 0b3c6b9868..c12d501a0d 100644 --- a/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExample.java +++ b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExample.java @@ -15,11 +15,13 @@ */ package de.learnlib.testsupport.example; +import de.learnlib.time.MMLTModelParams; import net.automatalib.alphabet.Alphabet; import net.automatalib.alphabet.ProceduralInputAlphabet; import net.automatalib.alphabet.VPAlphabet; import net.automatalib.automaton.UniversalAutomaton; import net.automatalib.automaton.fsa.DFA; +import net.automatalib.automaton.mmlt.MMLT; import net.automatalib.automaton.procedural.SBA; import net.automatalib.automaton.procedural.SPA; import net.automatalib.automaton.procedural.SPMM; @@ -28,6 +30,9 @@ import net.automatalib.automaton.transducer.StateLocalInputMealyMachine; import net.automatalib.automaton.transducer.SubsequentialTransducer; import net.automatalib.automaton.vpa.OneSEVPA; +import net.automatalib.symbol.time.TimeStepSequence; +import net.automatalib.symbol.time.TimedInput; +import net.automatalib.symbol.time.TimeoutSymbol; public interface LearningExample { @@ -62,6 +67,32 @@ interface StateLocalInputMealyLearningExample } + interface MMLTLearningExample extends LearningExample, MMLT> { + + MMLTModelParams getParams(); + + /** + * Returns the fully timed alphabet, including the {@link TimeoutSymbol} and {@link TimeStepSequence} symbol. + * + * @return the full (semantic) alphabet + * + * @see #getUntimedAlphabet() + */ + @Override + default Alphabet> getAlphabet() { + return getReferenceAutomaton().getSemantics().getInputAlphabet(); + } + + /** + * Returns the direct inputs of the {@link MMLT#getInputAlphabet() MMLT}. + * + * @return the direct input alphabet + */ + default Alphabet getUntimedAlphabet() { + return getReferenceAutomaton().getInputAlphabet(); + } + } + interface SPALearningExample extends LearningExample> { @Override diff --git a/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExamples.java b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExamples.java index cd2e5f8b0b..4c8fc99200 100644 --- a/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExamples.java +++ b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/LearningExamples.java @@ -22,6 +22,7 @@ import java.util.Random; import de.learnlib.testsupport.example.LearningExample.DFALearningExample; +import de.learnlib.testsupport.example.LearningExample.MMLTLearningExample; import de.learnlib.testsupport.example.LearningExample.MealyLearningExample; import de.learnlib.testsupport.example.LearningExample.MooreLearningExample; import de.learnlib.testsupport.example.LearningExample.OneSEVPALearningExample; @@ -41,6 +42,7 @@ import de.learnlib.testsupport.example.mealy.ExampleShahbazGroz; import de.learnlib.testsupport.example.mealy.ExampleStack; import de.learnlib.testsupport.example.mealy.ExampleTinyMealy; +import de.learnlib.testsupport.example.mmlt.MMLTExamples; import de.learnlib.testsupport.example.moore.ExampleRandomMoore; import de.learnlib.testsupport.example.sba.ExampleRandomSBA; import de.learnlib.testsupport.example.spa.ExamplePalindrome; @@ -129,6 +131,15 @@ public static List> createDFAExamples() { RANDOM_SST_PROPS)); } + public static List> createMMLTExamples() { + return Arrays.asList(MMLTExamples.hvac(), + MMLTExamples.sctp(), + MMLTExamples.sensorCollector(), + MMLTExamples.wm(), + MMLTExamples.oven(), + MMLTExamples.wsn()); + } + public static List> createSPAExamples() { return Arrays.asList(ExamplePalindrome.createExample(), ExampleRandomSPA.createExample(new Random(RANDOM_SEED), diff --git a/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/mmlt/MMLTExamples.java b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/mmlt/MMLTExamples.java new file mode 100644 index 0000000000..372c92a866 --- /dev/null +++ b/test-support/learning-examples/src/main/java/de/learnlib/testsupport/example/mmlt/MMLTExamples.java @@ -0,0 +1,178 @@ +/* Copyright (C) 2013-2025 TU Dortmund University + * This file is part of LearnLib . + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package de.learnlib.testsupport.example.mmlt; + +import java.io.IOException; +import java.io.InputStream; + +import de.learnlib.testsupport.example.LearningExample.MMLTLearningExample; +import de.learnlib.time.MMLTModelParams; +import net.automatalib.automaton.mmlt.MMLT; +import net.automatalib.automaton.mmlt.impl.CompactMMLT; +import net.automatalib.automaton.mmlt.impl.StringSymbolCombiner; +import net.automatalib.exception.FormatException; +import net.automatalib.serialization.dot.DOTInputModelData; +import net.automatalib.serialization.dot.DOTInputModelDeserializer; +import net.automatalib.serialization.dot.DOTParsers; +import net.automatalib.util.automaton.mmlt.MMLTs; + +/** + * A collection of {@link MMLT}-based learning examples. + */ +public final class MMLTExamples { + + private MMLTExamples() { + // prevent instantiation + } + + /** + * Returns an MMLT example of an HVAC system. + *

+ * The system has been adapted from: Taylor and Taylor: Patterns in the Machine + * + * @return a learning example for the specified machine + */ + public static MMLTLearningExample hvac() { + return new Example("HVAC"); + } + + /** + * Returns an MMLT example of an endpoint in the stream control and transmission protocol. + *

+ * The model has been adapted from: Stewart et al.: Stream Control Transmission Protocol (RFC 9260, Figure 3) + * + * @return a learning example for the specified machine + */ + public static MMLTLearningExample sctp() { + return new Example("SCTP"); + } + + /** + * Returns an MMLT example of a sensor collector. + *

+ * The sensor measures particulate matter and ambient noise. The measurement program automatically ends after some + * time. The program may be restarted at any time. Alternatively, a self-check program can be entered. This also + * ends after some time and may be aborted. At the end of either program, the collected data may be retrieved. + * + * @return a learning example for the specified machine + */ + public static MMLTLearningExample sensorCollector() { + return new Example("sensor_collector"); + } + + /** + * Returns an MMLT example of a washing machine. + *

+ * The machine is initially off. After powering it on and closing the door, the user can start either the short or + * the normal program. An open door prevents starting and triggers a warning. Not choosing a program within 10 + * seconds turns the machine off. + *

+ * In normal model, the machine fills the drum, heats the water, and starts the main wash. During this wash, it + * regularly adjusts the drum speed and maintains temperature. After 2 hours, the water is drained and the drum is + * spun at full speed for some time. Afterwards the remaining water is drained. The short program makes less + * adjustments, so that a wash ends after 1 hour. + *

+ * Both programs are interrupted when a leak is detected. Normal mode may also be interrupted by "stop". This drains + * the drum immediately. Once done, the door is unlocked, a message is shown, and the machine beeps repeatedly until + * the user presses any button or opens the door. + * + * @return a learning example for the specified machine + */ + public static MMLTLearningExample wm() { + return new Example("WM"); + } + + /** + * Returns an MMLT example of an oven with a time-controlled baking program. + *

+ * After powering the oven on, the oven remains idle until the program is started. During the program, the oven + * regularly measures and adjusts the temperature. At the end of the program, an alarm sounds. Then, the user may + * extend the program. If not extended, the program ends either when the user opens the door, presses a button, or a + * timeout occurs. + * + * @return a learning example for the specified machine + */ + public static MMLTLearningExample oven() { + return new Example("Oven"); + } + + /** + * Returns an MMLT example of a wireless sensor node. + *

+ * The node regularly collects and transmits data. If the battery is low, no data is transmitted. Then, a user may + * collect the data manually. The node can be shut down at any time. If the battery is empty, it is shut down + * automatically. + * + * @return a learning example for the specified machine + */ + public static MMLTLearningExample wsn() { + return new Example("WSN"); + } + + private static final class Example implements MMLTLearningExample { + + private static final int SCTP_TIMEOUT = 9000; // SCTP needs more waiting time + + private final String name; + private final MMLT mmlt; + private final MMLTModelParams params; + + private Example(String name) { + this.name = name; + + final String silentOutput = "void"; + final StringSymbolCombiner outputCombiner = StringSymbolCombiner.getInstance(); + final DOTInputModelDeserializer> parser = + DOTParsers.mmlt(silentOutput, outputCombiner); + + try (InputStream is = MMLTExamples.class.getResourceAsStream("/mmlt/" + name + ".dot")) { + final DOTInputModelData> model = parser.readModel(is); + final CompactMMLT automaton = model.model; + + final long maxTimeoutDelay = MMLTs.getMaximumTimeoutDelay(automaton); + final long maxTimerQueryWaitingFinal; + + if (name.contains("SCTP")) { + maxTimerQueryWaitingFinal = SCTP_TIMEOUT; + } else { + maxTimerQueryWaitingFinal = MMLTs.getMaximumInitialTimerValue(automaton) * 2; + } + + this.mmlt = automaton; + this.params = + new MMLTModelParams<>(silentOutput, outputCombiner, maxTimeoutDelay, maxTimerQueryWaitingFinal); + } catch (IOException | FormatException e) { + throw new IllegalStateException("Unable to load model " + name, e); + } + } + + @Override + public MMLTModelParams getParams() { + return this.params; + } + + @Override + public MMLT getReferenceAutomaton() { + return this.mmlt; + } + + @Override + public String toString() { + return this.name; + } + } + +} diff --git a/test-support/learning-examples/src/main/java/module-info.java b/test-support/learning-examples/src/main/java/module-info.java index ac70dd8719..222665195e 100644 --- a/test-support/learning-examples/src/main/java/module-info.java +++ b/test-support/learning-examples/src/main/java/module-info.java @@ -37,6 +37,7 @@ requires net.automatalib.serialization.learnlibv2; requires net.automatalib.util; requires org.slf4j; + requires net.automatalib.serialization.dot; // annotations are 'provided'-scoped and do not need to be loaded at runtime requires static org.checkerframework.checker.qual; @@ -50,4 +51,5 @@ exports de.learnlib.testsupport.example.spmm; exports de.learnlib.testsupport.example.sst; exports de.learnlib.testsupport.example.vpa; + exports de.learnlib.testsupport.example.mmlt; } diff --git a/test-support/learning-examples/src/main/resources/mmlt/HVAC.dot b/test-support/learning-examples/src/main/resources/mmlt/HVAC.dot new file mode 100644 index 0000000000..5c176a909b --- /dev/null +++ b/test-support/learning-examples/src/main/resources/mmlt/HVAC.dot @@ -0,0 +1,61 @@ +// Model of an HVAC system +// Adapted from Taylor and Taylor: Patterns in the Machine +digraph g { + + s0 [shape="circle"]; + s1 [timers="a=2000" shape="circle"]; + s2 [shape="circle"]; + s3 [shape="circle"]; + s4 [timers="a=2000" shape="circle"]; + s5 [shape="circle"]; + s6 [timers="a=2000" shape="circle"]; + s7 [timers="a=2000" shape="circle"]; + s8 [timers="a=2000" shape="circle"]; + s9 [timers="a=2000" shape="circle"]; + s10 [shape="circle"]; + s0 -> s1 [resets="a" label="Supplementing.Active / FromTransition.init"]; + s0 -> s2 [label="Supplementing.Inactive / Activity.initializeActive"]; + s1 -> s10 [label="Activity.OffMode / Stage.init"]; + s1 -> s2 [label="FromTransition.Completed / void"]; + s1 -> s1 [resets="a" label="to[a] / FromTransition.check"]; + s2 -> s10 [label="Activity.OffMode / Stage.init"]; + s2 -> s3 [label="Capacity.Excess / void"]; + s2 -> s5 [label="Capacity.NeedMore / Supplementing.enter"]; + s2 -> s6 [resets="a" label="OffCycle.IsStartInOffCycle / OffCycle.startCycling,OffCycle.startOffTime,Stage.startingOff"]; + s2 -> s8 [resets="a" label="OnCycle.IsStartInOnCycle / OnCycle.startCycling,OnCycle.startOnTime,Stage.startingOn"]; + s3 -> s10 [label="Activity.OffMode / Stage.init"]; + s3 -> s4 [resets="a" label="Activity.OnRequest / void"]; + s3 -> s10 [label="Supplementing.Inactive / Stage.shutdown"]; + s4 -> s10 [label="Activity.OffMode / Stage.init"]; + s4 -> s10 [label="BackTransition.Completed / Stage.notifyLower,Stage.shutdown"]; + s4 -> s4 [resets="a" label="to[a] / BackTransition.check"]; + s5 -> s10 [label="Activity.OffMode / Stage.init"]; + s5 -> s2 [label="Capacity.NeedLess / Supplementing.ext"]; + s6 -> s10 [label="Activity.OffMode / Stage.init"]; + s6 -> s3 [label="Capacity.Excess / void"]; + s6 -> s5 [label="Capacity.NeedMore / Supplementing.enter"]; + s6 -> s8 [resets="a" label="OffCycle.OffTimeExpired / OnCycle.startOnTime,Stage.startingOn"]; + s6 -> s7 [resets="a" label="OffCycle.StartingOffTimeExpired / Stage.off"]; + s6 -> s6 [resets="a" label="to[a] / OffCycle.checkStartingOffTime"]; + s7 -> s10 [label="Activity.OffMode / Stage.init"]; + s7 -> s3 [label="Capacity.Excess / void"]; + s7 -> s5 [label="Capacity.NeedMore / Supplementing.enter"]; + s7 -> s8 [resets="a" label="OffCycle.OffTimeExpired / OnCycle.startOnTime,Stage.startingOn"]; + s7 -> s7 [resets="a" label="to[a] / OffCycle.checkOffTime"]; + s8 -> s10 [label="Activity.OffMode / Stage.init"]; + s8 -> s3 [label="Capacity.Excess / void"]; + s8 -> s5 [label="Capacity.NeedMore / Supplementing.enter"]; + s8 -> s6 [resets="a" label="OnCycle.OnTimeExpired / OffCycle.startOffTime,Stage.startingOff"]; + s8 -> s9 [resets="a" label="OnCycle.StartingOnTimeExpired / Stage.on"]; + s8 -> s8 [resets="a" label="to[a] / OnCycle.checkStartingOnTime"]; + s9 -> s10 [label="Activity.OffMode / Stage.init"]; + s9 -> s3 [label="Capacity.Excess / void"]; + s9 -> s5 [label="Capacity.NeedMore / Supplementing.enter"]; + s9 -> s6 [resets="a" label="OnCycle.OnTimeExpired / OffCycle.startOffTime,Stage.startingOff"]; + s9 -> s9 [resets="a" label="to[a] / OnCycle.checkOnTime"]; + s10 -> s0 [label="Activity.OnRequest / void"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s10; + +} diff --git a/test-support/learning-examples/src/main/resources/mmlt/Oven.dot b/test-support/learning-examples/src/main/resources/mmlt/Oven.dot new file mode 100644 index 0000000000..4de25c32d9 --- /dev/null +++ b/test-support/learning-examples/src/main/resources/mmlt/Oven.dot @@ -0,0 +1,26 @@ +// Model of an oven with a time-controlled baking program. +// After powering the oven on, the oven remains idle until the program is started. +// During the program, the oven regularly measures and adjusts the temperature. +// At the end of the program, an alarm sounds. Then, the user may extend the program. +// If not extended, the program ends either when the user opens the door, presses a button, or a timeout occurs. +digraph g { + + s0 [shape="circle"]; + s1 [timers="a=3500,b=300000" shape="circle"]; + s2 [timers="a=5000" shape="circle"]; + s3 [shape="circle"]; + s0 -> s3 [label="User.Power / void"]; + s0 -> s1 [resets="a,b" label="User.Start / Temp.on"]; + s1 -> s0 [label="User.Stop / Temp.off"]; + s1 -> s1 [resets="a" label="to[a] / Temp.adjust"]; + s1 -> s2 [resets="a" label="to[b] / Alarm.start,Temp.off"]; + s2 -> s1 [resets="a,b" label="User.Extend / Alarm.stop,Temp.on"]; + s2 -> s0 [label="User.Open / Alarm.stop"]; + s2 -> s0 [label="User.Stop / Alarm.stop"]; + s2 -> s0 [label="to[a] / Alarm.stop"]; + s3 -> s0 [label="User.Power / void"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s3; + +} diff --git a/test-support/learning-examples/src/main/resources/mmlt/SCTP.dot b/test-support/learning-examples/src/main/resources/mmlt/SCTP.dot new file mode 100644 index 0000000000..9f2d239ad9 --- /dev/null +++ b/test-support/learning-examples/src/main/resources/mmlt/SCTP.dot @@ -0,0 +1,225 @@ +// Model of the endpoint association in the SCTP protocol. +// Adapted from Stewart et al.: Stream Control Transmission Protocol (RFC 9260, Figure 3) +digraph g { + + s0 [shape="circle" ]; + s1 [shape="circle" ]; + s2 [timers="a=1000" shape="circle" ]; + s3 [timers="a=1000" shape="circle" ]; + s4 [timers="a=1000" shape="circle" ]; + s5 [timers="a=1000" shape="circle" ]; + s6 [timers="a=1000" shape="circle" ]; + s7 [timers="a=1000" shape="circle" ]; + s8 [timers="a=1000" shape="circle" ]; + s9 [timers="a=1000" shape="circle" ]; + s10 [timers="a=1000" shape="circle" ]; + s11 [shape="circle" ]; + s12 [timers="a=1000" shape="circle" ]; + s13 [timers="a=1000" shape="circle" ]; + s14 [timers="a=1000" shape="circle" ]; + s15 [timers="a=1000" shape="circle" ]; + s16 [timers="a=1000" shape="circle" ]; + s17 [timers="a=1000" shape="circle" ]; + s18 [timers="a=1000" shape="circle" ]; + s19 [timers="a=1000" shape="circle" ]; + s20 [timers="a=1000" shape="circle" ]; + s21 [timers="a=1000" shape="circle" ]; + s22 [timers="a=1000" shape="circle" ]; + s23 [timers="a=1000" shape="circle" ]; + s24 [timers="a=1000" shape="circle" ]; + s25 [timers="a=1000" shape="circle" ]; + s26 [timers="a=1000" shape="circle" ]; + s27 [timers="a=1000" shape="circle" ]; + s28 [timers="a=1000" shape="circle" ]; + s29 [timers="a=1000" shape="circle" ]; + s30 [timers="a=1000" shape="circle" ]; + s31 [shape="circle" ]; + s32 [timers="a=1000" shape="circle" ]; + s33 [timers="a=1000" shape="circle" ]; + s34 [timers="a=1000" shape="circle" ]; + s35 [timers="a=1000" shape="circle" ]; + s36 [timers="a=1000" shape="circle" ]; + s37 [timers="a=1000" shape="circle" ]; + s38 [timers="a=1000" shape="circle" ]; + s39 [timers="a=1000" shape="circle" ]; + s40 [shape="circle" label="Closed"]; + s0 -> s40 [label="Receive.Abort / void"]; + s0 -> s1 [label="Receive.Shutdown / void"]; + s0 -> s40 [label="User.Abort / Send.abort"]; + s0 -> s11 [label="User.Shutdown / void"]; + s1 -> s2 [resets="a" label="no_outstanding / Send.shutdown_ack"]; + s1 -> s40 [label="Receive.Abort / void"]; + s1 -> s40 [label="User.Abort / Send.abort"]; + s2 -> s40 [label="Receive.Abort / void"]; + s2 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s2 -> s40 [label="Receive.Shutdown_complete / void"]; + s2 -> s40 [label="User.Abort / Send.abort"]; + s2 -> s3 [resets="a" label="to[a] / Send.shutdown_ack"]; + s3 -> s40 [label="Receive.Abort / void"]; + s3 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s3 -> s40 [label="Receive.Shutdown_complete / void"]; + s3 -> s40 [label="User.Abort / Send.abort"]; + s3 -> s4 [resets="a" label="to[a] / Send.shutdown_ack"]; + s4 -> s40 [label="Receive.Abort / void"]; + s4 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s4 -> s40 [label="Receive.Shutdown_complete / void"]; + s4 -> s40 [label="User.Abort / Send.abort"]; + s4 -> s5 [resets="a" label="to[a] / Send.shutdown_ack"]; + s5 -> s40 [label="Receive.Abort / void"]; + s5 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s5 -> s40 [label="Receive.Shutdown_complete / void"]; + s5 -> s40 [label="User.Abort / Send.abort"]; + s5 -> s6 [resets="a" label="to[a] / Send.shutdown_ack"]; + s6 -> s40 [label="Receive.Abort / void"]; + s6 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s6 -> s40 [label="Receive.Shutdown_complete / void"]; + s6 -> s40 [label="User.Abort / Send.abort"]; + s6 -> s7 [resets="a" label="to[a] / Send.shutdown_ack"]; + s7 -> s40 [label="Receive.Abort / void"]; + s7 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s7 -> s40 [label="Receive.Shutdown_complete / void"]; + s7 -> s40 [label="User.Abort / Send.abort"]; + s7 -> s8 [resets="a" label="to[a] / Send.shutdown_ack"]; + s8 -> s40 [label="Receive.Abort / void"]; + s8 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s8 -> s40 [label="Receive.Shutdown_complete / void"]; + s8 -> s40 [label="User.Abort / Send.abort"]; + s8 -> s9 [resets="a" label="to[a] / Send.shutdown_ack"]; + s9 -> s40 [label="Receive.Abort / void"]; + s9 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s9 -> s40 [label="Receive.Shutdown_complete / void"]; + s9 -> s40 [label="User.Abort / Send.abort"]; + s9 -> s10 [resets="a" label="to[a] / Send.shutdown_ack"]; + s10 -> s40 [label="Receive.Abort / void"]; + s10 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s10 -> s40 [label="Receive.Shutdown_complete / void"]; + s10 -> s40 [label="User.Abort / Send.abort"]; + s10 -> s31 [label="to[a] / User.error"]; + s11 -> s12 [resets="a" label="no_outstanding / Send.shutdown"]; + s11 -> s40 [label="Receive.Abort / void"]; + s11 -> s40 [label="User.Abort / Send.abort"]; + s12 -> s40 [label="Receive.Abort / void"]; + s12 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s12 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s12 -> s40 [label="User.Abort / Send.abort"]; + s12 -> s13 [resets="a" label="to[a] / Send.shutdown"]; + s13 -> s40 [label="Receive.Abort / void"]; + s13 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s13 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s13 -> s40 [label="User.Abort / Send.abort"]; + s13 -> s14 [resets="a" label="to[a] / Send.shutdown"]; + s14 -> s40 [label="Receive.Abort / void"]; + s14 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s14 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s14 -> s40 [label="User.Abort / Send.abort"]; + s14 -> s15 [resets="a" label="to[a] / Send.shutdown"]; + s15 -> s40 [label="Receive.Abort / void"]; + s15 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s15 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s15 -> s40 [label="User.Abort / Send.abort"]; + s15 -> s16 [resets="a" label="to[a] / Send.shutdown"]; + s16 -> s40 [label="Receive.Abort / void"]; + s16 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s16 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s16 -> s40 [label="User.Abort / Send.abort"]; + s16 -> s17 [resets="a" label="to[a] / Send.shutdown"]; + s17 -> s40 [label="Receive.Abort / void"]; + s17 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s17 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s17 -> s40 [label="User.Abort / Send.abort"]; + s17 -> s18 [resets="a" label="to[a] / Send.shutdown"]; + s18 -> s40 [label="Receive.Abort / void"]; + s18 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s18 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s18 -> s40 [label="User.Abort / Send.abort"]; + s18 -> s19 [resets="a" label="to[a] / Send.shutdown"]; + s19 -> s40 [label="Receive.Abort / void"]; + s19 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s19 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s19 -> s40 [label="User.Abort / Send.abort"]; + s19 -> s20 [resets="a" label="to[a] / Send.shutdown"]; + s20 -> s40 [label="Receive.Abort / void"]; + s20 -> s2 [resets="a" label="Receive.Shutdown / Send.shutdown_ack"]; + s20 -> s40 [label="Receive.Shutdown_ack / Send.shutdown_complete"]; + s20 -> s40 [label="User.Abort / Send.abort"]; + s20 -> s31 [label="to[a] / User.error"]; + s21 -> s40 [label="Receive.Abort / void"]; + s21 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s21 -> s40 [label="User.Abort / Send.abort"]; + s21 -> s32 [resets="a" label="to[a] / Send.init"]; + s22 -> s40 [label="Receive.Abort / void"]; + s22 -> s0 [label="Receive.Cookie_ack / void"]; + s22 -> s40 [label="User.Abort / Send.abort"]; + s22 -> s23 [resets="a" label="to[a] / Send.cookie_echo"]; + s23 -> s40 [label="Receive.Abort / void"]; + s23 -> s0 [label="Receive.Cookie_ack / void"]; + s23 -> s40 [label="User.Abort / Send.abort"]; + s23 -> s24 [resets="a" label="to[a] / Send.cookie_echo"]; + s24 -> s40 [label="Receive.Abort / void"]; + s24 -> s0 [label="Receive.Cookie_ack / void"]; + s24 -> s40 [label="User.Abort / Send.abort"]; + s24 -> s25 [resets="a" label="to[a] / Send.cookie_echo"]; + s25 -> s40 [label="Receive.Abort / void"]; + s25 -> s0 [label="Receive.Cookie_ack / void"]; + s25 -> s40 [label="User.Abort / Send.abort"]; + s25 -> s26 [resets="a" label="to[a] / Send.cookie_echo"]; + s26 -> s40 [label="Receive.Abort / void"]; + s26 -> s0 [label="Receive.Cookie_ack / void"]; + s26 -> s40 [label="User.Abort / Send.abort"]; + s26 -> s27 [resets="a" label="to[a] / Send.cookie_echo"]; + s27 -> s40 [label="Receive.Abort / void"]; + s27 -> s0 [label="Receive.Cookie_ack / void"]; + s27 -> s40 [label="User.Abort / Send.abort"]; + s27 -> s28 [resets="a" label="to[a] / Send.cookie_echo"]; + s28 -> s40 [label="Receive.Abort / void"]; + s28 -> s0 [label="Receive.Cookie_ack / void"]; + s28 -> s40 [label="User.Abort / Send.abort"]; + s28 -> s29 [resets="a" label="to[a] / Send.cookie_echo"]; + s29 -> s40 [label="Receive.Abort / void"]; + s29 -> s0 [label="Receive.Cookie_ack / void"]; + s29 -> s40 [label="User.Abort / Send.abort"]; + s29 -> s30 [resets="a" label="to[a] / Send.cookie_echo"]; + s30 -> s40 [label="Receive.Abort / void"]; + s30 -> s0 [label="Receive.Cookie_ack / void"]; + s30 -> s40 [label="User.Abort / Send.abort"]; + s30 -> s31 [label="to[a] / User.error"]; + s32 -> s40 [label="Receive.Abort / void"]; + s32 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s32 -> s40 [label="User.Abort / Send.abort"]; + s32 -> s33 [resets="a" label="to[a] / Send.init"]; + s33 -> s40 [label="Receive.Abort / void"]; + s33 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s33 -> s40 [label="User.Abort / Send.abort"]; + s33 -> s34 [resets="a" label="to[a] / Send.init"]; + s34 -> s40 [label="Receive.Abort / void"]; + s34 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s34 -> s40 [label="User.Abort / Send.abort"]; + s34 -> s35 [resets="a" label="to[a] / Send.init"]; + s35 -> s40 [label="Receive.Abort / void"]; + s35 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s35 -> s40 [label="User.Abort / Send.abort"]; + s35 -> s36 [resets="a" label="to[a] / Send.init"]; + s36 -> s40 [label="Receive.Abort / void"]; + s36 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s36 -> s40 [label="User.Abort / Send.abort"]; + s36 -> s37 [resets="a" label="to[a] / Send.init"]; + s37 -> s40 [label="Receive.Abort / void"]; + s37 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s37 -> s40 [label="User.Abort / Send.abort"]; + s37 -> s38 [resets="a" label="to[a] / Send.init"]; + s38 -> s40 [label="Receive.Abort / void"]; + s38 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s38 -> s40 [label="User.Abort / Send.abort"]; + s38 -> s39 [resets="a" label="to[a] / Send.init"]; + s39 -> s40 [label="Receive.Abort / void"]; + s39 -> s22 [resets="a" label="Receive.Init_ack / Send.cookie_echo"]; + s39 -> s40 [label="User.Abort / Send.abort"]; + s39 -> s31 [label="to[a] / User.error"]; + s40 -> s40 [label="Receive.Init / Send.init_ack"]; + s40 -> s0 [label="Receive.Valid / Send.cookie_ack"]; + s40 -> s21 [resets="a" label="User.Associate / Send.init"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s40; + +} diff --git a/test-support/learning-examples/src/main/resources/mmlt/WM.dot b/test-support/learning-examples/src/main/resources/mmlt/WM.dot new file mode 100644 index 0000000000..89f9ade3e3 --- /dev/null +++ b/test-support/learning-examples/src/main/resources/mmlt/WM.dot @@ -0,0 +1,196 @@ +// Model of a washing machine +// The machine is initially off. After powering it on and closing the door, +// the user can start either the short or the normal program. An open +// door prevents starting and triggers a warning. Not choosing a program within 10 seconds turns the machine off. +// In normal model, the machine fills the drum, heats the water, and starts the main wash. During this wash, +// it regularly adjusts the drum speed and maintains temperature. After 2 hours, +// the water is drained and the drum is spun at full speed for some time. Afterwards the remaining water is drained. +// The short program makes less adjustments, so that a wash ends after 1 hour. +// Both programs are interrupted when a leak is detected. Normal mode may also be interrupted by "stop". +// This drains the drum immediately. Once done, the door is unlocked, a message is shown, and the machine +// beeps repeatedly until the user presses any button or opens the door. +digraph g { + + s0 [timers="a=10000" shape="circle"]; + s1 [timers="a=10000" shape="circle"]; + s2 [shape="circle"]; + s3 [shape="circle"]; + s4 [timers="a=2000" shape="circle"]; + s5 [shape="circle"]; + s6 [timers="a=360000" shape="circle"]; + s7 [timers="a=360000" shape="circle"]; + s8 [timers="a=360000" shape="circle"]; + s9 [timers="a=360000" shape="circle"]; + s10 [timers="a=360000" shape="circle"]; + s11 [timers="a=360000" shape="circle"]; + s12 [timers="a=360000" shape="circle"]; + s13 [timers="a=360000" shape="circle"]; + s14 [timers="a=360000" shape="circle"]; + s15 [timers="a=360000" shape="circle"]; + s16 [timers="a=360000" shape="circle"]; + s17 [timers="a=360000" shape="circle"]; + s18 [timers="a=360000" shape="circle"]; + s19 [timers="a=360000" shape="circle"]; + s20 [timers="a=360000" shape="circle"]; + s21 [timers="a=360000" shape="circle"]; + s22 [timers="a=360000" shape="circle"]; + s23 [timers="a=360000" shape="circle"]; + s24 [timers="a=360000" shape="circle"]; + s25 [timers="a=360000" shape="circle"]; + s26 [shape="circle"]; + s27 [timers="a=180000" shape="circle"]; + s28 [shape="circle"]; + s29 [shape="circle"]; + s30 [timers="a=10000" shape="circle"]; + s31 [shape="circle"]; + s32 [timers="a=3600000" shape="circle"]; + s33 [shape="circle"]; + s34 [shape="circle"]; + s0 -> s30 [resets="a" label="Buttons.Start_short / void"]; + s0 -> s1 [resets="a" label="Buttons.Start / void"]; + s0 -> s34 [label="to[a] / Display.off"]; + s1 -> s2 [label="Door.Closed / Display.rem_normal,Door.lock,PumpIn.start"]; + s1 -> s0 [resets="a" label="Door.Open / Display.door_warning"]; + s1 -> s34 [label="to[a] / Display.off"]; + s2 -> s3 [label="Buttons.Stop / PumpIn.stop,PumpOut.start"]; + s2 -> s29 [label="Water.Leak / Display.alarm,PumpIn.stop"]; + s2 -> s5 [label="Water.Full / Heater.start,PumpIn.stop"]; + s3 -> s4 [resets="a" label="Water.Empty / Display.done,Door.unlock"]; + s4 -> s0 [resets="a" label="Buttons.Start_short / void"]; + s4 -> s0 [resets="a" label="Buttons.On / void"]; + s4 -> s0 [resets="a" label="Buttons.Start / void"]; + s4 -> s0 [resets="a" label="Buttons.Stop / void"]; + s4 -> s0 [resets="a" label="Door.Open / void"]; + s4 -> s4 [resets="a" label="to[a] / Beeper.beep"]; + s5 -> s3 [label="Buttons.Stop / Heater.stop,PumpOut.start"]; + s5 -> s29 [label="Water.Leak / Display.alarm,Heater.stop"]; + s5 -> s6 [resets="a" label="Water.Temp_ok / Detergent.add,Drum.normal_speed,Heater.stop"]; + s6 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s6 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s6 -> s6 [label="Water.Temp_low / Heater.start"]; + s6 -> s6 [label="Water.Temp_ok / Heater.stop"]; + s6 -> s7 [resets="a" label="to[a] / Drum.change_speed"]; + s7 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s7 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s7 -> s7 [label="Water.Temp_low / Heater.start"]; + s7 -> s7 [label="Water.Temp_ok / Heater.stop"]; + s7 -> s8 [resets="a" label="to[a] / Drum.change_speed"]; + s8 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s8 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s8 -> s8 [label="Water.Temp_low / Heater.start"]; + s8 -> s8 [label="Water.Temp_ok / Heater.stop"]; + s8 -> s9 [resets="a" label="to[a] / Drum.change_speed"]; + s9 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s9 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s9 -> s9 [label="Water.Temp_low / Heater.start"]; + s9 -> s9 [label="Water.Temp_ok / Heater.stop"]; + s9 -> s10 [resets="a" label="to[a] / Drum.change_speed"]; + s10 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s10 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s10 -> s10 [label="Water.Temp_low / Heater.start"]; + s10 -> s10 [label="Water.Temp_ok / Heater.stop"]; + s10 -> s11 [resets="a" label="to[a] / Drum.change_speed"]; + s11 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s11 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s11 -> s11 [label="Water.Temp_low / Heater.start"]; + s11 -> s11 [label="Water.Temp_ok / Heater.stop"]; + s11 -> s12 [resets="a" label="to[a] / Drum.change_speed"]; + s12 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s12 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s12 -> s12 [label="Water.Temp_low / Heater.start"]; + s12 -> s12 [label="Water.Temp_ok / Heater.stop"]; + s12 -> s13 [resets="a" label="to[a] / Drum.change_speed"]; + s13 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s13 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s13 -> s13 [label="Water.Temp_low / Heater.start"]; + s13 -> s13 [label="Water.Temp_ok / Heater.stop"]; + s13 -> s14 [resets="a" label="to[a] / Drum.change_speed"]; + s14 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s14 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s14 -> s14 [label="Water.Temp_low / Heater.start"]; + s14 -> s14 [label="Water.Temp_ok / Heater.stop"]; + s14 -> s15 [resets="a" label="to[a] / Drum.change_speed"]; + s15 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s15 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s15 -> s15 [label="Water.Temp_low / Heater.start"]; + s15 -> s15 [label="Water.Temp_ok / Heater.stop"]; + s15 -> s16 [resets="a" label="to[a] / Drum.change_speed"]; + s16 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s16 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s16 -> s16 [label="Water.Temp_low / Heater.start"]; + s16 -> s16 [label="Water.Temp_ok / Heater.stop"]; + s16 -> s17 [resets="a" label="to[a] / Drum.change_speed"]; + s17 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s17 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s17 -> s17 [label="Water.Temp_low / Heater.start"]; + s17 -> s17 [label="Water.Temp_ok / Heater.stop"]; + s17 -> s18 [resets="a" label="to[a] / Drum.change_speed"]; + s18 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s18 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s18 -> s18 [label="Water.Temp_low / Heater.start"]; + s18 -> s18 [label="Water.Temp_ok / Heater.stop"]; + s18 -> s19 [resets="a" label="to[a] / Drum.change_speed"]; + s19 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s19 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s19 -> s19 [label="Water.Temp_low / Heater.start"]; + s19 -> s19 [label="Water.Temp_ok / Heater.stop"]; + s19 -> s20 [resets="a" label="to[a] / Drum.change_speed"]; + s20 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s20 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s20 -> s20 [label="Water.Temp_low / Heater.start"]; + s20 -> s20 [label="Water.Temp_ok / Heater.stop"]; + s20 -> s21 [resets="a" label="to[a] / Drum.change_speed"]; + s21 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s21 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s21 -> s21 [label="Water.Temp_low / Heater.start"]; + s21 -> s21 [label="Water.Temp_ok / Heater.stop"]; + s21 -> s22 [resets="a" label="to[a] / Drum.change_speed"]; + s22 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s22 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s22 -> s22 [label="Water.Temp_low / Heater.start"]; + s22 -> s22 [label="Water.Temp_ok / Heater.stop"]; + s22 -> s23 [resets="a" label="to[a] / Drum.change_speed"]; + s23 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s23 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s23 -> s23 [label="Water.Temp_low / Heater.start"]; + s23 -> s23 [label="Water.Temp_ok / Heater.stop"]; + s23 -> s24 [resets="a" label="to[a] / Drum.change_speed"]; + s24 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s24 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s24 -> s24 [label="Water.Temp_low / Heater.start"]; + s24 -> s24 [label="Water.Temp_ok / Heater.stop"]; + s24 -> s25 [resets="a" label="to[a] / Drum.change_speed"]; + s25 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s25 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s25 -> s25 [label="Water.Temp_low / Heater.start"]; + s25 -> s25 [label="Water.Temp_ok / Heater.stop"]; + s25 -> s26 [label="to[a] / Drum.stop,Heater.stop,PumpOut.start"]; + s26 -> s3 [label="Buttons.Stop / PumpOut.start"]; + s26 -> s29 [label="Water.Leak / Display.alarm,PumpOut.stop"]; + s26 -> s27 [resets="a" label="Water.Low / Drum.full_speed,PumpOut.stop"]; + s27 -> s3 [label="Buttons.Stop / Drum.stop,PumpOut.start"]; + s27 -> s29 [label="Water.Leak / Display.alarm,Drum.stop"]; + s27 -> s28 [label="to[a] / Drum.stop,PumpOut.start"]; + s28 -> s3 [label="Buttons.Stop / PumpOut.start"]; + s28 -> s4 [resets="a" label="Water.Empty / Display.done,Door.unlock,PumpOut.stop"]; + s28 -> s29 [label="Water.Leak / Display.alarm,PumpOut.stop"]; + s30 -> s31 [label="Door.Closed / Display.rem_short,Door.lock,Heater.start,PumpIn.start"]; + s30 -> s0 [resets="a" label="Door.Open / Display.door_warning"]; + s30 -> s34 [label="to[a] / Display.off"]; + s31 -> s3 [label="Buttons.Stop / Heater.stop,PumpIn.stop,PumpOut.start"]; + s31 -> s29 [label="Water.Leak / Display.alarm,Heater.stop,PumpIn.stop"]; + s31 -> s32 [resets="a" label="Water.Full / Detergent.add,Drum.normal_speed,Heater.stop,PumpIn.stop"]; + s32 -> s3 [label="Buttons.Stop / Drum.stop,Heater.stop,PumpOut.start"]; + s32 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,Heater.stop"]; + s32 -> s32 [label="Water.Temp_low / Heater.start"]; + s32 -> s32 [label="Water.Temp_ok / Heater.stop"]; + s32 -> s33 [label="to[a] / Drum.normal_speed,Heater.stop,PumpOut.start"]; + s33 -> s3 [label="Buttons.Stop / Drum.stop,PumpOut.start"]; + s33 -> s4 [resets="a" label="Water.Empty / Display.done,Door.unlock,Drum.stop,PumpOut.stop"]; + s33 -> s29 [label="Water.Leak / Display.alarm,Drum.stop,PumpOut.stop"]; + s34 -> s0 [resets="a" label="Buttons.On / Display.welcome"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s34; + +} diff --git a/test-support/learning-examples/src/main/resources/mmlt/WSN.dot b/test-support/learning-examples/src/main/resources/mmlt/WSN.dot new file mode 100644 index 0000000000..12d305ae35 --- /dev/null +++ b/test-support/learning-examples/src/main/resources/mmlt/WSN.dot @@ -0,0 +1,24 @@ +// Model of a wireless sensor node that regularly collects and transmits data. +// If the battery is low, no data is transmitted. Then, +// a user may collect the data manually. +// The node can be shut down at any time. If the battery is empty, it is shut down automatically. +digraph g { + + s0 [timers="a=60000,b=3600000" shape="circle"]; + s1 [timers="a=300000" shape="circle"]; + s2 [shape="circle"]; + s3 [shape="circle"]; + s0 -> s1 [resets="a" label="Battery.Low / Tx.disable"]; + s0 -> s3 [label="User.Power / void"]; + s0 -> s0 [resets="a" label="to[a] / Sensor.sample"]; + s0 -> s0 [resets="b" label="to[b] / Tx.send"]; + s1 -> s2 [label="Battery.Empty / void"]; + s1 -> s1 [resets="a" label="User.Collect / Buffer.get"]; + s1 -> s3 [label="User.Power / void"]; + s1 -> s1 [resets="a" label="to[a] / Sensor.sample"]; + s3 -> s0 [resets="a,b" label="User.Power / void"]; + +__start0 [label="" shape="none" width="0" height="0"]; +__start0 -> s3; + +} diff --git a/test-support/learning-examples/src/main/resources/mmlt/sensor_collector.dot b/test-support/learning-examples/src/main/resources/mmlt/sensor_collector.dot new file mode 100644 index 0000000000..9035e0a600 --- /dev/null +++ b/test-support/learning-examples/src/main/resources/mmlt/sensor_collector.dot @@ -0,0 +1,26 @@ +// An MMLT model of a sensor that measures particulate matter and ambient noise. +// The measurement program automatically ends after some time. It may be restarted at any time. +// Alternatively, a self-check program can be entered. This also ends after some time and may be aborted. +// At the end of either program, the collected data may be retrieved. +digraph g { + s0 [label="L0" timers=""] + s1 [label="L1" timers="a=3,b=6,c=40"] + s2 [label="L2" timers="d=4"] + s3 [label="L3" timers=""] + + s0 -> s1 [label="p1/go"] + + s1 -> s1 [label="abort / ok" resets="a,b,c"] + s1 -> s1 [label="to[a] / part"] + s1 -> s1 [label="to[b] / noise"] + s1 -> s3 [label="to[c] / done"] + + s0 -> s2 [label="p2 / go"] + s2 -> s3 [label="abort / void"] + s2 -> s3 [label="to[d] / done"] + + s3 -> s0 [label="collect / void"] + + __start0 [label="" shape="none" width="0" height="0"]; + __start0 -> s0; +} \ No newline at end of file