diff --git a/.gitignore b/.gitignore index d4fb532b..d36b972c 100644 --- a/.gitignore +++ b/.gitignore @@ -6,5 +6,7 @@ **/*.iml **/*.swp **/*.versionsBackup +**/*.ipr +**/*.iws diff --git a/infrastructure/infrastructure/src/main/java/eu/f4sten/infra/kafka/DefaultTopics.java b/infrastructure/infrastructure/src/main/java/eu/f4sten/infra/kafka/DefaultTopics.java index 8dbff9db..6c8cabbf 100644 --- a/infrastructure/infrastructure/src/main/java/eu/f4sten/infra/kafka/DefaultTopics.java +++ b/infrastructure/infrastructure/src/main/java/eu/f4sten/infra/kafka/DefaultTopics.java @@ -24,5 +24,5 @@ private DefaultTopics() { public static final String INGEST = "fasten.mvn.releases"; public static final String POM_ANALYZER = "fasten.POMAnalyzer"; public static final String CALLABLE_INDEXER = "fasten.CallableIndexFastenPlugin"; - + public static final String VUL_CHAIN_FINDER = "fasten.VulChainFinder"; } \ No newline at end of file diff --git a/infrastructure/infrastructure/src/test/java/eu/f4sten/infra/kafka/DefaultTopicsTest.java b/infrastructure/infrastructure/src/test/java/eu/f4sten/infra/kafka/DefaultTopicsTest.java index e7f573d2..e54b49ca 100644 --- a/infrastructure/infrastructure/src/test/java/eu/f4sten/infra/kafka/DefaultTopicsTest.java +++ b/infrastructure/infrastructure/src/test/java/eu/f4sten/infra/kafka/DefaultTopicsTest.java @@ -25,5 +25,7 @@ public class DefaultTopicsTest { public void defaultValues() { assertEquals("fasten.mvn.releases", DefaultTopics.INGEST); assertEquals("fasten.POMAnalyzer", DefaultTopics.POM_ANALYZER); + assertEquals("fasten.CallableIndexFastenPlugin", DefaultTopics.CALLABLE_INDEXER); + assertEquals("fasten.VulChainFinder", DefaultTopics.VUL_CHAIN_FINDER); } } \ No newline at end of file diff --git a/infrastructure/loader/pom.xml b/infrastructure/loader/pom.xml index fca21501..ad50e5e6 100644 --- a/infrastructure/loader/pom.xml +++ b/infrastructure/loader/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 eu.fasten-project @@ -70,7 +69,11 @@ ingested-artifact-completion 0.0.8-SNAPSHOT - + + eu.fasten-project + vulnerable-chain-finder + 0.0.8-SNAPSHOT + org.reflections diff --git a/plugins/pom.xml b/plugins/pom.xml index c91da0f8..860d4b75 100644 --- a/plugins/pom.xml +++ b/plugins/pom.xml @@ -10,6 +10,7 @@ pom + vulnerable-chain-finder examples maven-crawler pom-analyzer diff --git a/plugins/vulnerable-chain-finder/pom.xml b/plugins/vulnerable-chain-finder/pom.xml new file mode 100644 index 00000000..1c0b1a28 --- /dev/null +++ b/plugins/vulnerable-chain-finder/pom.xml @@ -0,0 +1,32 @@ + + + 4.0.0 + + + eu.fasten-project + plugins + 0.0.8-SNAPSHOT + + vulnerable-chain-finder + + + + eu.fasten-project + pom-analyzer + 0.0.8-SNAPSHOT + + + eu.fasten.analyzer + javacg-opal + 0.0.8 + + + eu.fasten-project + infrastructure-impl + 0.0.8-SNAPSHOT + test + + + \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/Main.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/Main.java new file mode 100644 index 00000000..dadb7d70 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/Main.java @@ -0,0 +1,187 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder; + +import eu.f4sten.infra.AssertArgs; +import eu.f4sten.infra.Plugin; +import eu.f4sten.infra.json.TRef; +import eu.f4sten.infra.kafka.Kafka; +import eu.f4sten.infra.kafka.Lane; +import eu.f4sten.infra.kafka.Message; +import eu.f4sten.infra.kafka.MessageGenerator; +import eu.f4sten.pomanalyzer.data.MavenId; +import eu.f4sten.vulchainfinder.exceptions.RestApiError; +import eu.f4sten.vulchainfinder.utils.DatabaseUtils; +import eu.f4sten.vulchainfinder.utils.ImpactPropagator; +import eu.f4sten.vulchainfinder.utils.RestAPIDependencyResolver; +import eu.fasten.core.data.callableindex.RocksDao; +import eu.fasten.core.maven.data.Pom; +import eu.fasten.core.merge.CGMerger; +import eu.fasten.core.vulchains.VulnerableCallChain; +import eu.fasten.core.vulchains.VulnerableCallChainRepository; + +import java.io.File; +import java.util.HashSet; +import java.util.Set; +import javax.inject.Inject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class Main implements Plugin { + + private static final Logger LOG = LoggerFactory.getLogger(Main.class); + public final RestAPIDependencyResolver resolver; + private final DatabaseUtils db; + private final RocksDao dao; + private final Kafka kafka; + private final VulChainFinderArgs args; + private final MessageGenerator msgs; + private final VulnerableCallChainRepository repo; + + private MavenId curId; + + @Inject + public Main(DatabaseUtils db, RocksDao dao, Kafka kafka, VulChainFinderArgs args, MessageGenerator msgs, + RestAPIDependencyResolver resolver, VulnerableCallChainRepository repo) { + this.db = db; + this.dao = dao; + this.kafka = kafka; + this.args = args; + this.msgs = msgs; + this.resolver = resolver; + this.repo = repo; + } + + @Override + public void run() { + AssertArgs.assertFor(args) // + .notNull(a -> a.kafkaIn, "kafka input topic") // + .notNull(a -> a.kafkaOut, "kafka output topic"); + + LOG.info("Subscribing to '{}', will publish in '{}' ...", args.kafkaIn, args.kafkaOut); + + final var msgClass = new TRef, Object>, Object>, Object>>() {}; + + kafka.subscribe(args.kafkaIn, msgClass, (msg, l) -> { + final var pom = msg.input.input.input.payload; + curId = extractMavenIdFrom(pom); + LOG.info("Consuming next record ..."); + runOrPublishErr(this::process); + }); + while (true) { + LOG.debug("Polling ..."); + kafka.poll(); + } + } + + public void process() { + // NOTE: this can be a temporary FS-based check and can be replaced with a better approach or removed at all. + if (isCurIdProcessed()) { + LOG.info("Coordinate {} already processed!", curId.asCoordinate()); + return; + } + + LOG.info("Processing {}", curId.asCoordinate()); + + final var allDeps = resolver.resolveDependencyIds(curId); + + final var vulDeps = db.selectVulnerablePackagesExistingIn(allDeps); + + Set vulChains = new HashSet<>(); + if (curIdIsPackageLevelVulnerable(vulDeps)) { + vulChains = extractVulCallChains(allDeps, vulDeps); + } + + curIdIsMethodLevelVulnerable(vulChains); + // NOTE: it stores empty vuln. chains too to avoid re-processing records. + storeInVulRepo(vulChains); + } + + private boolean curIdIsMethodLevelVulnerable(final Set vulChains) { + return !vulChains.isEmpty(); + } + + private boolean curIdIsPackageLevelVulnerable(final Set vulDeps) { + return vulDeps != null && !vulDeps.isEmpty(); + } + + private void storeInVulRepo(final Set vulnerableCallChains) { + final var productName = String.format("%s:%s", curId.groupId, curId.artifactId); + repo.store(productName, curId.version, vulnerableCallChains); + } + + private Set extractVulCallChains(final Set allDeps, final Set vulDeps) { + Set result = new HashSet<>(); + + final var merger = new CGMerger(allDeps, db.getContext(), dao); + final var mergedCG = merger.mergeAllDeps(); + final var vulCallables = db.selectVulCallablesOf(vulDeps); + final var propagator = new ImpactPropagator(mergedCG, merger.getAllUrisFromDB(mergedCG)); + propagator.propagateUrisImpacts(vulCallables.keySet()); + LOG.info("Found {} distinct vulnerable paths", propagator.getImpacts().size()); + + if (!propagator.getImpacts().isEmpty()) { + result = propagator.extractApplicationVulChains(vulCallables, curId); + } + + return result; + } + + public static MavenId extractMavenIdFrom(final Pom pom) { + final var id = new MavenId(); + id.groupId = pom.groupId; + id.artifactId = pom.artifactId; + id.version = pom.version; + return id; + } + + private void runOrPublishErr(final Runnable r) { + try { + r.run(); + } catch (RestApiError e) { + LOG.error("Forced to stop the plug-in as the REST API is unavailable", e); + throw e; + } catch (Exception e) { + LOG.warn("Execution failed for input: {}", curId, e); + + var msg = msgs.getErr(curId, returnCause(e)); + kafka.publish(msg, args.kafkaOut, Lane.ERROR); + } + } + + private Throwable returnCause(final Exception e) { + final var isRunTime = RuntimeException.class.equals(e.getClass()); + final var causeNotNull = e.getCause() != null; + if (isRunTime && causeNotNull) { + return e.getCause(); + } + return e; + } + + public MavenId getCurId() { + return curId; + } + + public void setCurId(final MavenId curId) { + this.curId = curId; + } + + private boolean isCurIdProcessed() { + return new File(repo.getFilePath(String.format("%s:%s", curId.groupId, curId.artifactId), curId.version)).exists(); + } + +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/VulChainFinderArgs.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/VulChainFinderArgs.java new file mode 100644 index 00000000..3e6fcae6 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/VulChainFinderArgs.java @@ -0,0 +1,39 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder; + +import com.beust.jcommander.Parameter; +import eu.f4sten.infra.kafka.DefaultTopics; + +import java.io.File; + +public class VulChainFinderArgs { + + @Parameter(names = "--vulchainfinder.kafkaIn", arity = 1) + public String kafkaIn = DefaultTopics.CALLABLE_INDEXER; + + @Parameter(names = "--vulchainfinder.kafkaOut", arity = 1) + public String kafkaOut = DefaultTopics.VUL_CHAIN_FINDER; + + @Parameter(names = "--restApiBaseUrl", arity = 1) + public String restApiBaseURL; + + @Parameter(names = "--callableIndexPath", arity = 1) + public File callableIndexPath; + + @Parameter(names = "--vulnChainRepoPath", arity = 1) + public File vulnChainRepoPath; +} diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/VulChainFinderInjectorConfig.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/VulChainFinderInjectorConfig.java new file mode 100644 index 00000000..f4de1c23 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/VulChainFinderInjectorConfig.java @@ -0,0 +1,95 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder; + +import com.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import com.google.inject.multibindings.ProvidesIntoSet; +import eu.f4sten.infra.IInjectorConfig; +import eu.f4sten.infra.InjectorConfig; +import eu.f4sten.infra.json.JsonUtils; +import eu.f4sten.infra.utils.IoUtils; +import eu.f4sten.infra.utils.PostgresConnector; +import eu.f4sten.vulchainfinder.json.FastenURIJacksonModule; +import eu.f4sten.vulchainfinder.utils.DatabaseUtils; +import eu.f4sten.vulchainfinder.utils.RestAPIDependencyResolver; +import eu.fasten.core.data.callableindex.RocksDao; +import eu.fasten.core.vulchains.VulnerableCallChainRepository; +import java.io.FileNotFoundException; +import java.net.http.HttpClient; +import java.nio.file.Paths; + +import org.jooq.SQLDialect; +import org.jooq.impl.DSL; +import org.rocksdb.RocksDBException; + +import static eu.f4sten.infra.AssertArgs.assertFor; + +@InjectorConfig +public class VulChainFinderInjectorConfig implements IInjectorConfig { + + private VulChainFinderArgs args; + + public VulChainFinderInjectorConfig(VulChainFinderArgs args) { + this.args = args; + } + + @Override + public void configure(Binder binder) { + binder.bind(VulChainFinderArgs.class).toInstance(args); + } + + @Provides + public DatabaseUtils bindDatabaseUtils(PostgresConnector pc, JsonUtils json) { + var c = pc.getNewConnection(); + var dslContext = DSL.using(c, SQLDialect.POSTGRES); + return new DatabaseUtils(dslContext, json); + } + + @Provides + @Singleton + public RocksDao bindRocksDao() throws RocksDBException { + assertFor(args) // + .notNull(args -> args.callableIndexPath, "Provide path to the callable indexer, RocksDB folder!") // + .that(args -> args.callableIndexPath.exists(), "Path to the callable indexer does not exist!"); + return new RocksDao(args.callableIndexPath.getPath(), true); + } + + @Provides + public RestAPIDependencyResolver bindRestAPIDependencyResolver(){ + assertFor(args) // + .notNull(args -> args.restApiBaseURL, "Provide the REST API address!"); + return new RestAPIDependencyResolver(args.restApiBaseURL, HttpClient.newBuilder().build()); + } + + @Provides + public VulnerableCallChainRepository bindVulnerableCallChainRepository(IoUtils io){ + try { + assertFor(args) // + .notNull(args -> args.vulnChainRepoPath, "Provide a path to store vulnerable chain repos!"); + return new VulnerableCallChainRepository(Paths.get(io.getBaseFolder().toString(), args.vulnChainRepoPath.getPath()).toString()); + } catch (FileNotFoundException e) { + throw new RuntimeException(e); + } + } + + @ProvidesIntoSet + public Module bindJacksonModule() { + return new FastenURIJacksonModule(); + } +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/data/NodeReachability.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/data/NodeReachability.java new file mode 100644 index 00000000..7cb6b2f2 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/data/NodeReachability.java @@ -0,0 +1,72 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder.data; + +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +public class NodeReachability { + + public long targetNode; + public Map nextStepTowardsTarget = new HashMap<>(); + + public NodeReachability(long impactingNode) { + this.targetNode = impactingNode; + } + + public boolean isReachingTarget(long nodeId) { + return targetNode == nodeId || nextStepTowardsTarget.containsKey(nodeId); + } + + public List getShortestPath(long nodeId) { + if (!isReachingTarget(nodeId)) { + throw new InvalidParameterException("Node does not reach target"); + } + + final var path = new ArrayList(); + path.add(nodeId); + + var curNode = nodeId; + while (curNode != targetNode) { + curNode = nextStepTowardsTarget.get(curNode); + path.add(curNode); + } + return path; + } + + @Override + public boolean equals(Object o) { + return EqualsBuilder.reflectionEquals(this, o); + } + + @Override + public int hashCode() { + return HashCodeBuilder.reflectionHashCode(this); + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE); + } +} diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/exceptions/RestApiError.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/exceptions/RestApiError.java new file mode 100644 index 00000000..6a1a3a24 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/exceptions/RestApiError.java @@ -0,0 +1,26 @@ +/* + * Copyright 2022 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder.exceptions; + +/** + * A custom exception for Rest API-related exception to stop the plug-in's container. This error is unrecoverable. + */ +public class RestApiError extends Error { + public RestApiError(String errorMsg, Throwable err) { + super(errorMsg, err); + } +} diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/json/FastenURIJacksonModule.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/json/FastenURIJacksonModule.java new file mode 100644 index 00000000..bf1b58cf --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/json/FastenURIJacksonModule.java @@ -0,0 +1,57 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder.json; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import eu.fasten.core.data.FastenJavaURI; +import eu.fasten.core.data.FastenURI; +import java.io.IOException; + +public class FastenURIJacksonModule extends SimpleModule { + + public FastenURIJacksonModule() { + + addSerializer(FastenURI.class, new JsonSerializer<>() { + @Override + public void serialize(FastenURI value, JsonGenerator gen, + SerializerProvider serializer) throws IOException { + gen.writeString(value.toString()); + } + }); + + addDeserializer(FastenURI.class, new JsonDeserializer<>() { + @Override + public FastenURI deserialize(JsonParser p, + DeserializationContext deserializationContext) + throws IOException { + final var uri = p.getValueAsString(); + FastenURI fastenURI; + if (uri.startsWith("fasten://mvn")) { + fastenURI = FastenJavaURI.create(uri); + }else { + fastenURI = FastenURI.create(uri); + } + return fastenURI; + } + }); + } +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/DatabaseUtils.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/DatabaseUtils.java new file mode 100644 index 00000000..aa40e693 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/DatabaseUtils.java @@ -0,0 +1,121 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder.utils; + +import eu.f4sten.infra.json.JsonUtils; +import eu.f4sten.infra.json.TRef; +import eu.fasten.core.data.FastenURI; +import eu.fasten.core.data.metadatadb.MetadataDao; +import eu.fasten.core.data.metadatadb.codegen.tables.Modules; +import eu.fasten.core.data.vulnerability.Vulnerability; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.jooq.DSLContext; +import org.jooq.Record; + +public class DatabaseUtils { + + private final DSLContext context; + private final JsonUtils jsonUtils; + + public DatabaseUtils(DSLContext context, JsonUtils jsonUtils) { + this.context = context; + this.jsonUtils = jsonUtils; + } + + public DSLContext getContext() { + return context; + } + + public MetadataDao getDao(DSLContext ctx) { + return new MetadataDao(ctx); + } + + public Map> selectVulCallablesOf(final Set depIds) { + Map> vulCallables = new HashMap<>(); + for (Long depId : depIds) { + vulCallables = selectVulCallablesOf(depId); + } + return vulCallables; + } + + public Map> selectVulCallablesOf(final long depId) { + + final var moduleIds = selectAllModulesOf(depId); + if (moduleIds == null || moduleIds.isEmpty()) { + return Collections.emptyMap(); + } + return selectConcurrentlyVulCallablesOf(moduleIds); + } + + public Map> selectConcurrentlyVulCallablesOf( + final Set moduleIds) { + Map> result = new ConcurrentHashMap<>(); + moduleIds.parallelStream().forEach(moduleId -> + context.fetch(createStrForSelectVulCallablesWhereModuleIdIs(moduleId)) + .forEach(record -> { + final var vulMap = convertRecordToVulMap(record); + if (!vulMap.isEmpty()) { + result.put(createFastenUriFromPckgVersionUriFields(record), + new ArrayList<>(vulMap.values())); + } + })); + return result; + } + + private Map convertRecordToVulMap(final Record record) { + final var vulField = record.get(3); + if (vulField == null) { + return Collections.emptyMap(); + } + final var setType = new TRef>() {}; + return jsonUtils.fromJson(vulField.toString(), setType); + } + + public static FastenURI createFastenUriFromPckgVersionUriFields(final Record record) { + final var uriString = + String.format("%s%s$%s%s", + "fasten://mvn!", record.get(0), record.get(1), record.get(2)); + return FastenURI.create(uriString); + } + + public static String createStrForSelectVulCallablesWhereModuleIdIs(final Long moduleId) { + return "SELECT packages.package_name, package_versions.version, callables.fasten_uri, " + + "callables.metadata -> 'vulnerabilities' " + + "FROM callables, modules, package_versions, packages " + + "where modules.id = callables.module_id " + + "and package_versions.id = modules.package_version_id " + + "and packages.id = package_versions.package_id " + + "and callables.metadata -> 'vulnerabilities' IS NOT NULL " + + "and callables.module_id = " + moduleId; + } + + public Set selectAllModulesOf(final long depId) { + return context.select(Modules.MODULES.ID).from(Modules.MODULES) + .where(Modules.MODULES.PACKAGE_VERSION_ID.eq(depId)).fetch() + .intoSet(Modules.MODULES.ID); + } + + public Set selectVulnerablePackagesExistingIn(final Set depIds) { + return getDao(context).findVulnerablePackageVersions(depIds); + } +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/ImpactPropagator.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/ImpactPropagator.java new file mode 100644 index 00000000..1067d450 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/ImpactPropagator.java @@ -0,0 +1,131 @@ +package eu.f4sten.vulchainfinder.utils; + +import com.google.common.collect.BiMap; +import eu.f4sten.pomanalyzer.data.MavenId; +import eu.f4sten.vulchainfinder.data.NodeReachability; +import eu.fasten.core.data.DirectedGraph; +import eu.fasten.core.data.FastenURI; +import eu.fasten.core.data.vulnerability.Vulnerability; +import eu.fasten.core.vulchains.VulnerableCallChain; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class ImpactPropagator { + + private final DirectedGraph graph; + private final BiMap idUriMap; + + private Set impacts; + + public ImpactPropagator(DirectedGraph graph, BiMap idUriMap) { + this.graph = graph; + this.idUriMap = idUriMap; + } + + public Set getImpacts() { + return impacts; + } + + public void propagateUrisImpacts(final Set uris) { + + impacts = uris.stream() + .map(nodeUri -> idUriMap.inverse().get(nodeUri.toString())) + .filter(this::nodeIdExists) + .map(this::propagateNodeImpacts) + .collect(Collectors.toSet()); + } + + public NodeReachability propagateNodeImpacts(final Long nodeId) { + + final var result = new NodeReachability(nodeId); + + final var nodesToVisit = new LinkedList(); + nodesToVisit.add(nodeId); + + while (!nodesToVisit.isEmpty()) { + final var currentTarget = nodesToVisit.poll(); + + for (final var srcToTarget : graph.incomingEdgesOf(currentTarget)) { + final var source = srcToTarget.firstLong(); + if (result.nextStepTowardsTarget.containsKey(source) || currentTarget.equals(source)) { + continue; + } + + result.nextStepTowardsTarget.put(source, currentTarget); + nodesToVisit.add(source); + } + } + return result; + } + + private boolean nodeIdExists(final Long nodeId) { + return nodeId != null && graph.containsVertex(nodeId); + } + + public Set extractApplicationVulChains( + final Map> vulCallables, + final MavenId appId) { + return extractNodesVulChains(vulCallables, extractPackageNodes(appId)); + } + + public Set extractNodesVulChains( + final Map> vulCallables, + final Set longs) { + final Set result = new HashSet<>(); + + for (final var appNode : longs) { + for (final var impact : impacts) { + if (thereIsNoImpactForNode(appNode, impact)) { + continue; + } + + final var currentVul = + vulCallables.get(FastenURI.create(idUriMap.get(impact.targetNode))); + final var chains = extractUriChainsForNode(appNode, impact); + result.add(new VulnerableCallChain(currentVul, chains)); + } + } + + return result; + } + + private boolean thereIsNoImpactForNode(final Long appNode, final NodeReachability impact) { + return !impact.nextStepTowardsTarget.containsKey(appNode); + } + + private List extractUriChainsForNode(final Long appNode, + final NodeReachability impact) { + final var result = new ArrayList(); + + if (impact.isReachingTarget(appNode)) { + final var appVulIdChains = impact.getShortestPath(appNode); + for (final var nodeId : appVulIdChains) { + result.add(FastenURI.create(idUriMap.get(nodeId))); + } + } + return result; + } + + public HashSet extractPackageNodes(final MavenId id) { + final var result = new HashSet(); + + final var appUri = createPackageUri(id); + for (final var uri : idUriMap.values()) { + if (uri.startsWith(appUri)) { + result.add(idUriMap.inverse().get(uri)); + } + } + + return result; + } + + private String createPackageUri(final MavenId appId) { + return String.format("%s%s:%s$%s", "fasten://mvn!", + appId.groupId, appId.artifactId, appId.version); + } +} diff --git a/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/RestAPIDependencyResolver.java b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/RestAPIDependencyResolver.java new file mode 100644 index 00000000..cc96d216 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/main/java/eu/f4sten/vulchainfinder/utils/RestAPIDependencyResolver.java @@ -0,0 +1,149 @@ +package eu.f4sten.vulchainfinder.utils; + +import eu.f4sten.pomanalyzer.data.MavenId; +import eu.f4sten.vulchainfinder.exceptions.RestApiError; +import eu.fasten.core.data.metadatadb.codegen.tables.Dependencies; +import eu.fasten.core.data.metadatadb.codegen.tables.PackageVersions; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.net.http.HttpResponse.BodyHandler; +import java.util.HashSet; +import java.util.Set; +import org.json.JSONArray; +import org.json.JSONObject; + +public class RestAPIDependencyResolver { + private static final BodyHandler BODY_HANDLER = HttpResponse.BodyHandlers.ofString(); + private static final String DEPS_ENDPOINT; + public static final String PACKAGE_VERSION_ENDPOINT = + "/packages/{groupId}:{artifactId}/{version}"; + public static final String RESOLVE_DEPENDENCIES_ENDPOINT; + + static { + DEPS_ENDPOINT = PACKAGE_VERSION_ENDPOINT + "/deps"; + RESOLVE_DEPENDENCIES_ENDPOINT = PACKAGE_VERSION_ENDPOINT + "/resolve/dependencies"; + } + + private static final int OK = 200; + private final String restAPIBaseURL; + private final HttpClient client; + + public String getRestAPIBaseURL() { + return restAPIBaseURL; + } + + public RestAPIDependencyResolver(String restAPIBaseURL, HttpClient client) { + this.restAPIBaseURL = restAPIBaseURL; + this.client = client; + } + + public Set resolveDependencyGavs(final MavenId mavenId) { + final HttpResponse response = requestEndPoint(RESOLVE_DEPENDENCIES_ENDPOINT, mavenId); + return extractGavsFromResponse(response); + } + + private Set extractGavsFromResponse(final HttpResponse response) { + final Set result = new HashSet<>(); + + final var deps = new JSONArray(response.body()); + for (final var dep : deps) { + var g = extractStringFieldFromJSONObj((JSONObject) dep, "groupId"); + var a = extractLongFieldFromJSONObj((JSONObject) dep, "artifactId"); + var v = extractLongFieldFromJSONObj((JSONObject) dep, "version"); + result.add(g + ":" + a + ":" + v); + } + return result; + } + + public Set resolveDependencyIds(final MavenId mavenId) { + final HttpResponse response = requestEndPoint(DEPS_ENDPOINT, mavenId); + final var depIds = extractPackageIdsFromResponse(response); + final HttpResponse appResponse = requestEndPoint(PACKAGE_VERSION_ENDPOINT, mavenId); + final var fieldName = PackageVersions.PACKAGE_VERSIONS.ID.getName(); + depIds.add(extractLongFieldFromJSONObj(new JSONObject(appResponse.body()), fieldName)); + return depIds; + } + + private HttpResponse requestEndPoint(final String depsEndpoint, final MavenId id) { + final var uri = createUri(depsEndpoint, id); + final var request = HttpRequest.newBuilder().uri(uri).GET().build(); + return sendOrThrow(request); + } + + public String extractStringFieldFromJSONObj(final JSONObject response, final String field) { + return (String) response.get(field); + } + + public long extractLongFieldFromJSONObj(final JSONObject response, final String field) { + return (int) response.get(field); + } + + public Set extractPackageIdsFromResponse(final HttpResponse response) { + final Set result = new HashSet<>(); + + final var deps = new JSONArray(response.body()); + for (final var dep : deps) { + final var fieldName = Dependencies.DEPENDENCIES.DEPENDENCY_ID.getName(); + long id = extractLongFieldFromJSONObj((JSONObject) dep, fieldName); + result.add(id); + } + return result; + } + + public static boolean isNotOK(final HttpResponse response) { + return response.statusCode() != OK; + } + + public HttpResponse sendOrThrow(final HttpRequest request) { + HttpResponse response = null; + Exception exception = null; + + try { + response = this.client.send(request, BODY_HANDLER); + } catch (IOException | InterruptedException e) { + exception = e; + } + if (exception != null || response == null || isNotOK(response)) { + throw new RestApiError("Problem requesting Rest API.", exception); + } + + return response; + } + + public URI createUri(final String endpoint, final MavenId id) { + return returnFullUriOrThrow(placeIDInEndpoint(endpoint, id)); + } + + public String placeIDInEndpoint(final String uri, final MavenId id) { + return uri.replace("{groupId}", id.groupId).replace("{artifactId}", id.artifactId).replace("{version}", + id.version); + } + + public URI returnFullUriOrThrow(final String endpoint) { + try { + return new URI(makeCorrectFullURI(endpoint)); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + + public String makeCorrectFullURI(String endpoint) { + var delim = "/"; + if (endpoint == null || this.restAPIBaseURL == null) { + return null; + } + if (endpoint.startsWith("/")) { + endpoint = endpoint.substring(1); + } + String base = this.restAPIBaseURL; + if (this.restAPIBaseURL.endsWith("/")) { + base = base.replaceFirst(".$", ""); + } + + return String.format("%s%s%s", base, delim, endpoint); + } +} diff --git a/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/MainTest.java b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/MainTest.java new file mode 100644 index 00000000..dd83cca3 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/MainTest.java @@ -0,0 +1,127 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder; + +import static eu.f4sten.vulchainfinder.Main.extractMavenIdFrom; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.http.HttpClient; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +import org.jooq.impl.DSL; +import org.json.JSONException; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.rocksdb.RocksDBException; +import org.skyscreamer.jsonassert.JSONAssert; +import org.skyscreamer.jsonassert.JSONCompareMode; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.f4sten.infra.impl.json.JsonUtilsImpl; +import eu.f4sten.infra.kafka.Kafka; +import eu.f4sten.infra.kafka.MessageGenerator; +import eu.f4sten.pomanalyzer.data.MavenId; +import eu.f4sten.vulchainfinder.json.FastenURIJacksonModule; +import eu.f4sten.vulchainfinder.utils.DatabaseUtils; +import eu.f4sten.vulchainfinder.utils.RestAPIDependencyResolver; +import eu.fasten.core.data.callableindex.RocksDao; +import eu.fasten.core.maven.data.PomBuilder; +import eu.fasten.core.vulchains.VulnerableCallChainRepository; + +class MainTest { + + public static final String CI_URL = "/Users/mehdi/Desktop/MyMac/TUD/FASTEN/Repositories/MainRepo/fasten-docker-deployment/docker-volumes/fasten/java/callable-index"; + private static final String DB_URL = "jdbc:postgresql://localhost:5432/fasten_java"; + private static final String USR = "fasten"; + + public static final HttpClient HTTP_CLIENT = HttpClient.newBuilder().build(); + public static final String LOCAL_REST = "http://localhost:9080/"; + public static final Path VUL_REPO = Paths.get("src", "test", "resources", "vulnrepo"); + + @Test + void testExtractMavenIdFrom() { + final var pb = new PomBuilder(); + pb.groupId = "g"; + pb.artifactId = "g"; + pb.version = "v"; + + var actual = extractMavenIdFrom(pb.pom()); + final MavenId expected = new MavenId(); + expected.groupId = pb.groupId; + expected.artifactId = pb.artifactId; + expected.version = pb.version; + assertEquals(expected, actual); + } + + // TODO implement vulnerability inserter in the integration tests plugin and use + // that as a dependency to automate the vulnerability insertion + @Disabled("This is an integration test that checks if all the steps of vul-chain-finder work " + + "correctly. It is only for local development and debugging. It requires DC up and running" + + "with synthetic app:0.0.1 ingested, vulnerability inserted to wash method, and CI_URL " + + "constant available.") + @Test + void testProcess() throws RocksDBException, IOException, JSONException { + final var id = getMavenId("eu.fasten-project.tests.syntheticjars", "app", "0.0.1"); + final Main main = setUpMainFor(id); + + main.process(); + + final var actualStr = readResourceIntoString(createResourceNameFromID(id)); + final var expectedStr = readResourceIntoString("expected.json"); + + JSONAssert.assertEquals(expectedStr, actualStr, JSONCompareMode.LENIENT); + } + + private String createResourceNameFromID(final MavenId id) { + return id.groupId + "-" + id.artifactId + "-" + id.version + ".json"; + } + + private Main setUpMainFor(final MavenId id) throws RocksDBException, FileNotFoundException { + final var dbContext = DSL.using(DB_URL, USR, "fasten1234"); + final var om = new ObjectMapper().registerModule(new FastenURIJacksonModule()); + om.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + final var jsonUtils = new JsonUtilsImpl(om); + final var db = new DatabaseUtils(dbContext, jsonUtils); + final var ci = new RocksDao(CI_URL, true); + final var resolver = new RestAPIDependencyResolver(LOCAL_REST, HTTP_CLIENT); + final var repo = new VulnerableCallChainRepository(VUL_REPO.toString()); + final var main = new Main(db, ci, mock(Kafka.class), new VulChainFinderArgs(), mock(MessageGenerator.class), + resolver, repo); + main.setCurId(id); + return main; + } + + private String readResourceIntoString(final String name) throws IOException { + final var path = Paths.get(VUL_REPO.toAbsolutePath().toString(), name); + return Files.readString(path); + } + + private MavenId getMavenId(final String g, final String a, final String v) { + final var id = new MavenId(); + id.groupId = g; + id.artifactId = a; + id.version = v; + return id; + } +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/data/NodeReachabilityTest.java b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/data/NodeReachabilityTest.java new file mode 100644 index 00000000..1c6612e6 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/data/NodeReachabilityTest.java @@ -0,0 +1,144 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder.data; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.security.InvalidParameterException; +import java.util.List; +import org.junit.jupiter.api.Test; + +class NodeReachabilityTest { + + @Test + public void defaultValues() { + var sut = new NodeReachability(123); + assertEquals(123, sut.targetNode); + assertNotNull(sut.nextStepTowardsTarget); + assertTrue(sut.nextStepTowardsTarget.isEmpty()); + } + + @Test + public void equalityDefaults() { + var a = new NodeReachability(123); + var b = new NodeReachability(123); + assertEquals(a, b); + assertEquals(a.hashCode(), b.hashCode()); + } + + @Test + public void equalityWithContent() { + var a = new NodeReachability(123); + a.nextStepTowardsTarget.put(1L, 2L); + var b = new NodeReachability(123); + b.nextStepTowardsTarget.put(1L, 2L); + assertEquals(a, b); + assertEquals(a.hashCode(), b.hashCode()); + } + + @Test + public void equalityDifferentTargetNode() { + var a = new NodeReachability(123); + var b = new NodeReachability(234); + assertNotEquals(a, b); + assertNotEquals(a.hashCode(), b.hashCode()); + } + + @Test + public void equalityDifferentContent() { + var a = new NodeReachability(123); + a.nextStepTowardsTarget.put(1L, 2L); + var b = new NodeReachability(123); + assertNotEquals(a, b); + assertNotEquals(a.hashCode(), b.hashCode()); + } + + @Test + public void toStringIsImplemented() { + var actual = new NodeReachability(123).toString(); + assertTrue(actual.contains("\n")); + assertTrue(actual.contains("@")); + assertTrue(actual.contains(NodeReachability.class.getSimpleName())); + assertTrue(actual.contains("targetNode")); + } + + @Test + public void isReachingTargetSelf() { + var sut = new NodeReachability(123); + assertTrue(sut.isReachingTarget(123)); + } + + @Test + public void isReachingTargetNonExisting() { + var sut = new NodeReachability(1); + assertFalse(sut.isReachingTarget(2)); + } + + @Test + public void isReachingTargetDirectNeighbor() { + var sut = new NodeReachability(1); + sut.nextStepTowardsTarget.put(2L, 1L); + assertTrue(sut.isReachingTarget(2)); + } + + @Test + public void isReachingTargetTransitiveNeighbor() { + var sut = new NodeReachability(1); + sut.nextStepTowardsTarget.put(2L, 1L); + sut.nextStepTowardsTarget.put(3L, 2L); + assertTrue(sut.isReachingTarget(3)); + } + + @Test + public void getShortestPathSelf() { + var sut = new NodeReachability(1); + var actual = sut.getShortestPath(1); + var expected = List.of(1L); + assertEquals(expected, actual); + } + + @Test + public void failsWhenNoPathExists() { + assertThrows(InvalidParameterException.class, () -> { + new NodeReachability(1).getShortestPath(2); + }); + } + + @Test + public void getShortestPathDirectNeighbor() { + var sut = new NodeReachability(1); + sut.nextStepTowardsTarget.put(2L, 1L); + var actual = sut.getShortestPath(2); + var expected = List.of(2L, 1L); + assertEquals(expected, actual); + } + + @Test + public void getShortestPathTransitive() { + var sut = new NodeReachability(1); + sut.nextStepTowardsTarget.put(2L, 1L); + sut.nextStepTowardsTarget.put(3L, 2L); + var actual = sut.getShortestPath(3); + var expected = List.of(3L, 2L, 1L); + assertEquals(expected, actual); + } +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/json/FastenURIJacksonModuleTest.java b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/json/FastenURIJacksonModuleTest.java new file mode 100644 index 00000000..0008ebfb --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/json/FastenURIJacksonModuleTest.java @@ -0,0 +1,44 @@ +package eu.f4sten.vulchainfinder.json; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.fasten.core.data.FastenJavaURI; +import eu.fasten.core.data.FastenURI; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class FastenURIJacksonModuleTest { + + ObjectMapper om; + + @BeforeEach + void setUp() { + om = new ObjectMapper().registerModule(new FastenURIJacksonModule()); + } + + @Test + void testJavaUriRoundTrip() throws JsonProcessingException { + final var uriStr = + "fasten://mvn!eu.fasten-project.tests.syntheticjars:lib$0.0.1/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType"; + test(uriStr, FastenJavaURI.class); + } + + @Test + void testUriRoundTrip() throws JsonProcessingException { + final var uriStr = + "/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType"; + test(uriStr, FastenURI.class); + } + + private void test(final String uriStr, Class type) throws JsonProcessingException { + var first = FastenURI.create(uriStr); + final var middle = om.writeValueAsString(first); + final var end = om.readValue(middle, FastenURI.class); + assertInstanceOf(type, end); + assertEquals(end, first); + } + +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/DatabaseUtilsTest.java b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/DatabaseUtilsTest.java new file mode 100644 index 00000000..5e45dedb --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/DatabaseUtilsTest.java @@ -0,0 +1,257 @@ +/* + * Copyright 2021 Delft University of Technology + * + * 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 eu.f4sten.vulchainfinder.utils; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.f4sten.infra.impl.json.JsonUtilsImpl; +import eu.f4sten.infra.json.JsonUtils; +import eu.f4sten.infra.json.TRef; +import eu.f4sten.vulchainfinder.json.FastenURIJacksonModule; +import eu.fasten.core.data.FastenURI; +import eu.fasten.core.data.metadatadb.MetadataDao; +import eu.fasten.core.data.metadatadb.codegen.tables.Callables; +import eu.fasten.core.data.metadatadb.codegen.tables.Modules; +import eu.fasten.core.data.metadatadb.codegen.tables.PackageVersions; +import eu.fasten.core.data.metadatadb.codegen.tables.Packages; +import eu.fasten.core.data.vulnerability.Vulnerability; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.jooq.DSLContext; +import org.jooq.JSONB; +import org.jooq.Record; +import org.jooq.Result; +import org.jooq.SQLDialect; +import org.jooq.SelectConditionStep; +import org.jooq.SelectJoinStep; +import org.jooq.SelectSelectStep; +import org.jooq.impl.DSL; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +class DatabaseUtilsTest { + + public static final String SEE_WHY = + "This is an integration test. It requires Docker Compose up and running and the " + + "synthetic jars ingested, with inserted vulnerability in callable table."; + public static final String DB_URL = "jdbc:postgresql://localhost:5432/fasten_java"; + public static final String PG_PWD = System.getenv("PG_PWD"); + public static final String USR = "fasten"; + public static String VUL_RECORD; + public static DSLContext CONTEXT; + public static DatabaseUtils DB; + + public static Map> EXPECTED_VULNERABILITY; + + @BeforeEach + void setUp() throws JsonProcessingException { + CONTEXT = mock(DSLContext.class); + var om = new ObjectMapper(); + om.registerModule(new FastenURIJacksonModule()); + om.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + final var jutils = new JsonUtilsImpl(om); + DB = new DatabaseUtils(CONTEXT, jutils); + VUL_RECORD = " \"CVE-2017-3164\": {\n" + + " \"id\": \"CVE-2017-3164\",\n" + + " \"cwe_ids\": [\n" + + " \"CWE-918\"\n" + + " ],\n" + + " \"severity\": \"MEDIUM\",\n" + + " \"patch_date\": \"2019-01-15\",\n" + + " \"scoreCVSS2\": 5,\n" + + " \"scoreCVSS3\": 7.5,\n" + + " \"description\": \"Server Side Request Forgery in Apache Solr, versions 1.3 until 7.6 (inclusive). Since the \\\"shards\\\" parameter does not have a corresponding whitelist mechanism, a remote attacker with access to the server could make Solr perform an HTTP GET request to any reachable URL.\",\n" + + " \"vectorCVSS2\": \"AV:N/AC:L/Au:N/C:P/I:N/A:N\",\n" + + " \"vectorCVSS3\": \"CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:H/I:N/A:N\",\n" + + " \"published_date\": \"2019-03-08\",\n" + + " \"last_modified_date\": \"2020-12-09\"\n" + + " }\n"; + + var vulString = "{\n" + + " \"vulnerabilities\": {\n" + + VUL_RECORD + + " }\n" + + "}"; + + final var setType = new TRef>>() { + }; + + HashMap> VUL_ID_VUL_OBJECT_MAP = + om.readValue(vulString, setType); + + EXPECTED_VULNERABILITY = Map.of(FastenURI.create( + "fasten://mvn!eu.fasten-project.tests.syntheticjars:lib$0.0.1/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType"), + List.of(VUL_ID_VUL_OBJECT_MAP.get("vulnerabilities").get("CVE-2017-3164"))); + + } + + @Test + void testCreateFastenUri() { + final var mockRecord = mock(Record.class); + when(mockRecord.get(0)).thenReturn("eu.fasten-project.tests.syntheticjars:lib"); + when(mockRecord.get(1)).thenReturn("0.0.1"); + when(mockRecord.get(2)).thenReturn( + "/lib/BasicMotorVehicle.addDirt()%2Fjava.lang%2FVoidType"); + final var actual = DatabaseUtils.createFastenUriFromPckgVersionUriFields(mockRecord); + assertEquals(FastenURI.create("fasten://mvn!eu.fasten-project.tests.syntheticjars:lib$0.0" + + ".1/lib/BasicMotorVehicle.addDirt()%2Fjava.lang%2FVoidType"), actual); + } + + @Test + void testSelectVulCallablesWhereModuleIdIs() { + final var actual = DatabaseUtils.createStrForSelectVulCallablesWhereModuleIdIs(2L); + assertTrue(actual.endsWith("and callables.module_id = 2")); + } + + @Disabled(SEE_WHY) + @Test + void testIntegrationForSelectAllModulesOf() { + var actual = getDb().selectAllModulesOf(1); + assertEquals(Set.of(1L, 2L, 3L, 4L), actual); + } + + @Disabled(SEE_WHY) + @Test + void testSelectConcurrentlyVulCallablesOf() { + final var actual = getDb().selectConcurrentlyVulCallablesOf(Set.of(6L)); + assertEquals(EXPECTED_VULNERABILITY, actual); + } + + @Disabled(SEE_WHY) + @Test + void testSelectVulCallablesOfIn() { + final var actual = getDb().selectVulCallablesOf(2); + assertEquals(EXPECTED_VULNERABILITY, actual); + } + + @Disabled(SEE_WHY) + @Test + void testSelectVulCallablesOfWithSet() { + final var actual = getDb().selectVulCallablesOf(Set.of(2L)); + assertEquals(EXPECTED_VULNERABILITY, actual); + } + + @Test + void testSelectVulnerablePackagesExistingIn() { + var dao = mock(MetadataDao.class); + var json = mock(JsonUtils.class); + var dslContext = mock(DSLContext.class); + var db = new DatabaseUtils(dslContext, json) { + public MetadataDao getDao(DSLContext ctx) { + return dao; + } + }; + final var depIds = Set.of(1L, 2L); + db.selectVulnerablePackagesExistingIn(depIds); + + verify(dao).findVulnerablePackageVersions(depIds); + } + + @Test + void testSelectAllModulesOf(){ + final var expected = Set.of(1L, 2L, 3L); + setUpModuleSelection(expected); + + assertEquals(expected, DB.selectAllModulesOf(1)); + } + + @Test + void testSelectVulCallablesOfWithNullVulField(){ + setUpModuleSelection(Set.of(1L)); + setUpCallableSelection(null); + + final var actual = DB.selectVulCallablesOf(Set.of(1L)); + + assertEquals(Collections.emptyMap(), actual); + } + + @Test + void testSelectVulCallablesOfWithModuleEmpty(){ + setUpModuleSelection(Set.of()); + setUpCallableSelection(null); + + final var actual = DB.selectVulCallablesOf(Set.of(1L)); + + assertEquals(Collections.emptyMap(), actual); + } + + @Test + void testSelectVulCallablesOfWithModuleNull(){ + setUpModuleSelection(null); + setUpCallableSelection(null); + + final var actual = DB.selectVulCallablesOf(Set.of(1L)); + + assertEquals(Collections.emptyMap(), actual); + } + + @Test + void testSelectVulCallablesOf(){ + setUpModuleSelection(Set.of(1L)); + setUpCallableSelection(JSONB.valueOf("{" + VUL_RECORD + "}")); + + final var actual = DB.selectVulCallablesOf(Set.of(1L)); + + assertEquals(EXPECTED_VULNERABILITY, actual); + } + + private void setUpModuleSelection(Set expected) { + final var select = mock(SelectSelectStep.class); + final var selectJoin = mock(SelectJoinStep.class); + final var selectCondition = mock(SelectConditionStep.class); + final var result = mock(Result.class); + when(select.from(Modules.MODULES)).thenReturn(selectJoin); + when(CONTEXT.select(Modules.MODULES.ID)).thenReturn(select); + when(selectJoin.where(Modules.MODULES.PACKAGE_VERSION_ID.eq(1L))).thenReturn(selectCondition); + when(selectCondition.fetch()).thenReturn(result); + when(result.intoSet(Modules.MODULES.ID)).thenReturn(expected); + } + + private void setUpCallableSelection(final JSONB vul) { + DSLContext create = DSL.using(SQLDialect.DEFAULT); + final var pn = Packages.PACKAGES.PACKAGE_NAME; + final var v = PackageVersions.PACKAGE_VERSIONS.VERSION; + final var uri = Callables.CALLABLES.FASTEN_URI; + final var result = create.newResult(); + final var record = + create.newRecord(pn, v, uri, Callables.CALLABLES.METADATA).values( + "eu.fasten-project.tests.syntheticjars:lib", + "0.0.1", "/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType", vul); + result.add(record); + when(CONTEXT.fetch(anyString())).thenReturn(result); + } + + private DatabaseUtils getDb() { + final var om = new ObjectMapper().registerModule(new FastenURIJacksonModule()); + om.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + final var jsonUtils = new JsonUtilsImpl(om); + return new DatabaseUtils(DSL.using(DB_URL, USR, PG_PWD), jsonUtils); + } + +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/ImpactPropagatorTest.java b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/ImpactPropagatorTest.java new file mode 100644 index 00000000..5b5ceacd --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/ImpactPropagatorTest.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 eu.f4sten.vulchainfinder.utils; + +import static eu.fasten.analyzer.javacgopal.data.CGAlgorithm.RTA; +import static eu.fasten.analyzer.javacgopal.data.CallPreservationStrategy.ONLY_STATIC_CALLSITES; +import static eu.fasten.core.utils.TestUtils.getTestResource; + +import com.google.common.collect.HashBiMap; +import eu.f4sten.pomanalyzer.data.MavenId; +import eu.f4sten.vulchainfinder.data.NodeReachability; +import eu.fasten.analyzer.javacgopal.data.OPALCallGraphConstructor; +import eu.fasten.analyzer.javacgopal.data.OPALPartialCallGraphConstructor; +import eu.fasten.core.data.FastenURI; +import eu.fasten.core.data.MergedDirectedGraph; +import eu.fasten.core.data.PartialJavaCallGraph; +import eu.fasten.core.data.vulnerability.Vulnerability; +import eu.fasten.core.merge.CGMerger; +import eu.fasten.core.vulchains.VulnerableCallChain; +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +class ImpactPropagatorTest { + + @Test + void testPropagateUrisImpacts() { + + // ┌─────────┐ + // │ │ + // * v2 * * ▼ + // 4◄────3───►2◄───5───►6───►7* + // │ │ │ + // ▼ ▼ │ + // 8 1◄───┘ + // v1 + final ImpactPropagator propagator = propagageSyntheticGraph( + Set.of(FastenURI.create("v1"), FastenURI.create("v2")), + HashBiMap.create(Map.of(1L, "v1", 2L, "v2"))); + + Assertions.assertEquals(getExpected(), propagator.getImpacts()); + } + + private ImpactPropagator propagageSyntheticGraph(Set vulUris, + HashBiMap uris) { + final var dg = createDg(); + final var propagator = new ImpactPropagator(dg, uris); + + propagator.propagateUrisImpacts(vulUris); + return propagator; + } + + @Test + void testExtractAppVulChains(){ + final var vulUriStr = "fasten://mvn!g1:a1$1/name.space/V1.v1()%2Fjava.lang%2FVoidType"; + final var vulUri = FastenURI.create(vulUriStr); + final var appUriStr = "fasten://mvn!g:a$1/name.space/V6.v6()%2Fjava.lang%2FVoidType"; + final var appUri = FastenURI.create(appUriStr); + final ImpactPropagator propagator = propagageSyntheticGraph( + Set.of(vulUri), HashBiMap.create(Map.of(1L, vulUriStr, 6L, appUriStr))); + final var vulList = List.of(new Vulnerability()); + final var vulCallables = Map.of(vulUri, vulList); + final var id = new MavenId(); + id.groupId = "g"; + id.artifactId = "a"; + id.version = "1"; + final var actual = + propagator.extractApplicationVulChains(vulCallables, id); + final var expected = Set.of(new VulnerableCallChain(vulList, List.of(appUri, vulUri))); + Assertions.assertEquals(expected, actual); + } + + private Set getExpected() { + final var steps = Map.of( + 3L, 2L, + 5L, 2L + ); + final var nr1 = new NodeReachability(2L); + nr1.nextStepTowardsTarget.putAll(steps); + final var steps2 = Map.of( + 5L, 6L, + 6L, 1L, + 7L, 1L + ); + final var nr2 = new NodeReachability(1L); + nr2.nextStepTowardsTarget.putAll(steps2); + return Set.of(nr1, nr2); + } + + private static MergedDirectedGraph createDg() { + MergedDirectedGraph dg = new MergedDirectedGraph(); + addEdge(dg, 3, 4); + addEdge(dg, 3, 2); + addEdge(dg, 3, 3); + addEdge(dg, 2, 8); + addEdge(dg, 5, 2); + addEdge(dg, 5, 6); + addEdge(dg, 5, 7); + addEdge(dg, 6, 1); + addEdge(dg, 6, 7); + addEdge(dg, 7, 1); + // Self call to increase the coverage + addEdge(dg, 7,7); + return dg; + } + + @Disabled("This test is slow.") + @Test + void testARealWorldExample() { + final var propagator = instantiatePropagatorFromResources(); + final var vulUri = vulUri(); + final var appUri = appUri(); + final var expected = expected(); + final var vulCallables = Map.of(vulUri, List.of(new Vulnerability())); + + propagator.propagateUrisImpacts(vulCallables.keySet()); + final var vulChains = + propagator.extractApplicationVulChains(vulCallables, getMavenId()); + + final var actual = FindFirstTargetNodeInVulChain(vulChains, appUri, vulUri); + Assertions.assertEquals(expected, actual); + } + + private MavenId getMavenId() { + final var id = new MavenId(); + id.groupId = "com.alibaba.middleware"; + id.artifactId = "termd-core"; + id.version = "1.1.7.13-SNAPSHOT"; + return id; + } + + private static ImpactPropagator instantiatePropagatorFromResources() { + var deps = List.of( + generateCG(getTestResource("termd-core-1.1.7.13-SNAPSHOT.jar"), + "com.alibaba.middleware:termd-core", "1.1.7.13-SNAPSHOT"), + generateCG(getTestResource("jackson-databind-2.7.9.4.jar"), + "com.fasterxml.jackson.core:jackson-databind", "2.7.9.4")); + + final var merger = new CGMerger(deps); + final var mergedCG = merger.mergeAllDeps(); + final var uris = merger.getAllUris(); + return new ImpactPropagator(mergedCG, uris); + } + + private static FastenURI expected() { + return FastenURI.create("fasten://mvn!com.fasterxml.jackson" + + ".core:jackson-databind$2.7.9" + + ".4/com.fasterxml.jackson.databind/ObjectMapper.readValue(%2Fjava" + + ".lang%2FString,%2Fjava.lang%2FClass)%2Fjava.lang%2FObject"); + } + + private static FastenURI appUri() { + return FastenURI.create("fasten://mvn!com.alibaba.middleware:termd-core$1.1.7" + + ".13-SNAPSHOT/io.termd.core.http/HttpTtyConnection.writeToDecoder" + + "(%2Fjava.lang%2FString)%2Fjava.lang%2FVoidType"); + } + + private static FastenURI vulUri() { + return FastenURI.create( + "fasten://mvn!com.fasterxml.jackson.core:jackson-databind$2.7.9.4/" + + "com.fasterxml.jackson.databind.deser/BasicDeserializerFactory.createMapDeserializer" + + "(%2Fcom.fasterxml.jackson.databind%2FDeserializationContext," + + "%2Fcom.fasterxml.jackson.databind.type%2FMapType," + + "%2Fcom.fasterxml.jackson.databind%2FBeanDescription)" + + "%2Fcom.fasterxml.jackson.databind%2FJsonDeserializer"); + } + + private static void addEdge(MergedDirectedGraph dg, long source, long target) { + dg.addVertex(source); + dg.addVertex(target); + dg.addEdge(source, target); + } + + private FastenURI FindFirstTargetNodeInVulChain(final Set vulChains, + final FastenURI appUri, + final FastenURI vulUri) { + FastenURI result = null; + for (final var vulChain : vulChains) { + var chain = vulChain.getChain(); + if (chain.get(0).equals(appUri)) { + if (chain.get(chain.size() - 1).equals(vulUri)) { + result = chain.get(1); + } + } + } + return result; + } + + private static PartialJavaCallGraph generateCG(final File file, final String product, + final String version) { + final var opalCG = new OPALCallGraphConstructor().construct(file, RTA); + final var cg = + new OPALPartialCallGraphConstructor().construct(opalCG, ONLY_STATIC_CALLSITES); + + return new PartialJavaCallGraph("mvn", product, version, + -1, "opal", cg.classHierarchy, cg.graph); + } + +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/RestAPIDependencyResolverTest.java b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/RestAPIDependencyResolverTest.java new file mode 100644 index 00000000..2c31b44f --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/test/java/eu/f4sten/vulchainfinder/utils/RestAPIDependencyResolverTest.java @@ -0,0 +1,199 @@ +package eu.f4sten.vulchainfinder.utils; + +import static org.apache.http.HttpStatus.SC_OK; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.util.Set; + +import org.json.JSONArray; +import org.json.JSONException; +import org.json.JSONObject; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import eu.f4sten.pomanalyzer.data.MavenId; +import eu.fasten.core.data.metadatadb.codegen.tables.Dependencies; + +class RestAPIDependencyResolverTest { + + private static final HttpResponse SOME_RESPONSE = mock(HttpResponse.class); + private static final HttpRequest SOME_REQUEST = mock(HttpRequest.class); + public static String BASE_URL; + public static JSONObject DEP_JSON; + public static JSONArray DEPS_JSON_ARRAY; + public static MavenId ID; + public static HttpResponse MOCK_RESPONSE; + public static HttpClient MOCK_CLIENT; + public static String EXPECTED_URI_STRING; + public static String ENDPOINT_WITH_SLASH; + public static String ENDPOINT_WITHOUT_SLASH; + public static RestAPIDependencyResolver RESOLVER_WITHOUT_SLASH; + public static RestAPIDependencyResolver RESOLVER_WITH_SLASH; + + @BeforeEach + void setUp() throws JSONException { + BASE_URL = "https://api.fasten-project.eu"; + DEP_JSON = new JSONObject(" {\n" + " \"package_version_id\": 1,\n" + " \"dependency_id\": 2" + " }"); + DEPS_JSON_ARRAY = new JSONArray(); + DEPS_JSON_ARRAY.put(DEP_JSON); + + ID = new MavenId(); + ID.groupId = "org.apache.solr"; + ID.artifactId = "solr-core"; + ID.version = "6.6.1"; + + MOCK_CLIENT = mock(HttpClient.class); + MOCK_RESPONSE = mock(HttpResponse.class); + + // TODO break this up into multiple test cases + + ENDPOINT_WITHOUT_SLASH = "api/mvn/packages/org.apache.solr:solr-core/6.6.1/deps"; + ENDPOINT_WITH_SLASH = "/api/mvn/packages/org.apache.solr:solr-core/6.6.1/deps"; + RESOLVER_WITHOUT_SLASH = new RestAPIDependencyResolver(BASE_URL, MOCK_CLIENT); + RESOLVER_WITH_SLASH = new RestAPIDependencyResolver(BASE_URL, MOCK_CLIENT); + + // TODO DO NOT call sut methods in your setup... your are supposed to test + // these! + EXPECTED_URI_STRING = RESOLVER_WITHOUT_SLASH.getRestAPIBaseURL() + ENDPOINT_WITH_SLASH; + } + + @Disabled("Depends on the Rest Api and current state of the DB." + "Run while development and adjust accordingly") + @Test + void resolveServer() { + var resolver = new RestAPIDependencyResolver(BASE_URL, HttpClient.newBuilder().build()); + var actual = resolver.resolveDependencyIds(ID); + assertEquals(Set.of(4L, 8L, 42L, 53L, 55L, 56L, 59L, 70L, 71L, 159L, 10479L), actual); + } + + @Disabled + @Test + void testResolveDependencyGavs() { + var resolver = new RestAPIDependencyResolver(BASE_URL, HttpClient.newBuilder().build()); + var mavenId = new MavenId(); + mavenId.groupId = "org.springframework.hateoas"; + mavenId.artifactId = "org.springframework.hateoas:spring-hateoas"; + mavenId.version = "0.19.0.RELEASE"; + resolver.resolveDependencyGavs(mavenId); + + } + + @Disabled("Works with Docker Compose, only when synthetic jar app is inserted!" + + "Run while development and adjust accordingly") + @Test + void resolveLocal() { + var resolver = new RestAPIDependencyResolver("http://localhost:9080", HttpClient.newBuilder().build()); + var id = new MavenId(); + id.groupId = "eu.fasten-project.tests.syntheticjars"; + id.artifactId = "app"; + id.version = "0.0.1"; + var actual = resolver.resolveDependencyIds(id); + assertEquals(Set.of(2L, 1L), actual); + } + + @Test + void testExtractPackageIdsFromResponse() { + when(MOCK_RESPONSE.body()).thenReturn(DEPS_JSON_ARRAY.toString()); + var actual = RESOLVER_WITH_SLASH.extractPackageIdsFromResponse(MOCK_RESPONSE); + assertEquals(Set.of(2L), actual); + } + + @Test + void testExtractLongFieldFromJSONObj() { + final var actual = RESOLVER_WITH_SLASH.extractLongFieldFromJSONObj(DEP_JSON, + Dependencies.DEPENDENCIES.DEPENDENCY_ID.getName()); + assertEquals(2L, actual); + } + + @Test + void testIsOk() { + when(MOCK_RESPONSE.statusCode()).thenReturn(200); + assertFalse(RestAPIDependencyResolver.isNotOK(MOCK_RESPONSE)); + } + + @Test + void testIsNotOK() { + when(MOCK_RESPONSE.statusCode()).thenReturn(400); + assertTrue(RestAPIDependencyResolver.isNotOK(MOCK_RESPONSE)); + } + + @Test + @SuppressWarnings("unchecked") + void testSendOrThrowCallsSend() throws IOException, InterruptedException { + when(SOME_RESPONSE.statusCode()).thenReturn(SC_OK); + // TODO Urgently clarify use of matchers! + when(MOCK_CLIENT.send(any(), any())).thenReturn(SOME_RESPONSE); + RESOLVER_WITH_SLASH.sendOrThrow(SOME_REQUEST); + verify(MOCK_CLIENT, times(1)).send(any(), any()); + } + + @Test + void placeIDInEndpoint() { + var actual = RESOLVER_WITH_SLASH.placeIDInEndpoint(ENDPOINT_WITH_SLASH, ID); + assertEquals(ENDPOINT_WITH_SLASH, actual); + } + + @Test + void testReturnFullUri() throws URISyntaxException { + var actual = RESOLVER_WITH_SLASH.returnFullUriOrThrow(ENDPOINT_WITH_SLASH); + assertEquals(actual, new URI(EXPECTED_URI_STRING)); + } + + @Test + void testThrowsURISyntaxException() { + Exception exception = assertThrows(RuntimeException.class, () -> { + RESOLVER_WITH_SLASH.returnFullUriOrThrow(ENDPOINT_WITH_SLASH + "[]"); + }); + + var expectedMessage = "Illegal character in path at index 83"; + var actualMessage = exception.getMessage(); + + assertTrue(actualMessage.contains(expectedMessage)); + assertTrue(exception.getCause() instanceof URISyntaxException); + } + + @Test + void testThrowNullPointerException() { + assertThrows(NullPointerException.class, () -> { + RESOLVER_WITH_SLASH.returnFullUriOrThrow(null); + }); + } + + @Test + void testMakeCorrectFullURI1() { + final var actual = RESOLVER_WITHOUT_SLASH.makeCorrectFullURI(ENDPOINT_WITH_SLASH); + assertEquals(EXPECTED_URI_STRING, actual); + } + + @Test + void testMakeCorrectFullURI2() { + final var actual = RESOLVER_WITHOUT_SLASH.makeCorrectFullURI(ENDPOINT_WITHOUT_SLASH); + assertEquals(EXPECTED_URI_STRING, actual); + } + + @Test + void testMakeCorrectFullURI3() { + final var actual = RESOLVER_WITH_SLASH.makeCorrectFullURI(ENDPOINT_WITH_SLASH); + assertEquals(EXPECTED_URI_STRING, actual); + } + + @Test + void testMakeCorrectFullURI4() { + final var actual = RESOLVER_WITH_SLASH.makeCorrectFullURI(ENDPOINT_WITHOUT_SLASH); + assertEquals(EXPECTED_URI_STRING, actual); + } +} \ No newline at end of file diff --git a/plugins/vulnerable-chain-finder/src/test/resources/jackson-databind-2.7.9.4.jar b/plugins/vulnerable-chain-finder/src/test/resources/jackson-databind-2.7.9.4.jar new file mode 100644 index 00000000..a4ec29ec Binary files /dev/null and b/plugins/vulnerable-chain-finder/src/test/resources/jackson-databind-2.7.9.4.jar differ diff --git a/plugins/vulnerable-chain-finder/src/test/resources/termd-core-1.1.7.13-SNAPSHOT.jar b/plugins/vulnerable-chain-finder/src/test/resources/termd-core-1.1.7.13-SNAPSHOT.jar new file mode 100644 index 00000000..ceaa54b5 Binary files /dev/null and b/plugins/vulnerable-chain-finder/src/test/resources/termd-core-1.1.7.13-SNAPSHOT.jar differ diff --git a/plugins/vulnerable-chain-finder/src/test/resources/vulnrepo/expected.json b/plugins/vulnerable-chain-finder/src/test/resources/vulnrepo/expected.json new file mode 100644 index 00000000..dab1ed35 --- /dev/null +++ b/plugins/vulnerable-chain-finder/src/test/resources/vulnrepo/expected.json @@ -0,0 +1,98 @@ +[ + { + "vulnerabilities": [ + { + "id": "CVE-2017-3164", + "purls": [], + "first_patched_purls": [], + "scoreCVSS2": 5.0, + "scoreCVSS3": 7.5, + "vectorCVSS2": "AV:N/AC:L/Au:N/C:P/I:N/A:N", + "vectorCVSS3": "CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:H/I:N/A:N", + "severity": "MEDIUM", + "description": "Server Side Request Forgery in Apache Solr, versions 1.3 until 7.6 (inclusive). Since the \"shards\" parameter does not have a corresponding whitelist mechanism, a remote attacker with access to the server could make Solr perform an HTTP GET request to any reachable URL.", + "references": [], + "patch_links": [], + "exploits": [], + "patches": [] + } + ], + "chain": [ + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/null/Main.doCarStuff()%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/app/RoadTrip.doRoadTrip(%2Flib%2FMotorVehicle)%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:lib$0.0.1/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType" + ] + }, + { + "vulnerabilities": [ + { + "id": "CVE-2017-3164", + "purls": [], + "first_patched_purls": [], + "scoreCVSS2": 5.0, + "scoreCVSS3": 7.5, + "vectorCVSS2": "AV:N/AC:L/Au:N/C:P/I:N/A:N", + "vectorCVSS3": "CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:H/I:N/A:N", + "severity": "MEDIUM", + "description": "Server Side Request Forgery in Apache Solr, versions 1.3 until 7.6 (inclusive). Since the \"shards\" parameter does not have a corresponding whitelist mechanism, a remote attacker with access to the server could make Solr perform an HTTP GET request to any reachable URL.", + "references": [], + "patch_links": [], + "exploits": [], + "patches": [] + } + ], + "chain": [ + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/null/Main.main(%2Fjava.lang%2FString%5B%5D)%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/null/Main.doMotorCycleStuff()%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/app/RoadTrip.doRoadTrip(%2Flib%2FMotorVehicle)%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:lib$0.0.1/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType" + ] + }, + { + "vulnerabilities": [ + { + "id": "CVE-2017-3164", + "purls": [], + "first_patched_purls": [], + "scoreCVSS2": 5.0, + "scoreCVSS3": 7.5, + "vectorCVSS2": "AV:N/AC:L/Au:N/C:P/I:N/A:N", + "vectorCVSS3": "CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:H/I:N/A:N", + "severity": "MEDIUM", + "description": "Server Side Request Forgery in Apache Solr, versions 1.3 until 7.6 (inclusive). Since the \"shards\" parameter does not have a corresponding whitelist mechanism, a remote attacker with access to the server could make Solr perform an HTTP GET request to any reachable URL.", + "references": [], + "patch_links": [], + "exploits": [], + "patches": [] + } + ], + "chain": [ + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/null/Main.doMotorCycleStuff()%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/app/RoadTrip.doRoadTrip(%2Flib%2FMotorVehicle)%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:lib$0.0.1/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType" + ] + }, + { + "vulnerabilities": [ + { + "id": "CVE-2017-3164", + "purls": [], + "first_patched_purls": [], + "scoreCVSS2": 5.0, + "scoreCVSS3": 7.5, + "vectorCVSS2": "AV:N/AC:L/Au:N/C:P/I:N/A:N", + "vectorCVSS3": "CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:H/I:N/A:N", + "severity": "MEDIUM", + "description": "Server Side Request Forgery in Apache Solr, versions 1.3 until 7.6 (inclusive). Since the \"shards\" parameter does not have a corresponding whitelist mechanism, a remote attacker with access to the server could make Solr perform an HTTP GET request to any reachable URL.", + "references": [], + "patch_links": [], + "exploits": [], + "patches": [] + } + ], + "chain": [ + "fasten://mvn!eu.fasten-project.tests.syntheticjars:app$0.0.1/app/RoadTrip.doRoadTrip(%2Flib%2FMotorVehicle)%2Fjava.lang%2FVoidType", + "fasten://mvn!eu.fasten-project.tests.syntheticjars:lib$0.0.1/lib/VehicleWash.wash(MotorVehicle)%2Fjava.lang%2FVoidType" + ] + } +] \ No newline at end of file diff --git a/pom.xml b/pom.xml index 28bfa699..4c90cb7b 100644 --- a/pom.xml +++ b/pom.xml @@ -1,5 +1,4 @@ - + 4.0.0 @@ -60,6 +59,12 @@ jsonassert 1.5.0 test + + + com.vaadin.external.google + android-json + + com.github.stefanbirkner diff --git a/releng/pom.xml b/releng/pom.xml index f4846f70..44816a41 100644 --- a/releng/pom.xml +++ b/releng/pom.xml @@ -65,8 +65,8 @@ 3.8.0 11 - -g:vars - -g:vars + +