diff --git a/dump-schema/src/main/java/eu/dnetlib/dhp/skgif/model/MinProduct.java b/dump-schema/src/main/java/eu/dnetlib/dhp/skgif/model/MinProduct.java index 7126918..3884629 100644 --- a/dump-schema/src/main/java/eu/dnetlib/dhp/skgif/model/MinProduct.java +++ b/dump-schema/src/main/java/eu/dnetlib/dhp/skgif/model/MinProduct.java @@ -13,6 +13,15 @@ public class MinProduct implements Serializable { private String doi; private String pmcid; private String arxivid; + private String pmid; + + public String getPmid() { + return pmid; + } + + public void setPmid(String pmid) { + this.pmid = pmid; + } public String getLocal_identifier() { return local_identifier; diff --git a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/filterentities/SelectConnectedEntities.java b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/filterentities/SelectConnectedEntities.java index 0de1541..efc6cf2 100644 --- a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/filterentities/SelectConnectedEntities.java +++ b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/filterentities/SelectConnectedEntities.java @@ -17,6 +17,8 @@ import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -103,63 +105,56 @@ public class SelectConnectedEntities implements Serializable { Utils .getFunderName(p.getFundingtree().get(0).getValue()) .equalsIgnoreCase("European Commission")); + Dataset datasources = Utils .readPath(spark, inputPath + "datasource", Datasource.class) .filter((FilterFunction) d -> !d.getDataInfo().getDeletedbyinference()); // select relations having source in the set of identifiers selected for eosc - Dataset resultSource = resultIds - .joinWith(relation, resultIds.col("value").equalTo(relation.col("source"))) - .map((MapFunction, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class)); - - // write relations having source and target in the set - resultIds - .joinWith(resultSource, resultIds.col("value").equalTo(resultSource.col("target"))) - .map((MapFunction, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class)) + Dataset relationSource = relation + .join(resultIds, resultIds.col("value").equalTo(relation.col("source")), "left_semi"); + relationSource + .join(resultIds, resultIds.col("value").equalTo(relation.col("target")), "left_semi") .write() .option("compression", "gzip") .mode(SaveMode.Overwrite) - .json(workingDir + "relation"); - - // write relations between results and organizations - resultSource - .joinWith(organizations, resultSource.col("target").equalTo(organizations.col("id"))) - .map((MapFunction, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class)) + .json(workingDir + "resultrelation"); +// +// // write relations between results and organizations + relationSource + .joinWith(organizations, relation.col("target").equalTo(organizations.col("id")), "left_semi") .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(workingDir + "relation"); + .json(workingDir + "organizaitonrelation"); - resultSource - .joinWith(projects, resultSource.col("target").equalTo(projects.col("id"))) - .map((MapFunction, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class)) + relationSource + .joinWith(projects, relation.col("target").equalTo(projects.col("id")), "left_semi") .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(workingDir + "relation"); + .json(workingDir + "projectrelation"); // write organizations linked to results in the set - resultSource - .joinWith(organizations, resultSource.col("target").equalTo(organizations.col("id"))) - .map( - (MapFunction, Organization>) t2 -> t2._2(), - Encoders.bean(Organization.class)) + + organizations + .join(relationSource, relationSource.col("target").equalTo(organizations.col("id")), "left_semi") .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + "organization"); // write projects linked to results in the set - resultSource - .joinWith(projects, resultSource.col("target").equalTo(projects.col("id"))) - .map((MapFunction, Project>) t2 -> t2._2(), Encoders.bean(Project.class)) + projects + .join(relationSource, relationSource.col("target").equalTo(projects.col("id"))) .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + "project"); // read the results and select all the distinct instance.hostedbykey Dataset datasourceReferencedIds = spark.emptyDataset(Encoders.STRING()); + for (EntityType entity : ModelSupport.entityTypes.keySet()) if (ModelSupport.isResult(entity)) { Class resultClazz = ModelSupport.entityTypes.get(entity); @@ -171,45 +166,88 @@ public class SelectConnectedEntities implements Serializable { (FlatMapFunction) r -> r .getInstance() .stream() - .flatMap(i -> Stream.of(i.getHostedby().getKey(), i.getCollectedfrom().getKey())) + .filter(i -> i.getHostedby() != null && i.getHostedby().getKey() != null) + .map(i -> i.getHostedby().getKey()) + .collect(Collectors.toList()) + .iterator(), + Encoders.STRING())); + datasourceReferencedIds = datasourceReferencedIds + .union( + Utils + .readPath(spark, workingDir + entity.name(), resultClazz) + .flatMap( + (FlatMapFunction) r -> r + .getInstance() + .stream() + .filter(i -> i.getCollectedfrom() != null && i.getCollectedfrom().getKey() != null) + .map(i -> i.getCollectedfrom().getKey()) .collect(Collectors.toList()) .iterator(), Encoders.STRING())); } datasourceReferencedIds = datasourceReferencedIds.distinct(); // join with the datasources and write the datasource in the join - datasourceReferencedIds - .joinWith(datasources, datasourceReferencedIds.col("value").equalTo(datasources.col("id"))) - .map((MapFunction, Datasource>) t2 -> t2._2(), Encoders.bean(Datasource.class)) + datasources + .joinWith( + datasourceReferencedIds, datasourceReferencedIds.col("value").equalTo(datasources.col("id")), + "left_semi") .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + "datasource"); // selecting relations between organizations and projects in the selected set - Dataset organizationSbs = Utils.readPath(spark, workingDir + "organization", Organization.class); - Dataset projectSbs = Utils.readPath(spark, workingDir + "project", Project.class); - Dataset orgSourceRels = organizationSbs - .joinWith(relation, organizationSbs.col("id").equalTo(relation.col("source"))) - .map((MapFunction, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class)); - orgSourceRels - .joinWith(projectSbs, orgSourceRels.col("target").equalTo(projectSbs.col("id"))) - .map((MapFunction, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class)) + StructType tp = StructType.fromDDL("`id` STRING"); + Dataset organizationSbs = spark + .read() + .schema(tp) + .json(workingDir + "organization") + .select("id"); + + Dataset projectSbs = spark + .read() + .schema(tp) + .json(workingDir + "project") + .select("id"); +// + Dataset tmpRel; + tmpRel = relation + .join( + organizationSbs, organizationSbs + .col("id") + .equalTo(relation.col("source")), + "left_semi"); + tmpRel + .join(projectSbs, tmpRel.col("target").equalTo(projectSbs.col("id")), "left_semi") .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(workingDir + "relation"); + .json(workingDir + "orgprojelation"); // selecting relations between datasources and organizations in the selected set - Dataset datasourceSbs = Utils.readPath(spark, workingDir + "datasource", Datasource.class); - Dataset dsSourceRels = datasourceSbs - .joinWith(relation, datasourceSbs.col("id").as("dsId").equalTo(relation.col("source"))) - .map((MapFunction, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class)); - dsSourceRels - .joinWith(organizationSbs, dsSourceRels.col("target").equalTo(organizations.col("id").as("orgId"))) - .map((MapFunction, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class)) + Dataset datasourceSbs = spark + .read() + .schema(tp) + .json(workingDir + "datasource") + .select("id"); + + tmpRel = relation + .join(datasourceSbs, datasourceSbs.col("id").equalTo(relation.col("source")), "left_semi"); + tmpRel + .join(organizationSbs, tmpRel.col("target").equalTo(organizationSbs.col("id")), "left_semi") .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(workingDir + "datsorgrelation"); + + Utils + .readPath(spark, workingDir + "resultrelation", Relation.class) + .union(Utils.readPath(spark, workingDir + "organizaitonrelation", Relation.class)) + .union(Utils.readPath(spark, workingDir + "projectrelation", Relation.class)) + .union(Utils.readPath(spark, workingDir + "orgprojelation", Relation.class)) + .union(Utils.readPath(spark, workingDir + "datsorgrelation", Relation.class)) + .write() + .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + "relation"); diff --git a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResult.java b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResult.java index b0f862e..fea0a48 100644 --- a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResult.java +++ b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResult.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.oa.graph.dump.skgif; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static eu.dnetlib.dhp.oa.graph.dump.skgif.ResultMapper.map; import java.io.Serializable; import java.util.*; @@ -9,12 +10,15 @@ import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapGroupsFunction; +import org.apache.spark.rdd.RDD; import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; -import org.jetbrains.annotations.NotNull; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,10 +28,8 @@ import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.Datasource; -import eu.dnetlib.dhp.schema.oaf.Organization; import eu.dnetlib.dhp.skgif.model.*; import eu.dnetlib.dhp.skgif.model.AccessRight; -import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; /** @@ -323,7 +325,7 @@ public class DumpResult implements Serializable { results .joinWith(prr, results.col("id").equalTo(prr.col("resultId")), "left") .map((MapFunction, ResearchProduct>) t2 -> { - ResearchProduct rp = ResultMapper.map(t2._1()); + ResearchProduct rp = map(t2._1()); if (Optional.ofNullable(t2._2()).isPresent()) { if (Optional.ofNullable(t2._2().getRelated_products()).isPresent()) rp.setRelated_products(t2._2().getRelated_products()); @@ -357,47 +359,283 @@ public class DumpResult implements Serializable { } private static void selectRelations(SparkSession spark, String inputPath, String workingDir) { - Dataset relation = Utils - .readPath( - spark, - inputPath + "relation", Relation.class) - .filter( - (FilterFunction) r -> !r.getDataInfo().getDeletedbyinference() && - !r.getDataInfo().getInvisible()) - .filter( - (FilterFunction) r -> r - .getRelClass() - .equalsIgnoreCase(RelationType.RESULT_AFFILIATIED_TO_ORGANIZATION.label) || - r.getRelClass().equalsIgnoreCase(RelationType.RESULT_OUTCOME_FUNDING.label) || - r.getRelClass().equalsIgnoreCase(RelationType.SUPPLEMENT.label) || - r.getRelClass().equalsIgnoreCase(RelationType.DOCUMENTS.label) || - r.getRelClass().equalsIgnoreCase(RelationType.PART.label) || - r.getRelClass().equalsIgnoreCase(RelationType.VERSION.label) || - r.getRelClass().equalsIgnoreCase(RelationType.CITATION.label)); + List relationsProducts = Arrays + .asList( + RelationType.CITATION.label, + // RelationType.RESULT_AFFILIATIED_TO_ORGANIZATION.label, + RelationType.SUPPLEMENT.label, + // RelationType.RESULT_OUTCOME_FUNDING.label, + RelationType.DOCUMENTS.label, + RelationType.PART.label, + RelationType.VERSION.label); - Dataset encloseMinEntity = Utils - .readPath(spark, workingDir + "minEntity", EncloseMinElement.class); + // relationsProducts +// .stream() +// .forEach(r -> buildRelationPerProducts(spark, inputPath, workingDir, r)); +// buildRelationPerAffiliation( +// spark, inputPath, workingDir, RelationType.RESULT_AFFILIATIED_TO_ORGANIZATION.label); + buildRelationPerGrant(spark, inputPath, workingDir, RelationType.RESULT_OUTCOME_FUNDING.label); - relation - .joinWith(encloseMinEntity, relation.col("target").equalTo(encloseMinEntity.col("enclosedEntityId"))) - .map((MapFunction, EncloseMinElement>) t2 -> { - EncloseMinElement eme = t2._2(); - eme.setResultId(t2._1().getSource()); - eme.setSemantics(t2._1().getRelClass()); - return eme; - }, Encoders.bean(EncloseMinElement.class)) - .groupByKey((MapFunction) eme -> eme.getResultId(), Encoders.STRING()) - .mapGroups((MapGroupsFunction) (k, v) -> { - RelationPerProduct rpp = new RelationPerProduct(); - rpp.setResultId(k); - insertEnclosedElement(rpp, v.next()); - v.forEachRemaining(e -> insertEnclosedElement(rpp, e)); - return rpp; - }, Encoders.bean(RelationPerProduct.class)) + RDD temp = spark + .read() + .schema(Encoders.bean(RelationPerProduct.class).schema()) + .json(workingDir + "aggrelation_temp") + .as(Encoders.bean(RelationPerProduct.class)) + .toJavaRDD() + .mapToPair(v -> new Tuple2<>(v.getResultId(), v)) + .reduceByKey((a, b) -> { + mergeRelationPerProduct(a, b); + return a; + }) + .map(v -> v._2()) + .rdd(); + + spark + .createDataset(temp, Encoders.bean(RelationPerProduct.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + "/aggrelation"); + + } + + private static void buildRelationPerGrant(SparkSession spark, String inputPath, String workingDir, + String relationType) { + log.info("Relation: {}", relationType); + final StructType relationstructureSchema = new StructType() + .add( + "dataInfo", new StructType() + .add("deletedbyinference", DataTypes.BooleanType)) + .add("source", DataTypes.StringType) + .add("target", DataTypes.StringType) + .add("relClass", DataTypes.StringType); + + final StructType grantSchema = new StructType() + .add( + "local_identifier", DataTypes.StringType) + + .add("funder", DataTypes.StringType) + .add("code", DataTypes.StringType) + .add("title", DataTypes.StringType) + + ; + + Dataset relation = spark + .read() + .schema(relationstructureSchema) + .json(inputPath + "relation") + .filter( + "datainfo.deletedbyinference != true and " + + "relclass == '" + relationType + "'") + .drop("dataInfo"); + + Dataset minProduct = spark + .read() + .schema(grantSchema) + .json(workingDir + "minGrant"); + + relation + .joinWith( + minProduct, relation.col("target").equalTo(minProduct.col("local_identifier"))) + .selectExpr("_1.source as sourceResult", "_1.relClass as relClass", "_2.*") + .groupByKey((MapFunction) r -> r.getAs("sourceResult"), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> { + RelationPerProduct rpp = new RelationPerProduct(); + rpp.setResultId(k); + rpp.setRelatedProduct(new HashMap<>()); + updateRelevantGrant(rpp, it.next()); + it.forEachRemaining(r -> updateRelevantGrant(rpp, r)); + return rpp; + }, Encoders.bean(RelationPerProduct.class)) + // .show(false); + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(workingDir + "aggrelation_temp"); + } + + private static void updateRelevantGrant(RelationPerProduct rpp, Row next) { + if (!Optional.ofNullable(rpp.getFunding()).isPresent()) + rpp.setFunding(new ArrayList<>()); + MinGrant mo = new MinGrant(); + mo.setLocal_identifier(next.getAs("local_identifier")); + mo.setTitle(next.getAs("title")); + mo.setFunder(next.getAs("fundet")); + mo.setCode(next.getAs("code")); + + rpp.getFunding().add(mo); + } + + private static void buildRelationPerAffiliation(SparkSession spark, String inputPath, String workingDir, + String relationType) { + log.info("Relation: {}", relationType); + final StructType relationstructureSchema = new StructType() + .add( + "dataInfo", new StructType() + .add("deletedbyinference", DataTypes.BooleanType)) + .add("source", DataTypes.StringType) + .add("target", DataTypes.StringType) + .add("relClass", DataTypes.StringType); + + final StructType orgSchema = new StructType() + .add( + "local_identifier", DataTypes.StringType) + + .add("name", DataTypes.StringType) + .add("ror", DataTypes.StringType) + .add("isni", DataTypes.StringType) + .add("fundRef", DataTypes.StringType) + .add("rinGold", DataTypes.StringType) + .add("wikidata", DataTypes.StringType); + + Dataset relation = spark + .read() + .schema(relationstructureSchema) + .json(inputPath + "relation") + .filter( + "datainfo.deletedbyinference != true and " + + "relclass == '" + relationType + "'") + .drop("dataInfo"); + + Dataset minOrganization = spark + .read() + .schema(orgSchema) + .json(workingDir + "minOrganization"); + + relation + .joinWith( + minOrganization, relation.col("target").equalTo(minOrganization.col("local_identifier"))) + .selectExpr("_1.source as sourceResult", "_1.relClass as relClass", "_2.*") + .groupByKey((MapFunction) r -> r.getAs("sourceResult"), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> { + RelationPerProduct rpp = new RelationPerProduct(); + rpp.setResultId(k); + rpp.setRelatedProduct(new HashMap<>()); + updateRelevantOrganization(rpp, it.next()); + it.forEachRemaining(r -> updateRelevantOrganization(rpp, r)); + return rpp; + }, Encoders.bean(RelationPerProduct.class)) + // .show(false); + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(workingDir + "aggrelation_temp"); + } + + private static void updateRelevantOrganization(RelationPerProduct rpp, Row next) { + if (!Optional.ofNullable(rpp.getOrganizations()).isPresent()) + rpp.setOrganizations(new ArrayList<>()); + MinOrganization mo = new MinOrganization(); + mo.setLocal_identifier(next.getAs("local_identifier")); + mo.setIsni(next.getAs("isni")); + mo.setRor(next.getAs("ror")); + mo.setName(next.getAs("name")); + mo.setWikidata(next.getAs("wikidata")); + mo.setFundRef(next.getAs("fundRef")); + mo.setRinGold(next.getAs("rinGold")); + + rpp.getOrganizations().add(mo); + } + + private static void buildRelationPerProducts(SparkSession spark, String inputPath, String workingDir, + String relationType) { + log.info("Relation: {}", relationType); + final StructType relationstructureSchema = new StructType() + .add( + "dataInfo", new StructType() + .add("deletedbyinference", DataTypes.BooleanType)) + .add("source", DataTypes.StringType) + .add("target", DataTypes.StringType) + .add("relClass", DataTypes.StringType); + + final StructType productSchema = new StructType() + .add( + "local_identifier", DataTypes.StringType) + + .add("title", DataTypes.StringType) + .add("doi", DataTypes.StringType) + .add("pmcid", DataTypes.StringType) + .add("arxivid", DataTypes.StringType) + .add("pmid", DataTypes.StringType); + + Dataset relation = spark + .read() + .schema(relationstructureSchema) + .json(inputPath + "relation") + .filter( + "datainfo.deletedbyinference != true and " + + "relclass == '" + relationType + "'") + .drop("dataInfo"); + + Dataset minProduct = spark + .read() + .schema(productSchema) + .json(workingDir + "minProduct"); + + relation + .joinWith( + minProduct, relation.col("target").equalTo(minProduct.col("local_identifier"))) + .selectExpr("_1.source as sourceResult", "_1.relClass as relClass", "_2.*") + .groupByKey((MapFunction) r -> r.getAs("sourceResult"), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> { + RelationPerProduct rpp = new RelationPerProduct(); + rpp.setResultId(k); + rpp.setRelatedProduct(new HashMap<>()); + updateRelatedProduct(rpp, it.next()); + it.forEachRemaining(r -> updateRelatedProduct(rpp, r)); + return rpp; + }, Encoders.bean(RelationPerProduct.class)) + // .show(false); + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(workingDir + "aggrelation_temp"); +// .map((MapFunction, RelationPerProduct>) t2 -> { +// RelationPerProduct rpp = new RelationPerProduct(); +// t2._2().setResultId(t2._1().getAs("source")); +// t2._2().setSemantics(t2._1().getAs("relClass")); +// insertEnclosedElement(rpp, t2._2()); +// rpp.setResultId(t2._1().getAs("source")); +// return rpp; +// }, Encoders.bean(RelationPerProduct.class)) +// .filter(Objects::nonNull) +// .toJavaRDD() +// .mapToPair(value -> new Tuple2<>(value.getResultId(), value)) +// .reduceByKey((a, b) -> { +// mergeRelationPerProduct(a, b); +// return a; +// }) +// +// .map(value -> value._2) +// .rdd(), +// Encoders.bean(RelationPerProduct.class)) +//// .saveAsTextFile(workingDir + "/aggrelation", GzipCodec.class); +//// .groupByKey((MapFunction) eme -> eme.getResultId(), Encoders.STRING()) +//// .mapGroups((MapGroupsFunction) (k, v) -> { +//// RelationPerProduct rpp = new RelationPerProduct(); +//// rpp.setResultId(k); +//// insertEnclosedElement(rpp, v.next()); +//// v.forEachRemaining(e -> insertEnclosedElement(rpp, e)); +//// return rpp; +//// }, Encoders.bean(RelationPerProduct.class)) +// .write() +// .mode(SaveMode.Append) +// .option("compression", "gzip") +// .json(workingDir + "/aggrelation_temp"); + } + + private static void updateRelatedProduct(RelationPerProduct rpp, Row next) { + String key = next.getAs("relClass"); + if (!rpp.getRelatedProduct().keySet().contains(key)) + rpp.getRelatedProduct().put(key, new ArrayList<>()); + MinProduct mp = new MinProduct(); + mp.setLocal_identifier(next.getAs("local_identifier")); + mp.setTitle(next.getAs("title")); + mp.setPmid(next.getAs("pmid")); + mp.setArxivid(next.getAs("arxivid")); + mp.setPmcid(next.getAs("pmcid")); + mp.setDoi(next.getAs("doi")); + rpp.getRelatedProduct().get(key).add(mp); } private static void insertEnclosedElement(RelationPerProduct rpp, EncloseMinElement element) { @@ -414,19 +652,21 @@ public class DumpResult implements Serializable { } - private static Dataset getMinProduct(SparkSession spark, String inputPath, - Class clazz) { - return Utils - .readPath(spark, inputPath, clazz) - .filter( - (FilterFunction) r -> !r.getDataInfo().getDeletedbyinference() && - !r.getDataInfo().getInvisible()) - .map((MapFunction) r -> { - EncloseMinElement eme = new EncloseMinElement(); - eme.setEnclosedEntityId(r.getId()); - eme.setMinProduct(Utils.getMinProduct(r)); - return eme; - }, Encoders.bean(EncloseMinElement.class)); + private static void mergeRelationPerProduct(RelationPerProduct rpp1, RelationPerProduct rpp2) { + if (Optional.ofNullable(rpp2.getOrganizations()).isPresent()) + rpp1.getOrganizations().addAll(rpp2.getOrganizations()); + if (Optional.ofNullable(rpp2.getFunding()).isPresent()) + rpp1.getFunding().addAll(rpp2.getFunding()); + if (Optional.ofNullable(rpp2.getRelatedProduct()).isPresent()) { + Map> temp = rpp2.getRelatedProduct(); + for (String key : temp.keySet()) { + if (!rpp1.getRelatedProduct().containsKey(key)) + rpp1.getRelatedProduct().put(key, new ArrayList<>()); + for (MinProduct mp : rpp2.getRelatedProduct().get(key)) + rpp1.getRelatedProduct().get(key).add(mp); + } + } + } } diff --git a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/EmitFromEntities.java b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/EmitFromEntities.java index 212978f..2a8e0f7 100644 --- a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/EmitFromEntities.java +++ b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/EmitFromEntities.java @@ -13,10 +13,9 @@ import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapGroupsFunction; +import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,6 +23,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.oa.graph.dump.skgif.beans.EmitPerManifestation; import eu.dnetlib.dhp.oa.graph.dump.skgif.beans.EncloseMinElement; import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.Datasource; @@ -83,16 +83,17 @@ public class EmitFromEntities implements Serializable { Utils .readPath(spark, inputPath + "project", Project.class) .filter((FilterFunction) p -> !p.getDataInfo().getDeletedbyinference()) - .map((MapFunction) p -> { - EncloseMinElement eme = new EncloseMinElement(); - eme.setEnclosedEntityId(p.getId()); - eme.setMinGrant(Utils.getMinGrant(p)); - return eme; - }, Encoders.bean(EncloseMinElement.class)) + .map((MapFunction) p -> { +// EncloseMinElement eme = new EncloseMinElement(); +// eme.setEnclosedEntityId(p.getId()); +// eme.setMinGrant(Utils.getMinGrant(p)); +// return eme; + return Utils.getMinGrant(p); + }, Encoders.bean(MinGrant.class)) .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(workingDir + "/minEntity"); + .json(workingDir + "/minGrant"); } @@ -100,39 +101,56 @@ public class EmitFromEntities implements Serializable { Utils .readPath(spark, inputPath + "organization", Organization.class) .filter((FilterFunction) o -> !o.getDataInfo().getDeletedbyinference()) - .map((MapFunction) o -> { - EncloseMinElement eme = new EncloseMinElement(); - eme.setMinOrganization(Utils.getMinOrganization(o)); - eme.setEnclosedEntityId(o.getId()); - return eme; - }, - Encoders.bean(EncloseMinElement.class)) + .map( + (MapFunction) o -> { + // EncloseMinElement eme = new EncloseMinElement(); + return Utils.getMinOrganization(o); +// eme.setMinOrganization(Utils.getMinOrganization(o)); +// eme.setEnclosedEntityId(o.getId()); +// return eme; + }, + Encoders.bean(MinOrganization.class)) .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(workingDir + "/minEntity"); + .json(workingDir + "/minOrganization"); } private static void emitFromDatasource(SparkSession spark, String inputPath, String workingDir) { +// StructType daSchema = StructType +// .fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>, `officialname` STRUCT<`value` STRING>"); +// spark +// .read() +// .schema(Encoders.bean(Datasource.class).schema()) +// .json(inputPath + "datasource") +// .filter( +// "dataInfo.deletedbyinference != true " ) +// .selectExpr("id as enclosedEntity", " as minDatasource" ) +// .select(new Column("id").as("minDatasource.local_identifier"), new Column("officialname.value").as("minDatasource.name"), new Column("id").as("enclosedEntityId")) +// .write() +// .mode(SaveMode.Append) +// .option("compression","gzip") +// .json(workingDir + "/minEntity"); + Utils .readPath(spark, inputPath + "datasource", Datasource.class) .filter((FilterFunction) d -> !d.getDataInfo().getDeletedbyinference()) - .map((MapFunction) d -> { - EncloseMinElement eme = new EncloseMinElement(); - eme - .setMinDatsource( - MinVenue - .newInstance( - // Utils.getIdentifier(Prefixes.DATASOURCE, d.getId()), - // d.getOfficialname().getValue())); - d.getId(), d.getOfficialname().getValue())); - eme.setEnclosedEntityId(d.getId()); - return eme; - }, Encoders.bean(EncloseMinElement.class)) + .map((MapFunction) d -> { +// EncloseMinElement eme = new EncloseMinElement(); +// eme +// .setMinDatsource( + return MinVenue + .newInstance( + // Utils.getIdentifier(Prefixes.DATASOURCE, d.getId()), + // d.getOfficialname().getValue())); + d.getId(), d.getOfficialname().getValue());// ); + // eme.setEnclosedEntityId(d.getId()); + // return eme; + }, Encoders.bean(MinVenue.class)) .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(workingDir + "/minEntity"); + .json(workingDir + "/minDatasource"); Utils .readPath(spark, inputPath + "datasource", Datasource.class) @@ -142,36 +160,36 @@ public class EmitFromEntities implements Serializable { .getEoscdatasourcetype() .getClassid() .equalsIgnoreCase("Journal archive")) - .map((MapFunction) d -> { - EncloseMinElement eme = new EncloseMinElement(); - eme.setEnclosedEntityId(d.getId()); + .map((MapFunction) d -> { +// EncloseMinElement eme = new EncloseMinElement(); +// eme.setEnclosedEntityId(d.getId()); if (Optional.ofNullable(d.getJournal()).isPresent() && Optional.ofNullable(d.getJournal().getIssnPrinted()).isPresent()) { - eme - .setMinVenue( - MinVenue - .newInstance( - Utils.getIdentifier(Prefixes.VENUE, d.getJournal().getIssnPrinted()), - d.getOfficialname().getValue())); - return eme; + // eme + // .setMinVenue( + return MinVenue + .newInstance( + Utils.getIdentifier(Prefixes.VENUE, d.getJournal().getIssnPrinted()), + d.getOfficialname().getValue());// ); + // return eme; } if (Optional.ofNullable(d.getJournal()).isPresent() && Optional.ofNullable(d.getJournal().getIssnOnline()).isPresent()) { - eme - .setMinVenue( - MinVenue - .newInstance( - Utils.getIdentifier(Prefixes.VENUE, d.getJournal().getIssnOnline()), - d.getOfficialname().getValue())); - return eme; +// eme +// .setMinVenue( + return MinVenue + .newInstance( + Utils.getIdentifier(Prefixes.VENUE, d.getJournal().getIssnOnline()), + d.getOfficialname().getValue());// ); + // return eme; } return null; - }, Encoders.bean(EncloseMinElement.class)) + }, Encoders.bean(MinVenue.class)) .filter(Objects::nonNull) .write() - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(workingDir + "/minEntity"); + .json(workingDir + "/minVenue"); } @@ -186,23 +204,20 @@ public class EmitFromEntities implements Serializable { } private static void emitMinProduct(SparkSession spark, String inputPath, String workingDir) { - Utils.removeOutputDir(spark, workingDir + "minEntity"); + Utils.removeOutputDir(spark, workingDir + "minProduct"); ModelSupport.entityTypes.keySet().forEach(e -> { if (ModelSupport.isResult(e)) { Class resultClazz = ModelSupport.entityTypes.get(e); Utils .readPath(spark, inputPath + e.name(), resultClazz) - .map((MapFunction) p -> { - EncloseMinElement eme = new EncloseMinElement(); - eme.setMinProduct(Utils.getMinProduct(p)); - eme.setEnclosedEntityId(p.getId()); - return eme; - }, Encoders.bean(EncloseMinElement.class)) + .map( + (MapFunction) p -> Utils.getMinProduct(p), + Encoders.bean(MinProduct.class)) .write() .mode(SaveMode.Append) .option("compression", "gzip") - .json(workingDir + "/minEntity"); + .json(workingDir + "/minProduct"); } diff --git a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/Utils.java b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/Utils.java index 7c9fb62..beb70e4 100644 --- a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/Utils.java +++ b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/Utils.java @@ -155,6 +155,9 @@ public class Utils implements Serializable { case "arxiv": mp.setArxivid(pid.getValue()); break; + case "pmid": + mp.setPmid(pid.getValue()); + break; } } diff --git a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/beans/EncloseMinElement.java b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/beans/EncloseMinElement.java index 4557ec7..899a94b 100644 --- a/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/beans/EncloseMinElement.java +++ b/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/skgif/beans/EncloseMinElement.java @@ -85,4 +85,5 @@ public class EncloseMinElement implements Serializable { public void setMinProduct(MinProduct minProduct) { this.minProduct = minProduct; } + } diff --git a/dump/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/oozie_app/workflow.xml b/dump/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/oozie_app/workflow.xml index 20595aa..0107cee 100644 --- a/dump/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/oozie_app/workflow.xml +++ b/dump/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/oozie_app/workflow.xml @@ -62,7 +62,7 @@ - + @@ -218,19 +218,90 @@ --filterPath${filterPath} - + - + ${jobTracker} ${nameNode} - ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/* - ${nameNode}/${workingDir}/graph/ + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/publication + ${nameNode}/${workingDir}/graph/publication + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/dataset + ${nameNode}/${workingDir}/graph/dataset + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/otherresearchproduct + ${nameNode}/${workingDir}/graph/otherresearchproduct + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/software + ${nameNode}/${workingDir}/graph/software + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/datasource + ${nameNode}/${workingDir}/graph/datasource + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/project + ${nameNode}/${workingDir}/graph/project + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/organization + ${nameNode}/${workingDir}/graph/organization + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/relation + ${nameNode}/${workingDir}/graph/relation + yarn @@ -248,9 +319,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - --sourcePath${workingDir}/graph/ - --workingDir${workingDir}/ + + --outputPath${outputPath} + --workingDir/user/miriam.baglioni/oa/graph/dump/temp/working_dir/ + --sourcePath/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/ @@ -263,18 +336,22 @@ eu.dnetlib.dhp.oa.graph.dump.skgif.DumpResult dump-${projectVersion}.jar - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} + --executor-cores=4 + --executor-memory=8G --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=5G --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + --conf spark.sql.shuffle.partitions=15000 - --sourcePath${workingDir}/graph/ + + --sourcePath/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/ --outputPath${outputPath} - --workingDir${workingDir}/ + + --workingDir/user/miriam.baglioni/oa/graph/dump/temp/working_dir/ @@ -296,9 +373,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - --sourcePath${workingDir}/graph/ + + --sourcePath/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/ --outputPath${outputPath} - --workingDir${workingDir}/ + + --workingDir/user/miriam.baglioni/oa/graph/dump/temp/working_dir/ @@ -320,9 +399,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - --sourcePath${workingDir}/graph/ + + --sourcePath/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/ --outputPath${outputPath} - --workingDir${workingDir}/ + + --workingDir/user/miriam.baglioni/oa/graph/dump/temp/working_dir/ @@ -345,9 +426,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - --sourcePath${workingDir}/graph/ + + --sourcePath/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/ --outputPath${outputPath} - --workingDir${workingDir}/ + + --workingDir/user/miriam.baglioni/oa/graph/dump/temp/working_dir/ @@ -370,9 +453,11 @@ --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - --sourcePath${workingDir}/graph/ + + --sourcePath/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/ --outputPath${outputPath} - --workingDir${workingDir}/ + + --workingDir/user/miriam.baglioni/oa/graph/dump/temp/working_dir/ diff --git a/dump/src/test/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResultTest.java b/dump/src/test/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResultTest.java index dfe74a3..83ec0db 100644 --- a/dump/src/test/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResultTest.java +++ b/dump/src/test/java/eu/dnetlib/dhp/oa/graph/dump/skgif/DumpResultTest.java @@ -4,6 +4,8 @@ package eu.dnetlib.dhp.oa.graph.dump.skgif; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayList; +import java.util.HashMap; import java.util.stream.Collectors; import javax.validation.constraints.AssertTrue; @@ -14,9 +16,14 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.ForeachFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.MapGroupsFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; @@ -28,6 +35,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.oa.graph.dump.skgif.beans.EmitPerManifestation; import eu.dnetlib.dhp.oa.graph.dump.skgif.beans.RelationPerProduct; +import eu.dnetlib.dhp.oa.model.graph.Relation; import eu.dnetlib.dhp.skgif.model.*; import eu.dnetlib.dhp.utils.DHPUtils; @@ -489,6 +497,54 @@ public class DumpResultTest { } + @Test + public void testEmitFromApi() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/oa/graph/dump/skgif/graphForAPIExample/") + .getPath(); + + final String workingDir = getClass() + .getResource("/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/") + .getPath(); + + final StructType relationstructureSchema = new StructType() + .add( + "dataInfo", new StructType() + .add("deletedbyinference", DataTypes.BooleanType)) + .add("source", DataTypes.StringType) + .add("target", DataTypes.StringType) + .add("relClass", DataTypes.StringType); + + final StructType productSchema = new StructType() + .add("local_identifier", DataTypes.StringType) + .add("title", DataTypes.StringType) + .add("doi", DataTypes.StringType) + .add("pmcid", DataTypes.StringType) + .add("arxivid", DataTypes.StringType) + .add("pmid", DataTypes.StringType); + + Dataset relation = spark + .read() + .schema(relationstructureSchema) + .json(sourcePath + "relation") + .filter( + "datainfo.deletedbyinference != true and " + + "relclass == '" + RelationType.CITATION.label + "'") + .drop("dataInfo"); + + Dataset minProduct = spark + .read() + .schema(productSchema) + .json(workingDir + "minProduct"); + relation + .joinWith( + minProduct, relation.col("target").equalTo(minProduct.col("local_identifier"))) + .selectExpr("_1.source as sourceResult", "_2.*") + + .show(false); + + } + @Test public void testEmitFromApiDump() throws Exception { final String sourcePath = getClass() diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-2e174e76-c662-4a0e-bc26-a2779b0dc259-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-2e174e76-c662-4a0e-bc26-a2779b0dc259-c000.json new file mode 100644 index 0000000..b10607b --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-2e174e76-c662-4a0e-bc26-a2779b0dc259-c000.json @@ -0,0 +1,32 @@ +{"enclosedEntityId":"50|59796a29bd90::390b69a1e20d5af206fd6849c9b97d13","minProduct":{"local_identifier":"50|59796a29bd90::390b69a1e20d5af206fd6849c9b97d13","title":"Power to gas"}} +{"enclosedEntityId":"50|59796a29bd90::6098357dcb0f21a492ae2689961268ef","minProduct":{"local_identifier":"50|59796a29bd90::6098357dcb0f21a492ae2689961268ef","title":"Vrije Tijd? Tijd voor Deventer!"}} +{"enclosedEntityId":"50|59796a29bd90::d0545b3c7166358e16599aec2284b101","minProduct":{"local_identifier":"50|59796a29bd90::d0545b3c7166358e16599aec2284b101","title":"Sepsis at ICU admission does not decrease 30-day survival in very old patients"}} +{"enclosedEntityId":"50|73490d0e0f82::1f1e664683ae0035b0d64487d1e1bcbe","minProduct":{"local_identifier":"50|73490d0e0f82::1f1e664683ae0035b0d64487d1e1bcbe","title":"The Effects of Membrane Physical Properties on Insulin Receptor Autophosphorylation and Signalling"}} +{"enclosedEntityId":"50|73490d0e0f82::62d68e6a826109387d9b24d125dee5d1","minProduct":{"local_identifier":"50|73490d0e0f82::62d68e6a826109387d9b24d125dee5d1","title":"Protection of Migrant Agricultural Workers' Rights"}} +{"enclosedEntityId":"50|CSC_________::288c3eab977e8b76a7abe31f989c9901","minProduct":{"local_identifier":"50|CSC_________::288c3eab977e8b76a7abe31f989c9901","title":"A fabric design for Armani Casa"}} +{"enclosedEntityId":"50|CSC_________::710a52c4e2a9b3b697c682dd15595a4c","minProduct":{"local_identifier":"50|CSC_________::710a52c4e2a9b3b697c682dd15595a4c","title":"''Archpelago'' Apron, Navy : formgivning"}} +{"enclosedEntityId":"50|CSC_________::7a3ea1f6a35d738fb193f6fae3303d32","minProduct":{"local_identifier":"50|CSC_________::7a3ea1f6a35d738fb193f6fae3303d32","title":"Ragna Róbertsdóttir | Configured Landscapes"}} +{"enclosedEntityId":"50|RECOLECTA___::02e2efbf7bdfbdc708995bf8cb3c038b","minProduct":{"local_identifier":"50|RECOLECTA___::02e2efbf7bdfbdc708995bf8cb3c038b","title":"Por qué la educación a distancia (en línea)"}} +{"enclosedEntityId":"50|RECOLECTA___::04e03c6d2756db9d75874947a7ef224c","minProduct":{"local_identifier":"50|RECOLECTA___::04e03c6d2756db9d75874947a7ef224c","title":"Sistemas recomendadores aplicados a productos de ropa"}} +{"enclosedEntityId":"50|RECOLECTA___::05d59bf33bc35aeb975ca399180a051f","minProduct":{"local_identifier":"50|RECOLECTA___::05d59bf33bc35aeb975ca399180a051f","title":"Origen y desarrollo de los derechos fundamentales laborales en Venezuela (1830-Constitución de 1999) = Origin and development of fundamental labor rights in Venezuela (1830-1999 Constitution)"}} +{"enclosedEntityId":"50|RECOLECTA___::074a83f6b5813016364f2f5c4651f026","minProduct":{"local_identifier":"50|RECOLECTA___::074a83f6b5813016364f2f5c4651f026","title":"Página 3"}} +{"enclosedEntityId":"50|RECOLECTA___::1111974150e3c89ed32a8fabe4785e32","minProduct":{"local_identifier":"50|RECOLECTA___::1111974150e3c89ed32a8fabe4785e32","title":"Diseño y aplicaciones de materiales híbridos fotoactivos"}} +{"enclosedEntityId":"50|RECOLECTA___::2694fc6101c2a0454d91fa047997ea57","minProduct":{"local_identifier":"50|RECOLECTA___::2694fc6101c2a0454d91fa047997ea57","title":"BIM-based decision support for building condition assessment"}} +{"enclosedEntityId":"50|RECOLECTA___::4c67143df007c7fdfd9af06ead811612","minProduct":{"local_identifier":"50|RECOLECTA___::4c67143df007c7fdfd9af06ead811612","title":"Máster profesorado - Sesión práctica 1: Diseño y desarrollo de programaciones y actividades formativas"}} +{"enclosedEntityId":"50|RECOLECTA___::554f00ffc38fa55e66208b19a8576b7d","minProduct":{"local_identifier":"50|RECOLECTA___::554f00ffc38fa55e66208b19a8576b7d","title":"Uncertainty analysis methods to select the optimal alternative in the design of parking facilities"}} +{"enclosedEntityId":"50|59796a29bd90::1a4c7bbe17709e01cb361b2159aa3296","minProduct":{"local_identifier":"50|59796a29bd90::1a4c7bbe17709e01cb361b2159aa3296","title":"Sociale voorwaarden in aanbestedingen: beleid en uitvoering"}} +{"enclosedEntityId":"50|59796a29bd90::227dfff51dc04032c0fc5c96a5ebc405","minProduct":{"local_identifier":"50|59796a29bd90::227dfff51dc04032c0fc5c96a5ebc405","title":"In situ detection of product age and argon concentration as measure of the re-use potential of insulating glass units in buildings"}} +{"enclosedEntityId":"50|59796a29bd90::2a8c8c2e29955006521983ec30226a23","minProduct":{"local_identifier":"50|59796a29bd90::2a8c8c2e29955006521983ec30226a23","title":"Annual conference REFRAME"}} +{"enclosedEntityId":"50|59796a29bd90::e8ef028372d784a26a451f6b50d0006d","minProduct":{"local_identifier":"50|59796a29bd90::e8ef028372d784a26a451f6b50d0006d","title":"Kruidenrijk grasland : kans voor koe en boer"}} +{"enclosedEntityId":"50|73490d0e0f82::831b9da5b9523d3a954471c111f03fcc","minProduct":{"local_identifier":"50|73490d0e0f82::831b9da5b9523d3a954471c111f03fcc","title":"Using Reputation in Repeated Selfish Routing with Incomplete Information"}} +{"enclosedEntityId":"50|73490d0e0f82::8ef7550c8b45953789005b0af1626c4e","minProduct":{"local_identifier":"50|73490d0e0f82::8ef7550c8b45953789005b0af1626c4e","title":"Refractor Plates Optimization and Data Analysis of a Correlation Spectrometer, Used as a Remote Sensing Instrument"}} +{"enclosedEntityId":"50|73490d0e0f82::d94d2b644fe36e0398c55f894f01bf13","minProduct":{"local_identifier":"50|73490d0e0f82::d94d2b644fe36e0398c55f894f01bf13","title":"links between labour-market experiences and health"}} +{"enclosedEntityId":"50|CSC_________::0335521d3fcd008e1a73cc500a4911c2","minProduct":{"local_identifier":"50|CSC_________::0335521d3fcd008e1a73cc500a4911c2","title":"10 kohtausta seksuaalisesta vallankäytöstä (teatteriesitys) : valosuunnittelu"}} +{"enclosedEntityId":"50|CSC_________::33b62b7fd4cb5305580ff6b5ad3b110d","minProduct":{"local_identifier":"50|CSC_________::33b62b7fd4cb5305580ff6b5ad3b110d","title":"Lyhytkapillaarielektroforeesi on-line -prosessiseurantaan"}} +{"enclosedEntityId":"50|CSC_________::a3bfbbaaee8b7d79a42a334bc5fb7f00","minProduct":{"local_identifier":"50|CSC_________::a3bfbbaaee8b7d79a42a334bc5fb7f00","title":"Jam Session w/ Dr. Sandberg (konsert) : musiker"}} +{"enclosedEntityId":"50|RECOLECTA___::00f011c5c9490e47c91bdf36701221f3","minProduct":{"local_identifier":"50|RECOLECTA___::00f011c5c9490e47c91bdf36701221f3","title":"Aló"}} +{"enclosedEntityId":"50|RECOLECTA___::0197025e456a3e0a4922b82c990d6bd8","minProduct":{"local_identifier":"50|RECOLECTA___::0197025e456a3e0a4922b82c990d6bd8","title":"Ficha 489"}} +{"enclosedEntityId":"50|RECOLECTA___::2757c2e94b79fe7822c3a59005d07870","minProduct":{"local_identifier":"50|RECOLECTA___::2757c2e94b79fe7822c3a59005d07870","title":"Clinical spectrum of premature pubarche: Links to metabolic syndrome and ovarian hyperandrogenism"}} +{"enclosedEntityId":"50|RECOLECTA___::3099980dac34189351509d429ba16866","minProduct":{"local_identifier":"50|RECOLECTA___::3099980dac34189351509d429ba16866","title":"La incidencia del factor nuclear en el terrorismo"}} +{"enclosedEntityId":"50|RECOLECTA___::3227a354fbf54668c379229297c71c0e","minProduct":{"local_identifier":"50|RECOLECTA___::3227a354fbf54668c379229297c71c0e","title":"Pre-training of deep neural networks through Real-Coded Genetic Algorithms"}} +{"enclosedEntityId":"50|RECOLECTA___::37a152ef82da05df86d1414a0bc200b4","minProduct":{"local_identifier":"50|RECOLECTA___::37a152ef82da05df86d1414a0bc200b4","title":"the UN 2030 Agenda to Improve Global Communicative Competence"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-2e174e76-c662-4a0e-bc26-a2779b0dc259-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-2e174e76-c662-4a0e-bc26-a2779b0dc259-c000.json.gz deleted file mode 100644 index 6845b9c..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-2e174e76-c662-4a0e-bc26-a2779b0dc259-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-3828fd20-3909-42ee-88dc-5ec33eff050d-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-3828fd20-3909-42ee-88dc-5ec33eff050d-c000.json new file mode 100644 index 0000000..68b669b --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-3828fd20-3909-42ee-88dc-5ec33eff050d-c000.json @@ -0,0 +1,44 @@ +{"enclosedEntityId":"10|06cdd3ff4700::69433f5cb51fe4c3959d9a892c8eff4a","minDatsource":{"local_identifier":"10|06cdd3ff4700::69433f5cb51fe4c3959d9a892c8eff4a","name":"Analyse de réseaux pour les sciences sociales"}} +{"enclosedEntityId":"10|doajarticles::008cf70ae79b184f4fe3c0547e98a582","minDatsource":{"local_identifier":"10|doajarticles::008cf70ae79b184f4fe3c0547e98a582","name":"Revista Electrónica Sinéctica"}} +{"enclosedEntityId":"10|doajarticles::00eb4160e4726f60887b60fa7df15e32","minDatsource":{"local_identifier":"10|doajarticles::00eb4160e4726f60887b60fa7df15e32","name":"Jurnal Soeropati"}} +{"enclosedEntityId":"10|doajarticles::0c217533c6e0bfe1a4a36a093f8c23e2","minDatsource":{"local_identifier":"10|doajarticles::0c217533c6e0bfe1a4a36a093f8c23e2","name":"Acta Zoologica Academiae Scientiarum Hungaricae"}} +{"enclosedEntityId":"10|doajarticles::0cccd429850ed396c90777b54cf52a1c","minDatsource":{"local_identifier":"10|doajarticles::0cccd429850ed396c90777b54cf52a1c","name":"Патология кровообращения и кардиохирургия"}} +{"enclosedEntityId":"10|doajarticles::0d957b8eca7fe947571d6400474c95de","minDatsource":{"local_identifier":"10|doajarticles::0d957b8eca7fe947571d6400474c95de","name":"Journal of the Illuminating Engineering Institute of Japan"}} +{"enclosedEntityId":"10|doajarticles::0db249d213651f07c9dabeb3edc53efc","minDatsource":{"local_identifier":"10|doajarticles::0db249d213651f07c9dabeb3edc53efc","name":"Drug Metabolism and Pharmacokinetics"}} +{"enclosedEntityId":"10|doajarticles::0ee8ccdef95fedf8ed8f0a62e7c2c8af","minDatsource":{"local_identifier":"10|doajarticles::0ee8ccdef95fedf8ed8f0a62e7c2c8af","name":"BMC Medical Imaging"}} +{"enclosedEntityId":"10|doajarticles::0f7fe9973717cb7ac2bba8685bc88a2a","minDatsource":{"local_identifier":"10|doajarticles::0f7fe9973717cb7ac2bba8685bc88a2a","name":"Reflektika"}} +{"enclosedEntityId":"10|doajarticles::10769bf5b219fbdf75d6bd2d8bd45406","minDatsource":{"local_identifier":"10|doajarticles::10769bf5b219fbdf75d6bd2d8bd45406","name":"e-Finanse"}} +{"enclosedEntityId":"10|doajarticles::10960937f086496a32324c9eeb1cf380","minDatsource":{"local_identifier":"10|doajarticles::10960937f086496a32324c9eeb1cf380","name":"ILIRIA International Review"}} +{"enclosedEntityId":"10|doajarticles::15215d4c9ae17e4e738c1a09c06aacee","minDatsource":{"local_identifier":"10|doajarticles::15215d4c9ae17e4e738c1a09c06aacee","name":"Storia e Futuro"}} +{"enclosedEntityId":"10|doajarticles::15d8c3c4f653bffd815dbdbb1ce56d93","minDatsource":{"local_identifier":"10|doajarticles::15d8c3c4f653bffd815dbdbb1ce56d93","name":"پژوهش‌های ریاضی"}} +{"enclosedEntityId":"10|doajarticles::176acb5e4cef928cfb0ec4adeacaee89","minDatsource":{"local_identifier":"10|doajarticles::176acb5e4cef928cfb0ec4adeacaee89","name":"Indian Journal of Occupational and Environmental Medicine"}} +{"enclosedEntityId":"10|doajarticles::18c7e558fdce9c26c97cb0336b0584ff","minDatsource":{"local_identifier":"10|doajarticles::18c7e558fdce9c26c97cb0336b0584ff","name":"Mercator : Revista de Geografia da UFC"}} +{"enclosedEntityId":"10|doajarticles::192e590353f6cf9af3fdba01b1e1cfde","minDatsource":{"local_identifier":"10|doajarticles::192e590353f6cf9af3fdba01b1e1cfde","name":"Journal of International Translational Medicine"}} +{"enclosedEntityId":"10|doajarticles::1a4560f88542bcb1b8d7f49b98244d57","minDatsource":{"local_identifier":"10|doajarticles::1a4560f88542bcb1b8d7f49b98244d57","name":"Емінак"}} +{"enclosedEntityId":"10|doajarticles::1c8148b0dbade0f654ed77b6213f5926","minDatsource":{"local_identifier":"10|doajarticles::1c8148b0dbade0f654ed77b6213f5926","name":"Terra Sebus: Acta Musei Sabesiensis"}} +{"enclosedEntityId":"10|doajarticles::201d90c44e936c5fc512731c567bfafe","minDatsource":{"local_identifier":"10|doajarticles::201d90c44e936c5fc512731c567bfafe","name":"AIMS Allergy and Immunology"}} +{"enclosedEntityId":"10|doajarticles::2102c11ea98b6a8196ceec8c07444bd0","minDatsource":{"local_identifier":"10|doajarticles::2102c11ea98b6a8196ceec8c07444bd0","name":"Revista Organizações em Contexto"}} +{"enclosedEntityId":"10|doajarticles::2226acf8a08b9d804c69f409383353c3","minDatsource":{"local_identifier":"10|doajarticles::2226acf8a08b9d804c69f409383353c3","name":"Frontiers in Bird Science"}} +{"enclosedEntityId":"10|doajarticles::27be52df8acc8b659e75ec6221f4ccc9","minDatsource":{"local_identifier":"10|doajarticles::27be52df8acc8b659e75ec6221f4ccc9","name":"Jurnal Mediakita"}} +{"enclosedEntityId":"10|doajarticles::27f5e80d5e90370663ced5e93f343922","minDatsource":{"local_identifier":"10|doajarticles::27f5e80d5e90370663ced5e93f343922","name":"Frontiers in Bioengineering and Biotechnology"}} +{"enclosedEntityId":"10|doajarticles::29c549c62b532a88d2b700a8e0959140","minDatsource":{"local_identifier":"10|doajarticles::29c549c62b532a88d2b700a8e0959140","name":"Applied Engineering Letters"}} +{"enclosedEntityId":"10|doajarticles::29c6094ddd0e95cfbbf6904b671276f2","minDatsource":{"local_identifier":"10|doajarticles::29c6094ddd0e95cfbbf6904b671276f2","name":"Zoonoses"}} +{"enclosedEntityId":"10|doajarticles::29e93cc8b85cb7b8e97efd5a1e54f750","minDatsource":{"local_identifier":"10|doajarticles::29e93cc8b85cb7b8e97efd5a1e54f750","name":"Wirtschaftsdienst"}} +{"enclosedEntityId":"10|doajarticles::2d7182c08442361426031a606d5dfa51","minDatsource":{"local_identifier":"10|doajarticles::2d7182c08442361426031a606d5dfa51","name":"Jurnal Mekanik Terapan"}} +{"enclosedEntityId":"10|doajarticles::2f4f6ae448e6d8894402e1e578d8add5","minDatsource":{"local_identifier":"10|doajarticles::2f4f6ae448e6d8894402e1e578d8add5","name":"International Journal of Environmental Research and Public Health"}} +{"enclosedEntityId":"10|doajarticles::3169f7cfe0a07828e3cefe88f624a2be","minDatsource":{"local_identifier":"10|doajarticles::3169f7cfe0a07828e3cefe88f624a2be","name":"Phytopathology Research"}} +{"enclosedEntityId":"10|doajarticles::3184d8b36fce56f3b2f003d71a482208","minDatsource":{"local_identifier":"10|doajarticles::3184d8b36fce56f3b2f003d71a482208","name":"International Journal of Racket Sports Science"}} +{"enclosedEntityId":"10|doajarticles::381006a9dc56ce643cd07d1626032951","minDatsource":{"local_identifier":"10|doajarticles::381006a9dc56ce643cd07d1626032951","name":"Economy"}} +{"enclosedEntityId":"10|doajarticles::3c46128f8958c5c4eecedcaa4f901107","minDatsource":{"local_identifier":"10|doajarticles::3c46128f8958c5c4eecedcaa4f901107","name":"GiLE Journal of Skills Development"}} +{"enclosedEntityId":"10|doajarticles::3d68963fdb08bef6733f6eceba6d4529","minDatsource":{"local_identifier":"10|doajarticles::3d68963fdb08bef6733f6eceba6d4529","name":"The Internet Journal of Gynecology and Obstetrics"}} +{"enclosedEntityId":"10|doajarticles::3d8d349471917017eb42012997ccc9c4","minDatsource":{"local_identifier":"10|doajarticles::3d8d349471917017eb42012997ccc9c4","name":"International Journal on Applications of Graph Theory in Wireless ad hoc Networks and Sensor Networks"}} +{"enclosedEntityId":"10|doajarticles::3e182197bc8f29afb81f18af8576bc87","minDatsource":{"local_identifier":"10|doajarticles::3e182197bc8f29afb81f18af8576bc87","name":"اسلام و مدیریت"}} +{"enclosedEntityId":"10|doajarticles::3e8734d0869c179053138585b8148c6d","minDatsource":{"local_identifier":"10|doajarticles::3e8734d0869c179053138585b8148c6d","name":"Duke Journal of Gender Law & Policy"}} +{"enclosedEntityId":"10|doajarticles::3ebbbfe48c7b3ed215d75501cefcbafa","minDatsource":{"local_identifier":"10|doajarticles::3ebbbfe48c7b3ed215d75501cefcbafa","name":"Journal of BioScience and Biotechnology"}} +{"enclosedEntityId":"10|doajarticles::3f6a9897445d696f70272c1799472ccb","minDatsource":{"local_identifier":"10|doajarticles::3f6a9897445d696f70272c1799472ccb","name":"Arabic Literatures for Academic Zealots"}} +{"enclosedEntityId":"10|issn___print::0c5c2b0effc9281e07a078747254e8b7","minDatsource":{"local_identifier":"10|issn___print::0c5c2b0effc9281e07a078747254e8b7","name":"Peptides"}} +{"enclosedEntityId":"10|doajarticles::fcf556693aaa9aaf5e9d466cabb47001","minDatsource":{"local_identifier":"10|doajarticles::fcf556693aaa9aaf5e9d466cabb47001","name":"Records of the Australian Museum"}} +{"enclosedEntityId":"10|issn___print::adfbde8d7e865d2507cb180b9b43712b","minDatsource":{"local_identifier":"10|issn___print::adfbde8d7e865d2507cb180b9b43712b","name":"Current Opinion in Lipidology"}} +{"enclosedEntityId":"10|issn___print::094790b7d3268cb0aa34ef1a4eb073c8","minDatsource":{"local_identifier":"10|issn___print::094790b7d3268cb0aa34ef1a4eb073c8","name":"Genome Research"}} +{"enclosedEntityId":"10|issn___print::a212960a5e88e1d2e9169c10e8f5e345","minDatsource":{"local_identifier":"10|issn___print::a212960a5e88e1d2e9169c10e8f5e345","name":"Journal of Physics D Applied Physics"}} +{"enclosedEntityId":"10|issn___print::32a67ba964e0afcb945f2cb69952f00d","minDatsource":{"local_identifier":"10|issn___print::32a67ba964e0afcb945f2cb69952f00d","name":"Environmental Modeling & Assessment"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-3828fd20-3909-42ee-88dc-5ec33eff050d-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-3828fd20-3909-42ee-88dc-5ec33eff050d-c000.json.gz deleted file mode 100644 index edb75b9..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-3828fd20-3909-42ee-88dc-5ec33eff050d-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-5c1cd3a3-b306-457d-b39d-1e46b4f3f00b-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-5c1cd3a3-b306-457d-b39d-1e46b4f3f00b-c000.json new file mode 100644 index 0000000..8278b9c --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-5c1cd3a3-b306-457d-b39d-1e46b4f3f00b-c000.json @@ -0,0 +1,40 @@ +{"enclosedEntityId":"40|aka_________::01430745354b8004f6b2af63031fe3af","minGrant":{"code":"201295","funder":"Academy of Finland","local_identifier":"40|aka_________::01430745354b8004f6b2af63031fe3af","title":"Mechanisms of learning and memory: The role of CaMKII in the function of hippocampal place cells"}} +{"enclosedEntityId":"40|aka_________::054007a36d79c3c85f1871b2eb991d35","minGrant":{"code":"336408","funder":"Academy of Finland","local_identifier":"40|aka_________::054007a36d79c3c85f1871b2eb991d35","title":"Sustainable nanocellulose-based bioinks towards diverse material functionalities and therapeutic delivery of bioactive cues (SusCellInk)"}} +{"enclosedEntityId":"40|aka_________::0641940b9ee5df7fe02447e22c3cbf9f","minGrant":{"code":"317918","funder":"Academy of Finland","local_identifier":"40|aka_________::0641940b9ee5df7fe02447e22c3cbf9f","title":"Active Control of Spin Waves in YIG-based Magnonics"}} +{"enclosedEntityId":"40|aka_________::066705d02acb0c978dee1dbf89f235dd","minGrant":{"code":"128508","funder":"Academy of Finland","local_identifier":"40|aka_________::066705d02acb0c978dee1dbf89f235dd","title":"Insect herbivores in a multi-enemy world: the role of plant allelochemicals"}} +{"enclosedEntityId":"40|aka_________::0807f683e88d6d75f1dc152fd35ad613","minGrant":{"code":"118591","funder":"Academy of Finland","local_identifier":"40|aka_________::0807f683e88d6d75f1dc152fd35ad613","title":"Physical Activity and Genetic Factors in the Etiology of Metabolic Syndrome, Diabetes and Cardiovascular Disease"}} +{"enclosedEntityId":"40|aka_________::0a586655b6e4f6fc2870aef381061d83","minGrant":{"code":"131998","funder":"Academy of Finland","local_identifier":"40|aka_________::0a586655b6e4f6fc2870aef381061d83","title":"Post-war constructions of youth social problems: Policing youth, gender and delinquency in Finland, 1945-1970"}} +{"enclosedEntityId":"40|aka_________::0c0c832f6d9bb2045ade8e2b1b7b5202","minGrant":{"code":"127041","funder":"Academy of Finland","local_identifier":"40|aka_________::0c0c832f6d9bb2045ade8e2b1b7b5202","title":"Local Development, Social Innovation and Social Capital ¿ the Bulgarian case"}} +{"enclosedEntityId":"40|aka_________::0ee3adf25cc07bf65846f379d2ee882f","minGrant":{"code":"134278","funder":"Academy of Finland","local_identifier":"40|aka_________::0ee3adf25cc07bf65846f379d2ee882f","title":"The aquatic chemistry of aluminium silicates; Kinetics and Dynamics"}} +{"enclosedEntityId":"40|aka_________::1255b7377e2e41a81cd9ac9e0aea9739","minGrant":{"code":"78871","funder":"Academy of Finland","local_identifier":"40|aka_________::1255b7377e2e41a81cd9ac9e0aea9739","title":"Effects of magnesium lithospermate B on intracellular calsium release in endothelial cells"}} +{"enclosedEntityId":"40|aka_________::186c9e5f4705d0e85dcc7483abe4a7b5","minGrant":{"code":"120205","funder":"Academy of Finland","local_identifier":"40|aka_________::186c9e5f4705d0e85dcc7483abe4a7b5","title":"Seminar series on the nature and relations of social science knowledge and social epistemology."}} +{"enclosedEntityId":"40|aka_________::18814e2f047f4861e47dfd3261bf2021","minGrant":{"code":"207310","funder":"Academy of Finland","local_identifier":"40|aka_________::18814e2f047f4861e47dfd3261bf2021","title":"Factors contributingto the co-existence of invertebrate predators and fish in clay-turbid lakes - implications for lake restoration"}} +{"enclosedEntityId":"40|aka_________::1f59ecb9bd518ace77289e2c8bcd622f","minGrant":{"code":"294142","funder":"Academy of Finland","local_identifier":"40|aka_________::1f59ecb9bd518ace77289e2c8bcd622f","title":"Understanding and harnessing the quantum chemical origins of Nature's catalytic power"}} +{"enclosedEntityId":"40|aka_________::207d893289892d9a60527e095f6f57b5","minGrant":{"code":"286712","funder":"Academy of Finland","local_identifier":"40|aka_________::207d893289892d9a60527e095f6f57b5","title":"The role of lipids and membrane subdomains in B cell receptor organization and signaling"}} +{"enclosedEntityId":"40|aka_________::218329e2400634a14718b0c608659a73","minGrant":{"code":"124614","funder":"Academy of Finland","local_identifier":"40|aka_________::218329e2400634a14718b0c608659a73","title":"Res Publica: coordinating humans and things"}} +{"enclosedEntityId":"40|aka_________::228e875c365128b6df66f6a16e11ee3d","minGrant":{"code":"127624","funder":"Academy of Finland","local_identifier":"40|aka_________::228e875c365128b6df66f6a16e11ee3d","title":"Models and methods for integrating magnetoencephalography and dynamic magnetic resonance inverse imaging"}} +{"enclosedEntityId":"40|aka_________::22a615a6799c9d36261ff95cf268624a","minGrant":{"code":"118195","funder":"Academy of Finland","local_identifier":"40|aka_________::22a615a6799c9d36261ff95cf268624a","title":"Optical and Surface Properties of Nanoparticles"}} +{"enclosedEntityId":"40|aka_________::27326e10246f0ace453cd9ea7e4fe2cc","minGrant":{"code":"205917","funder":"Academy of Finland","local_identifier":"40|aka_________::27326e10246f0ace453cd9ea7e4fe2cc","title":"Development Ethics and the Rhetoric of Global Justice"}} +{"enclosedEntityId":"40|aka_________::282f59c4bb7fbc413b4b5d20be82ff2c","minGrant":{"code":"216312","funder":"Academy of Finland","local_identifier":"40|aka_________::282f59c4bb7fbc413b4b5d20be82ff2c","title":"Yhteishanke 15 RAS-SA"}} +{"enclosedEntityId":"40|aka_________::2a79dc440bea90acb421cb4262dd6ef3","minGrant":{"code":"118247","funder":"Academy of Finland","local_identifier":"40|aka_________::2a79dc440bea90acb421cb4262dd6ef3","title":"Paper, bioenergy and green chemicals from nonwood residues by a novel biorefinery (PEGRES)"}} +{"enclosedEntityId":"40|aka_________::2a7ab7393f164a2ae11b8a088d2972fb","minGrant":{"code":"115098","funder":"Academy of Finland","local_identifier":"40|aka_________::2a7ab7393f164a2ae11b8a088d2972fb","title":"Homology theory: A non-standard approach"}} +{"enclosedEntityId":"40|aka_________::2abaa9ec52576bcdfff43d49907f0f0b","minGrant":{"code":"81062","funder":"Academy of Finland","local_identifier":"40|aka_________::2abaa9ec52576bcdfff43d49907f0f0b","title":"Professional-client interaction in genetic counselling"}} +{"enclosedEntityId":"40|aka_________::31d52dbb6806e6bb7713bf36cad6241b","minGrant":{"code":"200554","funder":"Academy of Finland","local_identifier":"40|aka_________::31d52dbb6806e6bb7713bf36cad6241b","title":"CERC 3; Partisipation to the Workshop"}} +{"enclosedEntityId":"40|aka_________::36cd37c4efeb56bfe23c45b6d6d2f82a","minGrant":{"code":"218490","funder":"Academy of Finland","local_identifier":"40|aka_________::36cd37c4efeb56bfe23c45b6d6d2f82a","title":"Molecular mechanisms of STAT6 mediated transcription"}} +{"enclosedEntityId":"40|aka_________::371d2e092629b03f39bef41058d2266f","minGrant":{"code":"259176","funder":"Academy of Finland","local_identifier":"40|aka_________::371d2e092629b03f39bef41058d2266f","title":"6th Japan-Finnish Biotechnology Symposium in Sendai, Japan, June 2012"}} +{"enclosedEntityId":"40|aka_________::39ebb6f9d7647c3588d1de5d6bdb1ea6","minGrant":{"code":"259562","funder":"Academy of Finland","local_identifier":"40|aka_________::39ebb6f9d7647c3588d1de5d6bdb1ea6","title":"Comprehensive view on process of dispersal with individual tracking, genetic analysis and modelling"}} +{"enclosedEntityId":"40|aka_________::3dfbc1c05c6829be1142741f50637c70","minGrant":{"code":"251623","funder":"Academy of Finland","local_identifier":"40|aka_________::3dfbc1c05c6829be1142741f50637c70","title":"Stereospecific distribution of fatty acids in acylglycerols of nutritionally important lipids"}} +{"enclosedEntityId":"40|aka_________::40e9dd5da112258f9f8cab8fe9089341","minGrant":{"code":"203484","funder":"Academy of Finland","local_identifier":"40|aka_________::40e9dd5da112258f9f8cab8fe9089341","title":"research of microfossiles"}} +{"enclosedEntityId":"40|aka_________::4146425b0e0da208229fd572c7f06f3c","minGrant":{"code":"218149","funder":"Academy of Finland","local_identifier":"40|aka_________::4146425b0e0da208229fd572c7f06f3c","title":"Development of radiation hard silicon particle detectors for CERN LHC Upgrade"}} +{"enclosedEntityId":"40|aka_________::42207d9e78fa2daa52ff57d052defc5d","minGrant":{"code":"108291","funder":"Academy of Finland","local_identifier":"40|aka_________::42207d9e78fa2daa52ff57d052defc5d","title":"Reconstruction of Holocene climatic changes on the basis of microfossils in Finnish and Polish lake sediments"}} +{"enclosedEntityId":"40|aka_________::42fdc9c8db6e540ddd9698c1e0039d5f","minGrant":{"code":"325117","funder":"Academy of Finland","local_identifier":"40|aka_________::42fdc9c8db6e540ddd9698c1e0039d5f","title":"Model Management Systems: Machine learning meets Database Systems (MLDB) / Consortium: MLDB"}} +{"enclosedEntityId":"40|aka_________::434daecbc44532c03b0313928f0c321d","minGrant":{"code":"128040","funder":"Academy of Finland","local_identifier":"40|aka_________::434daecbc44532c03b0313928f0c321d","title":"A geological evolution model for effective utilization of the lithium deposits of Central Ostrobothnia region"}} +{"enclosedEntityId":"40|aka_________::45331b6aaab07314eddb989a074e597c","minGrant":{"code":"212376","funder":"Academy of Finland","local_identifier":"40|aka_________::45331b6aaab07314eddb989a074e597c","title":"Vernacular universals vs. contact-induced language change"}} +{"enclosedEntityId":"40|aka_________::45dc0e38ea84f1d81c05067da286733c","minGrant":{"code":"211611","funder":"Academy of Finland","local_identifier":"40|aka_________::45dc0e38ea84f1d81c05067da286733c","title":"Julkaisutoimintaan"}} +{"enclosedEntityId":"40|aka_________::463c01e55d9aae7434e5d34119020f97","minGrant":{"code":"133603","funder":"Academy of Finland","local_identifier":"40|aka_________::463c01e55d9aae7434e5d34119020f97","title":"The effects of intensive bio-fuel production and use on regional air quality and global climate"}} +{"enclosedEntityId":"40|aka_________::48c72a18f56e02deeab699caf23aa2cc","minGrant":{"code":"131129","funder":"Academy of Finland","local_identifier":"40|aka_________::48c72a18f56e02deeab699caf23aa2cc","title":"Postdoctoral Research at the University of Victoria in theoretical physics, in Canada"}} +{"enclosedEntityId":"40|aka_________::4a3a847ec9bf91c49da51296973a1b4c","minGrant":{"code":"203047","funder":"Academy of Finland","local_identifier":"40|aka_________::4a3a847ec9bf91c49da51296973a1b4c","title":"Kansainvälinen sellu- ja paperitekniikan tutkijakoulu"}} +{"enclosedEntityId":"40|aka_________::4b64d3704910ea2d6951d5f2c6a39cf8","minGrant":{"code":"109793","funder":"Academy of Finland","local_identifier":"40|aka_________::4b64d3704910ea2d6951d5f2c6a39cf8","title":"Environments and Evolutionary Dynamics of Fossil and Recent Land Mammal Communities: Combining Field Work, Data Mining and Modelling Approaches"}} +{"enclosedEntityId":"40|sshrc_______::1e5e62235d094afd01cd56e65112fc63","minGrant":{"code":"unidentified","funder":"Social Sciences and Humanities Research Council","local_identifier":"40|sshrc_______::1e5e62235d094afd01cd56e65112fc63","title":"unidentified"}} +{"enclosedEntityId":"40|wt__________::1e5e62235d094afd01cd56e65112fc63","minGrant":{"code":"unidentified","funder":"Wellcome Trust","local_identifier":"40|wt__________::1e5e62235d094afd01cd56e65112fc63","title":"unidentified"}} +{"enclosedEntityId":"40|nsf_________::7a5b4fbb6ed8545679194fe45a984ec9","minGrant":{"code":"1202095","funder":"National Science Foundation","local_identifier":"40|nsf_________::7a5b4fbb6ed8545679194fe45a984ec9","title":"Plasma-dynamics in Nano/Micro-Structures for RF to THz Applications"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-5c1cd3a3-b306-457d-b39d-1e46b4f3f00b-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-5c1cd3a3-b306-457d-b39d-1e46b4f3f00b-c000.json.gz deleted file mode 100644 index 05216b2..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-5c1cd3a3-b306-457d-b39d-1e46b4f3f00b-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-7879c42e-0911-493f-90a4-5517475c8c49-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-7879c42e-0911-493f-90a4-5517475c8c49-c000.json new file mode 100644 index 0000000..adb4d74 --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-7879c42e-0911-493f-90a4-5517475c8c49-c000.json @@ -0,0 +1,20 @@ +{"enclosedEntityId":"50|doi_________::0840e1cfdf65dd3abdeee453e234e034","minProduct":{"doi":"10.1177/0267323118790163","local_identifier":"50|doi_________::0840e1cfdf65dd3abdeee453e234e034","title":"Celebrities in Czech politics in 1996–2013"}} +{"enclosedEntityId":"50|doi_________::479f67164e2f1d21baafbfbc3b12851d","minProduct":{"doi":"10.1038/ejhg.2014.31","local_identifier":"50|doi_________::479f67164e2f1d21baafbfbc3b12851d"}} +{"enclosedEntityId":"50|doi_________::7078f75053e8c1bff562adbce2d81fe2","minProduct":{"doi":"10.1016/j.marmicro.2013.04.005","local_identifier":"50|doi_________::7078f75053e8c1bff562adbce2d81fe2","title":"Effect of dissolved oxygen concentration on planktonic foraminifera through laboratory culture experiments and implications for oceanic anoxic events"}} +{"enclosedEntityId":"50|doi_________::86d7aecc7029c70836d268b4ea9b974d","minProduct":{"doi":"10.1111/bij.12378","local_identifier":"50|doi_________::86d7aecc7029c70836d268b4ea9b974d","title":"Phylogeography of the humbug damselfish,Dascyllus aruanus(Linnaeus, 1758): evidence of Indo-Pacific vicariance and genetic differentiation of peripheral populations"}} +{"enclosedEntityId":"50|doi_________::a83b2a6d96883a5f2d360fb5fb7db8ae","minProduct":{"doi":"10.3389/fimmu.2018.01146","local_identifier":"50|doi_________::a83b2a6d96883a5f2d360fb5fb7db8ae"}} +{"enclosedEntityId":"50|doi_________::af75a7dc6b0ba2ca2349a68588ef877e","minProduct":{"doi":"10.1038/srep37852","local_identifier":"50|doi_________::af75a7dc6b0ba2ca2349a68588ef877e"}} +{"enclosedEntityId":"50|doi_dedup___::079ebd73b673282559d6f4befe003d47","minProduct":{"doi":"10.1002/cne.23131","local_identifier":"50|doi_dedup___::079ebd73b673282559d6f4befe003d47","title":"Galanin gene expression and effects of its knock‐down on the development of the nervous system in larval zebrafish"}} +{"enclosedEntityId":"50|doi_________::0363a8e9be16f62cdfd58b3def423a57","minProduct":{"doi":"10.3853/j.0067-1975.49.1997.1260","local_identifier":"50|doi_________::0363a8e9be16f62cdfd58b3def423a57","title":"Amphipoda from the South Pacific: Western Samoa"}} +{"enclosedEntityId":"50|doi_________::0888444d70ea8e3bcbc12d9d281fce22","minProduct":{"doi":"10.1063/1.4928566","local_identifier":"50|doi_________::0888444d70ea8e3bcbc12d9d281fce22","title":"Dark-to-arc transition in field emission dominated atmospheric microdischarges"}} +{"enclosedEntityId":"50|doi_________::0941e6d7170c6bbfb40721aa9ecff245","minProduct":{"doi":"10.1007/s10666-018-9616-4","local_identifier":"50|doi_________::0941e6d7170c6bbfb40721aa9ecff245","title":"The Strategic Impact of Adaptation in a Transboundary Pollution Dynamic Game"}} +{"enclosedEntityId":"50|doi_________::1f56d1b5a2e7c9223c3479476c9c3491","minProduct":{"doi":"10.1534/genetics.116.190215","local_identifier":"50|doi_________::1f56d1b5a2e7c9223c3479476c9c3491"}} +{"enclosedEntityId":"50|doi_________::579e3d4992475c1b375494c9e19b24ce","minProduct":{"doi":"10.1186/s13023-016-0515-y","local_identifier":"50|doi_________::579e3d4992475c1b375494c9e19b24ce","title":"8th European Conference on Rare Diseases & Orphan Products (ECRD 2016)"}} +{"enclosedEntityId":"50|doi_dedup___::893d0a5c08409822dbb82b8973a42909","minProduct":{"doi":"10.1111/j.1471-4159.2004.02190.x","local_identifier":"50|doi_dedup___::893d0a5c08409822dbb82b8973a42909","title":"Neurochemical and behavioural changes in zebrafish Danio rerio after systemic administration of 6‐hydroxydopamine and 1‐methyl‐4‐phenyl‐1,2,3,6‐tetrahydropyridine"}} +{"enclosedEntityId":"50|doi_dedup___::8b0484d2afbd9b09116ef60c18044964","minProduct":{"doi":"10.1016/0196-9781(89)90045-4","local_identifier":"50|doi_dedup___::8b0484d2afbd9b09116ef60c18044964","title":"Tissue-specific sex differences in galanin-like immunoreactivity and galanin mRNA during development in the rat"}} +{"enclosedEntityId":"50|doi_________::1034996a07e1496dde6048008e2187de","minProduct":{"doi":"10.1007/s10666-016-9501-y","local_identifier":"50|doi_________::1034996a07e1496dde6048008e2187de","title":"Mitigation vs. Adaptation: Analytic Models for Policy Assessment"}} +{"enclosedEntityId":"50|doi_________::81a64d0689f4a2f93995b81574d2fcbe","minProduct":{"doi":"10.1016/s0003-3472(71)80005-2","local_identifier":"50|doi_________::81a64d0689f4a2f93995b81574d2fcbe","title":"Apparent effect of prior experience on a habitat preference exhibited by the reef fish, Dascyllus aruanus (Pisces: Pomacentridae)"}} +{"enclosedEntityId":"50|doi_dedup___::7a40bb2e9ad2e96d6de26787ab1c310b","minProduct":{"doi":"10.1101/gr.107987.110","local_identifier":"50|doi_dedup___::7a40bb2e9ad2e96d6de26787ab1c310b","title":"Reshaping the gut microbiome with bacterial transplantation and antibiotic intake"}} +{"enclosedEntityId":"50|doi_dedup___::233bd897c6654f274059272b8740b47f","minProduct":{"doi":"10.1097/mol.0b013e32835918cd","local_identifier":"50|doi_dedup___::233bd897c6654f274059272b8740b47f","title":"Paternal transmission, cardiovascular risk factors and epigenetics"}} +{"enclosedEntityId":"50|doi_________::1647515a356946e4857c817afc7bfa38","minProduct":{"doi":"10.1088/0022-3727/47/50/503001","local_identifier":"50|doi_________::1647515a356946e4857c817afc7bfa38","title":"Microscale gas breakdown: ion-enhanced field emission and the modified Paschen’s curve"}} +{"enclosedEntityId":"50|doi_dedup___::6481edac8ee74c1f1bfa9c0fdd3e6f11","minProduct":{"doi":"10.1038/nature10811","local_identifier":"50|doi_dedup___::6481edac8ee74c1f1bfa9c0fdd3e6f11","title":"The Drosophila melanogaster Genetic Reference Panel"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-7879c42e-0911-493f-90a4-5517475c8c49-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-7879c42e-0911-493f-90a4-5517475c8c49-c000.json.gz deleted file mode 100644 index 4222c29..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-7879c42e-0911-493f-90a4-5517475c8c49-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b06ba3f4-176a-4fde-9a5c-746b4e9624a3-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b06ba3f4-176a-4fde-9a5c-746b4e9624a3-c000.json new file mode 100644 index 0000000..2bc399c --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b06ba3f4-176a-4fde-9a5c-746b4e9624a3-c000.json @@ -0,0 +1,32 @@ +{"enclosedEntityId":"50|__bioTools__::3640ff1dd63f33b085265311638a2ef1","minProduct":{"local_identifier":"50|__bioTools__::3640ff1dd63f33b085265311638a2ef1","title":"Information-based Similarity Toolbox"}} +{"enclosedEntityId":"50|__bioTools__::6a8a9601934e1d566bdc9e387f106cad","minProduct":{"local_identifier":"50|__bioTools__::6a8a9601934e1d566bdc9e387f106cad","title":"MuSiC suite: PoPMuSiC, HoTMuSiC, and SNPMuSiC"}} +{"enclosedEntityId":"50|__bioTools__::8e5431247fb491ae7cf46b4806abbfaf","minProduct":{"local_identifier":"50|__bioTools__::8e5431247fb491ae7cf46b4806abbfaf","title":"A GUI for bacterial identification"}} +{"enclosedEntityId":"50|__bioTools__::cee88972c28033daaca8786a265da604","minProduct":{"local_identifier":"50|__bioTools__::cee88972c28033daaca8786a265da604","title":"OligoCalc"}} +{"enclosedEntityId":"50|__bioTools__::e1594e9122869da8eec86671ab713259","minProduct":{"local_identifier":"50|__bioTools__::e1594e9122869da8eec86671ab713259","title":"PaSS"}} +{"enclosedEntityId":"50|__bioTools__::ec9a52d4c30788879d1e5fd097171f5f","minProduct":{"local_identifier":"50|__bioTools__::ec9a52d4c30788879d1e5fd097171f5f","title":"BiSeq"}} +{"enclosedEntityId":"50|__bioTools__::f50acbc8c02cd875a9e83a78ed8f285d","minProduct":{"local_identifier":"50|__bioTools__::f50acbc8c02cd875a9e83a78ed8f285d","title":"BlastKOALA"}} +{"enclosedEntityId":"50|dedup_wf_002::47f351f4d17708772645eb4f556d802d","minProduct":{"local_identifier":"50|dedup_wf_002::47f351f4d17708772645eb4f556d802d","title":"CodeSearchNet software on GitHub"}} +{"enclosedEntityId":"50|dedup_wf_002::48d5bd6174ba13c5e20b577e71f3a330","minProduct":{"local_identifier":"50|dedup_wf_002::48d5bd6174ba13c5e20b577e71f3a330","title":"Logiciel WAFA (Welfare assessment of farm animal). Outil d'aide à l'évaluation du bien-être des animaux (outil de saisie, base de données, logiciel de calcul de scores de bien-être, outil de simulation, ressource d'informations, pages web)"}} +{"enclosedEntityId":"50|dedup_wf_002::4d37f1fd55ae2c11290d4b161060f302","minProduct":{"local_identifier":"50|dedup_wf_002::4d37f1fd55ae2c11290d4b161060f302","title":"DeepCCA software on GitHub"}} +{"enclosedEntityId":"50|dedup_wf_002::a36d36bcd9e8fe405cfc82fc8b84294f","minProduct":{"local_identifier":"50|dedup_wf_002::a36d36bcd9e8fe405cfc82fc8b84294f","title":"questionr"}} +{"enclosedEntityId":"50|dedup_wf_002::b82b856ebd89cdd36b8107e475280e9e","minProduct":{"local_identifier":"50|dedup_wf_002::b82b856ebd89cdd36b8107e475280e9e","title":"snmalloc software on GitHub"}} +{"enclosedEntityId":"50|doi_________::07ca00c92c3093a76fcae84b2185ed85","minProduct":{"doi":"10.24433/co.2868322.v1","local_identifier":"50|doi_________::07ca00c92c3093a76fcae84b2185ed85","title":"Image Quality Environment for FPGA Floating-Point Video Processing Library"}} +{"enclosedEntityId":"50|doi_________::094b6aff40ccf64df47063a59ab83da1","minProduct":{"doi":"10.5281/zenodo.3724469","local_identifier":"50|doi_________::094b6aff40ccf64df47063a59ab83da1","title":"stonerlab/Stoner-PythonCode: v0.9.5 Bugfix and improvement release"}} +{"enclosedEntityId":"50|doi_________::15b9e4e4e1102ae67e1b0e567a7593fc","minProduct":{"doi":"10.5281/zenodo.4763538","local_identifier":"50|doi_________::15b9e4e4e1102ae67e1b0e567a7593fc","title":"poliastro/poliastro: poliastro 0.15.0 (Earth edition)"}} +{"enclosedEntityId":"50|doi_________::16be0dfa4fa679bed49d01fc4471cabd","minProduct":{"doi":"10.5281/zenodo.1251581","local_identifier":"50|doi_________::16be0dfa4fa679bed49d01fc4471cabd","title":"sotorrent/posthistory-extractor: Refactored release"}} +{"enclosedEntityId":"50|doi_________::185d0f8b38dfeb9f567f30a3190a916c","minProduct":{"doi":"10.5281/zenodo.3979689","local_identifier":"50|doi_________::185d0f8b38dfeb9f567f30a3190a916c","title":"InsightSoftwareConsortium/ITKElastix: ITKElastix 0.6.3"}} +{"enclosedEntityId":"50|__bioTools__::022faad7a573c4f12301c486815a3fa5","minProduct":{"local_identifier":"50|__bioTools__::022faad7a573c4f12301c486815a3fa5","title":"petalNet"}} +{"enclosedEntityId":"50|__bioTools__::0c42b0be275910c096d2bacaccff89ad","minProduct":{"local_identifier":"50|__bioTools__::0c42b0be275910c096d2bacaccff89ad","title":"Diversity Workbench"}} +{"enclosedEntityId":"50|__bioTools__::58861f6ae561f40b585e44c0734b4a70","minProduct":{"local_identifier":"50|__bioTools__::58861f6ae561f40b585e44c0734b4a70","title":"HMCan"}} +{"enclosedEntityId":"50|__bioTools__::85fb610b34dc395431f8eda781566c4e","minProduct":{"local_identifier":"50|__bioTools__::85fb610b34dc395431f8eda781566c4e","title":"iAssembler"}} +{"enclosedEntityId":"50|__bioTools__::97b678c1b70fe0b13197be61111a3513","minProduct":{"local_identifier":"50|__bioTools__::97b678c1b70fe0b13197be61111a3513","title":"DNA Baser"}} +{"enclosedEntityId":"50|__bioTools__::9961460fe6c9852c36bdbfea8b7fbf2c","minProduct":{"local_identifier":"50|__bioTools__::9961460fe6c9852c36bdbfea8b7fbf2c","title":"lncRNAdb"}} +{"enclosedEntityId":"50|__bioTools__::c802825f3b3a94343e30a18ed23d4786","minProduct":{"local_identifier":"50|__bioTools__::c802825f3b3a94343e30a18ed23d4786","title":"Gene Set Builder"}} +{"enclosedEntityId":"50|dedup_wf_002::629f6117216829e2582934ac1d845d9e","minProduct":{"local_identifier":"50|dedup_wf_002::629f6117216829e2582934ac1d845d9e","title":"Support tool for the overall assessment of animal welfare"}} +{"enclosedEntityId":"50|dedup_wf_002::6ae16f2f40fa342f71a5e1a0a11eb962","minProduct":{"local_identifier":"50|dedup_wf_002::6ae16f2f40fa342f71a5e1a0a11eb962","title":"transforms3d software on GitHub"}} +{"enclosedEntityId":"50|dedup_wf_002::b4f06f3b28fa9cd2d7b9185e3f6604eb","minProduct":{"local_identifier":"50|dedup_wf_002::b4f06f3b28fa9cd2d7b9185e3f6604eb","title":"varcmp software on GitHub"}} +{"enclosedEntityId":"50|dedup_wf_002::e3373c97b277d3f0f3f6c21d688ae520","minProduct":{"local_identifier":"50|dedup_wf_002::e3373c97b277d3f0f3f6c21d688ae520","title":"OpenDSS-G software on SourceForge"}} +{"enclosedEntityId":"50|doi_________::0abc081de6a2fe9506b20e8b58389cb2","minProduct":{"doi":"10.5281/zenodo.7254817","local_identifier":"50|doi_________::0abc081de6a2fe9506b20e8b58389cb2","title":"napari/magicgui: v0.6.0"}} +{"enclosedEntityId":"50|doi_________::0c6f89082f07ea5c737eaf25d0712e01","minProduct":{"doi":"10.18129/b9.bioc.adam","local_identifier":"50|doi_________::0c6f89082f07ea5c737eaf25d0712e01","title":"ADAM"}} +{"enclosedEntityId":"50|doi_________::115332ff9a5e3d15a4c79bc3e819e165","minProduct":{"doi":"10.5281/zenodo.573248","local_identifier":"50|doi_________::115332ff9a5e3d15a4c79bc3e819e165","title":"PeterMulhair/planets: First release of Science software"}} +{"enclosedEntityId":"50|doi_________::115a8028953b0488dbb04c0f99b9dd62","minProduct":{"doi":"10.5281/zenodo.50641","local_identifier":"50|doi_________::115a8028953b0488dbb04c0f99b9dd62","title":"libfreenect2: Release 0.2"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b06ba3f4-176a-4fde-9a5c-746b4e9624a3-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b06ba3f4-176a-4fde-9a5c-746b4e9624a3-c000.json.gz deleted file mode 100644 index 7539d50..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b06ba3f4-176a-4fde-9a5c-746b4e9624a3-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b720d208-b8b9-4fa6-8898-444f4172f408-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b720d208-b8b9-4fa6-8898-444f4172f408-c000.json new file mode 100644 index 0000000..4d30d2d --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b720d208-b8b9-4fa6-8898-444f4172f408-c000.json @@ -0,0 +1,34 @@ +{"enclosedEntityId":"20|openorgs____::131a553780dbaac8a5a9d2ed72184a4e","minOrganization":{"local_identifier":"20|openorgs____::131a553780dbaac8a5a9d2ed72184a4e","name":"Balter Medical (Norway)","ror":"https://ror.org/028521m47"}} +{"enclosedEntityId":"20|openorgs____::261fc6624613dfa43a9112ed1f3eeb24","minOrganization":{"local_identifier":"20|openorgs____::261fc6624613dfa43a9112ed1f3eeb24","name":"Seminario Bíblico de Colombia Fundación Universitaria","ror":"https://ror.org/0454xt523"}} +{"enclosedEntityId":"20|openorgs____::4638f0f5f68a09d88bdad3d219969e87","minOrganization":{"local_identifier":"20|openorgs____::4638f0f5f68a09d88bdad3d219969e87","name":"Global Health Association of Miami","ror":"https://ror.org/03vwqmf10"}} +{"enclosedEntityId":"20|openorgs____::a6e7e05b0f3f2b5fad7636b5c91ca564","minOrganization":{"isni":"0000000417629411","local_identifier":"20|openorgs____::a6e7e05b0f3f2b5fad7636b5c91ca564","name":"Al Ain University of Science and Technology","ror":"https://ror.org/023abrt21","wikidata":"Q4703454"}} +{"enclosedEntityId":"20|openorgs____::d425e923bef9e604ceba2582e98655d0","minOrganization":{"local_identifier":"20|openorgs____::d425e923bef9e604ceba2582e98655d0","name":"Dà-Jiāng Innovations Science and Technology (China)","ror":"https://ror.org/04fmkfb67","wikidata":"Q16924332"}} +{"enclosedEntityId":"20|openorgs____::dea7c5dc0120ae8387e55f73f06f15b9","minOrganization":{"local_identifier":"20|openorgs____::dea7c5dc0120ae8387e55f73f06f15b9","name":"Prombiom Tecnologia (Brazil)","ror":"https://ror.org/00cm1b035"}} +{"enclosedEntityId":"20|openorgs____::fdf2b3cf0d29f81fe747f98f925ddd25","minOrganization":{"local_identifier":"20|openorgs____::fdf2b3cf0d29f81fe747f98f925ddd25","name":"Oita Prefectural Yufu Support School","ror":"https://ror.org/00bxxsn28"}} +{"enclosedEntityId":"20|openorgs____::1d0d3689a72a8d8e72acb42938875c37","minOrganization":{"local_identifier":"20|openorgs____::1d0d3689a72a8d8e72acb42938875c37","name":"Grosses LF Foghts Fund","ror":"https://ror.org/01bn4mn58"}} +{"enclosedEntityId":"20|openorgs____::2ffbe7a78ecfe5cc6fc6747acafbf38f","minOrganization":{"isni":"0000000404018291","local_identifier":"20|openorgs____::2ffbe7a78ecfe5cc6fc6747acafbf38f","name":"Western Hospital","ror":"https://ror.org/05qv7gx64"}} +{"enclosedEntityId":"20|openorgs____::470cd1fef48a9d135b6e1b839305fa77","minOrganization":{"isni":"0000000417713349","local_identifier":"20|openorgs____::470cd1fef48a9d135b6e1b839305fa77","name":"China-Japan Friendship Hospital","ror":"https://ror.org/037cjxp13","wikidata":"Q5013040"}} +{"enclosedEntityId":"20|openorgs____::5f5774bad85a5a5cabc37ba96f68fc31","minOrganization":{"local_identifier":"20|openorgs____::5f5774bad85a5a5cabc37ba96f68fc31","name":"Freedom to Live","ror":"https://ror.org/04m7y9d03","wikidata":"Q30287250"}} +{"enclosedEntityId":"20|openorgs____::69d8c22104a233aed4fb91a66964f314","minOrganization":{"isni":"0000000406469053","local_identifier":"20|openorgs____::69d8c22104a233aed4fb91a66964f314","name":"Beijing Academy of Agricultural and Forestry Sciences","ror":"https://ror.org/04trzn023"}} +{"enclosedEntityId":"20|openorgs____::71eda1b0e57bbe2289bccf5dd6deda96","minOrganization":{"local_identifier":"20|openorgs____::71eda1b0e57bbe2289bccf5dd6deda96","name":"PRO-BIO Association of Organic Farmers","ror":"https://ror.org/022z4zx74"}} +{"enclosedEntityId":"20|openorgs____::d697e623f6f0070df329832637772a26","minOrganization":{"isni":"0000000417708099","local_identifier":"20|openorgs____::d697e623f6f0070df329832637772a26","name":"Universidad Tecnológica del Sureste de Veracruz","ror":"https://ror.org/003gh8k45","wikidata":"Q30293735"}} +{"enclosedEntityId":"20|fct_________::d257dbe94f60771eca25f07ea1ca2446","minOrganization":{"isni":"0000000119310410","local_identifier":"20|fct_________::d257dbe94f60771eca25f07ea1ca2446","name":"Governo da Republica Portuguesa Ministério da Defesa Nacional","rinGold":"RingGold:70945"}} +{"enclosedEntityId":"20|openorgs____::3e4d9491ea8eb63ce3f680f279f2b36a","minOrganization":{"local_identifier":"20|openorgs____::3e4d9491ea8eb63ce3f680f279f2b36a","name":"Uncopiers","ror":"https://ror.org/01f0erc76"}} +{"enclosedEntityId":"20|openorgs____::6f2c0aef6169d86ff799b705bf0da2cc","minOrganization":{"isni":"0000000404597684","local_identifier":"20|openorgs____::6f2c0aef6169d86ff799b705bf0da2cc","name":"Sarah Cannon Research Institute","ror":"https://ror.org/03cp5cj42","wikidata":"Q7422164"}} +{"enclosedEntityId":"20|openorgs____::cfb2ec0149c1b30a7577416f936b4f9a","minOrganization":{"isni":"0000000094855579","local_identifier":"20|openorgs____::cfb2ec0149c1b30a7577416f936b4f9a","name":"Alabama State University","ror":"https://ror.org/01eedy375","wikidata":"Q941524"}} +{"enclosedEntityId":"20|openorgs____::2d60bcec8a36ee26b92286dd541b91db","minOrganization":{"local_identifier":"20|openorgs____::2d60bcec8a36ee26b92286dd541b91db","name":"Cardiaccs (Norway)","ror":"https://ror.org/039726g63"}} +{"enclosedEntityId":"20|openorgs____::566c614eb2fbe378e1c5e536fc897a81","minOrganization":{"local_identifier":"20|openorgs____::566c614eb2fbe378e1c5e536fc897a81","name":"UniCredit (United States)","ror":"https://ror.org/01mrzcs29","wikidata":"Q30338823"}} +{"enclosedEntityId":"20|openorgs____::6879960765450300a5e651f6775f3337","minOrganization":{"isni":"0000000453456247","local_identifier":"20|openorgs____::6879960765450300a5e651f6775f3337","name":"Universidad Nacional de Pilar","ror":"https://ror.org/03p5xhb29","wikidata":"Q1570663"}} +{"enclosedEntityId":"20|openorgs____::756f0578d1599a0232c37048829aa1a8","minOrganization":{"isni":"0000000446599514","local_identifier":"20|openorgs____::756f0578d1599a0232c37048829aa1a8","name":"Biomer Technology (United Kingdom)","ror":"https://ror.org/01szvzc53"}} +{"enclosedEntityId":"20|openorgs____::a1399ef06ded575a2d8b763f0626a8d8","minOrganization":{"isni":"0000000119457711","local_identifier":"20|openorgs____::a1399ef06ded575a2d8b763f0626a8d8","name":"Instituto de Investigaciones Marinas","ror":"https://ror.org/01603fg59"}} +{"enclosedEntityId":"20|openorgs____::a569e7bc1c32ff065802c88e456892bc","minOrganization":{"local_identifier":"20|openorgs____::a569e7bc1c32ff065802c88e456892bc","name":"MSD (Lithuania)","ror":"https://ror.org/03cnjaa38","wikidata":"Q30269105"}} +{"enclosedEntityId":"20|openorgs____::b8b8ca674452579f3f593d9f5e557483","minOrganization":{"isni":"0000000123318773","local_identifier":"20|openorgs____::b8b8ca674452579f3f593d9f5e557483","name":"University College Cork","ror":"https://ror.org/03265fv13","wikidata":"Q1574185"}} +{"enclosedEntityId":"20|openorgs____::33f2ec599c4e086e750c126f19552f67","minOrganization":{"isni":"0000000404841472","local_identifier":"20|openorgs____::33f2ec599c4e086e750c126f19552f67","name":"Purdue University System","ror":"https://ror.org/05p8z3f47","wikidata":"Q929715"}} +{"enclosedEntityId":"20|openorgs____::45327df9862294107f84a90daa1e0cbc","minOrganization":{"isni":"0000000405460241","local_identifier":"20|openorgs____::45327df9862294107f84a90daa1e0cbc","name":"National Taiwan University of Arts","ror":"https://ror.org/05bqach95","wikidata":"Q6978838"}} +{"enclosedEntityId":"20|openorgs____::c9f6cad9752276cb556e88a5111e86aa","minOrganization":{"isni":"0000000419369684","local_identifier":"20|openorgs____::c9f6cad9752276cb556e88a5111e86aa","name":"University of California, Davis","ror":"https://ror.org/05rrcem69","wikidata":"Q129421"}} +{"enclosedEntityId":"20|openorgs____::6a7b1b4c40a067a1f209de6867fe094d","minOrganization":{"isni":"0000000419368948","local_identifier":"20|openorgs____::6a7b1b4c40a067a1f209de6867fe094d","name":"University of Oxford","ror":"https://ror.org/052gg0110","wikidata":"Q6786826"}} +{"enclosedEntityId":"20|openorgs____::013584bf907fdcb40bcec00b5a78fc12","minOrganization":{"isni":"0000000419372197","local_identifier":"20|openorgs____::013584bf907fdcb40bcec00b5a78fc12","name":"Purdue University West Lafayette","ror":"https://ror.org/04keq6987","wikidata":"Q217741"}} +{"enclosedEntityId":"20|openorgs____::4dee7ea0d1fd1b48c199ee744e21dc13","minOrganization":{"isni":"0000000119425154","local_identifier":"20|openorgs____::4dee7ea0d1fd1b48c199ee744e21dc13","name":"Helmholtz Association of German Research Centres","ror":"https://ror.org/0281dp749","wikidata":"Q679913"}} +{"enclosedEntityId":"20|openorgs____::0ff89de99d4a8f4b04cb162bcb5740cf","minOrganization":{"isni":"0000000121807477","local_identifier":"20|openorgs____::0ff89de99d4a8f4b04cb162bcb5740cf","name":"Australian National University","ror":"https://ror.org/019wvm592","wikidata":"Q127990"}} +{"enclosedEntityId":"20|openorgs____::9d3eb607509fcacae1c5a7f25e36e3e5","minOrganization":{"isni":"0000000121496795","local_identifier":"20|openorgs____::9d3eb607509fcacae1c5a7f25e36e3e5","name":"University of Warmia and Mazury in Olsztyn","ror":"https://ror.org/05s4feg49","wikidata":"Q1733433"}} +{"enclosedEntityId":"20|openorgs____::0ae431b820e4c33db8967fbb2b919150","minOrganization":{"isni":"0000000404102071","local_identifier":"20|openorgs____::0ae431b820e4c33db8967fbb2b919150","name":"University of Helsinki","ror":"https://ror.org/040af2s02","wikidata":"Q28695"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b720d208-b8b9-4fa6-8898-444f4172f408-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b720d208-b8b9-4fa6-8898-444f4172f408-c000.json.gz deleted file mode 100644 index 3a1889d..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-b720d208-b8b9-4fa6-8898-444f4172f408-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-d82887dd-9c95-49fc-9efc-0feee839fa0b-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-d82887dd-9c95-49fc-9efc-0feee839fa0b-c000.json new file mode 100644 index 0000000..0596d75 --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-d82887dd-9c95-49fc-9efc-0feee839fa0b-c000.json @@ -0,0 +1,44 @@ +{"enclosedEntityId":"10|06cdd3ff4700::69433f5cb51fe4c3959d9a892c8eff4a","minVenue":{"local_identifier":"temp_venue__::69433f5cb51fe4c3959d9a892c8eff4a","name":"Analyse de réseaux pour les sciences sociales"}} +{"enclosedEntityId":"10|doajarticles::008cf70ae79b184f4fe3c0547e98a582","minVenue":{"local_identifier":"temp_venue__::008cf70ae79b184f4fe3c0547e98a582","name":"Revista Electrónica Sinéctica"}} +{"enclosedEntityId":"10|doajarticles::00eb4160e4726f60887b60fa7df15e32","minVenue":{"local_identifier":"temp_venue__::00eb4160e4726f60887b60fa7df15e32","name":"Jurnal Soeropati"}} +{"enclosedEntityId":"10|doajarticles::0c217533c6e0bfe1a4a36a093f8c23e2","minVenue":{"local_identifier":"temp_venue__::0c217533c6e0bfe1a4a36a093f8c23e2","name":"Acta Zoologica Academiae Scientiarum Hungaricae"}} +{"enclosedEntityId":"10|doajarticles::0cccd429850ed396c90777b54cf52a1c","minVenue":{"local_identifier":"temp_venue__::0cccd429850ed396c90777b54cf52a1c","name":"Патология кровообращения и кардиохирургия"}} +{"enclosedEntityId":"10|doajarticles::0d957b8eca7fe947571d6400474c95de","minVenue":{"local_identifier":"temp_venue__::0d957b8eca7fe947571d6400474c95de","name":"Journal of the Illuminating Engineering Institute of Japan"}} +{"enclosedEntityId":"10|doajarticles::0db249d213651f07c9dabeb3edc53efc","minVenue":{"local_identifier":"temp_venue__::0db249d213651f07c9dabeb3edc53efc","name":"Drug Metabolism and Pharmacokinetics"}} +{"enclosedEntityId":"10|doajarticles::0ee8ccdef95fedf8ed8f0a62e7c2c8af","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"BMC Medical Imaging"}} +{"enclosedEntityId":"10|doajarticles::0f7fe9973717cb7ac2bba8685bc88a2a","minVenue":{"local_identifier":"temp_venue__::0f7fe9973717cb7ac2bba8685bc88a2a","name":"Reflektika"}} +{"enclosedEntityId":"10|doajarticles::10769bf5b219fbdf75d6bd2d8bd45406","minVenue":{"local_identifier":"temp_venue__::10769bf5b219fbdf75d6bd2d8bd45406","name":"e-Finanse"}} +{"enclosedEntityId":"10|doajarticles::10960937f086496a32324c9eeb1cf380","minVenue":{"local_identifier":"temp_venue__::10960937f086496a32324c9eeb1cf380","name":"ILIRIA International Review"}} +{"enclosedEntityId":"10|doajarticles::15215d4c9ae17e4e738c1a09c06aacee","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"Storia e Futuro"}} +{"enclosedEntityId":"10|doajarticles::15d8c3c4f653bffd815dbdbb1ce56d93","minVenue":{"local_identifier":"temp_venue__::15d8c3c4f653bffd815dbdbb1ce56d93","name":"پژوهش‌های ریاضی"}} +{"enclosedEntityId":"10|doajarticles::176acb5e4cef928cfb0ec4adeacaee89","minVenue":{"local_identifier":"temp_venue__::176acb5e4cef928cfb0ec4adeacaee89","name":"Indian Journal of Occupational and Environmental Medicine"}} +{"enclosedEntityId":"10|doajarticles::18c7e558fdce9c26c97cb0336b0584ff","minVenue":{"local_identifier":"temp_venue__::18c7e558fdce9c26c97cb0336b0584ff","name":"Mercator : Revista de Geografia da UFC"}} +{"enclosedEntityId":"10|doajarticles::192e590353f6cf9af3fdba01b1e1cfde","minVenue":{"local_identifier":"temp_venue__::192e590353f6cf9af3fdba01b1e1cfde","name":"Journal of International Translational Medicine"}} +{"enclosedEntityId":"10|doajarticles::1a4560f88542bcb1b8d7f49b98244d57","minVenue":{"local_identifier":"temp_venue__::1a4560f88542bcb1b8d7f49b98244d57","name":"Емінак"}} +{"enclosedEntityId":"10|doajarticles::1c8148b0dbade0f654ed77b6213f5926","minVenue":{"local_identifier":"temp_venue__::1c8148b0dbade0f654ed77b6213f5926","name":"Terra Sebus: Acta Musei Sabesiensis"}} +{"enclosedEntityId":"10|doajarticles::201d90c44e936c5fc512731c567bfafe","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"AIMS Allergy and Immunology"}} +{"enclosedEntityId":"10|doajarticles::2102c11ea98b6a8196ceec8c07444bd0","minVenue":{"local_identifier":"temp_venue__::2102c11ea98b6a8196ceec8c07444bd0","name":"Revista Organizações em Contexto"}} +{"enclosedEntityId":"10|doajarticles::2226acf8a08b9d804c69f409383353c3","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"Frontiers in Bird Science"}} +{"enclosedEntityId":"10|doajarticles::27be52df8acc8b659e75ec6221f4ccc9","minVenue":{"local_identifier":"temp_venue__::27be52df8acc8b659e75ec6221f4ccc9","name":"Jurnal Mediakita"}} +{"enclosedEntityId":"10|doajarticles::27f5e80d5e90370663ced5e93f343922","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"Frontiers in Bioengineering and Biotechnology"}} +{"enclosedEntityId":"10|doajarticles::29c549c62b532a88d2b700a8e0959140","minVenue":{"local_identifier":"temp_venue__::29c549c62b532a88d2b700a8e0959140","name":"Applied Engineering Letters"}} +{"enclosedEntityId":"10|doajarticles::29c6094ddd0e95cfbbf6904b671276f2","minVenue":{"local_identifier":"temp_venue__::29c6094ddd0e95cfbbf6904b671276f2","name":"Zoonoses"}} +{"enclosedEntityId":"10|doajarticles::29e93cc8b85cb7b8e97efd5a1e54f750","minVenue":{"local_identifier":"temp_venue__::29e93cc8b85cb7b8e97efd5a1e54f750","name":"Wirtschaftsdienst"}} +{"enclosedEntityId":"10|doajarticles::2d7182c08442361426031a606d5dfa51","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"Jurnal Mekanik Terapan"}} +{"enclosedEntityId":"10|doajarticles::2f4f6ae448e6d8894402e1e578d8add5","minVenue":{"local_identifier":"temp_venue__::2f4f6ae448e6d8894402e1e578d8add5","name":"International Journal of Environmental Research and Public Health"}} +{"enclosedEntityId":"10|doajarticles::3169f7cfe0a07828e3cefe88f624a2be","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"Phytopathology Research"}} +{"enclosedEntityId":"10|doajarticles::3184d8b36fce56f3b2f003d71a482208","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"International Journal of Racket Sports Science"}} +{"enclosedEntityId":"10|doajarticles::381006a9dc56ce643cd07d1626032951","minVenue":{"local_identifier":"temp_venue__::381006a9dc56ce643cd07d1626032951","name":"Economy"}} +{"enclosedEntityId":"10|doajarticles::3c46128f8958c5c4eecedcaa4f901107","minVenue":{"local_identifier":"temp_venue__::d41d8cd98f00b204e9800998ecf8427e","name":"GiLE Journal of Skills Development"}} +{"enclosedEntityId":"10|doajarticles::3d68963fdb08bef6733f6eceba6d4529","minVenue":{"local_identifier":"temp_venue__::3d68963fdb08bef6733f6eceba6d4529","name":"The Internet Journal of Gynecology and Obstetrics"}} +{"enclosedEntityId":"10|doajarticles::3d8d349471917017eb42012997ccc9c4","minVenue":{"local_identifier":"temp_venue__::3d8d349471917017eb42012997ccc9c4","name":"International Journal on Applications of Graph Theory in Wireless ad hoc Networks and Sensor Networks"}} +{"enclosedEntityId":"10|doajarticles::3e182197bc8f29afb81f18af8576bc87","minVenue":{"local_identifier":"temp_venue__::3e182197bc8f29afb81f18af8576bc87","name":"اسلام و مدیریت"}} +{"enclosedEntityId":"10|doajarticles::3e8734d0869c179053138585b8148c6d","minVenue":{"local_identifier":"temp_venue__::3e8734d0869c179053138585b8148c6d","name":"Duke Journal of Gender Law & Policy"}} +{"enclosedEntityId":"10|doajarticles::3ebbbfe48c7b3ed215d75501cefcbafa","minVenue":{"local_identifier":"temp_venue__::3ebbbfe48c7b3ed215d75501cefcbafa","name":"Journal of BioScience and Biotechnology"}} +{"enclosedEntityId":"10|doajarticles::3f6a9897445d696f70272c1799472ccb","minVenue":{"local_identifier":"temp_venue__::3f6a9897445d696f70272c1799472ccb","name":"Arabic Literatures for Academic Zealots"}} +{"enclosedEntityId":"10|issn___print::0c5c2b0effc9281e07a078747254e8b7","minVenue":{"local_identifier":"temp_venue__::0c5c2b0effc9281e07a078747254e8b7","name":"Peptides"}} +{"enclosedEntityId":"10|doajarticles::fcf556693aaa9aaf5e9d466cabb47001","minVenue":{"local_identifier":"temp_venue__::fcf556693aaa9aaf5e9d466cabb47001","name":"Records of the Australian Museum"}} +{"enclosedEntityId":"10|issn___print::adfbde8d7e865d2507cb180b9b43712b","minVenue":{"local_identifier":"temp_venue__::adfbde8d7e865d2507cb180b9b43712b","name":"Current Opinion in Lipidology"}} +{"enclosedEntityId":"10|issn___print::094790b7d3268cb0aa34ef1a4eb073c8","minVenue":{"local_identifier":"temp_venue__::094790b7d3268cb0aa34ef1a4eb073c8","name":"Genome Research"}} +{"enclosedEntityId":"10|issn___print::a212960a5e88e1d2e9169c10e8f5e345","minVenue":{"local_identifier":"temp_venue__::a212960a5e88e1d2e9169c10e8f5e345","name":"Journal of Physics D Applied Physics"}} +{"enclosedEntityId":"10|issn___print::32a67ba964e0afcb945f2cb69952f00d","minVenue":{"local_identifier":"temp_venue__::32a67ba964e0afcb945f2cb69952f00d","name":"Environmental Modeling & Assessment"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-d82887dd-9c95-49fc-9efc-0feee839fa0b-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-d82887dd-9c95-49fc-9efc-0feee839fa0b-c000.json.gz deleted file mode 100644 index b17fb70..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-d82887dd-9c95-49fc-9efc-0feee839fa0b-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-fcd5f272-7aab-4f7d-807d-cac431cfa416-c000.json b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-fcd5f272-7aab-4f7d-807d-cac431cfa416-c000.json new file mode 100644 index 0000000..beaa4a8 --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-fcd5f272-7aab-4f7d-807d-cac431cfa416-c000.json @@ -0,0 +1,54 @@ +{"enclosedEntityId":"50|475c1990cbb2::100f1774e6061b0926892da5e93d6c9d","minProduct":{"local_identifier":"50|475c1990cbb2::100f1774e6061b0926892da5e93d6c9d","title":"CAGDB - NATGAM - Campbellton/Bathurst"}} +{"enclosedEntityId":"50|475c1990cbb2::4646d85eae5eec4bf2f18fcbad21f57b","minProduct":{"local_identifier":"50|475c1990cbb2::4646d85eae5eec4bf2f18fcbad21f57b","title":"Travel Activities and Motivation Survey, 2006"}} +{"enclosedEntityId":"50|475c1990cbb2::4f36c055af3184c6216cc4eae5988556","minProduct":{"local_identifier":"50|475c1990cbb2::4f36c055af3184c6216cc4eae5988556","title":"Centres d'artistes autogérés du Québec"}} +{"enclosedEntityId":"50|475c1990cbb2::aaf64bbe32caceb6975d06961daa8755","minProduct":{"local_identifier":"50|475c1990cbb2::aaf64bbe32caceb6975d06961daa8755","title":"Profile for Statistical Area Classification, 2006 Census"}} +{"enclosedEntityId":"50|475c1990cbb2::c3851713a5d4af37f222d512402b92ce","minProduct":{"local_identifier":"50|475c1990cbb2::c3851713a5d4af37f222d512402b92ce","title":"Territorial provincial borders - 1M"}} +{"enclosedEntityId":"50|475c1990cbb2::c857f2c92ae1fd825eb7d5ac4a49d528","minProduct":{"local_identifier":"50|475c1990cbb2::c857f2c92ae1fd825eb7d5ac4a49d528","title":"Lithoprobe AG Transect MT; 13 sites from AG pontiac; magnetotelluric"}} +{"enclosedEntityId":"50|475c1990cbb2::d766f95e68bbbc591c182617ab1d8c17","minProduct":{"local_identifier":"50|475c1990cbb2::d766f95e68bbbc591c182617ab1d8c17","title":"Cadastral Information for Chemawawin 1"}} +{"enclosedEntityId":"50|475c1990cbb2::dc450205363661be28029a55eca89e21","minProduct":{"local_identifier":"50|475c1990cbb2::dc450205363661be28029a55eca89e21","title":"Bedrock geology compilation of the eastern Rackla belt, NTS 105N/15, 105N/16, 105O/13, 106B/4, 106C/1, 106C/2, east-central Yukon"}} +{"enclosedEntityId":"50|475c1990cbb2::df09e45d51554cc803b2a83d1cde48bb","minProduct":{"local_identifier":"50|475c1990cbb2::df09e45d51554cc803b2a83d1cde48bb","title":"Petroleum and Natural Gas Grid Unit"}} +{"enclosedEntityId":"50|475c1990cbb2::dfc9ec4f6f0dd6b7b7319065b976cb6a","minProduct":{"local_identifier":"50|475c1990cbb2::dfc9ec4f6f0dd6b7b7319065b976cb6a","title":"Open Data Goal 3"}} +{"enclosedEntityId":"50|475c1990cbb2::eab4efdde43a6e6755a5bb8f13819257","minProduct":{"local_identifier":"50|475c1990cbb2::eab4efdde43a6e6755a5bb8f13819257","title":"Chronic Wasting Disease Survey of Caribou"}} +{"enclosedEntityId":"50|RECOLECTA___::0c2e85681af5fcd303b56c26d0f1df8c","minProduct":{"local_identifier":"50|RECOLECTA___::0c2e85681af5fcd303b56c26d0f1df8c","title":"Braque : 5 juillet-30 septembre 1980 : Fondation Maeght"}} +{"enclosedEntityId":"50|_____OmicsDI::0faf5884d90f019521587b6f43a4f32c","minProduct":{"local_identifier":"50|_____OmicsDI::0faf5884d90f019521587b6f43a4f32c","title":"Mus musculus"}} +{"enclosedEntityId":"50|_____OmicsDI::10c057b3c8b3c2ed707a115547ed27da","minProduct":{"local_identifier":"50|_____OmicsDI::10c057b3c8b3c2ed707a115547ed27da","title":"ST5 MRSA in New Zealand"}} +{"enclosedEntityId":"50|_____OmicsDI::10c733d1dd55d609a1fe24d9b85df85a","minProduct":{"local_identifier":"50|_____OmicsDI::10c733d1dd55d609a1fe24d9b85df85a","title":"Expression data from CtBP knockdown MCF-7 cells"}} +{"enclosedEntityId":"50|_____OmicsDI::17cd78747bf733510ed811513532838c","minProduct":{"local_identifier":"50|_____OmicsDI::17cd78747bf733510ed811513532838c","title":"Notch pathway is overexpressed and is a therapeutic target in clear cell renal cancer"}} +{"enclosedEntityId":"50|_____OmicsDI::1b52cbcc49ff394df4c7e1dcaaa9b171","minProduct":{"local_identifier":"50|_____OmicsDI::1b52cbcc49ff394df4c7e1dcaaa9b171","title":"Homo sapiens"}} +{"enclosedEntityId":"50|475c1990cbb2::2cc74f33f75b047c6194ae4fc56aa462","minProduct":{"local_identifier":"50|475c1990cbb2::2cc74f33f75b047c6194ae4fc56aa462","title":"Purchasing Card Expenditures - Ministry of Forests and Range - FYE11"}} +{"enclosedEntityId":"50|RECOLECTA___::13d528828cfb560910b3a5d6c7bc0598","minProduct":{"local_identifier":"50|RECOLECTA___::13d528828cfb560910b3a5d6c7bc0598","title":"Tanger: Place de France = Plaza de Francia = France's Place"}} +{"enclosedEntityId":"50|RECOLECTA___::93c8f349e12876e593e24ac7ab25de5a","minProduct":{"local_identifier":"50|RECOLECTA___::93c8f349e12876e593e24ac7ab25de5a","title":"Caja para cenizas de cocina baja"}} +{"enclosedEntityId":"50|_____OmicsDI::04a55d471467fd107e560eeaea5e5d2b","minProduct":{"local_identifier":"50|_____OmicsDI::04a55d471467fd107e560eeaea5e5d2b","title":"Dictyostelium intermedium"}} +{"enclosedEntityId":"50|_____OmicsDI::060082611f4b9ee955d9ae70096bf5b8","minProduct":{"local_identifier":"50|_____OmicsDI::060082611f4b9ee955d9ae70096bf5b8","title":"Anopheles melas strain:CM1002058"}} +{"enclosedEntityId":"50|_____OmicsDI::06df8fca5361d17a5515a72982acd41b","minProduct":{"local_identifier":"50|_____OmicsDI::06df8fca5361d17a5515a72982acd41b","title":"Mus musculus"}} +{"enclosedEntityId":"50|_____OmicsDI::09c6e0e564c4b51d71724e79604f88ca","minProduct":{"local_identifier":"50|_____OmicsDI::09c6e0e564c4b51d71724e79604f88ca","title":"alpha proteobacterium SCGC AAA298-K06"}} +{"enclosedEntityId":"50|_____OmicsDI::0a95732def325605b25c71cda5e5089f","minProduct":{"local_identifier":"50|_____OmicsDI::0a95732def325605b25c71cda5e5089f","title":"Draft Genome Sequence of Burkholderia gladioli Strain UCD-UG_CHAPALOTE"}} +{"enclosedEntityId":"50|_____OmicsDI::0d1703ba052d60c2acc74a97e75d33c3","minProduct":{"local_identifier":"50|_____OmicsDI::0d1703ba052d60c2acc74a97e75d33c3","title":"Pseudomonas sp. WW_13 strain:WW_13"}} +{"enclosedEntityId":"50|_____OmicsDI::0e36ef668edffce1b2cb31d2ed27032c","minProduct":{"local_identifier":"50|_____OmicsDI::0e36ef668edffce1b2cb31d2ed27032c","title":"Vitis vinifera"}} +{"enclosedEntityId":"50|_____OmicsDI::0e68778fcdd0b9560347598c0e9b4278","minProduct":{"local_identifier":"50|_____OmicsDI::0e68778fcdd0b9560347598c0e9b4278","title":"Bacillus cereus 03BB102"}} +{"enclosedEntityId":"50|_____OmicsDI::131469bb810daf1e4c1158eacdfe07fb","minProduct":{"local_identifier":"50|_____OmicsDI::131469bb810daf1e4c1158eacdfe07fb","title":"The MEF2B Regulatory Network - Expression microarray data"}} +{"enclosedEntityId":"50|_____OmicsDI::1c3313b790f4cea27f4d075525878871","minProduct":{"local_identifier":"50|_____OmicsDI::1c3313b790f4cea27f4d075525878871","title":"null"}} +{"enclosedEntityId":"50|_____OmicsDI::21a36a20149a35a70d4beb358af18443","minProduct":{"local_identifier":"50|_____OmicsDI::21a36a20149a35a70d4beb358af18443","title":"The Paramecium Ezl1 protein mediates dual methylation of H3 lysines 9 and 27"}} +{"enclosedEntityId":"50|_____OmicsDI::2298e7316cdc24a7ad529fafc9f5477b","minProduct":{"local_identifier":"50|_____OmicsDI::2298e7316cdc24a7ad529fafc9f5477b","title":"Deinococcus maricopensis"}} +{"enclosedEntityId":"50|_____OmicsDI::36e02acd25698bd97082f438db5d4cf0","minProduct":{"local_identifier":"50|_____OmicsDI::36e02acd25698bd97082f438db5d4cf0","title":"Homo sapiens"}} +{"enclosedEntityId":"50|_____OmicsDI::3b95d6d56f1866c4d00ab5d70d6dff56","minProduct":{"local_identifier":"50|_____OmicsDI::3b95d6d56f1866c4d00ab5d70d6dff56","title":"Puccinia graminis f. sp. tritici 06ND76C"}} +{"enclosedEntityId":"50|_____OmicsDI::40255e6a0ce5be99ede6905950f48f33","minProduct":{"local_identifier":"50|_____OmicsDI::40255e6a0ce5be99ede6905950f48f33","title":"Mus musculus"}} +{"enclosedEntityId":"50|_____OmicsDI::44046a700c424550af38a3eedf5e250b","minProduct":{"local_identifier":"50|_____OmicsDI::44046a700c424550af38a3eedf5e250b","title":"Nutritional effects by beta-carotene in lung in males and females of control mice versus BCMO knockout mice"}} +{"enclosedEntityId":"50|_____OmicsDI::46fbcedddb9ad77014ef2e286015d84f","minProduct":{"local_identifier":"50|_____OmicsDI::46fbcedddb9ad77014ef2e286015d84f","title":"null"}} +{"enclosedEntityId":"50|_____OmicsDI::4a1cdf0609094d699f4c36f3605265e8","minProduct":{"local_identifier":"50|_____OmicsDI::4a1cdf0609094d699f4c36f3605265e8","title":"Homo sapiens"}} +{"enclosedEntityId":"50|_____OmicsDI::4bd8625a0db3e7f35b27c131d928a9c4","minProduct":{"local_identifier":"50|_____OmicsDI::4bd8625a0db3e7f35b27c131d928a9c4","title":"POG_patient_31043 - samples"}} +{"enclosedEntityId":"50|_____OmicsDI::4dd82a0bb25bc5d8e3ba6a32f430738c","minProduct":{"local_identifier":"50|_____OmicsDI::4dd82a0bb25bc5d8e3ba6a32f430738c","title":"Solanum lycopersicum strain:TLB220"}} +{"enclosedEntityId":"50|_____OmicsDI::52b13189a46d8a7148436757d0b2bee1","minProduct":{"local_identifier":"50|_____OmicsDI::52b13189a46d8a7148436757d0b2bee1","title":"Homo sapiens"}} +{"enclosedEntityId":"50|_____OmicsDI::5a5e0168b632c354799e9ade6d23865b","minProduct":{"local_identifier":"50|_____OmicsDI::5a5e0168b632c354799e9ade6d23865b","title":"null"}} +{"enclosedEntityId":"50|_____OmicsDI::6a7dd028c859df2e976d5406f76f8b0b","minProduct":{"local_identifier":"50|_____OmicsDI::6a7dd028c859df2e976d5406f76f8b0b","title":"freshwater metagenome"}} +{"enclosedEntityId":"50|_____OmicsDI::6fa118bd596b943d802562bd3f8104aa","minProduct":{"local_identifier":"50|_____OmicsDI::6fa118bd596b943d802562bd3f8104aa","title":"Proliferating Langerhans cells dampen inflammation in established mouse psoriatic lesions"}} +{"enclosedEntityId":"50|_____OmicsDI::710f99d06d7433275c1c123b3845480d","minProduct":{"local_identifier":"50|_____OmicsDI::710f99d06d7433275c1c123b3845480d","title":"Characterizing commonalities and differences between the breast and prostate cancer metabotypes in African-American cohorts"}} +{"enclosedEntityId":"50|_____OmicsDI::71c6f3c958a052a3bc88ea6bee442ad1","minProduct":{"local_identifier":"50|_____OmicsDI::71c6f3c958a052a3bc88ea6bee442ad1","title":"GNPS - Mouse BXD liver metabolomics"}} +{"enclosedEntityId":"50|_____OmicsDI::7b4accbdae8e4782d6395b3b024641d8","minProduct":{"local_identifier":"50|_____OmicsDI::7b4accbdae8e4782d6395b3b024641d8","title":"Neurospora crassa strain:2489x8790_390a"}} +{"enclosedEntityId":"50|_____OmicsDI::7ee47638c285148bda6764f37b3a5537","minProduct":{"local_identifier":"50|_____OmicsDI::7ee47638c285148bda6764f37b3a5537","title":"Homo sapiens"}} +{"enclosedEntityId":"50|_____OmicsDI::89d0864591fabb3d90be26af88ba8081","minProduct":{"local_identifier":"50|_____OmicsDI::89d0864591fabb3d90be26af88ba8081","title":"Salmonella enterica strain:SEH-16"}} +{"enclosedEntityId":"50|_____OmicsDI::8cb322209941ef6e8b107b5fc5ca800c","minProduct":{"local_identifier":"50|_____OmicsDI::8cb322209941ef6e8b107b5fc5ca800c","title":"Methylobacter luteus IMV-B-3098"}} +{"enclosedEntityId":"50|_____OmicsDI::8de599bd144ef54d1d954cb83274fa2f","minProduct":{"local_identifier":"50|_____OmicsDI::8de599bd144ef54d1d954cb83274fa2f","title":"Homo sapiens"}} +{"enclosedEntityId":"50|_____OmicsDI::94010edd576f861d025bf0aecf97d54e","minProduct":{"local_identifier":"50|_____OmicsDI::94010edd576f861d025bf0aecf97d54e","title":"A regulatory CD9+ B cell subset controls HDM-induced allergic airway inflammation"}} +{"enclosedEntityId":"50|_____OmicsDI::95f4b23d1e4410d6d3b192a03fd89692","minProduct":{"local_identifier":"50|_____OmicsDI::95f4b23d1e4410d6d3b192a03fd89692","title":"Bartonella bacilliformis"}} +{"enclosedEntityId":"50|_____OmicsDI::9e9f2b443aaf8b637500d28b86122b6d","minProduct":{"local_identifier":"50|_____OmicsDI::9e9f2b443aaf8b637500d28b86122b6d","title":"Genetics of gene expression in primary human immune cells"}} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-fcd5f272-7aab-4f7d-807d-cac431cfa416-c000.json.gz b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-fcd5f272-7aab-4f7d-807d-cac431cfa416-c000.json.gz deleted file mode 100644 index 3cd9ea3..0000000 Binary files a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minEntity/part-00000-fcd5f272-7aab-4f7d-807d-cac431cfa416-c000.json.gz and /dev/null differ diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/paort0 b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/paort0 new file mode 100644 index 0000000..c297bc7 --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/paort0 @@ -0,0 +1,32 @@ +{"local_identifier":"50|59796a29bd90::390b69a1e20d5af206fd6849c9b97d13","title":"Power to gas"} +{"local_identifier":"50|59796a29bd90::6098357dcb0f21a492ae2689961268ef","title":"Vrije Tijd? Tijd voor Deventer!"} +{"local_identifier":"50|59796a29bd90::d0545b3c7166358e16599aec2284b101","title":"Sepsis at ICU admission does not decrease 30-day survival in very old patients"} +{"local_identifier":"50|73490d0e0f82::1f1e664683ae0035b0d64487d1e1bcbe","title":"The Effects of Membrane Physical Properties on Insulin Receptor Autophosphorylation and Signalling"} +{"local_identifier":"50|73490d0e0f82::62d68e6a826109387d9b24d125dee5d1","title":"Protection of Migrant Agricultural Workers' Rights"} +{"local_identifier":"50|CSC_________::288c3eab977e8b76a7abe31f989c9901","title":"A fabric design for Armani Casa"} +{"local_identifier":"50|CSC_________::710a52c4e2a9b3b697c682dd15595a4c","title":"''Archpelago'' Apron, Navy : formgivning"} +{"local_identifier":"50|CSC_________::7a3ea1f6a35d738fb193f6fae3303d32","title":"Ragna Róbertsdóttir | Configured Landscapes"} +{"local_identifier":"50|RECOLECTA___::02e2efbf7bdfbdc708995bf8cb3c038b","title":"Por qué la educación a distancia (en línea)"} +{"local_identifier":"50|RECOLECTA___::04e03c6d2756db9d75874947a7ef224c","title":"Sistemas recomendadores aplicados a productos de ropa"} +{"local_identifier":"50|RECOLECTA___::05d59bf33bc35aeb975ca399180a051f","title":"Origen y desarrollo de los derechos fundamentales laborales en Venezuela (1830-Constitución de 1999) = Origin and development of fundamental labor rights in Venezuela (1830-1999 Constitution)"} +{"local_identifier":"50|RECOLECTA___::074a83f6b5813016364f2f5c4651f026","title":"Página 3"} +{"local_identifier":"50|RECOLECTA___::1111974150e3c89ed32a8fabe4785e32","title":"Diseño y aplicaciones de materiales híbridos fotoactivos"} +{"local_identifier":"50|RECOLECTA___::2694fc6101c2a0454d91fa047997ea57","title":"BIM-based decision support for building condition assessment"} +{"local_identifier":"50|RECOLECTA___::4c67143df007c7fdfd9af06ead811612","title":"Máster profesorado - Sesión práctica 1: Diseño y desarrollo de programaciones y actividades formativas"} +{"local_identifier":"50|RECOLECTA___::554f00ffc38fa55e66208b19a8576b7d","title":"Uncertainty analysis methods to select the optimal alternative in the design of parking facilities"} +{"local_identifier":"50|59796a29bd90::1a4c7bbe17709e01cb361b2159aa3296","title":"Sociale voorwaarden in aanbestedingen: beleid en uitvoering"} +{"local_identifier":"50|59796a29bd90::227dfff51dc04032c0fc5c96a5ebc405","title":"In situ detection of product age and argon concentration as measure of the re-use potential of insulating glass units in buildings"} +{"local_identifier":"50|59796a29bd90::2a8c8c2e29955006521983ec30226a23","title":"Annual conference REFRAME"} +{"local_identifier":"50|59796a29bd90::e8ef028372d784a26a451f6b50d0006d","title":"Kruidenrijk grasland : kans voor koe en boer"} +{"local_identifier":"50|73490d0e0f82::831b9da5b9523d3a954471c111f03fcc","title":"Using Reputation in Repeated Selfish Routing with Incomplete Information"} +{"local_identifier":"50|73490d0e0f82::8ef7550c8b45953789005b0af1626c4e","title":"Refractor Plates Optimization and Data Analysis of a Correlation Spectrometer, Used as a Remote Sensing Instrument"} +{"local_identifier":"50|73490d0e0f82::d94d2b644fe36e0398c55f894f01bf13","title":"links between labour-market experiences and health"} +{"local_identifier":"50|CSC_________::0335521d3fcd008e1a73cc500a4911c2","title":"10 kohtausta seksuaalisesta vallankäytöstä (teatteriesitys) : valosuunnittelu"} +{"local_identifier":"50|CSC_________::33b62b7fd4cb5305580ff6b5ad3b110d","title":"Lyhytkapillaarielektroforeesi on-line -prosessiseurantaan"} +{"local_identifier":"50|CSC_________::a3bfbbaaee8b7d79a42a334bc5fb7f00","title":"Jam Session w/ Dr. Sandberg (konsert) : musiker"} +{"local_identifier":"50|RECOLECTA___::00f011c5c9490e47c91bdf36701221f3","title":"Aló"} +{"local_identifier":"50|RECOLECTA___::0197025e456a3e0a4922b82c990d6bd8","title":"Ficha 489"} +{"local_identifier":"50|RECOLECTA___::2757c2e94b79fe7822c3a59005d07870","title":"Clinical spectrum of premature pubarche: Links to metabolic syndrome and ovarian hyperandrogenism"} +{"local_identifier":"50|RECOLECTA___::3099980dac34189351509d429ba16866","title":"La incidencia del factor nuclear en el terrorismo"} +{"local_identifier":"50|RECOLECTA___::3227a354fbf54668c379229297c71c0e","title":"Pre-training of deep neural networks through Real-Coded Genetic Algorithms"} +{"local_identifier":"50|RECOLECTA___::37a152ef82da05df86d1414a0bc200b4","title":"the UN 2030 Agenda to Improve Global Communicative Competence"} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part1 b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part1 new file mode 100644 index 0000000..ccb44dd --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part1 @@ -0,0 +1,20 @@ +{"doi":"10.1177/0267323118790163","local_identifier":"50|doi_________::0840e1cfdf65dd3abdeee453e234e034","title":"Celebrities in Czech politics in 1996–2013"} +{"doi":"10.1038/ejhg.2014.31","local_identifier":"50|doi_________::479f67164e2f1d21baafbfbc3b12851d"} +{"doi":"10.1016/j.marmicro.2013.04.005","local_identifier":"50|doi_________::7078f75053e8c1bff562adbce2d81fe2","title":"Effect of dissolved oxygen concentration on planktonic foraminifera through laboratory culture experiments and implications for oceanic anoxic events"} +{"doi":"10.1111/bij.12378","local_identifier":"50|doi_________::86d7aecc7029c70836d268b4ea9b974d","title":"Phylogeography of the humbug damselfish,Dascyllus aruanus(Linnaeus, 1758): evidence of Indo-Pacific vicariance and genetic differentiation of peripheral populations"} +{"doi":"10.3389/fimmu.2018.01146","local_identifier":"50|doi_________::a83b2a6d96883a5f2d360fb5fb7db8ae"} +{"doi":"10.1038/srep37852","local_identifier":"50|doi_________::af75a7dc6b0ba2ca2349a68588ef877e"} +{"doi":"10.1002/cne.23131","local_identifier":"50|doi_dedup___::079ebd73b673282559d6f4befe003d47","title":"Galanin gene expression and effects of its knock‐down on the development of the nervous system in larval zebrafish"} +{"doi":"10.3853/j.0067-1975.49.1997.1260","local_identifier":"50|doi_________::0363a8e9be16f62cdfd58b3def423a57","title":"Amphipoda from the South Pacific: Western Samoa"} +{"doi":"10.1063/1.4928566","local_identifier":"50|doi_________::0888444d70ea8e3bcbc12d9d281fce22","title":"Dark-to-arc transition in field emission dominated atmospheric microdischarges"} +{"doi":"10.1007/s10666-018-9616-4","local_identifier":"50|doi_________::0941e6d7170c6bbfb40721aa9ecff245","title":"The Strategic Impact of Adaptation in a Transboundary Pollution Dynamic Game"} +{"doi":"10.1534/genetics.116.190215","local_identifier":"50|doi_________::1f56d1b5a2e7c9223c3479476c9c3491"} +{"doi":"10.1186/s13023-016-0515-y","local_identifier":"50|doi_________::579e3d4992475c1b375494c9e19b24ce","title":"8th European Conference on Rare Diseases & Orphan Products (ECRD 2016)"} +{"doi":"10.1111/j.1471-4159.2004.02190.x","local_identifier":"50|doi_dedup___::893d0a5c08409822dbb82b8973a42909","title":"Neurochemical and behavioural changes in zebrafish Danio rerio after systemic administration of 6‐hydroxydopamine and 1‐methyl‐4‐phenyl‐1,2,3,6‐tetrahydropyridine"} +{"doi":"10.1016/0196-9781(89)90045-4","local_identifier":"50|doi_dedup___::8b0484d2afbd9b09116ef60c18044964","title":"Tissue-specific sex differences in galanin-like immunoreactivity and galanin mRNA during development in the rat"} +{"doi":"10.1007/s10666-016-9501-y","local_identifier":"50|doi_________::1034996a07e1496dde6048008e2187de","title":"Mitigation vs. Adaptation: Analytic Models for Policy Assessment"} +{"doi":"10.1016/s0003-3472(71)80005-2","local_identifier":"50|doi_________::81a64d0689f4a2f93995b81574d2fcbe","title":"Apparent effect of prior experience on a habitat preference exhibited by the reef fish, Dascyllus aruanus (Pisces: Pomacentridae)"} +{"doi":"10.1101/gr.107987.110","local_identifier":"50|doi_dedup___::7a40bb2e9ad2e96d6de26787ab1c310b","title":"Reshaping the gut microbiome with bacterial transplantation and antibiotic intake"} +{"doi":"10.1097/mol.0b013e32835918cd","local_identifier":"50|doi_dedup___::233bd897c6654f274059272b8740b47f","title":"Paternal transmission, cardiovascular risk factors and epigenetics"} +{"doi":"10.1088/0022-3727/47/50/503001","local_identifier":"50|doi_________::1647515a356946e4857c817afc7bfa38","title":"Microscale gas breakdown: ion-enhanced field emission and the modified Paschen’s curve"} +{"doi":"10.1038/nature10811","local_identifier":"50|doi_dedup___::6481edac8ee74c1f1bfa9c0fdd3e6f11","title":"The Drosophila melanogaster Genetic Reference Panel"} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part3 b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part3 new file mode 100644 index 0000000..6bf2ae2 --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part3 @@ -0,0 +1,32 @@ +{"local_identifier":"50|__bioTools__::3640ff1dd63f33b085265311638a2ef1","title":"Information-based Similarity Toolbox"} +{"local_identifier":"50|__bioTools__::6a8a9601934e1d566bdc9e387f106cad","title":"MuSiC suite: PoPMuSiC, HoTMuSiC, and SNPMuSiC"} +{"local_identifier":"50|__bioTools__::8e5431247fb491ae7cf46b4806abbfaf","title":"A GUI for bacterial identification"} +{"local_identifier":"50|__bioTools__::cee88972c28033daaca8786a265da604","title":"OligoCalc"} +{"local_identifier":"50|__bioTools__::e1594e9122869da8eec86671ab713259","title":"PaSS"} +{"local_identifier":"50|__bioTools__::ec9a52d4c30788879d1e5fd097171f5f","title":"BiSeq"} +{"local_identifier":"50|__bioTools__::f50acbc8c02cd875a9e83a78ed8f285d","title":"BlastKOALA"} +{"local_identifier":"50|dedup_wf_002::47f351f4d17708772645eb4f556d802d","title":"CodeSearchNet software on GitHub"} +{"local_identifier":"50|dedup_wf_002::48d5bd6174ba13c5e20b577e71f3a330","title":"Logiciel WAFA (Welfare assessment of farm animal). Outil d'aide à l'évaluation du bien-être des animaux (outil de saisie, base de données, logiciel de calcul de scores de bien-être, outil de simulation, ressource d'informations, pages web)"} +{"local_identifier":"50|dedup_wf_002::4d37f1fd55ae2c11290d4b161060f302","title":"DeepCCA software on GitHub"} +{"local_identifier":"50|dedup_wf_002::a36d36bcd9e8fe405cfc82fc8b84294f","title":"questionr"} +{"local_identifier":"50|dedup_wf_002::b82b856ebd89cdd36b8107e475280e9e","title":"snmalloc software on GitHub"} +{"doi":"10.24433/co.2868322.v1","local_identifier":"50|doi_________::07ca00c92c3093a76fcae84b2185ed85","title":"Image Quality Environment for FPGA Floating-Point Video Processing Library"} +{"doi":"10.5281/zenodo.3724469","local_identifier":"50|doi_________::094b6aff40ccf64df47063a59ab83da1","title":"stonerlab/Stoner-PythonCode: v0.9.5 Bugfix and improvement release"} +{"doi":"10.5281/zenodo.4763538","local_identifier":"50|doi_________::15b9e4e4e1102ae67e1b0e567a7593fc","title":"poliastro/poliastro: poliastro 0.15.0 (Earth edition)"} +{"doi":"10.5281/zenodo.1251581","local_identifier":"50|doi_________::16be0dfa4fa679bed49d01fc4471cabd","title":"sotorrent/posthistory-extractor: Refactored release"} +{"doi":"10.5281/zenodo.3979689","local_identifier":"50|doi_________::185d0f8b38dfeb9f567f30a3190a916c","title":"InsightSoftwareConsortium/ITKElastix: ITKElastix 0.6.3"} +{"local_identifier":"50|__bioTools__::022faad7a573c4f12301c486815a3fa5","title":"petalNet"} +{"local_identifier":"50|__bioTools__::0c42b0be275910c096d2bacaccff89ad","title":"Diversity Workbench"} +{"local_identifier":"50|__bioTools__::58861f6ae561f40b585e44c0734b4a70","title":"HMCan"} +{"local_identifier":"50|__bioTools__::85fb610b34dc395431f8eda781566c4e","title":"iAssembler"} +{"local_identifier":"50|__bioTools__::97b678c1b70fe0b13197be61111a3513","title":"DNA Baser"} +{"local_identifier":"50|__bioTools__::9961460fe6c9852c36bdbfea8b7fbf2c","title":"lncRNAdb"} +{"local_identifier":"50|__bioTools__::c802825f3b3a94343e30a18ed23d4786","title":"Gene Set Builder"} +{"local_identifier":"50|dedup_wf_002::629f6117216829e2582934ac1d845d9e","title":"Support tool for the overall assessment of animal welfare"} +{"local_identifier":"50|dedup_wf_002::6ae16f2f40fa342f71a5e1a0a11eb962","title":"transforms3d software on GitHub"} +{"local_identifier":"50|dedup_wf_002::b4f06f3b28fa9cd2d7b9185e3f6604eb","title":"varcmp software on GitHub"} +{"local_identifier":"50|dedup_wf_002::e3373c97b277d3f0f3f6c21d688ae520","title":"OpenDSS-G software on SourceForge"} +{"doi":"10.5281/zenodo.7254817","local_identifier":"50|doi_________::0abc081de6a2fe9506b20e8b58389cb2","title":"napari/magicgui: v0.6.0"} +{"doi":"10.18129/b9.bioc.adam","local_identifier":"50|doi_________::0c6f89082f07ea5c737eaf25d0712e01","title":"ADAM"} +{"doi":"10.5281/zenodo.573248","local_identifier":"50|doi_________::115332ff9a5e3d15a4c79bc3e819e165","title":"PeterMulhair/planets: First release of Science software"} +{"doi":"10.5281/zenodo.50641","local_identifier":"50|doi_________::115a8028953b0488dbb04c0f99b9dd62","title":"libfreenect2: Release 0.2"} diff --git a/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part4 b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part4 new file mode 100644 index 0000000..a65e5c2 --- /dev/null +++ b/dump/src/test/resources/eu/dnetlib/dhp/oa/graph/dump/skgif/workingDirApiExample/minProduct/part4 @@ -0,0 +1,54 @@ +{"local_identifier":"50|475c1990cbb2::100f1774e6061b0926892da5e93d6c9d","title":"CAGDB - NATGAM - Campbellton/Bathurst"} +{"local_identifier":"50|475c1990cbb2::4646d85eae5eec4bf2f18fcbad21f57b","title":"Travel Activities and Motivation Survey, 2006"} +{"local_identifier":"50|475c1990cbb2::4f36c055af3184c6216cc4eae5988556","title":"Centres d'artistes autogérés du Québec"} +{"local_identifier":"50|475c1990cbb2::aaf64bbe32caceb6975d06961daa8755","title":"Profile for Statistical Area Classification, 2006 Census"} +{"local_identifier":"50|475c1990cbb2::c3851713a5d4af37f222d512402b92ce","title":"Territorial provincial borders - 1M"} +{"local_identifier":"50|475c1990cbb2::c857f2c92ae1fd825eb7d5ac4a49d528","title":"Lithoprobe AG Transect MT; 13 sites from AG pontiac; magnetotelluric"} +{"local_identifier":"50|475c1990cbb2::d766f95e68bbbc591c182617ab1d8c17","title":"Cadastral Information for Chemawawin 1"} +{"local_identifier":"50|475c1990cbb2::dc450205363661be28029a55eca89e21","title":"Bedrock geology compilation of the eastern Rackla belt, NTS 105N/15, 105N/16, 105O/13, 106B/4, 106C/1, 106C/2, east-central Yukon"} +{"local_identifier":"50|475c1990cbb2::df09e45d51554cc803b2a83d1cde48bb","title":"Petroleum and Natural Gas Grid Unit"} +{"local_identifier":"50|475c1990cbb2::dfc9ec4f6f0dd6b7b7319065b976cb6a","title":"Open Data Goal 3"} +{"local_identifier":"50|475c1990cbb2::eab4efdde43a6e6755a5bb8f13819257","title":"Chronic Wasting Disease Survey of Caribou"} +{"local_identifier":"50|RECOLECTA___::0c2e85681af5fcd303b56c26d0f1df8c","title":"Braque : 5 juillet-30 septembre 1980 : Fondation Maeght"} +{"local_identifier":"50|_____OmicsDI::0faf5884d90f019521587b6f43a4f32c","title":"Mus musculus"} +{"local_identifier":"50|_____OmicsDI::10c057b3c8b3c2ed707a115547ed27da","title":"ST5 MRSA in New Zealand"} +{"local_identifier":"50|_____OmicsDI::10c733d1dd55d609a1fe24d9b85df85a","title":"Expression data from CtBP knockdown MCF-7 cells"} +{"local_identifier":"50|_____OmicsDI::17cd78747bf733510ed811513532838c","title":"Notch pathway is overexpressed and is a therapeutic target in clear cell renal cancer"} +{"local_identifier":"50|_____OmicsDI::1b52cbcc49ff394df4c7e1dcaaa9b171","title":"Homo sapiens"} +{"local_identifier":"50|475c1990cbb2::2cc74f33f75b047c6194ae4fc56aa462","title":"Purchasing Card Expenditures - Ministry of Forests and Range - FYE11"} +{"local_identifier":"50|RECOLECTA___::13d528828cfb560910b3a5d6c7bc0598","title":"Tanger: Place de France = Plaza de Francia = France's Place"} +{"local_identifier":"50|RECOLECTA___::93c8f349e12876e593e24ac7ab25de5a","title":"Caja para cenizas de cocina baja"} +{"local_identifier":"50|_____OmicsDI::04a55d471467fd107e560eeaea5e5d2b","title":"Dictyostelium intermedium"} +{"local_identifier":"50|_____OmicsDI::060082611f4b9ee955d9ae70096bf5b8","title":"Anopheles melas strain:CM1002058"} +{"local_identifier":"50|_____OmicsDI::06df8fca5361d17a5515a72982acd41b","title":"Mus musculus"} +{"local_identifier":"50|_____OmicsDI::09c6e0e564c4b51d71724e79604f88ca","title":"alpha proteobacterium SCGC AAA298-K06"} +{"local_identifier":"50|_____OmicsDI::0a95732def325605b25c71cda5e5089f","title":"Draft Genome Sequence of Burkholderia gladioli Strain UCD-UG_CHAPALOTE"} +{"local_identifier":"50|_____OmicsDI::0d1703ba052d60c2acc74a97e75d33c3","title":"Pseudomonas sp. WW_13 strain:WW_13"} +{"local_identifier":"50|_____OmicsDI::0e36ef668edffce1b2cb31d2ed27032c","title":"Vitis vinifera"} +{"local_identifier":"50|_____OmicsDI::0e68778fcdd0b9560347598c0e9b4278","title":"Bacillus cereus 03BB102"} +{"local_identifier":"50|_____OmicsDI::131469bb810daf1e4c1158eacdfe07fb","title":"The MEF2B Regulatory Network - Expression microarray data"} +{"local_identifier":"50|_____OmicsDI::1c3313b790f4cea27f4d075525878871","title":"null"} +{"local_identifier":"50|_____OmicsDI::21a36a20149a35a70d4beb358af18443","title":"The Paramecium Ezl1 protein mediates dual methylation of H3 lysines 9 and 27"} +{"local_identifier":"50|_____OmicsDI::2298e7316cdc24a7ad529fafc9f5477b","title":"Deinococcus maricopensis"} +{"local_identifier":"50|_____OmicsDI::36e02acd25698bd97082f438db5d4cf0","title":"Homo sapiens"} +{"local_identifier":"50|_____OmicsDI::3b95d6d56f1866c4d00ab5d70d6dff56","title":"Puccinia graminis f. sp. tritici 06ND76C"} +{"local_identifier":"50|_____OmicsDI::40255e6a0ce5be99ede6905950f48f33","title":"Mus musculus"} +{"local_identifier":"50|_____OmicsDI::44046a700c424550af38a3eedf5e250b","title":"Nutritional effects by beta-carotene in lung in males and females of control mice versus BCMO knockout mice"} +{"local_identifier":"50|_____OmicsDI::46fbcedddb9ad77014ef2e286015d84f","title":"null"} +{"local_identifier":"50|_____OmicsDI::4a1cdf0609094d699f4c36f3605265e8","title":"Homo sapiens"} +{"local_identifier":"50|_____OmicsDI::4bd8625a0db3e7f35b27c131d928a9c4","title":"POG_patient_31043 - samples"} +{"local_identifier":"50|_____OmicsDI::4dd82a0bb25bc5d8e3ba6a32f430738c","title":"Solanum lycopersicum strain:TLB220"} +{"local_identifier":"50|_____OmicsDI::52b13189a46d8a7148436757d0b2bee1","title":"Homo sapiens"} +{"local_identifier":"50|_____OmicsDI::5a5e0168b632c354799e9ade6d23865b","title":"null"} +{"local_identifier":"50|_____OmicsDI::6a7dd028c859df2e976d5406f76f8b0b","title":"freshwater metagenome"} +{"local_identifier":"50|_____OmicsDI::6fa118bd596b943d802562bd3f8104aa","title":"Proliferating Langerhans cells dampen inflammation in established mouse psoriatic lesions"} +{"local_identifier":"50|_____OmicsDI::710f99d06d7433275c1c123b3845480d","title":"Characterizing commonalities and differences between the breast and prostate cancer metabotypes in African-American cohorts"} +{"local_identifier":"50|_____OmicsDI::71c6f3c958a052a3bc88ea6bee442ad1","title":"GNPS - Mouse BXD liver metabolomics"} +{"local_identifier":"50|_____OmicsDI::7b4accbdae8e4782d6395b3b024641d8","title":"Neurospora crassa strain:2489x8790_390a"} +{"local_identifier":"50|_____OmicsDI::7ee47638c285148bda6764f37b3a5537","title":"Homo sapiens"} +{"local_identifier":"50|_____OmicsDI::89d0864591fabb3d90be26af88ba8081","title":"Salmonella enterica strain:SEH-16"} +{"local_identifier":"50|_____OmicsDI::8cb322209941ef6e8b107b5fc5ca800c","title":"Methylobacter luteus IMV-B-3098"} +{"local_identifier":"50|_____OmicsDI::8de599bd144ef54d1d954cb83274fa2f","title":"Homo sapiens"} +{"local_identifier":"50|_____OmicsDI::94010edd576f861d025bf0aecf97d54e","title":"A regulatory CD9+ B cell subset controls HDM-induced allergic airway inflammation"} +{"local_identifier":"50|_____OmicsDI::95f4b23d1e4410d6d3b192a03fd89692","title":"Bartonella bacilliformis"} +{"local_identifier":"50|_____OmicsDI::9e9f2b443aaf8b637500d28b86122b6d","title":"Genetics of gene expression in primary human immune cells"} \ No newline at end of file