[SKG-IF] tries to make the process finish. need to change the strategy
This commit is contained in:
parent
98bec3d2d2
commit
3c4c4e8ce0
|
@ -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;
|
||||
|
|
|
@ -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<Datasource> datasources = Utils
|
||||
.readPath(spark, inputPath + "datasource", Datasource.class)
|
||||
.filter((FilterFunction<Datasource>) d -> !d.getDataInfo().getDeletedbyinference());
|
||||
|
||||
// select relations having source in the set of identifiers selected for eosc
|
||||
Dataset<Relation> resultSource = resultIds
|
||||
.joinWith(relation, resultIds.col("value").equalTo(relation.col("source")))
|
||||
.map((MapFunction<Tuple2<String, Relation>, 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<Tuple2<String, Relation>, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class))
|
||||
Dataset<Row> 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<Tuple2<Relation, Organization>, 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<Tuple2<Relation, Project>, 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<Tuple2<Relation, Organization>, 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<Tuple2<Relation, Project>, 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<String> datasourceReferencedIds = spark.emptyDataset(Encoders.STRING());
|
||||
|
||||
for (EntityType entity : ModelSupport.entityTypes.keySet())
|
||||
if (ModelSupport.isResult(entity)) {
|
||||
Class<R> resultClazz = ModelSupport.entityTypes.get(entity);
|
||||
|
@ -171,45 +166,88 @@ public class SelectConnectedEntities implements Serializable {
|
|||
(FlatMapFunction<R, String>) 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, String>) 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<Tuple2<String, Datasource>, 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<Organization> organizationSbs = Utils.readPath(spark, workingDir + "organization", Organization.class);
|
||||
Dataset<Project> projectSbs = Utils.readPath(spark, workingDir + "project", Project.class);
|
||||
Dataset<Relation> orgSourceRels = organizationSbs
|
||||
.joinWith(relation, organizationSbs.col("id").equalTo(relation.col("source")))
|
||||
.map((MapFunction<Tuple2<Organization, Relation>, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class));
|
||||
orgSourceRels
|
||||
.joinWith(projectSbs, orgSourceRels.col("target").equalTo(projectSbs.col("id")))
|
||||
.map((MapFunction<Tuple2<Relation, Project>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
|
||||
StructType tp = StructType.fromDDL("`id` STRING");
|
||||
Dataset<Row> organizationSbs = spark
|
||||
.read()
|
||||
.schema(tp)
|
||||
.json(workingDir + "organization")
|
||||
.select("id");
|
||||
|
||||
Dataset<Row> projectSbs = spark
|
||||
.read()
|
||||
.schema(tp)
|
||||
.json(workingDir + "project")
|
||||
.select("id");
|
||||
//
|
||||
Dataset<Row> 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<Datasource> datasourceSbs = Utils.readPath(spark, workingDir + "datasource", Datasource.class);
|
||||
Dataset<Relation> dsSourceRels = datasourceSbs
|
||||
.joinWith(relation, datasourceSbs.col("id").as("dsId").equalTo(relation.col("source")))
|
||||
.map((MapFunction<Tuple2<Datasource, Relation>, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class));
|
||||
dsSourceRels
|
||||
.joinWith(organizationSbs, dsSourceRels.col("target").equalTo(organizations.col("id").as("orgId")))
|
||||
.map((MapFunction<Tuple2<Relation, Organization>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
|
||||
Dataset<Row> 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");
|
||||
|
||||
|
|
|
@ -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<Tuple2<R, PartialResearchProduct>, 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> relation = Utils
|
||||
.readPath(
|
||||
spark,
|
||||
inputPath + "relation", Relation.class)
|
||||
.filter(
|
||||
(FilterFunction<Relation>) r -> !r.getDataInfo().getDeletedbyinference() &&
|
||||
!r.getDataInfo().getInvisible())
|
||||
.filter(
|
||||
(FilterFunction<Relation>) 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<String> 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<EncloseMinElement> 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<Tuple2<Relation, EncloseMinElement>, EncloseMinElement>) t2 -> {
|
||||
EncloseMinElement eme = t2._2();
|
||||
eme.setResultId(t2._1().getSource());
|
||||
eme.setSemantics(t2._1().getRelClass());
|
||||
return eme;
|
||||
}, Encoders.bean(EncloseMinElement.class))
|
||||
.groupByKey((MapFunction<EncloseMinElement, String>) eme -> eme.getResultId(), Encoders.STRING())
|
||||
.mapGroups((MapGroupsFunction<String, EncloseMinElement, RelationPerProduct>) (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<RelationPerProduct> 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<Row> relation = spark
|
||||
.read()
|
||||
.schema(relationstructureSchema)
|
||||
.json(inputPath + "relation")
|
||||
.filter(
|
||||
"datainfo.deletedbyinference != true and " +
|
||||
"relclass == '" + relationType + "'")
|
||||
.drop("dataInfo");
|
||||
|
||||
Dataset<Row> 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<Row, String>) r -> r.getAs("sourceResult"), Encoders.STRING())
|
||||
.mapGroups((MapGroupsFunction<String, Row, RelationPerProduct>) (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<Row> relation = spark
|
||||
.read()
|
||||
.schema(relationstructureSchema)
|
||||
.json(inputPath + "relation")
|
||||
.filter(
|
||||
"datainfo.deletedbyinference != true and " +
|
||||
"relclass == '" + relationType + "'")
|
||||
.drop("dataInfo");
|
||||
|
||||
Dataset<Row> 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<Row, String>) r -> r.getAs("sourceResult"), Encoders.STRING())
|
||||
.mapGroups((MapGroupsFunction<String, Row, RelationPerProduct>) (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<Row> relation = spark
|
||||
.read()
|
||||
.schema(relationstructureSchema)
|
||||
.json(inputPath + "relation")
|
||||
.filter(
|
||||
"datainfo.deletedbyinference != true and " +
|
||||
"relclass == '" + relationType + "'")
|
||||
.drop("dataInfo");
|
||||
|
||||
Dataset<Row> 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<Row, String>) r -> r.getAs("sourceResult"), Encoders.STRING())
|
||||
.mapGroups((MapGroupsFunction<String, Row, RelationPerProduct>) (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<Tuple2<Row, EncloseMinElement>, 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<EncloseMinElement, String>) eme -> eme.getResultId(), Encoders.STRING())
|
||||
//// .mapGroups((MapGroupsFunction<String, EncloseMinElement, RelationPerProduct>) (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 <R extends Result> Dataset<EncloseMinElement> getMinProduct(SparkSession spark, String inputPath,
|
||||
Class<R> clazz) {
|
||||
return Utils
|
||||
.readPath(spark, inputPath, clazz)
|
||||
.filter(
|
||||
(FilterFunction<R>) r -> !r.getDataInfo().getDeletedbyinference() &&
|
||||
!r.getDataInfo().getInvisible())
|
||||
.map((MapFunction<R, EncloseMinElement>) 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<String, List<MinProduct>> 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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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<Project>) p -> !p.getDataInfo().getDeletedbyinference())
|
||||
.map((MapFunction<Project, EncloseMinElement>) p -> {
|
||||
EncloseMinElement eme = new EncloseMinElement();
|
||||
eme.setEnclosedEntityId(p.getId());
|
||||
eme.setMinGrant(Utils.getMinGrant(p));
|
||||
return eme;
|
||||
}, Encoders.bean(EncloseMinElement.class))
|
||||
.map((MapFunction<Project, MinGrant>) 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<Organization>) o -> !o.getDataInfo().getDeletedbyinference())
|
||||
.map((MapFunction<Organization, EncloseMinElement>) o -> {
|
||||
EncloseMinElement eme = new EncloseMinElement();
|
||||
eme.setMinOrganization(Utils.getMinOrganization(o));
|
||||
eme.setEnclosedEntityId(o.getId());
|
||||
return eme;
|
||||
},
|
||||
Encoders.bean(EncloseMinElement.class))
|
||||
.map(
|
||||
(MapFunction<Organization, MinOrganization>) 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", "<id as local_identifier> 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<Datasource>) d -> !d.getDataInfo().getDeletedbyinference())
|
||||
.map((MapFunction<Datasource, EncloseMinElement>) 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<Datasource, MinVenue>) 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<Datasource, EncloseMinElement>) d -> {
|
||||
EncloseMinElement eme = new EncloseMinElement();
|
||||
eme.setEnclosedEntityId(d.getId());
|
||||
.map((MapFunction<Datasource, MinVenue>) 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 <R extends Result> 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<R> resultClazz = ModelSupport.entityTypes.get(e);
|
||||
|
||||
Utils
|
||||
.readPath(spark, inputPath + e.name(), resultClazz)
|
||||
.map((MapFunction<R, EncloseMinElement>) p -> {
|
||||
EncloseMinElement eme = new EncloseMinElement();
|
||||
eme.setMinProduct(Utils.getMinProduct(p));
|
||||
eme.setEnclosedEntityId(p.getId());
|
||||
return eme;
|
||||
}, Encoders.bean(EncloseMinElement.class))
|
||||
.map(
|
||||
(MapFunction<R, MinProduct>) p -> Utils.getMinProduct(p),
|
||||
Encoders.bean(MinProduct.class))
|
||||
.write()
|
||||
.mode(SaveMode.Append)
|
||||
.option("compression", "gzip")
|
||||
.json(workingDir + "/minEntity");
|
||||
.json(workingDir + "/minProduct");
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -155,6 +155,9 @@ public class Utils implements Serializable {
|
|||
case "arxiv":
|
||||
mp.setArxivid(pid.getValue());
|
||||
break;
|
||||
case "pmid":
|
||||
mp.setPmid(pid.getValue());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -85,4 +85,5 @@ public class EncloseMinElement implements Serializable {
|
|||
public void setMinProduct(MinProduct minProduct) {
|
||||
this.minProduct = minProduct;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -62,7 +62,7 @@
|
|||
</property>
|
||||
</configuration>
|
||||
</global>
|
||||
<start to="select_relevant_graph_subset"/>
|
||||
<start to="dump_result"/>
|
||||
|
||||
|
||||
<decision name="filter_set">
|
||||
|
@ -218,19 +218,90 @@
|
|||
<arg>--filterPath</arg><arg>${filterPath}</arg>
|
||||
</spark>
|
||||
<!-- <ok to="emit_from_result"/>-->
|
||||
<ok to="copy_graph_subset"/>
|
||||
<ok to="copy_graph_publication"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_subset">
|
||||
<action name="copy_graph_publication">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/*</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/</arg>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/publication</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/publication</arg>
|
||||
</distcp>
|
||||
<ok to="copy_graph_dataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_dataset">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/dataset</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/dataset</arg>
|
||||
</distcp>
|
||||
<ok to="copy_graph_otherresearchproduct"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_otherresearchproduct">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/otherresearchproduct</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/otherresearchproduct</arg>
|
||||
</distcp>
|
||||
<ok to="copy_graph_software"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_software">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/software</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/software</arg>
|
||||
</distcp>
|
||||
<ok to="copy_graph_datasource"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_datasource">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/datasource</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/datasource</arg>
|
||||
</distcp>
|
||||
<ok to="copy_graph_project"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_project">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/project</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/project</arg>
|
||||
</distcp>
|
||||
<ok to="copy_graph_organization"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_organization">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/organization</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/organization</arg>
|
||||
</distcp>
|
||||
<ok to="copy_graph_relation"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="copy_graph_relation">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/relation</arg>
|
||||
<arg>${nameNode}/${workingDir}/graph/relation</arg>
|
||||
</distcp>
|
||||
<ok to="emit_from_result"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="emit_from_result">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
|
@ -248,9 +319,11 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
|
||||
<arg>--workingDir</arg><arg>${workingDir}/</arg>
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>-->
|
||||
<!-- <arg>--workingDir</arg><arg>${workingDir}/</arg>-->
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
<arg>--workingDir</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/</arg>
|
||||
<arg>--sourcePath</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/</arg>
|
||||
</spark>
|
||||
<ok to="dump_result"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -263,18 +336,22 @@
|
|||
<class>eu.dnetlib.dhp.oa.graph.dump.skgif.DumpResult</class>
|
||||
<jar>dump-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--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
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>-->
|
||||
<arg>--sourcePath</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
<arg>--workingDir</arg><arg>${workingDir}/</arg>
|
||||
<!-- <arg>--workingDir</arg><arg>${workingDir}/</arg>-->
|
||||
<arg>--workingDir</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/</arg>
|
||||
</spark>
|
||||
<ok to="dump_datasource"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -296,9 +373,11 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>-->
|
||||
<arg>--sourcePath</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
<arg>--workingDir</arg><arg>${workingDir}/</arg>
|
||||
<!-- <arg>--workingDir</arg><arg>${workingDir}/</arg>-->
|
||||
<arg>--workingDir</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/</arg>
|
||||
</spark>
|
||||
<ok to="dump_venue"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -320,9 +399,11 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>-->
|
||||
<arg>--sourcePath</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
<arg>--workingDir</arg><arg>${workingDir}/</arg>
|
||||
<!-- <arg>--workingDir</arg><arg>${workingDir}/</arg>-->
|
||||
<arg>--workingDir</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/</arg>
|
||||
</spark>
|
||||
<ok to="dump_organization"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -345,9 +426,11 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>-->
|
||||
<arg>--sourcePath</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
<arg>--workingDir</arg><arg>${workingDir}/</arg>
|
||||
<!-- <arg>--workingDir</arg><arg>${workingDir}/</arg>-->
|
||||
<arg>--workingDir</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/</arg>
|
||||
</spark>
|
||||
<ok to="dump_grant"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -370,9 +453,11 @@
|
|||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
|
||||
<!-- <arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>-->
|
||||
<arg>--sourcePath</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/graph/</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
<arg>--workingDir</arg><arg>${workingDir}/</arg>
|
||||
<!-- <arg>--workingDir</arg><arg>${workingDir}/</arg>-->
|
||||
<arg>--workingDir</arg><arg>/user/miriam.baglioni/oa/graph/dump/temp/working_dir/</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
|
|
|
@ -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<Row> relation = spark
|
||||
.read()
|
||||
.schema(relationstructureSchema)
|
||||
.json(sourcePath + "relation")
|
||||
.filter(
|
||||
"datainfo.deletedbyinference != true and " +
|
||||
"relclass == '" + RelationType.CITATION.label + "'")
|
||||
.drop("dataInfo");
|
||||
|
||||
Dataset<Row> 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()
|
||||
|
|
|
@ -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"}}
|
Binary file not shown.
|
@ -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"}}
|
Binary file not shown.
|
@ -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"}}
|
Binary file not shown.
|
@ -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,<i>Dascyllus aruanus</i>(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 <i>Danio rerio</i> 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"}}
|
Binary file not shown.
|
@ -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"}}
|
Binary file not shown.
|
@ -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"}}
|
Binary file not shown.
|
@ -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"}}
|
Binary file not shown.
|
@ -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"}}
|
Binary file not shown.
|
@ -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"}
|
|
@ -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,<i>Dascyllus aruanus</i>(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 <i>Danio rerio</i> 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"}
|
|
@ -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"}
|
|
@ -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"}
|
Loading…
Reference in New Issue