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.*; import java.util.stream.Collectors; import javax.xml.crypto.Data; 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.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.oa.graph.dump.skgif.beans.*; 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.skgif.model.*; import eu.dnetlib.dhp.skgif.model.AccessRight; import eu.dnetlib.dhp.skgif.model.Organization; import scala.Tuple2; /** * @author miriam.baglioni * @Date 06/02/24 */ public class DumpResult implements Serializable { private static final Logger log = LoggerFactory.getLogger(DumpResult.class); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( DumpResult.class .getResourceAsStream( "/eu/dnetlib/dhp/oa/graph/dump/skgif/dump_result_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); Boolean isSparkSessionManaged = Optional .ofNullable(parser.get("isSparkSessionManaged")) .map(Boolean::valueOf) .orElse(Boolean.TRUE); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); final String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); final String workingDir = parser.get("workingDir"); log.info("workingDir: {}", workingDir); final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); SparkConf conf = new SparkConf(); runWithSparkSession( conf, isSparkSessionManaged, spark -> { Utils.removeOutputDir(spark, workingDir + "aggrelation"); mapResult(spark, inputPath, workingDir, outputPath); }); } //per ogni result emetto id + journal se esiste + istanza + hosted by dell'istanza public static void mapResult(SparkSession spark, String inputPath, String workingDir, String outputPath) { // merge of relations and manifestation for the same result getRelationAndManifestation(spark, workingDir, inputPath); // dump of the result and enrichment with relevant information for relations and manifestations dumpResult(spark, inputPath, workingDir, outputPath); } private static void getRelationAndManifestation(SparkSession spark, String workingDir, String inputPath) { // Dataset aggRelations = Utils // .readPath(spark, workingDir + "aggrelation", RelationPerProduct.class); final StructType rp = new StructType() .add( "dataInfo", new StructType() .add("deletedbyinference", DataTypes.BooleanType)) .add( "eoscdatasourcetype", new StructType() .add("classid", DataTypes.StringType)) .add("id", DataTypes.StringType) ; ModelSupport.entityTypes .keySet() .stream() .filter(ModelSupport::isResult) .forEach(e -> { Utils.removeOutputDir(spark, workingDir + e.name() + "/partialresearchproduct"); log.info("executing on {}", e.name()); Dataset datasource = spark .read() .schema(rp) .json(inputPath + "/datasource") .filter(("datainfo.deletedbyinference != true and eoscdatasourcetype.classid == 'Journal archive'")) .drop("datainfo", "eoscdatasourcetype"); Dataset man = spark .read() .schema(Encoders.bean(EmitPerManifestation.class).schema()) .json(workingDir + e.name() + "/manifestation"); // Dataset partialResearchProduct = man .joinWith(datasource, man.col("hostedby").equalTo(datasource.col("id")), "left") .groupByKey( (MapFunction, String>) t2 -> t2._1().getAs("resultId"), Encoders.STRING()) .mapGroups( (MapGroupsFunction, PartialResearchProduct>) ( k, v) -> { PartialResearchProduct prp = new PartialResearchProduct(); prp.setResultId(k); List manifestationList = new ArrayList<>(); Tuple2 first = v.next(); manifestationList.add(getManifestation(first)); v.forEachRemaining(value -> manifestationList.add(getManifestation(value))); prp.setManifestations(manifestationList); return prp; }, Encoders.bean(PartialResearchProduct.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + e.name() + "/temp_partitalresearchproduct"); Dataset partialResearchProduct = spark .read() .schema(Encoders.bean(PartialResearchProduct.class).schema()) .json(workingDir + e.name() + "/temp_partitalresearchproduct") .as(Encoders.bean(PartialResearchProduct.class)); Dataset affiliations = Utils .readPath( spark, workingDir + "relations/result_relevant_organizations", ExtendingOrganization.class); // Dataset prrWithAffiliation = partialResearchProduct .joinWith( affiliations, partialResearchProduct.col("resultId").equalTo(affiliations.col("entityId")), "left") .map( (MapFunction, PartialResearchProduct>) t2 -> { if (t2._2() == null) return t2._1(); t2._1().setRelevant_organizations(t2._2().getRelevant_organization()); return t2._1(); }, Encoders.bean(PartialResearchProduct.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + e.name() + "/temp_partitalresearchproductaff"); Utils.removeOutputDir(spark, workingDir + e.name() + "/temp_partitalresearchproduct"); Dataset prrWithAffiliation = spark .read() .schema(Encoders.bean(PartialResearchProduct.class).schema()) .json(workingDir + e.name() + "/temp_partitalresearchproductaff") .as(Encoders.bean(PartialResearchProduct.class)); Dataset grants = Utils .readPath(spark, workingDir + "relations/funding", GrantRelation.class); // Dataset pprWitGrants = prrWithAffiliation .joinWith(grants, prrWithAffiliation.col("resultId").equalTo(grants.col("resultId")), "left") .map((MapFunction, PartialResearchProduct>) t2 -> { if (t2._2() == null) return t2._1(); t2._1().setFunding(t2._2().getFunding()); return t2._1(); }, Encoders.bean(PartialResearchProduct.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + e.name() + "/temp_partitalresearchproductgrant"); Utils.removeOutputDir(spark, workingDir + e.name() + "/temp_partitalresearchproductaff"); Dataset pprWitGrants = spark .read() .schema(Encoders.bean(PartialResearchProduct.class).schema()) .json(workingDir + e.name() + "/temp_partitalresearchproductgrant") .as(Encoders.bean(PartialResearchProduct.class)); Dataset relatedResults = Utils .readPath(spark, workingDir + "/relations/related_products", ProductsRelation.class); pprWitGrants .joinWith( relatedResults, pprWitGrants.col("resultId").equalTo(relatedResults.col("resultId")), "left") .map( (MapFunction, PartialResearchProduct>) t2 -> { if (t2._2() == null) return t2._1(); t2._1().setRelated_products(t2._2().getRelated_products()); return t2._1(); }, Encoders.bean(PartialResearchProduct.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + e.name() + "/partialresearchproduct"); Utils.removeOutputDir(spark, workingDir + e.name() + "/temp_partitalresearchproductgrant"); }); } private static Manifestation getManifestation(Tuple2 t2) { // se il lato sinistro c'e' allora ho la biblio e la venue // se non c'e' allora ho solo gli altri valori Row epm = t2._1(); Manifestation manifestation = new Manifestation(); manifestation.setProduct_local_type(epm.getAs("product_local_type")); manifestation.setProduct_local_type_schema(epm.getAs("product_local_type_schema")); if (Optional.ofNullable(epm.getAs("publishing_date")).isPresent()) manifestation .setDates( Arrays .asList( Dates.newInstance(epm.getAs("publishing_date"), "publishing"))); manifestation.setPeer_review(epm.getAs("peer_reviewed")); manifestation.setMetadata_curation("unavailable"); manifestation.setAccess_right(epm.getAs("access_right")); manifestation.setLicence(epm.getAs("licence")); manifestation.setUrl(epm.getAs("url")); manifestation.setPid(epm.getAs("pid")); if (Optional.ofNullable(t2._2()).isPresent()) { Biblio biblio = getBiblio(epm); // if (biblio == null) // log.info("null biblio fo {} ", epm.getAs("resultId")); manifestation.setBiblio(getBiblio(epm)); if (Optional.ofNullable(epm.getAs("journal")).isPresent() && Optional.ofNullable(epm.getAs("journal.issnPrinted")).isPresent()) manifestation .setVenue( MinVenue .newInstance( Utils.getIdentifier(Prefixes.VENUE, epm.getAs("journal.issnPrinted")), epm.getAs("hostedbyvalue"))); else if (Optional.ofNullable(epm.getAs("journal")).isPresent() && Optional.ofNullable(epm.getAs("journal.issnOnline")).isPresent()) manifestation .setVenue( MinVenue .newInstance( Utils.getIdentifier(Prefixes.VENUE, epm.getAs("journal.issnOnline")), epm.getAs("hostedbyvalue"))); } manifestation .setHosting_datasource( MinVenue .newInstance( // Utils.getIdentifier(Prefixes.DATASOURCE, epm.getInstance().getHostedby().getKey()), epm.getAs("hostedBy"), epm.getAs("hostedbyvalue"))); return manifestation; } private static Biblio getBiblio(Row epm) { Biblio biblio = new Biblio(); if (!Optional.ofNullable(epm.getAs("journal")).isPresent()) { return null; } if (Optional.ofNullable(epm.getAs("journal.edition")).isPresent()) biblio.setEdition(epm.getAs("journal.edition")); if (Optional.ofNullable(epm.getAs("journal.iss")).isPresent()) biblio.setIssue(epm.getAs("journal.iss")); if (Optional.ofNullable(epm.getAs("publisher")).isPresent()) biblio.setPublisher(epm.getAs("publisher")); if (Optional.ofNullable(epm.getAs("journal.vol")).isPresent()) biblio.setVolume(epm.getAs("journal.vol")); if (Optional.ofNullable(epm.getAs("journal.ep")).isPresent()) biblio.setEnd_page(epm.getAs("journal.ep")); if (Optional.ofNullable(epm.getAs("journal.sp")).isPresent()) biblio.setStart_page(epm.getAs("journal.sp")); return biblio; } private static void dumpResult(SparkSession spark, String inputPath, String workingDir, String outputPath) { ModelSupport.entityTypes .keySet() .parallelStream() .filter(ModelSupport::isResult) .forEach(e -> { Class resultClazz = ModelSupport.entityTypes.get(e); Utils.removeOutputDir(spark, workingDir + e.name() + "/researchproduct"); Dataset results = Utils.readPath(spark, inputPath + e.name(), resultClazz); Dataset prr = Utils .readPath(spark, workingDir + e.name() + "/partialresearchproduct", PartialResearchProduct.class); results .joinWith(prr, results.col("id").equalTo(prr.col("resultId")), "left") .map((MapFunction, ResearchProduct>) t2 -> { 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()); if (Optional.ofNullable(t2._2().getFunding()).isPresent()) rp.setFunding(t2._2().getFunding()); if (Optional.ofNullable(t2._2().getRelevant_organizations()).isPresent()) rp.setRelevant_organizations(t2._2().getRelevant_organizations()); if (Optional.ofNullable(t2._2().getManifestations()).isPresent()) rp.setManifestations(t2._2().getManifestations()); } return rp; }, Encoders.bean(ResearchProduct.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + e.name() + "/researchproduct"); }); Dataset researchProducts = spark.emptyDataset(Encoders.bean(ResearchProduct.class)); for (EntityType e : ModelSupport.entityTypes.keySet()) { if (ModelSupport.isResult(e)) researchProducts = researchProducts .union(Utils.readPath(spark, workingDir + e.name() + "/researchproduct", ResearchProduct.class)); } researchProducts .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputPath + "ResearchProduct"); } private static void selectRelations(SparkSession spark, String inputPath, String workingDir) { List relationsProducts = Arrays .asList( RelationType.CITATION.label, // RelationType.RESULT_AFFILIATIED_TO_ORGANIZATION.label, RelationType.SUPPLEMENT.label, // RelationType.RESULT_OUTCOME_FUNDING.label, RelationType.DOCUMENTS.label, RelationType.PART.label, RelationType.VERSION.label); Dataset relation = spark .read() .schema(Encoders.bean(Relation.class).schema()) .json(inputPath + "relation") .filter( "datainfo.deletedbyinference != true and " + "relClass == 'hasAuthorInstitution") .select("source", "target"); Dataset organization = spark .read() .schema(Encoders.bean(Organization.class).schema()) .json(inputPath + "organization") .filter("datainfo.deletedbyinference != true") .select("id", "pid", "legalname.value"); // result = spark.read().schema(Encoders.bean(Result.class).schema()) // .json(inputPath + ) // 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); RDD temp = spark .read() .schema(Encoders.bean(RelationPerProduct.class).schema()) .json(workingDir + "aggrelation_temp") .as(Encoders.bean(RelationPerProduct.class)) .toJavaRDD() .mapToPair(v -> new Tuple2<>(v.getResultId(), v)) .reduceByKey((a, b) -> { mergeRelationPerProduct(a, b); return a; }) .map(v -> v._2()) .rdd(); spark .createDataset(temp, Encoders.bean(RelationPerProduct.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(workingDir + "/aggrelation"); } private static void buildRelationPerGrant(SparkSession spark, String inputPath, String workingDir, String relationType) { log.info("Relation: {}", relationType); final StructType relationstructureSchema = new StructType() .add( "dataInfo", new StructType() .add("deletedbyinference", DataTypes.BooleanType)) .add("source", DataTypes.StringType) .add("target", DataTypes.StringType) .add("relClass", DataTypes.StringType); final StructType grantSchema = new StructType() .add( "local_identifier", DataTypes.StringType) .add("funder", DataTypes.StringType) .add("code", DataTypes.StringType) .add("title", DataTypes.StringType) ; Dataset relation = spark .read() .schema(relationstructureSchema) .json(inputPath + "relation") .filter( "datainfo.deletedbyinference != true and " + "relclass == '" + relationType + "'") .drop("dataInfo"); Dataset minProduct = spark .read() .schema(grantSchema) .json(workingDir + "minGrant"); relation .joinWith( minProduct, relation.col("target").equalTo(minProduct.col("local_identifier"))) .selectExpr("_1.source as sourceResult", "_1.relClass as relClass", "_2.*") .groupByKey((MapFunction) r -> r.getAs("sourceResult"), Encoders.STRING()) .mapGroups((MapGroupsFunction) (k, it) -> { RelationPerProduct rpp = new RelationPerProduct(); rpp.setResultId(k); rpp.setRelatedProduct(new HashMap<>()); updateRelevantGrant(rpp, it.next()); it.forEachRemaining(r -> updateRelevantGrant(rpp, r)); return rpp; }, Encoders.bean(RelationPerProduct.class)) // .show(false); .write() .mode(SaveMode.Append) .option("compression", "gzip") .json(workingDir + "aggrelation_temp"); } private static void updateRelevantGrant(RelationPerProduct rpp, Row next) { if (!Optional.ofNullable(rpp.getFunding()).isPresent()) rpp.setFunding(new ArrayList<>()); MinGrant mo = new MinGrant(); mo.setLocal_identifier(next.getAs("local_identifier")); mo.setTitle(next.getAs("title")); mo.setFunder(next.getAs("fundet")); mo.setCode(next.getAs("code")); rpp.getFunding().add(mo); } private static void buildRelationPerAffiliation(SparkSession spark, String inputPath, String workingDir, String relationType) { log.info("Relation: {}", relationType); final StructType relationstructureSchema = new StructType() .add( "dataInfo", new StructType() .add("deletedbyinference", DataTypes.BooleanType)) .add("source", DataTypes.StringType) .add("target", DataTypes.StringType) .add("relClass", DataTypes.StringType); final StructType orgSchema = new StructType() .add( "local_identifier", DataTypes.StringType) .add("name", DataTypes.StringType) .add("ror", DataTypes.StringType) .add("isni", DataTypes.StringType) .add("fundRef", DataTypes.StringType) .add("rinGold", DataTypes.StringType) .add("wikidata", DataTypes.StringType); Dataset relation = spark .read() .schema(relationstructureSchema) .json(inputPath + "relation") .filter( "datainfo.deletedbyinference != true and " + "relclass == '" + relationType + "'") .drop("dataInfo"); Dataset minOrganization = spark .read() .schema(orgSchema) .json(workingDir + "minOrganization"); relation .joinWith( minOrganization, relation.col("target").equalTo(minOrganization.col("local_identifier"))) .selectExpr("_1.source as sourceResult", "_1.relClass as relClass", "_2.*") .groupByKey((MapFunction) r -> r.getAs("sourceResult"), Encoders.STRING()) .mapGroups((MapGroupsFunction) (k, it) -> { RelationPerProduct rpp = new RelationPerProduct(); rpp.setResultId(k); rpp.setRelatedProduct(new HashMap<>()); updateRelevantOrganization(rpp, it.next()); it.forEachRemaining(r -> updateRelevantOrganization(rpp, r)); return rpp; }, Encoders.bean(RelationPerProduct.class)) // .show(false); .write() .mode(SaveMode.Append) .option("compression", "gzip") .json(workingDir + "aggrelation_temp"); } private static void updateRelevantOrganization(RelationPerProduct rpp, Row next) { if (!Optional.ofNullable(rpp.getOrganizations()).isPresent()) rpp.setOrganizations(new ArrayList<>()); MinOrganization mo = new MinOrganization(); mo.setLocal_identifier(next.getAs("local_identifier")); mo.setIsni(next.getAs("isni")); mo.setRor(next.getAs("ror")); mo.setName(next.getAs("name")); mo.setWikidata(next.getAs("wikidata")); mo.setFundRef(next.getAs("fundRef")); mo.setRinGold(next.getAs("rinGold")); rpp.getOrganizations().add(mo); } private static void buildRelationPerProducts(SparkSession spark, String inputPath, String workingDir, String relationType) { log.info("Relation: {}", relationType); final StructType relationstructureSchema = new StructType() .add( "dataInfo", new StructType() .add("deletedbyinference", DataTypes.BooleanType)) .add("source", DataTypes.StringType) .add("target", DataTypes.StringType) .add("relClass", DataTypes.StringType); final StructType productSchema = new StructType() .add( "local_identifier", DataTypes.StringType) .add("title", DataTypes.StringType) .add("doi", DataTypes.StringType) .add("pmcid", DataTypes.StringType) .add("arxivid", DataTypes.StringType) .add("pmid", DataTypes.StringType); Dataset relation = spark .read() .schema(relationstructureSchema) .json(inputPath + "relation") .filter( "datainfo.deletedbyinference != true and " + "relclass == '" + relationType + "'") .drop("dataInfo"); Dataset minProduct = spark .read() .schema(productSchema) .json(workingDir + "minProduct"); relation .joinWith( minProduct, relation.col("target").equalTo(minProduct.col("local_identifier"))) .selectExpr("_1.source as sourceResult", "_1.relClass as relClass", "_2.*") .groupByKey((MapFunction) r -> r.getAs("sourceResult"), Encoders.STRING()) .mapGroups((MapGroupsFunction) (k, it) -> { RelationPerProduct rpp = new RelationPerProduct(); rpp.setResultId(k); rpp.setRelatedProduct(new HashMap<>()); updateRelatedProduct(rpp, it.next()); it.forEachRemaining(r -> updateRelatedProduct(rpp, r)); return rpp; }, Encoders.bean(RelationPerProduct.class)) // .show(false); .write() .mode(SaveMode.Append) .option("compression", "gzip") .json(workingDir + "aggrelation_temp"); // .map((MapFunction, RelationPerProduct>) t2 -> { // RelationPerProduct rpp = new RelationPerProduct(); // t2._2().setResultId(t2._1().getAs("source")); // t2._2().setSemantics(t2._1().getAs("relClass")); // insertEnclosedElement(rpp, t2._2()); // rpp.setResultId(t2._1().getAs("source")); // return rpp; // }, Encoders.bean(RelationPerProduct.class)) // .filter(Objects::nonNull) // .toJavaRDD() // .mapToPair(value -> new Tuple2<>(value.getResultId(), value)) // .reduceByKey((a, b) -> { // mergeRelationPerProduct(a, b); // return a; // }) // // .map(value -> value._2) // .rdd(), // Encoders.bean(RelationPerProduct.class)) //// .saveAsTextFile(workingDir + "/aggrelation", GzipCodec.class); //// .groupByKey((MapFunction) eme -> eme.getResultId(), Encoders.STRING()) //// .mapGroups((MapGroupsFunction) (k, v) -> { //// RelationPerProduct rpp = new RelationPerProduct(); //// rpp.setResultId(k); //// insertEnclosedElement(rpp, v.next()); //// v.forEachRemaining(e -> insertEnclosedElement(rpp, e)); //// return rpp; //// }, Encoders.bean(RelationPerProduct.class)) // .write() // .mode(SaveMode.Append) // .option("compression", "gzip") // .json(workingDir + "/aggrelation_temp"); } private static void updateRelatedProduct(RelationPerProduct rpp, Row next) { String key = next.getAs("relClass"); if (!rpp.getRelatedProduct().keySet().contains(key)) rpp.getRelatedProduct().put(key, new ArrayList<>()); MinProduct mp = new MinProduct(); mp.setLocal_identifier(next.getAs("local_identifier")); mp.setTitle(next.getAs("title")); mp.setPmid(next.getAs("pmid")); mp.setArxivid(next.getAs("arxivid")); mp.setPmcid(next.getAs("pmcid")); mp.setDoi(next.getAs("doi")); rpp.getRelatedProduct().get(key).add(mp); } private static void insertEnclosedElement(RelationPerProduct rpp, EncloseMinElement element) { if (Optional.ofNullable(element.getMinOrganization()).isPresent()) rpp.getOrganizations().add(element.getMinOrganization()); if (Optional.ofNullable(element.getMinGrant()).isPresent()) rpp.getFunding().add(element.getMinGrant()); if (Optional.ofNullable(element.getMinProduct()).isPresent()) { String sem = element.getSemantics(); if (!rpp.getRelatedProduct().containsKey(sem)) rpp.getRelatedProduct().put(sem, new ArrayList<>()); rpp.getRelatedProduct().get(sem).add(element.getMinProduct()); } } private static void mergeRelationPerProduct(RelationPerProduct rpp1, RelationPerProduct rpp2) { if (Optional.ofNullable(rpp2.getOrganizations()).isPresent()) rpp1.getOrganizations().addAll(rpp2.getOrganizations()); if (Optional.ofNullable(rpp2.getFunding()).isPresent()) rpp1.getFunding().addAll(rpp2.getFunding()); if (Optional.ofNullable(rpp2.getRelatedProduct()).isPresent()) { Map> temp = rpp2.getRelatedProduct(); for (String key : temp.keySet()) { if (!rpp1.getRelatedProduct().containsKey(key)) rpp1.getRelatedProduct().put(key, new ArrayList<>()); for (MinProduct mp : rpp2.getRelatedProduct().get(key)) rpp1.getRelatedProduct().get(key).add(mp); } } } }