dedupRecord = DedupRecordFactory
- .createDedupRecord(
- sc,
- spark,
- DedupUtility.createMergeRelPath(dedupPath, entity),
- DedupUtility.createEntityPath(sourcePath, entity),
- OafEntityType.valueOf(entity),
- dedupConf);
- spark
- .createDataset(dedupRecord.rdd(), Encoders.kryo(OafEntity.class))
- .write()
- .mode(SaveMode.Overwrite)
- .save(dedupPath + "/" + entity + "/dedup_records");
-//
-//
-// dedupRecord
-// .map(
-// r -> {
-// ObjectMapper mapper = new ObjectMapper();
-// return mapper.writeValueAsString(r);
-// })
-// .saveAsTextFile(dedupPath + "/" + entity + "/dedup_records");
- }
-}
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java
deleted file mode 100644
index 7adf992cd..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java
+++ /dev/null
@@ -1,92 +0,0 @@
-
-package eu.dnetlib.dedup;
-
-import java.util.List;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.MapFunction;
-import org.apache.spark.sql.Encoders;
-import org.apache.spark.sql.SparkSession;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-import eu.dnetlib.dhp.schema.oaf.Oaf;
-import eu.dnetlib.dhp.schema.oaf.Relation;
-import eu.dnetlib.pace.config.DedupConfig;
-import eu.dnetlib.pace.model.MapDocument;
-import eu.dnetlib.pace.util.MapDocumentUtil;
-import scala.Tuple2;
-
-/**
- * This Spark class creates similarity relations between entities, saving result
- *
- * param request: sourcePath entityType target Path
- */
-public class SparkCreateSimRels {
-
- public static void main(String[] args) throws Exception {
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils
- .toString(
- SparkCreateSimRels.class
- .getResourceAsStream(
- "/eu/dnetlib/dhp/sx/dedup/dedup_parameters.json")));
- parser.parseArgument(args);
- final SparkSession spark = SparkSession
- .builder()
- .appName(SparkCreateSimRels.class.getSimpleName())
- .master(parser.get("master"))
- .getOrCreate();
-
- final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
- final String inputPath = parser.get("sourcePath");
- final String entity = parser.get("entity");
- final String targetPath = parser.get("targetPath");
- // final DedupConfig dedupConf =
- // DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json")));
- final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf"));
-
- JavaPairRDD mapDocument = spark
- .read()
- .load(inputPath + "/" + entity)
- .as(Encoders.kryo(Oaf.class))
- .map((MapFunction) p -> new ObjectMapper().writeValueAsString(p), Encoders.STRING())
- .javaRDD()
- .repartition(1000)
- .mapToPair(
- s -> {
- MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
- return new Tuple2<>(d.getIdentifier(), d);
- });
-
- // create blocks for deduplication
- JavaPairRDD> blocks = Deduper.createsortedBlocks(sc, mapDocument, dedupConf);
- // JavaPairRDD> blocks = Deduper.createBlocks(sc,
- // mapDocument, dedupConf);
-
- // create relations by comparing only elements in the same group
- final JavaPairRDD dedupRels = Deduper.computeRelations2(sc, blocks, dedupConf);
- // final JavaPairRDD dedupRels = Deduper.computeRelations(sc, blocks,
- // dedupConf);
-
- final JavaRDD isSimilarToRDD = dedupRels
- .map(
- simRel -> {
- final Relation r = new Relation();
- r.setSource(simRel._1());
- r.setTarget(simRel._2());
- r.setRelClass("isSimilarTo");
- return r;
- });
-
- spark
- .createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class))
- .write()
- .mode("overwrite")
- .save(DedupUtility.createSimRelPath(targetPath, entity));
- }
-}
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkReporter.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkReporter.java
deleted file mode 100644
index 21e72b5b8..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkReporter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-
-package eu.dnetlib.dedup;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.spark.util.LongAccumulator;
-
-import eu.dnetlib.pace.util.Reporter;
-import scala.Serializable;
-import scala.Tuple2;
-
-public class SparkReporter implements Serializable, Reporter {
-
- final List> relations = new ArrayList<>();
- private static final Log log = LogFactory.getLog(SparkReporter.class);
- Map accumulators;
-
- public SparkReporter(Map accumulators) {
- this.accumulators = accumulators;
- }
-
- public void incrementCounter(
- String counterGroup,
- String counterName,
- long delta,
- Map accumulators) {
-
- final String accumulatorName = String.format("%s::%s", counterGroup, counterName);
- if (accumulators.containsKey(accumulatorName)) {
- accumulators.get(accumulatorName).add(delta);
- }
- }
-
- @Override
- public void incrementCounter(String counterGroup, String counterName, long delta) {
-
- incrementCounter(counterGroup, counterName, delta, accumulators);
- }
-
- @Override
- public void emit(String type, String from, String to) {
- relations.add(new Tuple2<>(from, to));
- }
-
- public List> getRelations() {
- return relations;
- }
-}
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java
deleted file mode 100644
index 79a3114fd..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java
+++ /dev/null
@@ -1,84 +0,0 @@
-
-package eu.dnetlib.dedup.graph;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Set;
-
-import org.apache.commons.lang.StringUtils;
-import org.codehaus.jackson.annotate.JsonIgnore;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import eu.dnetlib.dedup.DedupUtility;
-import eu.dnetlib.pace.util.PaceException;
-
-public class ConnectedComponent implements Serializable {
-
- private Set docIds;
- private String ccId;
-
- public ConnectedComponent() {
- }
-
- public ConnectedComponent(Set docIds) {
- this.docIds = docIds;
- createID();
- }
-
- public String createID() {
- if (docIds.size() > 1) {
- final String s = getMin();
- String prefix = s.split("\\|")[0];
- ccId = prefix + "|dedup_wf_001::" + DedupUtility.md5(s);
- return ccId;
- } else {
- return docIds.iterator().next();
- }
- }
-
- @JsonIgnore
- public String getMin() {
-
- final StringBuilder min = new StringBuilder();
- docIds
- .forEach(
- i -> {
- if (StringUtils.isBlank(min.toString())) {
- min.append(i);
- } else {
- if (min.toString().compareTo(i) > 0) {
- min.setLength(0);
- min.append(i);
- }
- }
- });
- return min.toString();
- }
-
- @Override
- public String toString() {
- ObjectMapper mapper = new ObjectMapper();
- try {
- return mapper.writeValueAsString(this);
- } catch (IOException e) {
- throw new PaceException("Failed to create Json: ", e);
- }
- }
-
- public Set getDocIds() {
- return docIds;
- }
-
- public void setDocIds(Set docIds) {
- this.docIds = docIds;
- }
-
- public String getCcId() {
- return ccId;
- }
-
- public void setCcId(String ccId) {
- this.ccId = ccId;
- }
-}
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala
deleted file mode 100644
index 38c695152..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala
+++ /dev/null
@@ -1,37 +0,0 @@
-package eu.dnetlib.dedup.graph
-
-import org.apache.spark.graphx._
-import org.apache.spark.rdd.RDD
-
-import scala.collection.JavaConversions;
-
-object GraphProcessor {
-
- def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int): RDD[ConnectedComponent] = {
- val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby
- val cc = graph.connectedComponents(maxIterations).vertices
-
- val joinResult = vertexes.leftOuterJoin(cc).map {
- case (id, (openaireId, cc)) => {
- if (cc.isEmpty) {
- (id, openaireId)
- }
- else {
- (cc.get, openaireId)
- }
- }
- }
- val connectedComponents = joinResult.groupByKey()
- .map[ConnectedComponent](cc => asConnectedComponent(cc))
- connectedComponents
- }
-
-
-
- def asConnectedComponent(group: (VertexId, Iterable[String])): ConnectedComponent = {
- val docs = group._2.toSet[String]
- val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs));
- connectedComponent
- }
-
-}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkPropagateRelationsJob.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkPropagateRelationsJob.java
deleted file mode 100644
index 3134f9400..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkPropagateRelationsJob.java
+++ /dev/null
@@ -1,78 +0,0 @@
-
-package eu.dnetlib.dedup.sx;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.spark.api.java.function.MapFunction;
-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 eu.dnetlib.dhp.application.ArgumentApplicationParser;
-import eu.dnetlib.dhp.schema.oaf.Relation;
-import eu.dnetlib.dhp.schema.scholexplorer.OafUtils;
-import scala.Tuple2;
-
-public class SparkPropagateRelationsJob {
-
- public static void main(String[] args) throws Exception {
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils
- .toString(
- SparkPropagateRelationsJob.class
- .getResourceAsStream(
- "/eu/dnetlib/dhp/sx/dedup/dedup_propagate_relation_parameters.json")));
- parser.parseArgument(args);
- final SparkSession spark = SparkSession
- .builder()
- .appName(SparkUpdateEntityJob.class.getSimpleName())
- .master(parser.get("master"))
- .getOrCreate();
-
- final String relationPath = parser.get("relationPath");
- final String mergeRelPath = parser.get("mergeRelPath");
- final String targetRelPath = parser.get("targetRelPath");
-
- final Dataset merge = spark
- .read()
- .load(mergeRelPath)
- .as(Encoders.bean(Relation.class))
- .where("relClass == 'merges'");
-
- final Dataset rels = spark
- .read()
- .load(relationPath)
- .as(Encoders.kryo(Relation.class))
- .map(
- (MapFunction) r -> r,
- Encoders.bean(Relation.class));
-
- final Dataset firstJoin = rels
- .joinWith(merge, merge.col("target").equalTo(rels.col("source")), "left_outer")
- .map(
- (MapFunction, Relation>) r -> {
- final Relation mergeRelation = r._2();
- final Relation relation = r._1();
- if (mergeRelation != null)
- relation.setSource(mergeRelation.getSource());
- if (relation.getDataInfo() == null)
- relation.setDataInfo(OafUtils.generateDataInfo("0.9", false));
- return relation;
- },
- Encoders.bean(Relation.class));
-
- final Dataset secondJoin = firstJoin
- .joinWith(merge, merge.col("target").equalTo(firstJoin.col("target")), "left_outer")
- .map(
- (MapFunction, Relation>) r -> {
- final Relation mergeRelation = r._2();
- final Relation relation = r._1();
- if (mergeRelation != null)
- relation.setTarget(mergeRelation.getSource());
- return relation;
- },
- Encoders.kryo(Relation.class));
-
- secondJoin.write().mode(SaveMode.Overwrite).save(targetRelPath);
- }
-}
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityJob.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityJob.java
deleted file mode 100644
index a847ad612..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityJob.java
+++ /dev/null
@@ -1,102 +0,0 @@
-
-package eu.dnetlib.dedup.sx;
-
-import java.io.IOException;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.PairFunction;
-import org.apache.spark.sql.*;
-
-import com.fasterxml.jackson.databind.DeserializationFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-import eu.dnetlib.dhp.schema.oaf.DataInfo;
-import eu.dnetlib.dhp.schema.oaf.Oaf;
-import eu.dnetlib.dhp.schema.oaf.Relation;
-import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
-import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
-import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown;
-import eu.dnetlib.dhp.utils.DHPUtils;
-import scala.Tuple2;
-
-public class SparkUpdateEntityJob {
-
- static final String IDJSONPATH = "$.id";
-
- public static void main(String[] args) throws Exception {
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils
- .toString(
- SparkUpdateEntityJob.class
- .getResourceAsStream(
- "/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json")));
- parser.parseArgument(args);
- final SparkSession spark = SparkSession
- .builder()
- .appName(SparkUpdateEntityJob.class.getSimpleName())
- .master(parser.get("master"))
- .getOrCreate();
-
- final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
- final String entityPath = parser.get("entityPath");
- final String mergeRelPath = parser.get("mergeRelPath");
- final String dedupRecordPath = parser.get("dedupRecordPath");
- final String entity = parser.get("entity");
- final String destination = parser.get("targetPath");
-
- final Dataset df = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class));
- final JavaPairRDD mergedIds = df
- .where("relClass == 'merges'")
- .select(df.col("target"))
- .distinct()
- .toJavaRDD()
- .mapToPair((PairFunction) r -> new Tuple2<>(r.getString(0), "d"));
- final JavaRDD sourceEntity = sc.textFile(entityPath);
-
- final JavaRDD dedupEntity = sc.textFile(dedupRecordPath);
- JavaPairRDD entitiesWithId = sourceEntity
- .mapToPair(
- (PairFunction) s -> new Tuple2<>(DHPUtils.getJPathString(IDJSONPATH, s), s));
- Class extends Oaf> mainClass;
- switch (entity) {
- case "publication":
- mainClass = DLIPublication.class;
- break;
- case "dataset":
- mainClass = DLIDataset.class;
- break;
- case "unknown":
- mainClass = DLIUnknown.class;
- break;
- default:
- throw new IllegalArgumentException("Illegal type " + entity);
- }
- JavaRDD map = entitiesWithId
- .leftOuterJoin(mergedIds)
- .map(
- k -> k._2()._2().isPresent()
- ? updateDeletedByInference(k._2()._1(), mainClass)
- : k._2()._1());
- map.union(dedupEntity).saveAsTextFile(destination, GzipCodec.class);
- }
-
- private static String updateDeletedByInference(
- final String json, final Class clazz) {
- final ObjectMapper mapper = new ObjectMapper();
- mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
- try {
- Oaf entity = mapper.readValue(json, clazz);
- if (entity.getDataInfo() == null)
- entity.setDataInfo(new DataInfo());
- entity.getDataInfo().setDeletedbyinference(true);
- return mapper.writeValueAsString(entity);
- } catch (IOException e) {
- throw new RuntimeException("Unable to convert json", e);
- }
- }
-}
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityWithDedupInfo.scala b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityWithDedupInfo.scala
deleted file mode 100644
index ce883e207..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityWithDedupInfo.scala
+++ /dev/null
@@ -1,75 +0,0 @@
-package eu.dnetlib.dedup.sx
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser
-import eu.dnetlib.dhp.schema.oaf.{Oaf, OafEntity, Relation}
-import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown, OafUtils}
-import org.apache.commons.io.IOUtils
-import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
-import org.slf4j.LoggerFactory
-import org.apache.spark.sql.functions.col
-
-object SparkUpdateEntityWithDedupInfo {
-
- def main(args: Array[String]): Unit = {
- val parser = new ArgumentApplicationParser(IOUtils.toString(SparkUpdateEntityWithDedupInfo.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json")))
- val logger = LoggerFactory.getLogger(SparkUpdateEntityWithDedupInfo.getClass)
- parser.parseArgument(args)
-
- val workingPath: String = parser.get("workingPath")
- logger.info(s"Working dir path = $workingPath")
-
- implicit val oafEncoder: Encoder[OafEntity] = Encoders.kryo[OafEntity]
- implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
-
- implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
- implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
- implicit val unkEncoder: Encoder[DLIUnknown] = Encoders.kryo[DLIUnknown]
-
-
-
- val spark: SparkSession = SparkSession
- .builder()
- .appName(SparkUpdateEntityWithDedupInfo.getClass.getSimpleName)
- .master(parser.get("master"))
- .getOrCreate()
-
-
- val entityPath = parser.get("entityPath")
- val mergeRelPath = parser.get("mergeRelPath")
- val dedupRecordPath = parser.get("dedupRecordPath")
- val entity = parser.get("entity")
- val destination = parser.get("targetPath")
-
- val mergedIds = spark.read.load(mergeRelPath).as[Relation]
- .where("relClass == 'merges'")
- .select(col("target"))
-
-
- val entities: Dataset[(String, OafEntity)] = spark
- .read
- .load(entityPath).as[OafEntity]
- .map(o => (o.getId, o))(Encoders.tuple(Encoders.STRING, oafEncoder))
-
-
- val finalDataset:Dataset[OafEntity] = entities.joinWith(mergedIds, entities("_1").equalTo(mergedIds("target")), "left")
- .map(k => {
- val e: OafEntity = k._1._2
- val t = k._2
- if (t != null && t.getString(0).nonEmpty) {
- if (e.getDataInfo == null) {
- e.setDataInfo(OafUtils.generateDataInfo())
- }
- e.getDataInfo.setDeletedbyinference(true)
- }
- e
- })
-
- val dedupRecords :Dataset[OafEntity] = spark.read.load(dedupRecordPath).as[OafEntity]
-
- finalDataset.union(dedupRecords)
- .repartition(1200).write
- .mode(SaveMode.Overwrite).save(destination)
-
- }
-
-}
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedupRecord_parameters.json b/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedupRecord_parameters.json
deleted file mode 100644
index de744dfb6..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedupRecord_parameters.json
+++ /dev/null
@@ -1,33 +0,0 @@
-[
- {
- "paramName": "mt",
- "paramLongName": "master",
- "paramDescription": "should be local or yarn",
- "paramRequired": true
- },
- {
- "paramName": "s",
- "paramLongName": "sourcePath",
- "paramDescription": "the path of the sequential file to read",
- "paramRequired": true
- },
- {
- "paramName": "e",
- "paramLongName": "entity",
- "paramDescription": "the type of entity to be deduped",
- "paramRequired": true
- },
- {
- "paramName": "c",
- "paramLongName": "dedupConf",
- "paramDescription": "dedup configuration to be used",
- "compressed": true,
- "paramRequired": true
- },
- {
- "paramName": "d",
- "paramLongName": "dedupPath",
- "paramDescription": "dedup path to load mergeRelation",
- "paramRequired": true
- }
-]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json b/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json
deleted file mode 100644
index 69428a296..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json
+++ /dev/null
@@ -1,38 +0,0 @@
-[
- {
- "paramName": "mt",
- "paramLongName": "master",
- "paramDescription": "should be local or yarn",
- "paramRequired": true
- },
- {
- "paramName": "ep",
- "paramLongName": "entityPath",
- "paramDescription": "the input entity path",
- "paramRequired": true
- },
- {
- "paramName": "mr",
- "paramLongName": "mergeRelPath",
- "paramDescription": "the input path of merge Rel",
- "paramRequired": true
- },
- {
- "paramName": "dr",
- "paramLongName": "dedupRecordPath",
- "paramDescription": "the inputPath of dedup record",
- "paramRequired": true
- },
- {
- "paramName": "e",
- "paramLongName": "entity",
- "paramDescription": "the type of entity",
- "paramRequired": true
- },
- {
- "paramName": "t",
- "paramLongName": "targetPath",
- "paramDescription": "the targetPath",
- "paramRequired": true
- }
-]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_parameters.json b/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_parameters.json
deleted file mode 100644
index 8ba8515d0..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_parameters.json
+++ /dev/null
@@ -1,33 +0,0 @@
-[
- {
- "paramName": "mt",
- "paramLongName": "master",
- "paramDescription": "should be local or yarn",
- "paramRequired": true
- },
- {
- "paramName": "s",
- "paramLongName": "sourcePath",
- "paramDescription": "the path of the sequential file to read",
- "paramRequired": true
- },
- {
- "paramName": "e",
- "paramLongName": "entity",
- "paramDescription": "the type of entity to be deduped",
- "paramRequired": true
- },
- {
- "paramName": "c",
- "paramLongName": "dedupConf",
- "paramDescription": "dedup configuration to be used",
- "compressed": true,
- "paramRequired": true
- },
- {
- "paramName": "t",
- "paramLongName": "targetPath",
- "paramDescription": "target path to save dedup result",
- "paramRequired": true
- }
-]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_propagate_relation_parameters.json b/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_propagate_relation_parameters.json
deleted file mode 100644
index 2ce78440f..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/dedup_propagate_relation_parameters.json
+++ /dev/null
@@ -1,26 +0,0 @@
-[
- {
- "paramName": "mt",
- "paramLongName": "master",
- "paramDescription": "should be local or yarn",
- "paramRequired": true
- },
- {
- "paramName": "ep",
- "paramLongName": "relationPath",
- "paramDescription": "the input relation path",
- "paramRequired": true
- },
- {
- "paramName": "mr",
- "paramLongName": "mergeRelPath",
- "paramDescription": "the input path of merge Rel",
- "paramRequired": true
- },
- {
- "paramName": "t",
- "paramLongName": "targetRelPath",
- "paramDescription": "the output Rel Path",
- "paramRequired": true
- }
-]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/oozie_app/config-default.xml
deleted file mode 100644
index 2e0ed9aee..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/oozie_app/config-default.xml
+++ /dev/null
@@ -1,18 +0,0 @@
-
-
- jobTracker
- yarnRM
-
-
- nameNode
- hdfs://nameservice1
-
-
- oozie.use.system.libpath
- true
-
-
- oozie.action.sharelib.for.spark
- spark2
-
-
\ No newline at end of file
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/oozie_app/workflow.xml
deleted file mode 100644
index 2214fd20a..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/dedup/oozie_app/workflow.xml
+++ /dev/null
@@ -1,182 +0,0 @@
-
-
-
- sourcePath
- the source path
-
-
- entity
- the entity that should be processed
-
-
- dedupConf
- the dedup Configuration
-
-
- targetPath
- the target path
-
-
- sparkDriverMemory
- memory for driver process
-
-
- sparkExecutorMemory
- memory for individual executor
-
-
-
-
-
- Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
-
-
-
-
-
-
-
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- yarn-cluster
- cluster
- Create Similarity Relations
- eu.dnetlib.dedup.SparkCreateSimRels
- dhp-dedup-scholexplorer-${projectVersion}.jar
-
- --executor-memory ${sparkExecutorMemory}
- --driver-memory=${sparkDriverMemory}
- --executor-cores=${sparkExecutorCores}
- ${sparkExtraOPT}
-
- -mtyarn-cluster
- --sourcePath${sourcePath}
- --targetPath${targetPath}
- --entity${entity}
- --dedupConf${dedupConf}
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- yarn-cluster
- cluster
- Create Connected Components
- eu.dnetlib.dedup.SparkCreateConnectedComponent
- dhp-dedup-scholexplorer-${projectVersion}.jar
-
- --executor-memory ${sparkExecutorMemory}
- --driver-memory=${sparkDriverMemory}
- --executor-cores=${sparkExecutorCores}
- ${sparkExtraOPT}
-
- -mtyarn-cluster
- --sourcePath${sourcePath}
- --targetPath${targetPath}
- --entity${entity}
- --dedupConf${dedupConf}
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- yarn-cluster
- cluster
- Create Dedup Record
- eu.dnetlib.dedup.SparkCreateDedupRecord
- dhp-dedup-scholexplorer-${projectVersion}.jar
-
- --executor-memory ${sparkExecutorMemory}
- --driver-memory=${sparkDriverMemory}
- --executor-cores=${sparkExecutorCores}
- ${sparkExtraOPT}
-
- -mtyarn-cluster
- --sourcePath${sourcePath}
- --dedupPath${targetPath}
- --entity${entity}
- --dedupConf${dedupConf}
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- yarn-cluster
- cluster
- Propagate Dedup Relations
- eu.dnetlib.dedup.sx.SparkPropagateRelationsJob
- dhp-dedup-scholexplorer-${projectVersion}.jar
-
- --executor-memory ${sparkExecutorMemory}
- --driver-memory=${sparkDriverMemory}
- --executor-cores=${sparkExecutorCores}
- ${sparkExtraOPT}
-
- -mtyarn-cluster
- --mergeRelPath${targetPath}/${entity}/mergeRel
- --relationPath${sourcePath}/relation
- --targetRelPath${targetPath}/${entity}/updated_relation
-
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- yarn-cluster
- cluster
- Update ${entity} and add DedupRecord
- eu.dnetlib.dedup.sx.SparkUpdateEntityWithDedupInfo
- dhp-dedup-scholexplorer-${projectVersion}.jar
-
- --executor-memory ${sparkExecutorMemory}
- --driver-memory=${sparkDriverMemory}
- --executor-cores=${sparkExecutorCores}
- ${sparkExtraOPT}
-
- -mtyarn-cluster
- --entityPath${sourcePath}/${entity}
- --mergeRelPath${targetPath}/${entity}/mergeRel
- --entity${entity}
- --dedupRecordPath${targetPath}/${entity}/dedup_records
- --targetPath${targetPath}/${entity}/updated_record
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
\ No newline at end of file
diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/test/resources/eu/dnetlib/dedup/sx/conf/pub_scholix.conf.json b/dhp-workflows/dhp-dedup-scholexplorer/src/test/resources/eu/dnetlib/dedup/sx/conf/pub_scholix.conf.json
deleted file mode 100644
index d91419853..000000000
--- a/dhp-workflows/dhp-dedup-scholexplorer/src/test/resources/eu/dnetlib/dedup/sx/conf/pub_scholix.conf.json
+++ /dev/null
@@ -1,378 +0,0 @@
-{
- "wf": {
- "threshold": "0.99",
- "dedupRun": "001",
- "entityType": "result",
- "subEntityType": "resulttype",
- "subEntityValue": "publication",
- "orderField": "title",
- "queueMaxSize": "2000",
- "groupMaxSize": "100",
- "maxChildren": "100",
- "slidingWindowSize": "200",
- "rootBuilder": [
- ],
- "includeChildren": "true",
- "maxIterations": 20,
- "idPath": "$.id"
- },
- "pace": {
- "clustering": [
- {
- "name": "ngrampairs",
- "fields": [
- "title"
- ],
- "params": {
- "max": "1",
- "ngramLen": "3"
- }
- },
- {
- "name": "suffixprefix",
- "fields": [
- "title"
- ],
- "params": {
- "max": "1",
- "len": "3"
- }
- }
- ],
- "decisionTree": {
- "start": {
- "fields": [
- {
- "field": "pid",
- "comparator": "jsonListMatch",
- "weight": 1.0,
- "countIfUndefined": "false",
- "params": {
- "jpath_value": "$.value",
- "jpath_classid": "$.qualifier.classid"
- }
- }
- ],
- "threshold": 0.5,
- "aggregation": "AVG",
- "positive": "MATCH",
- "negative": "layer2",
- "undefined": "layer2",
- "ignoreUndefined": "true"
- },
- "layer2": {
- "fields": [
- {
- "field": "title",
- "comparator": "titleVersionMatch",
- "weight": 1.0,
- "countIfUndefined": "false",
- "params": {}
- },
- {
- "field": "authors",
- "comparator": "sizeMatch",
- "weight": 1.0,
- "countIfUndefined": "false",
- "params": {}
- }
- ],
- "threshold": 1.0,
- "aggregation": "AND",
- "positive": "layer3",
- "negative": "NO_MATCH",
- "undefined": "layer3",
- "ignoreUndefined": "false"
- },
- "layer3": {
- "fields": [
- {
- "field": "title",
- "comparator": "levensteinTitle",
- "weight": 1.0,
- "countIfUndefined": "true",
- "params": {}
- }
- ],
- "threshold": 0.99,
- "aggregation": "AVG",
- "positive": "MATCH",
- "negative": "NO_MATCH",
- "undefined": "NO_MATCH",
- "ignoreUndefined": "true"
- }
- },
- "model": [
- {
- "name": "pid",
- "type": "JSON",
- "path": "$.pid",
- "overrideMatch": "true"
- },
- {
- "name": "title",
- "type": "String",
- "path": "$.title[*].value",
- "length": 250,
- "size": 5
- },
- {
- "name": "authors",
- "type": "List",
- "path": "$.author[*].fullname",
- "size": 200
- },
- {
- "name": "resulttype",
- "type": "String",
- "path": "$.resulttype.classid"
- }
- ],
- "blacklists": {
- "title": [
- "^Inside Front Cover$",
- "^CORR Insights$",
- "^Index des notions$",
- "^Department of Error.$",
- "^Untitled Item$",
- "^Department of Error$",
- "^Tome II : 1598 à 1605$",
- "^(à l’exception de roi, prince, royauté, pouvoir, image… qui sont omniprésents)$",
- "^Museen und Ausstellungsinstitute in Nürnberg$",
- "^Text/Conference Paper$",
- "^Table des illustrations$",
- "^An Intimate Insight on Psychopathy and a Novel Hermeneutic Psychological Science$",
- "^Index des noms$",
- "^Reply by Authors.$",
- "^Titelblatt - Inhalt$",
- "^Index des œuvres,$",
- "(?i)^Poster presentations$",
- "^THE ASSOCIATION AND THE GENERAL MEDICAL COUNCIL$",
- "^Problems with perinatal pathology\\.?$",
- "(?i)^Cases? of Puerperal Convulsions$",
- "(?i)^Operative Gyna?ecology$",
- "(?i)^Mind the gap\\!?\\:?$",
- "^Chronic fatigue syndrome\\.?$",
- "^Cartas? ao editor Letters? to the Editor$",
- "^Note from the Editor$",
- "^Anesthesia Abstract$",
- "^Annual report$",
- "(?i)^“?THE RADICAL PREVENTION OF VENEREAL DISEASE\\.?”?$",
- "(?i)^Graph and Table of Infectious Diseases?$",
- "^Presentation$",
- "(?i)^Reviews and Information on Publications$",
- "(?i)^PUBLIC HEALTH SERVICES?$",
- "(?i)^COMBINED TEXT-?BOOK OF OBSTETRICS AND GYN(Æ|ae)COLOGY$",
- "(?i)^Adrese autora$",
- "(?i)^Systematic Part .*\\. Catalogus Fossilium Austriae, Band 2: Echinoidea neogenica$",
- "(?i)^Acknowledgement to Referees$",
- "(?i)^Behçet's disease\\.?$",
- "(?i)^Isolation and identification of restriction endonuclease.*$",
- "(?i)^CEREBROVASCULAR DISEASES?.?$",
- "(?i)^Screening for abdominal aortic aneurysms?\\.?$",
- "^Event management$",
- "(?i)^Breakfast and Crohn's disease.*\\.?$",
- "^Cálculo de concentraciones en disoluciones acuosas. Ejercicio interactivo\\..*\\.$",
- "(?i)^Genetic and functional analyses of SHANK2 mutations suggest a multiple hit model of Autism spectrum disorders?\\.?$",
- "^Gushi hakubutsugaku$",
- "^Starobosanski nadpisi u Bosni i Hercegovini \\(.*\\)$",
- "^Intestinal spirocha?etosis$",
- "^Treatment of Rodent Ulcer$",
- "(?i)^\\W*Cloud Computing\\W*$",
- "^Compendio mathematico : en que se contienen todas las materias mas principales de las Ciencias que tratan de la cantidad$",
- "^Free Communications, Poster Presentations: Session [A-F]$",
- "^“The Historical Aspects? of Quackery\\.?”$",
- "^A designated centre for people with disabilities operated by St John of God Community Services (Limited|Ltd), Louth$",
- "^P(er|re)-Mile Premiums for Auto Insurance\\.?$",
- "(?i)^Case Report$",
- "^Boletín Informativo$",
- "(?i)^Glioblastoma Multiforme$",
- "(?i)^Nuevos táxones animales descritos en la península Ibérica y Macaronesia desde 1994 \\(.*\\)$",
- "^Zaměstnanecké výhody$",
- "(?i)^The Economics of Terrorism and Counter-Terrorism: A Survey \\(Part .*\\)$",
- "(?i)^Carotid body tumours?\\.?$",
- "(?i)^\\[Españoles en Francia : La condición Emigrante.*\\]$",
- "^Avant-propos$",
- "(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Head(s)? and Capital(s)?$",
- "(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Bases?$",
- "(?i)^PUBLIC HEALTH VERSUS THE STATE$",
- "^Viñetas de Cortázar$",
- "(?i)^Search for heavy neutrinos and W(\\[|_|\\(|_\\{|-)?R(\\]|\\)|\\})? bosons with right-handed couplings in a left-right symmetric model in pp collisions at.*TeV(\\.)?$",
- "(?i)^Measurement of the pseudorapidity and centrality dependence of the transverse energy density in Pb(-?)Pb collisions at.*tev(\\.?)$",
- "(?i)^Search for resonances decaying into top-quark pairs using fully hadronic decays in pp collisions with ATLAS at.*TeV$",
- "(?i)^Search for neutral minimal supersymmetric standard model Higgs bosons decaying to tau pairs in pp collisions at.*tev$",
- "(?i)^Relatório de Estágio (de|em) Angiologia e Cirurgia Vascular$",
- "^Aus der AGMB$",
- "^Znanstveno-stručni prilozi$",
- "(?i)^Zhodnocení finanční situace podniku a návrhy na zlepšení$",
- "(?i)^Evaluation of the Financial Situation in the Firm and Proposals to its Improvement$",
- "(?i)^Hodnocení finanční situace podniku a návrhy na její zlepšení$",
- "^Finanční analýza podniku$",
- "^Financial analysis( of business)?$",
- "(?i)^Textbook of Gyn(a)?(Æ)?(e)?cology$",
- "^Jikken nihon shūshinsho$",
- "(?i)^CORONER('|s)(s|') INQUESTS$",
- "(?i)^(Μελέτη παραγόντων )?risk management( για ανάπτυξη και εφαρμογή ενός πληροφοριακού συστήματος| και ανάπτυξη συστήματος)?$",
- "(?i)^Consultants' contract(s)?$",
- "(?i)^Upute autorima$",
- "(?i)^Bijdrage tot de Kennis van den Godsdienst der Dajaks van Lan(d|f)ak en Tajan$",
- "^Joshi shin kokubun$",
- "^Kōtō shōgaku dokuhon nōson'yō$",
- "^Jinjō shōgaku shōka$",
- "^Shōgaku shūjichō$",
- "^Nihon joshi dokuhon$",
- "^Joshi shin dokuhon$",
- "^Chūtō kanbun dokuhon$",
- "^Wabun dokuhon$",
- "(?i)^(Analysis of economy selected village or town|Rozbor hospodaření vybrané obce či města)$",
- "(?i)^cardiac rehabilitation$",
- "(?i)^Analytical summary$",
- "^Thesaurus resolutionum Sacrae Congregationis Concilii$",
- "(?i)^Sumario analítico(\\s{1})?(Analitic summary)?$",
- "^Prikazi i osvrti$",
- "^Rodinný dům s provozovnou$",
- "^Family house with an establishment$",
- "^Shinsei chūtō shin kokugun$",
- "^Pulmonary alveolar proteinosis(\\.?)$",
- "^Shinshū kanbun$",
- "^Viñeta(s?) de Rodríguez$",
- "(?i)^RUBRIKA UREDNIKA$",
- "^A Matching Model of the Academic Publication Market$",
- "^Yōgaku kōyō$",
- "^Internetový marketing$",
- "^Internet marketing$",
- "^Chūtō kokugo dokuhon$",
- "^Kokugo dokuhon$",
- "^Antibiotic Cover for Dental Extraction(s?)$",
- "^Strategie podniku$",
- "^Strategy of an Enterprise$",
- "(?i)^respiratory disease(s?)(\\.?)$",
- "^Award(s?) for Gallantry in Civil Defence$",
- "^Podniková kultura$",
- "^Corporate Culture$",
- "^Severe hyponatraemia in hospital inpatient(s?)(\\.?)$",
- "^Pracovní motivace$",
- "^Work Motivation$",
- "^Kaitei kōtō jogaku dokuhon$",
- "^Konsolidovaná účetní závěrka$",
- "^Consolidated Financial Statements$",
- "(?i)^intracranial tumour(s?)$",
- "^Climate Change Mitigation Options and Directed Technical Change: A Decentralized Equilibrium Analysis$",
- "^\\[CERVECERIAS MAHOU(\\.|\\:) INTERIOR\\] \\[Material gráfico\\]$",
- "^Housing Market Dynamics(\\:|\\.) On the Contribution of Income Shocks and Credit Constraint(s?)$",
- "^\\[Funciones auxiliares de la música en Radio París,.*\\]$",
- "^Úroveň motivačního procesu jako způsobu vedení lidí$",
- "^The level of motivation process as a leadership$",
- "^Pay-beds in N(\\.?)H(\\.?)S(\\.?) Hospitals$",
- "(?i)^news and events$",
- "(?i)^NOVOSTI I DOGAĐAJI$",
- "^Sansū no gakushū$",
- "^Posouzení informačního systému firmy a návrh změn$",
- "^Information System Assessment and Proposal for ICT Modification$",
- "^Stresové zatížení pracovníků ve vybrané profesi$",
- "^Stress load in a specific job$",
- "^Sunday: Poster Sessions, Pt.*$",
- "^Monday: Poster Sessions, Pt.*$",
- "^Wednesday: Poster Sessions, Pt.*",
- "^Tuesday: Poster Sessions, Pt.*$",
- "^Analýza reklamy$",
- "^Analysis of advertising$",
- "^Shōgaku shūshinsho$",
- "^Shōgaku sansū$",
- "^Shintei joshi kokubun$",
- "^Taishō joshi kokubun dokuhon$",
- "^Joshi kokubun$",
- "^Účetní uzávěrka a účetní závěrka v ČR$",
- "(?i)^The \"?Causes\"? of Cancer$",
- "^Normas para la publicación de artículos$",
- "^Editor('|s)(s|') [Rr]eply$",
- "^Editor(’|s)(s|’) letter$",
- "^Redaktoriaus žodis$",
- "^DISCUSSION ON THE PRECEDING PAPER$",
- "^Kōtō shōgaku shūshinsho jidōyō$",
- "^Shōgaku nihon rekishi$",
- "^(Theory of the flow of action currents in isolated myelinated nerve fibers).*$",
- "^Préface$",
- "^Occupational [Hh]ealth [Ss]ervices.$",
- "^In Memoriam Professor Toshiyuki TAKESHIMA$",
- "^Účetní závěrka ve vybraném podniku.*$",
- "^Financial statements in selected company$",
- "^Abdominal [Aa]ortic [Aa]neurysms.*$",
- "^Pseudomyxoma peritonei$",
- "^Kazalo autora$",
- "(?i)^uvodna riječ$",
- "^Motivace jako způsob vedení lidí$",
- "^Motivation as a leadership$",
- "^Polyfunkční dům$",
- "^Multi\\-funkcional building$",
- "^Podnikatelský plán$",
- "(?i)^Podnikatelský záměr$",
- "(?i)^Business Plan$",
- "^Oceňování nemovitostí$",
- "^Marketingová komunikace$",
- "^Marketing communication$",
- "^Sumario Analítico$",
- "^Riječ uredništva$",
- "^Savjetovanja i priredbe$",
- "^Índice$",
- "^(Starobosanski nadpisi).*$",
- "^Vzdělávání pracovníků v organizaci$",
- "^Staff training in organization$",
- "^(Life Histories of North American Geometridae).*$",
- "^Strategická analýza podniku$",
- "^Strategic Analysis of an Enterprise$",
- "^Sadržaj$",
- "^Upute suradnicima$",
- "^Rodinný dům$",
- "(?i)^Fami(l)?ly house$",
- "^Upute autorima$",
- "^Strategic Analysis$",
- "^Finanční analýza vybraného podniku$",
- "^Finanční analýza$",
- "^Riječ urednika$",
- "(?i)^Content(s?)$",
- "(?i)^Inhalt$",
- "^Jinjō shōgaku shūshinsho jidōyō$",
- "(?i)^Index$",
- "^Chūgaku kokubun kyōkasho$",
- "^Retrato de una mujer$",
- "^Retrato de un hombre$",
- "^Kōtō shōgaku dokuhon$",
- "^Shotōka kokugo$",
- "^Shōgaku dokuhon$",
- "^Jinjō shōgaku kokugo dokuhon$",
- "^Shinsei kokugo dokuhon$",
- "^Teikoku dokuhon$",
- "^Instructions to Authors$",
- "^KİTAP TAHLİLİ$",
- "^PRZEGLĄD PIŚMIENNICTWA$",
- "(?i)^Presentación$",
- "^İçindekiler$",
- "(?i)^Tabl?e of contents$",
- "^(CODICE DEL BEATO DE LOS REYES FERNANDO I Y SANCHA).*$",
- "^(\\[MADRID\\. BIBL\\. NAC\\. N.*KING FERDINAND I.*FROM SAN ISIDORO DE LEON\\. FACUNDUS SCRIPSIT DATED.*\\]).*",
- "^Editorial( Board)?$",
- "(?i)^Editorial \\(English\\)$",
- "^Editörden$",
- "^(Corpus Oral Dialectal \\(COD\\)\\.).*$",
- "^(Kiri Karl Morgensternile).*$",
- "^(\\[Eksliibris Aleksandr).*\\]$",
- "^(\\[Eksliibris Aleksandr).*$",
- "^(Eksliibris Aleksandr).*$",
- "^(Kiri A\\. de Vignolles).*$",
- "^(2 kirja Karl Morgensternile).*$",
- "^(Pirita kloostri idaosa arheoloogilised).*$",
- "^(Kiri tundmatule).*$",
- "^(Kiri Jenaer Allgemeine Literaturzeitung toimetusele).*$",
- "^(Eksliibris Nikolai Birukovile).*$",
- "^(Eksliibris Nikolai Issakovile).*$",
- "^(WHP Cruise Summary Information of section).*$",
- "^(Measurement of the top quark\\-pair production cross section with ATLAS in pp collisions at).*$",
- "^(Measurement of the spin\\-dependent structure function).*",
- "(?i)^.*authors['’′]? reply\\.?$",
- "(?i)^.*authors['’′]? response\\.?$"
- ]
- },
- "synonyms": {}
- }
-}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml
deleted file mode 100644
index 188e0debc..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml
+++ /dev/null
@@ -1,79 +0,0 @@
-
-
-
- dhp-workflows
- eu.dnetlib.dhp
- 1.2.4-SNAPSHOT
-
- 4.0.0
-
- dhp-graph-provision-scholexplorer
-
-
-
-
- net.alchim31.maven
- scala-maven-plugin
- 4.0.1
-
-
- scala-compile-first
- initialize
-
- add-source
- compile
-
-
-
- scala-test-compile
- process-test-resources
-
- testCompile
-
-
-
-
- ${scala.version}
-
-
-
-
-
-
-
-
- org.apache.spark
- spark-core_2.11
-
-
-
- org.apache.spark
- spark-sql_2.11
-
-
-
- eu.dnetlib.dhp
- dhp-common
- ${project.version}
-
-
-
- org.apache.httpcomponents
- httpmime
-
-
-
- org.elasticsearch
- elasticsearch-hadoop
-
-
-
-
- org.apache.httpcomponents
- httpclient
-
-
-
-
-
-
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala
deleted file mode 100644
index b71b7f054..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala
+++ /dev/null
@@ -1,425 +0,0 @@
-package eu.dnetlib.dhp.export
-
-import com.fasterxml.jackson.databind.ObjectMapper
-
-import java.time.LocalDateTime
-import java.time.format.DateTimeFormatter
-import eu.dnetlib.dhp.common.PacePerson
-import eu.dnetlib.dhp.schema.action.AtomicAction
-import eu.dnetlib.dhp.schema.common.ModelConstants
-import eu.dnetlib.dhp.schema.oaf.{Author, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty}
-import eu.dnetlib.dhp.utils.DHPUtils
-import org.apache.commons.lang3.StringUtils
-
-
-import scala.collection.JavaConverters._
-
-
-case class DLIExternalReference(id: String, url: String, sitename: String, label: String, pid: String, classId: String) {}
-
-object DLIToOAF {
-
-
- val collectedFromMap: Map[String, KeyValue] = Map(
- "dli_________::r3d100010527" -> generateKeyValue("10|re3data_____::c2a591f440598b63d854556beaf01591", "European Nucleotide Archive"),
- "dli_________::r3d100010255" -> generateKeyValue("10|re3data_____::480d275ed6f9666ee76d6a1215eabf26", "Inter-university Consortium for Political and Social Research"),
- "dli_________::r3d100011868" -> generateKeyValue("10|re3data_____::db814dc656a911b556dba42a331cebe9", "Mendeley Data"),
- "dli_________::elsevier" -> generateKeyValue("10|openaire____::8f87e10869299a5fe80b315695296b88", "Elsevier"),
- "dli_________::openaire" -> generateKeyValue("10|infrastruct_::f66f1bd369679b5b077dcdf006089556", "OpenAIRE"),
- "dli_________::thomsonreuters" -> generateKeyValue("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "Crossref"),
- "dli_________::r3d100010216" -> generateKeyValue("10|re3data_____::0fd79429de04343dbbec705d9b5f429f", "4TU.Centre for Research Data"),
- "dli_________::r3d100010134" -> generateKeyValue("10|re3data_____::9633d1e8c4309c833c2c442abeb0cfeb", "PANGAEA"),
- "dli_________::ieee" -> generateKeyValue("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "Crossref"),
- "dli_________::r3d100010197" -> generateKeyValue("10|re3data_____::9fd1d79973f7fda60cbe1d82e3819a68", "The Cambridge Structural Database"),
- "dli_________::nature" -> generateKeyValue("10|openaire____::6e380d9cf51138baec8480f5a0ce3a2e", "Springer Nature"),
- "dli_________::datacite" -> generateKeyValue("10|openaire____::9e3be59865b2c1c335d32dae2fe7b254", "Datacite"),
- "dli_________::r3d100010578" -> generateKeyValue("10|re3data_____::c4d751f29a7568011a4c80136b30b444", "IEDA"),
- "dli_________::r3d100010464" -> generateKeyValue("10|re3data_____::23e2a81591099828f6b83a1c83150666", "Research Data Australia"),
- "dli_________::r3d100010327" -> generateKeyValue("10|re3data_____::a644620b81135243dc9acc15d2362246", "Worldwide Protein Data Bank"),
- "dli_________::pubmed" -> generateKeyValue("10|opendoar____::eda80a3d5b344bc40f3bc04f65b7a357", "PubMed Central"),
- "dli_________::europe_pmc__" -> generateKeyValue("10|opendoar____::8b6dd7db9af49e67306feb59a8bdc52c", "Europe PubMed Central"),
- "dli_________::crossref" -> generateKeyValue("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "Crossref")
- )
-
-
- val relationTypeMapping: Map[String, (String, String)] = Map(
- "IsReferencedBy" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP),
- "References" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP),
- "IsRelatedTo" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP),
- "IsSupplementedBy" -> (ModelConstants.IS_SUPPLEMENTED_BY, ModelConstants.SUPPLEMENT),
- "Documents" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP),
- "Cites" -> (ModelConstants.CITES, ModelConstants.CITATION),
- "Unknown" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP),
- "IsSourceOf" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP),
- "IsCitedBy" -> (ModelConstants.IS_CITED_BY, ModelConstants.CITATION),
- "Reviews" -> (ModelConstants.REVIEWS, ModelConstants.REVIEW),
- "Describes" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP),
- "HasAssociationWith" -> (ModelConstants.IS_RELATED_TO, ModelConstants.RELATIONSHIP)
- )
-
- val expectecdPidType = List("uniprot", "ena", "chembl", "ncbi-n", "ncbi-p", "genbank", "pdb", "url")
-
-
- val filteredURL = List(
- "www.ebi.ac.uk",
- "www.uniprot.org",
- "f1000.com",
- "en.wikipedia.org",
- "flybase.org",
- "www.yeastgenome.org",
- "research.bioinformatics.udel.edu",
- "cancer.sanger.ac.uk",
- "www.iedb.org",
- "www.crd.york.ac.uk",
- "www.wormbase.org",
- "web.expasy.org",
- "www.hal.inserm.fr",
- "sabiork.h-its.org",
- "zfin.org",
- "www.pombase.org",
- "www.guidetopharmacology.org",
- "reactome.org"
- )
-
-
- val rel_inverse: Map[String, String] = Map(
- ModelConstants.IS_RELATED_TO -> ModelConstants.IS_RELATED_TO,
- ModelConstants.IS_SUPPLEMENTED_BY -> ModelConstants.IS_SUPPLEMENT_TO,
- ModelConstants.CITES -> ModelConstants.IS_CITED_BY,
- ModelConstants.IS_CITED_BY -> ModelConstants.CITES,
- ModelConstants.REVIEWS -> ModelConstants.IS_REVIEWED_BY
- )
-
-
- val PidTypeMap: Map[String, String] = Map(
- "pbmid" -> "pmid",
- "pmcid" -> "pmc",
- "pmid" -> "pmid",
- "pubmedid" -> "pmid",
- "DOI" -> "doi",
- "doi" -> "doi"
- )
-
-
- def fixInstance(r:Publication) :Publication = {
- val collectedFrom = r.getCollectedfrom.asScala.head
- r.getInstance().asScala.foreach(i => i.setCollectedfrom(collectedFrom))
- r
- }
-
-
- def fixInstanceDataset(r:Dataset) :Dataset = {
- val collectedFrom = r.getCollectedfrom.asScala.head
- r.getInstance().asScala.foreach(i => i.setCollectedfrom(collectedFrom))
- r
- }
-
-
- def toActionSet(item: Oaf): (String, String) = {
- val mapper = new ObjectMapper()
-
- item match {
- case dataset: Dataset =>
- val a: AtomicAction[Dataset] = new AtomicAction[Dataset]
- a.setClazz(classOf[Dataset])
- a.setPayload(dataset)
- (dataset.getClass.getCanonicalName, mapper.writeValueAsString(a))
- case publication: Publication =>
- val a: AtomicAction[Publication] = new AtomicAction[Publication]
- a.setClazz(classOf[Publication])
- a.setPayload(publication)
- (publication.getClass.getCanonicalName, mapper.writeValueAsString(a))
- case relation: Relation =>
- val a: AtomicAction[Relation] = new AtomicAction[Relation]
- a.setClazz(classOf[Relation])
- a.setPayload(relation)
- (relation.getClass.getCanonicalName, mapper.writeValueAsString(a))
- case _ =>
- null
- }
- }
-
- def convertClinicalTrial(dataset: DLIDataset): (String, String) = {
- val currentId = generateId(dataset.getId)
- val pids = dataset.getPid.asScala.filter(p => "clinicaltrials.gov".equalsIgnoreCase(p.getQualifier.getClassname)).map(p => s"50|r3111dacbab5::${DHPUtils.md5(p.getValue.toLowerCase())}")
- if (pids.isEmpty)
- null
- else
- (currentId, pids.head)
- }
-
-
- def insertExternalRefs(publication: Publication, externalReferences: List[DLIExternalReference]): Publication = {
-
- val eRefs = externalReferences.map(e => {
- val result = new ExternalReference()
- result.setSitename(e.sitename)
- result.setLabel(e.label)
- result.setUrl(e.url)
- result.setRefidentifier(e.pid)
- result.setDataInfo(generateDataInfo())
- result.setQualifier(createQualifier(e.classId, ModelConstants.DNET_EXTERNAL_REFERENCE_TYPE))
- result
- })
- publication.setExternalReference(eRefs.asJava)
- publication
-
- }
-
- def filterPid(p: StructuredProperty): Boolean = {
- if (expectecdPidType.contains(p.getQualifier.getClassname) && p.getQualifier.getClassname.equalsIgnoreCase("url"))
- if (filteredURL.exists(u => p.getValue.contains(u)))
- return true
- else
- return false
- expectecdPidType.contains(p.getQualifier.getClassname)
- }
-
-
- def extractTitle(titles: java.util.List[StructuredProperty]): String = {
-
- if (titles == null)
- return null
-
- val label = titles.asScala.map(p => p.getValue).find(p => p.nonEmpty)
- label.orNull
- }
-
- def convertDLIDatasetToExternalReference(dataset: DLIDataset): DLIExternalReference = {
- val pids = dataset.getPid.asScala.filter(filterPid)
-
- if (pids == null || pids.isEmpty)
- return null
-
- val pid: StructuredProperty = pids.head
-
-
- pid.getQualifier.getClassname match {
- case "uniprot" => DLIExternalReference(generateId(dataset.getId), s"https://www.uniprot.org/uniprot/${pid.getValue}", "UniProt", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
- case "ena" =>
- if (pid.getValue != null && pid.getValue.nonEmpty && pid.getValue.length > 7)
- DLIExternalReference(generateId(dataset.getId), s"https://www.ebi.ac.uk/ena/data/view/${pid.getValue.substring(0, 8)}", "European Nucleotide Archive", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
- else
- null
- case "chembl" => DLIExternalReference(generateId(dataset.getId), s"https://www.ebi.ac.uk/chembl/compound_report_card/${pid.getValue}", "ChEMBL", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
- case "ncbi-n" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "Nucleotide Database", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
- case "ncbi-p" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "Nucleotide Database", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
- case "genbank" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "GenBank", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
- case "pdb" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "Protein Data Bank", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
- case "url" => DLIExternalReference(generateId(dataset.getId), pid.getValue, "", extractTitle(dataset.getTitle), pid.getValue, "url")
-
- }
-
-
- }
-
-
- def convertDLIPublicationToOAF(inputPublication: DLIPublication): Publication = {
- val result = new Publication
- val cleanedPids = inputPublication.getPid.asScala.filter(p => PidTypeMap.contains(p.getQualifier.getClassid))
- .map(p => {
- p.setQualifier(createQualifier(PidTypeMap(p.getQualifier.getClassid), p.getQualifier.getSchemeid))
- p
- })
- if (cleanedPids.isEmpty)
- return null
- result.setId(generateId(inputPublication.getId))
- result.setDataInfo(generateDataInfo(invisible = true))
- if (inputPublication.getCollectedfrom == null || inputPublication.getCollectedfrom.size() == 0 || (inputPublication.getCollectedfrom.size() == 1 && inputPublication.getCollectedfrom.get(0) == null))
- return null
- result.setCollectedfrom(inputPublication.getCollectedfrom.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).filter(p => p != null).asJava)
- if(result.getCollectedfrom.isEmpty)
- return null
- result.setPid(cleanedPids.asJava)
- result.setDateofcollection(inputPublication.getDateofcollection)
- result.setOriginalId(inputPublication.getPid.asScala.map(p => p.getValue).asJava)
- result.setDateoftransformation(LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'")))
- if (inputPublication.getAuthor == null || inputPublication.getAuthor.isEmpty)
- return null
- result.setAuthor(inputPublication.getAuthor.asScala.map(convertAuthor).asJava)
- result.setResulttype(createQualifier(inputPublication.getResulttype.getClassid, inputPublication.getResulttype.getClassname, ModelConstants.DNET_RESULT_TYPOLOGIES, ModelConstants.DNET_RESULT_TYPOLOGIES))
-
- if (inputPublication.getSubject != null)
- result.setSubject(inputPublication.getSubject.asScala.map(convertSubject).asJava)
-
- if (inputPublication.getTitle == null || inputPublication.getTitle.isEmpty)
- return null
-
- result.setTitle(List(patchTitle(inputPublication.getTitle.get(0))).asJava)
-
- if (inputPublication.getRelevantdate == null || inputPublication.getRelevantdate.size() == 0)
- return null
-
- result.setRelevantdate(inputPublication.getRelevantdate.asScala.map(patchRelevantDate).asJava)
-
-
- result.setDescription(inputPublication.getDescription)
-
- result.setDateofacceptance(asField(inputPublication.getRelevantdate.get(0).getValue))
- result.setPublisher(inputPublication.getPublisher)
- result.setSource(inputPublication.getSource)
- result.setBestaccessright(createAccessRight(ModelConstants.UNKNOWN, ModelConstants.NOT_AVAILABLE, ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES))
-
- val dois = result.getPid.asScala.filter(p => "doi".equalsIgnoreCase(p.getQualifier.getClassname)).map(p => p.getValue)
- if (dois.isEmpty)
- return null
-
-
- val i: Instance = createInstance(s"https://dx.doi.org/${dois.head}", firstInstanceOrNull(inputPublication.getInstance()), result.getDateofacceptance)
-
- if (i != null)
- result.setInstance(List(i).asJava)
-
- result
- }
-
-
- def convertDLIRelation(r: Relation): Relation = {
-
- val rt = r.getRelType
- if (!relationTypeMapping.contains(rt))
- return null
- r.setRelType(ModelConstants.RESULT_RESULT)
- r.setRelClass(relationTypeMapping(rt)._1)
- r.setSubRelType(relationTypeMapping(rt)._2)
- r.setSource(generateId(r.getSource))
- r.setTarget(generateId(r.getTarget))
- r
- }
-
-
- def convertDLIDatasetTOOAF(d: DLIDataset): Dataset = {
-
- if (d.getCollectedfrom == null || d.getCollectedfrom.size() == 0 || (d.getCollectedfrom.size() == 1 && d.getCollectedfrom.get(0) == null))
- return null
- val result: Dataset = new Dataset
- result.setId(generateId(d.getId))
- result.setDataInfo(generateDataInfo())
- result.setCollectedfrom(d.getCollectedfrom.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).filter(p => p != null).asJava)
- if(result.getCollectedfrom.isEmpty)
- return null
-
-
- result.setPid(d.getPid)
-
- val fpids = result.getPid.asScala.filter(p => "doi".equalsIgnoreCase(p.getQualifier.getClassname) ||
- "pdb".equalsIgnoreCase(p.getQualifier.getClassname)
- ).map(p => p.getValue)
-
- if (fpids == null || fpids.isEmpty)
- return null
-
-
- result.setDateofcollection(d.getDateofcollection)
- result.setOriginalId(d.getPid.asScala.map(d => d.getValue).asJava)
- result.setDateoftransformation(LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'")))
- if (d.getAuthor == null || d.getAuthor.isEmpty)
- return null
- result.setAuthor(d.getAuthor.asScala.map(convertAuthor).asJava)
- result.setResulttype(createQualifier(d.getResulttype.getClassid, d.getResulttype.getClassname, ModelConstants.DNET_RESULT_TYPOLOGIES, ModelConstants.DNET_RESULT_TYPOLOGIES))
-
- if (d.getSubject != null)
- result.setSubject(d.getSubject.asScala.map(convertSubject).asJava)
-
- if (d.getTitle == null || d.getTitle.isEmpty)
- return null
-
- result.setTitle(List(patchTitle(d.getTitle.get(0))).asJava)
-
- if (d.getRelevantdate == null || d.getRelevantdate.size() == 0)
- return null
-
- result.setRelevantdate(d.getRelevantdate.asScala.map(patchRelevantDate).asJava)
-
-
- result.setDescription(d.getDescription)
-
- result.setDateofacceptance(asField(d.getRelevantdate.get(0).getValue))
- result.setPublisher(d.getPublisher)
- result.setSource(d.getSource)
- result.setBestaccessright(createAccessRight(ModelConstants.UNKNOWN, ModelConstants.NOT_AVAILABLE, ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES))
-
-
- val instance_urls = if (fpids.head.length < 5) s"https://www.rcsb.org/structure/${fpids.head}" else s"https://dx.doi.org/${fpids.head}"
-
- val i: Instance = createInstance(instance_urls, firstInstanceOrNull(d.getInstance()), result.getDateofacceptance, true)
-
- // Ticket #6281 added pid to Instance
- i.setPid(result.getPid)
- if (i != null)
- result.setInstance(List(i).asJava)
-
- result
- }
-
-
- def firstInstanceOrNull(instances: java.util.List[Instance]): Instance = {
-
- if (instances == null || instances.size() == 0)
- return null
- instances.get(0)
-
- }
-
-
- def createInstance(url: String, originalInstance: Instance, doa: Field[String], dataset: Boolean = false): Instance = {
-
- val i = new Instance
- i.setUrl(List(url).asJava)
- if (dataset)
- i.setInstancetype(createQualifier("0021", "Dataset", ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
- else
- i.setInstancetype(createQualifier("0000", "Unknown", ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
- if (originalInstance != null && originalInstance.getHostedby != null)
- i.setHostedby(originalInstance.getHostedby)
-
- i.setAccessright(createAccessRight(ModelConstants.UNKNOWN, ModelConstants.NOT_AVAILABLE, ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES))
- i.setDateofacceptance(doa)
-
- i
-
-
- }
-
-
- def patchRelevantDate(d: StructuredProperty): StructuredProperty = {
- d.setQualifier(createQualifier(ModelConstants.UNKNOWN, ModelConstants.DNET_DATACITE_DATE))
- d
-
- }
-
- def patchTitle(t: StructuredProperty): StructuredProperty = {
- t.setQualifier(ModelConstants.MAIN_TITLE_QUALIFIER)
- t
- }
-
-
- def convertSubject(s: StructuredProperty): StructuredProperty = {
- s.setQualifier(createQualifier("keyword", ModelConstants.DNET_SUBJECT_TYPOLOGIES))
- s
-
-
- }
-
-
- def convertAuthor(a: Author): Author = {
- if (a == null)
- return a
- val p = new PacePerson(a.getFullname, false)
- if (p.isAccurate) {
- a.setName(p.getNameString)
- a.setSurname(p.getSurnameString)
- }
- a
- }
-
-
- def generateId(id: String): String = {
- val md5 = if (id.contains("::")) StringUtils.substringAfter(id, "::") else StringUtils.substringAfter(id, "|")
- s"50|scholix_____::$md5"
- }
-
-
-
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/SparkExportContentForOpenAire.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/SparkExportContentForOpenAire.scala
deleted file mode 100644
index 3f632af22..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/SparkExportContentForOpenAire.scala
+++ /dev/null
@@ -1,175 +0,0 @@
-package eu.dnetlib.dhp.`export`
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser
-import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
-import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
-import org.apache.commons.io.IOUtils
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.io.compress.GzipCodec
-import org.apache.hadoop.mapred.SequenceFileOutputFormat
-import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
-import org.apache.spark.sql.functions._
-import org.apache.spark.sql.expressions.Window
-import org.apache.spark.SparkConf
-
-import scala.collection.mutable.ArrayBuffer
-
-object SparkExportContentForOpenAire {
-
- def main(args: Array[String]): Unit = {
- val conf: SparkConf = new SparkConf()
- val parser = new ArgumentApplicationParser(IOUtils.toString(SparkExportContentForOpenAire.getClass.getResourceAsStream("input_export_content_parameters.json")))
- parser.parseArgument(args)
- val spark: SparkSession =
- SparkSession
- .builder()
- .config(conf)
- .appName(SparkExportContentForOpenAire.getClass.getSimpleName)
- .master(parser.get("master")).getOrCreate()
-
-
- val workingPath = parser.get("workingDirPath")
-
- implicit val dliPubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
- implicit val dliDatEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
- implicit val pubEncoder: Encoder[Publication] = Encoders.bean(classOf[Publication])
- implicit val datEncoder: Encoder[OafDataset] = Encoders.bean(classOf[OafDataset])
- implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
-
- import spark.implicits._
-
- val dsRel = spark.read.load(s"$workingPath/relation_b").as[Relation]
- dsRel.filter(r => r.getDataInfo==null || r.getDataInfo.getDeletedbyinference ==false)
- .map(DLIToOAF.convertDLIRelation)
- .filter(r => r!= null)
- .write.mode(SaveMode.Overwrite).save(s"$workingPath/export/relationDS")
-
-
- val dsPubs = spark.read.load(s"$workingPath/publication").as[DLIPublication]
- dsPubs
- .filter(p=>p.getDataInfo.getDeletedbyinference == false)
- .map(DLIToOAF.convertDLIPublicationToOAF)
- .filter(p=>p!= null)
- .write.mode(SaveMode.Overwrite).save(s"$workingPath/export/publicationDS")
-
-
- val dsDataset = spark.read.load(s"$workingPath/dataset").as[DLIDataset]
- dsDataset
- .filter(p => p.getDataInfo.getDeletedbyinference == false)
- .map(DLIToOAF.convertDLIDatasetTOOAF).filter(p=>p!= null)
- .write.mode(SaveMode.Overwrite).save(s"$workingPath/export/datasetDS")
-
-
-
-
- val pubs:Dataset[Publication] = spark.read.load(s"$workingPath/export/publicationDS").as[Publication]
- val dats :Dataset[OafDataset] = spark.read.load(s"$workingPath/export/datasetDS").as[OafDataset]
- val relDS1 :Dataset[Relation] = spark.read.load(s"$workingPath/export/relationDS").as[Relation]
-
-
- val pub_id = pubs.select("id").distinct()
- val dat_id = dats.select("id").distinct()
-
-
- pub_id.joinWith(relDS1, pub_id("id").equalTo(relDS1("source"))).map(k => k._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/relationDS_f1")
-
- val relDS2= spark.read.load(s"$workingPath/export/relationDS_f1").as[Relation]
-
- relDS2.joinWith(dat_id, relDS2("target").equalTo(dats("id"))).map(k => k._1).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/relationDS_filtered")
-
-
- val r_source = relDS2.select(relDS2("source")).distinct()
- val r_target = relDS2.select(relDS2("target")).distinct()
-
-
- val w2 = Window.partitionBy("id").orderBy("lastupdatetimestamp")
-
- pubs.joinWith(r_source, pubs("id").equalTo(r_source("source")), "inner").map(k => k._1)
- .withColumn("row",row_number.over(w2)).where($"row" === 1).drop("row")
- .write.mode(SaveMode.Overwrite).save(s"$workingPath/export/publicationDS_filtered")
-
- dats.joinWith(r_target, dats("id").equalTo(r_target("target")), "inner").map(k => k._1)
- .withColumn("row",row_number.over(w2)).where($"row" === 1).drop("row")
- .write.mode(SaveMode.Overwrite).save(s"$workingPath/export/datasetAS")
-
-
- dsDataset.map(DLIToOAF.convertDLIDatasetToExternalReference).filter(p => p != null).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/externalReference")
-
- val pf = spark.read.load(s"$workingPath/export/publicationDS_filtered").select("id")
- val relDS3 = spark.read.load(s"$workingPath/export/relationDS").as[Relation]
- val relationTo = pf.joinWith(relDS3, pf("id").equalTo(relDS3("source")),"inner").map(t =>t._2)
-
- val extRef = spark.read.load(s"$workingPath/export/externalReference").as[DLIExternalReference]
-
- spark.createDataset(relationTo.joinWith(extRef, relationTo("target").equalTo(extRef("id")), "inner").map(d => {
- val r = d._1
- val ext = d._2
- (r.getSource, ext)
- }).rdd.groupByKey.map(f => {
- var dli_ext = ArrayBuffer[DLIExternalReference]()
- f._2.foreach(d => if (dli_ext.size < 100) dli_ext += d )
- (f._1, dli_ext)
- })).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/externalReference_grouped")
-
- val pubf :Dataset[Publication] = spark.read.load(s"$workingPath/export/publicationDS_filtered").as[Publication]
-
- val groupedERf:Dataset[(String, List[DLIExternalReference])]= spark.read.load(s"$workingPath/export/externalReference_grouped").as[(String, List[DLIExternalReference])]
-
- groupedERf.joinWith(pubf,pubf("id").equalTo(groupedERf("_1"))).map(t =>
- {
- val publication = t._2
- if (t._1 != null) {
- val eRefs = t._1._2
- DLIToOAF.insertExternalRefs(publication, eRefs)
-
- } else
- publication
- }
- ).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/publicationAS")
-
-
- dsDataset
- .map(DLIToOAF.convertClinicalTrial)
- .filter(p => p != null)
- .write.mode(SaveMode.Overwrite).save(s"$workingPath/export/clinicalTrials")
-
- val ct:Dataset[(String,String)] = spark.read.load(s"$workingPath/export/clinicalTrials").as[(String,String)]
-
- val relDS= spark.read.load(s"$workingPath/export/relationDS_f1").as[Relation]
-
- relDS.joinWith(ct, relDS("target").equalTo(ct("_1")), "inner")
- .map(k =>{
- val currentRel = k._1
- currentRel.setTarget(k._2._2)
- currentRel
- }).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/clinicalTrialsRels")
-
-
- val clRels:Dataset[Relation] = spark.read.load(s"$workingPath/export/clinicalTrialsRels").as[Relation]
- val rels:Dataset[Relation] = spark.read.load(s"$workingPath/export/relationDS_filtered").as[Relation]
-
- rels.union(clRels).flatMap(r => {
- val inverseRel = new Relation
- inverseRel.setSource(r.getTarget)
- inverseRel.setTarget(r.getSource)
- inverseRel.setDataInfo(r.getDataInfo)
- inverseRel.setCollectedfrom(r.getCollectedfrom)
- inverseRel.setRelType(r.getRelType)
- inverseRel.setSubRelType(r.getSubRelType)
- inverseRel.setRelClass(DLIToOAF.rel_inverse(r.getRelClass))
- List(r, inverseRel)
- }).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/relationAS")
-
-
-
- spark.read.load(s"$workingPath/export/publicationAS").as[Publication].map(DLIToOAF.fixInstance).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/publicationAS_fixed")
- spark.read.load(s"$workingPath/export/datasetAS").as[OafDataset].map(DLIToOAF.fixInstanceDataset).write.mode(SaveMode.Overwrite).save(s"$workingPath/export/datasetAS_fixed")
-
- val fRels:Dataset[(String,String)] = spark.read.load(s"$workingPath/export/relationAS").as[Relation].map(DLIToOAF.toActionSet)
- val fpubs:Dataset[(String,String)] = spark.read.load(s"$workingPath/export/publicationAS_fixed").as[Publication].map(DLIToOAF.toActionSet)
- val fdats:Dataset[(String,String)] = spark.read.load(s"$workingPath/export/datasetAS_fixed").as[OafDataset].map(DLIToOAF.toActionSet)
-
- fRels.union(fpubs).union(fdats).rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$workingPath/export/rawset", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec])
- }
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/MakeTar.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/MakeTar.java
deleted file mode 100644
index e19432f29..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/MakeTar.java
+++ /dev/null
@@ -1,112 +0,0 @@
-
-package eu.dnetlib.dhp.export.zenodo;
-
-import java.io.*;
-
-import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-import eu.dnetlib.dhp.common.MakeTarArchive;
-
-public class MakeTar implements Serializable {
-
- private static final Logger log = LoggerFactory.getLogger(MakeTar.class);
-
- public static void main(String[] args) throws Exception {
- String jsonConfiguration = IOUtils
- .toString(
- MakeTar.class
- .getResourceAsStream(
- "/eu/dnetlib/dhp/export/input_maketar_parameters.json"));
-
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
- parser.parseArgument(args);
-
- final String outputPath = parser.get("targetPath");
- log.info("hdfsPath: {}", outputPath);
-
- final String hdfsNameNode = parser.get("nameNode");
- log.info("nameNode: {}", hdfsNameNode);
-
- final String inputPath = parser.get("sourcePath");
- log.info("input path : {}", inputPath);
-
- Configuration conf = new Configuration();
- conf.set("fs.defaultFS", hdfsNameNode);
-
- FileSystem fileSystem = FileSystem.get(conf);
-
- MakeTarArchive.tarMaxSize(fileSystem, inputPath, outputPath, "scholix_dump", 25);
-
- }
-
-// public static void makeTArArchive(FileSystem fileSystem, String inputPath, String outputPath) throws IOException {
-//
-// RemoteIterator dir_iterator = fileSystem.listLocatedStatus(new Path(inputPath));
-//
-// while (dir_iterator.hasNext()) {
-// LocatedFileStatus fileStatus = dir_iterator.next();
-//
-// Path p = fileStatus.getPath();
-// String p_string = p.toString();
-// String entity = p_string.substring(p_string.lastIndexOf("/") + 1);
-//
-// write(fileSystem, p_string, outputPath + "/" + entity + ".tar", entity);
-// }
-//
-// }
-//
-// private static void write(FileSystem fileSystem, String inputPath, String outputPath, String dir_name)
-// throws IOException {
-//
-// Path hdfsWritePath = new Path(outputPath);
-// FSDataOutputStream fsDataOutputStream = null;
-// if (fileSystem.exists(hdfsWritePath)) {
-// fileSystem.delete(hdfsWritePath, true);
-//
-// }
-// fsDataOutputStream = fileSystem.create(hdfsWritePath);
-//
-// TarArchiveOutputStream ar = new TarArchiveOutputStream(fsDataOutputStream.getWrappedStream());
-//
-// RemoteIterator fileStatusListIterator = fileSystem
-// .listFiles(
-// new Path(inputPath), true);
-//
-// while (fileStatusListIterator.hasNext()) {
-// LocatedFileStatus fileStatus = fileStatusListIterator.next();
-//
-// Path p = fileStatus.getPath();
-// String p_string = p.toString();
-// if (!p_string.endsWith("_SUCCESS")) {
-// String name = p_string.substring(p_string.lastIndexOf("/") + 1);
-// TarArchiveEntry entry = new TarArchiveEntry(dir_name + "/" + name + ".json.gz");
-// entry.setSize(fileStatus.getLen());
-// ar.putArchiveEntry(entry);
-//
-// InputStream is = fileSystem.open(fileStatus.getPath());
-//
-// BufferedInputStream bis = new BufferedInputStream(is);
-//
-// int count;
-// byte data[] = new byte[1024];
-// while ((count = bis.read(data, 0, data.length)) != -1) {
-// ar.write(data, 0, count);
-// }
-// bis.close();
-// ar.closeArchiveEntry();
-//
-// }
-//
-// }
-//
-// ar.close();
-// }
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/SendToZenodoHDFS.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/SendToZenodoHDFS.java
deleted file mode 100644
index 2e2b7bc26..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/SendToZenodoHDFS.java
+++ /dev/null
@@ -1,80 +0,0 @@
-
-package eu.dnetlib.dhp.export.zenodo;
-
-import java.io.Serializable;
-import java.util.Optional;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-import eu.dnetlib.dhp.common.api.MissingConceptDoiException;
-import eu.dnetlib.dhp.common.api.ZenodoAPIClient;
-
-public class SendToZenodoHDFS implements Serializable {
-
- private static final Log log = LogFactory.getLog(SendToZenodoHDFS.class);
-
- public static void main(final String[] args) throws Exception, MissingConceptDoiException {
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils
- .toString(
- SendToZenodoHDFS.class
- .getResourceAsStream(
- "/eu/dnetlib/dhp/export/upload_zenodo.json")));
-
- parser.parseArgument(args);
-
- final String hdfsPath = parser.get("hdfsPath");
- final String hdfsNameNode = parser.get("nameNode");
- final String access_token = parser.get("accessToken");
- final String connection_url = parser.get("connectionUrl");
- final String metadata = parser.get("metadata");
- final Boolean newDeposition = Boolean.valueOf(parser.get("newDeposition"));
- final String concept_rec_id = Optional
- .ofNullable(parser.get("conceptRecordId"))
- .orElse(null);
-
- Configuration conf = new Configuration();
- conf.set("fs.defaultFS", hdfsNameNode);
-
- FileSystem fileSystem = FileSystem.get(conf);
-
- RemoteIterator fileStatusListIterator = fileSystem
- .listFiles(
- new Path(hdfsPath), true);
- ZenodoAPIClient zenodoApiClient = new ZenodoAPIClient(connection_url, access_token);
- if (newDeposition) {
- zenodoApiClient.newDeposition();
- } else {
- if (concept_rec_id == null) {
- throw new MissingConceptDoiException("No concept record id has been provided");
- }
- zenodoApiClient.newVersion(concept_rec_id);
- }
-
- while (fileStatusListIterator.hasNext()) {
- LocatedFileStatus fileStatus = fileStatusListIterator.next();
-
- Path p = fileStatus.getPath();
- String p_string = p.toString();
- if (!p_string.endsWith("_SUCCESS")) {
- // String tmp = p_string.substring(0, p_string.lastIndexOf("/"));
- String name = p_string.substring(p_string.lastIndexOf("/") + 1);
- log.info("Sending information for community: " + name);
- FSDataInputStream inputStream = fileSystem.open(p);
- zenodoApiClient.uploadIS(inputStream, name, fileStatus.getLen());
-
- }
-
- }
-
- zenodoApiClient.sendMretadata(metadata);
-// zenodoApiClient.publish();
-
- }
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/DropAndCreateESIndex.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/DropAndCreateESIndex.java
deleted file mode 100644
index 7598fd957..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/DropAndCreateESIndex.java
+++ /dev/null
@@ -1,98 +0,0 @@
-
-package eu.dnetlib.dhp.provision;
-
-import java.util.Map;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.client.methods.HttpDelete;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.entity.StringEntity;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-
-public class DropAndCreateESIndex {
-
- public static void main(String[] args) throws Exception {
-
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils
- .toString(
- DropAndCreateESIndex.class
- .getResourceAsStream(
- "/eu/dnetlib/dhp/provision/dropAndCreateIndex.json")));
- parser.parseArgument(args);
-
- final String index = parser.get("index");
-
- final String cluster = parser.get("cluster");
- final String clusterJson = IOUtils
- .toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/cluster.json"));
-
- final Map clusterMap = new ObjectMapper().readValue(clusterJson, Map.class);
-
- final String ip = clusterMap.get(cluster).split(",")[0];
-
- System.out.println(ip);
-
- final String url = "http://%s:9200/%s_%s";
-
- CloseableHttpClient client = HttpClients.createDefault();
-
- HttpDelete delete = new HttpDelete(String.format(url, ip, index, "object"));
-
- CloseableHttpResponse response = client.execute(delete);
-
- System.out.println("deleting Index SUMMARY");
- System.out.println(response.getStatusLine());
- client.close();
- client = HttpClients.createDefault();
-
- delete = new HttpDelete(String.format(url, ip, index, "scholix"));
-
- response = client.execute(delete);
-
- System.out.println("deleting Index SCHOLIX");
- System.out.println(response.getStatusLine());
- client.close();
- client = HttpClients.createDefault();
-
- final String summaryConf = IOUtils
- .toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/summary_index.json"));
-
- final String scholixConf = IOUtils
- .toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/scholix_index.json"));
-
- HttpPut put = new HttpPut(String.format(url, ip, index, "object"));
-
- StringEntity entity = new StringEntity(summaryConf);
- put.setEntity(entity);
- put.setHeader("Accept", "application/json");
- put.setHeader("Content-type", "application/json");
-
- System.out.println("creating First Index SUMMARY");
- response = client.execute(put);
-
- client.close();
- client = HttpClients.createDefault();
-
- System.out.println(response.getStatusLine());
-
- System.out.println("creating Index SCHOLIX");
- put = new HttpPut(String.format(url, ip, index, "scholix"));
-
- entity = new StringEntity(scholixConf);
- put.setEntity(entity);
- put.setHeader("Accept", "application/json");
- put.setHeader("Content-type", "application/json");
-
- response = client.execute(put);
- System.out.println(response.getStatusLine());
- client.close();
-
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/ProvisionUtil.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/ProvisionUtil.java
deleted file mode 100644
index 1b0cb4d05..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/ProvisionUtil.java
+++ /dev/null
@@ -1,48 +0,0 @@
-
-package eu.dnetlib.dhp.provision;
-
-import org.apache.commons.lang3.StringUtils;
-
-import eu.dnetlib.dhp.provision.scholix.summary.Typology;
-import eu.dnetlib.dhp.utils.DHPUtils;
-
-public class ProvisionUtil {
-
- public static final String deletedByInferenceJPATH = "$.dataInfo.deletedbyinference";
- public static final String TARGETJSONPATH = "$.target";
- public static final String SOURCEJSONPATH = "$.source";
-
- // public static RelatedItemInfo getItemType(final String item, final String idPath) {
- // String targetId = DHPUtils.getJPathString(idPath, item);
- // switch (StringUtils.substringBefore(targetId, "|")) {
- // case "50":
- // return new RelatedItemInfo(null,0,1,0);
- // case "60":
- // return new RelatedItemInfo(null,1,0,0);
- // case "70":
- // return new RelatedItemInfo(null,0,0,1);
- // default:
- // throw new RuntimeException("Unknonw target ID");
- //
- // }
- //
- // }
-
- public static Boolean isNotDeleted(final String item) {
- return !"true".equalsIgnoreCase(DHPUtils.getJPathString(deletedByInferenceJPATH, item));
- }
-
- public static Typology getItemTypeFromId(String id) {
-
- switch (StringUtils.substringBefore(id, "|")) {
- case "50":
- return Typology.publication;
- case "60":
- return Typology.dataset;
- case "70":
- return Typology.unknown;
- default:
- throw new RuntimeException("Unknonw ID type");
- }
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/RelatedItemInfo.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/RelatedItemInfo.java
deleted file mode 100644
index 28826612d..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/RelatedItemInfo.java
+++ /dev/null
@@ -1,59 +0,0 @@
-
-package eu.dnetlib.dhp.provision;
-
-import java.io.Serializable;
-
-/** This class models the information of related items */
-public class RelatedItemInfo implements Serializable {
-
- private String source;
-
- private long relatedDataset = 0;
-
- private long relatedPublication = 0;
-
- private long relatedUnknown = 0;
-
- public RelatedItemInfo() {
- }
-
- public RelatedItemInfo(
- String source, long relatedDataset, long relatedPublication, long relatedUnknown) {
- this.source = source;
- this.relatedDataset = relatedDataset;
- this.relatedPublication = relatedPublication;
- this.relatedUnknown = relatedUnknown;
- }
-
- public String getSource() {
- return source;
- }
-
- public void setSource(String source) {
- this.source = source;
- }
-
- public long getRelatedDataset() {
- return relatedDataset;
- }
-
- public void setRelatedDataset(long relatedDataset) {
- this.relatedDataset = relatedDataset;
- }
-
- public long getRelatedPublication() {
- return relatedPublication;
- }
-
- public void setRelatedPublication(long relatedPublication) {
- this.relatedPublication = relatedPublication;
- }
-
- public long getRelatedUnknown() {
- return relatedUnknown;
- }
-
- public void setRelatedUnknown(int relatedUnknown) {
- this.relatedUnknown = relatedUnknown;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkConvertDatasetToJson.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkConvertDatasetToJson.scala
deleted file mode 100644
index 8133666a6..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkConvertDatasetToJson.scala
+++ /dev/null
@@ -1,38 +0,0 @@
-package eu.dnetlib.dhp.provision
-
-import com.fasterxml.jackson.databind.ObjectMapper
-import eu.dnetlib.dhp.application.ArgumentApplicationParser
-import eu.dnetlib.dhp.provision.scholix.Scholix
-import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary
-import org.apache.commons.io.IOUtils
-import org.apache.hadoop.io.compress.GzipCodec
-import org.apache.spark.SparkConf
-import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
-
-object SparkConvertDatasetToJson {
-
- def main(args: Array[String]): Unit = {
- val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertDatasetToJson.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/dataset2Json.json")))
- parser.parseArgument(args)
- val conf = new SparkConf
- val spark = SparkSession.builder.config(conf).appName(SparkConvertDatasetToJson.getClass.getSimpleName).master(parser.get("master")).getOrCreate
-
- implicit val summaryEncoder: Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
- implicit val scholixEncoder: Encoder[Scholix] = Encoders.kryo[Scholix]
-
-
- val workingPath = parser.get("workingPath")
-
-
-
- spark.read.load(s"$workingPath/summary").as[ScholixSummary]
- .map(s => new ObjectMapper().writeValueAsString(s))(Encoders.STRING)
- .rdd.repartition(500).saveAsTextFile(s"$workingPath/summary_json", classOf[GzipCodec])
-
- spark.read.load(s"$workingPath/scholix").as[Scholix]
- .map(s => new ObjectMapper().writeValueAsString(s))(Encoders.STRING)
- .rdd.repartition(2000).saveAsTextFile(s"$workingPath/scholix_json", classOf[GzipCodec])
-
- }
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkExtractRelationCount.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkExtractRelationCount.scala
deleted file mode 100644
index d6e36ac87..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkExtractRelationCount.scala
+++ /dev/null
@@ -1,60 +0,0 @@
-package eu.dnetlib.dhp.provision
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser
-import eu.dnetlib.dhp.schema.oaf.Relation
-import org.apache.commons.io.IOUtils
-import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
-import org.apache.spark.sql.functions.{coalesce, col, count, lit}
-
-
-/**
- * SparkExtractRelationCount is a spark job that takes in input relation RDD and retrieve for each item in relation
- * which are the number of - Related Dataset - Related Publication - Related Unknown
- */
-object SparkExtractRelationCount {
-
-
- def main(args: Array[String]): Unit = {
-
- val parser = new ArgumentApplicationParser(IOUtils.toString(SparkExtractRelationCount.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/input_related_entities_parameters.json")))
- parser.parseArgument(args)
- val spark = SparkSession.builder.appName(SparkExtractRelationCount.getClass.getSimpleName).master(parser.get("master")).getOrCreate
-
- val workingDirPath = parser.get("workingDirPath")
-
- val relationPath = parser.get("relationPath")
-
- implicit val relEncoder: Encoder[Relation] = Encoders.kryo[Relation]
-
- val relation = spark.read.load(relationPath).as[Relation].map(r =>r)(Encoders.bean(classOf[Relation]))
-
- val relatedPublication = relation
- .where("target like '50%'")
- .groupBy("source")
- .agg(count("target").as("publication"))
- .select(col("source"). alias("p_source"), col("publication"))
- val relatedDataset = relation
- .where("target like '60%'")
- .groupBy("source")
- .agg(count("target").as("dataset"))
- .select(col("source"). alias("d_source"), col("dataset"))
- val relatedUnknown = relation
- .where("target like '70%'")
- .groupBy("source")
- .agg(count("target").as("unknown"))
- .select(col("source"). alias("u_source"), col("unknown"))
- val firstJoin = relatedPublication
- .join(relatedDataset,col("p_source").equalTo(col("d_source")),"full")
- .select( coalesce( col("p_source"), col("d_source")).alias("id"),
- col("publication"),
- col("dataset"))
- .join(relatedUnknown, col("u_source").equalTo(col("id")),"full")
- .select( coalesce(col("u_source"), col("id")).alias("source"),
- coalesce(col("publication"),lit(0)).alias("relatedPublication"),
- coalesce(col("dataset"),lit(0)).alias("relatedDataset"),
- coalesce(col("unknown"),lit(0)).alias("relatedUnknown")
- )
- firstJoin.write.mode(SaveMode.Overwrite).save(s"$workingDirPath/relatedItemCount")
- }
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateScholixIndex.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateScholixIndex.scala
deleted file mode 100644
index d39e38bfc..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateScholixIndex.scala
+++ /dev/null
@@ -1,94 +0,0 @@
-package eu.dnetlib.dhp.provision
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser
-import eu.dnetlib.dhp.provision.scholix.{Scholix, ScholixResource}
-import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary
-import eu.dnetlib.dhp.schema.oaf.Relation
-import org.apache.commons.io.IOUtils
-import org.apache.spark.SparkConf
-import org.apache.spark.sql.expressions.Aggregator
-import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
-
-object SparkGenerateScholixIndex {
-
-
-
- def getScholixAggregator(): Aggregator[(String, Scholix), Scholix, Scholix] = new Aggregator[(String, Scholix), Scholix, Scholix]{
-
- override def zero: Scholix = new Scholix()
-
- override def reduce(b: Scholix, a: (String, Scholix)): Scholix = {
- b.mergeFrom(a._2)
- b
- }
-
- override def merge(wx: Scholix, wy: Scholix): Scholix = {
- wx.mergeFrom(wy)
- wx
- }
- override def finish(reduction: Scholix): Scholix = reduction
-
- override def bufferEncoder: Encoder[Scholix] =
- Encoders.kryo(classOf[Scholix])
-
- override def outputEncoder: Encoder[Scholix] =
- Encoders.kryo(classOf[Scholix])
- }
-
-
- def main(args: Array[String]): Unit = {
- val parser = new ArgumentApplicationParser(IOUtils.toString(SparkGenerateScholixIndex.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json")))
- parser.parseArgument(args)
- val conf = new SparkConf
- conf.set("spark.sql.shuffle.partitions", "4000")
- val spark = SparkSession.builder.config(conf).appName(SparkGenerateScholixIndex.getClass.getSimpleName).master(parser.get("master")).getOrCreate
-
- val graphPath = parser.get("graphPath")
- val workingDirPath = parser.get("workingDirPath")
-
-
- implicit val summaryEncoder:Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
- implicit val relEncoder:Encoder[Relation] = Encoders.kryo[Relation]
- implicit val scholixEncoder:Encoder[Scholix] = Encoders.kryo[Scholix]
- implicit val tupleScholix:Encoder[(String,Scholix)]=Encoders.tuple(Encoders.STRING, scholixEncoder)
-
-
- val scholixSummary:Dataset[(String,ScholixSummary)] = spark.read.load(s"$workingDirPath/summary").as[ScholixSummary]
- .map(s => (s.getId, s))(Encoders.tuple(Encoders.STRING, summaryEncoder))
- val sourceRelations:Dataset[(String,Relation)]= spark.read.load(s"$graphPath/relation").as[Relation]
- .map(r => (r.getSource,r))(Encoders.tuple(Encoders.STRING, relEncoder))
-
- scholixSummary.joinWith(sourceRelations, scholixSummary("_1").equalTo(sourceRelations("_1")), "inner")
- .map(r=> {
- val summary = r._1._2
- val relation = r._2._2
-
- (relation.getTarget, Scholix.generateScholixWithSource(summary,relation))
-
- }).repartition(6000).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/scholix_source")
-
- val sTarget:Dataset[(String,Scholix)] = spark.read.load(s"$workingDirPath/scholix_source").as[(String, Scholix)]
-
- sTarget.joinWith(scholixSummary, sTarget("_1").equalTo(scholixSummary("_1")), "inner").map(i => {
- val summary = i._2._2
- val scholix = i._1._2
-
- val scholixResource = ScholixResource.fromSummary(summary)
- scholix.setTarget(scholixResource)
- scholix.generateIdentifier()
- scholix.generatelinkPublisher()
- scholix
- }).repartition(6000).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/scholix_r")
-
-
- val finalScholix:Dataset[Scholix] = spark.read.load(s"$workingDirPath/scholix_r").as[Scholix]
-
- finalScholix.map(d => (d.getIdentifier, d))(Encoders.tuple(Encoders.STRING, scholixEncoder))
- .groupByKey(_._1)(Encoders.STRING)
- .agg(getScholixAggregator().toColumn)
- .map(p => p._2)
- .write.mode(SaveMode.Overwrite).save(s"$workingDirPath/scholix")
-
- }
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateSummaryIndex.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateSummaryIndex.scala
deleted file mode 100644
index bf3d0342b..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateSummaryIndex.scala
+++ /dev/null
@@ -1,70 +0,0 @@
-package eu.dnetlib.dhp.provision
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser
-import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary
-import eu.dnetlib.dhp.schema.oaf.{Oaf, OafEntity, Relation}
-import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown}
-import org.apache.commons.io.IOUtils
-import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
-
-object SparkGenerateSummaryIndex {
-
- def main(args: Array[String]): Unit = {
- val parser = new ArgumentApplicationParser(IOUtils.toString(SparkGenerateSummaryIndex.getClass.getResourceAsStream("/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json")))
- parser.parseArgument(args)
- val spark = SparkSession.builder.appName(SparkGenerateSummaryIndex.getClass.getSimpleName).master(parser.get("master")).getOrCreate
-
- val graphPath = parser.get("graphPath")
- val workingDirPath = parser.get("workingDirPath")
-
- implicit val relatedItemInfoEncoders: Encoder[RelatedItemInfo] = Encoders.bean(classOf[RelatedItemInfo])
- implicit val datasetEncoder:Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
- implicit val publicationEncoder:Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
- implicit val relationEncoder:Encoder[Relation] = Encoders.kryo[Relation]
- implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
- implicit val oafWithIdEncoder: Encoder[(String, Oaf)] = Encoders.tuple(Encoders.STRING, oafEncoder)
- implicit val scholixSummaryEncoder: Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
- implicit val scholixSummaryEncoderTuple: Encoder[(String,ScholixSummary)] = Encoders.tuple(Encoders.STRING,scholixSummaryEncoder)
-
-
- val pubs = spark.read.load(s"$graphPath/publication").as[Oaf].map(o => (o.asInstanceOf[DLIPublication].getId, o))
- val dats = spark.read.load(s"$graphPath/dataset").as[Oaf].map(o => (o.asInstanceOf[DLIDataset].getId, o))
- val ukn = spark.read.load(s"$graphPath/unknown").as[Oaf].map(o => (o.asInstanceOf[DLIUnknown].getId, o))
-
-
- val summary:Dataset[(String,ScholixSummary)] = pubs.union(dats).union(ukn).map(o =>{
- val s = ScholixSummary.fromOAF(o._2)
- (s.getId,s)
- })
-
-
- val relatedItemInfoDs:Dataset[RelatedItemInfo] = spark.read.load(s"$workingDirPath/relatedItemCount").as[RelatedItemInfo]
-
-
- summary.joinWith(relatedItemInfoDs, summary("_1").equalTo(relatedItemInfoDs("source")), "inner")
- .map(i => {
- val summary = i._1._2
- val relatedItemInfo = i._2
- summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset)
- summary.setRelatedPublications(relatedItemInfo.getRelatedPublication)
- summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown)
- summary
- }).filter(s => s.getLocalIdentifier != null).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/summary")
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
- }
-
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java
deleted file mode 100644
index f1eb3992d..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java
+++ /dev/null
@@ -1,61 +0,0 @@
-
-package eu.dnetlib.dhp.provision;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.sql.SparkSession;
-import org.elasticsearch.spark.rdd.api.java.JavaEsSpark;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-
-public class
-SparkIndexCollectionOnES {
-
- public static void main(String[] args) throws Exception {
-
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils
- .toString(
- SparkIndexCollectionOnES.class
- .getResourceAsStream(
- "/eu/dnetlib/dhp/provision/index_on_es.json")));
- parser.parseArgument(args);
-
- SparkConf conf = new SparkConf()
- .setAppName(SparkIndexCollectionOnES.class.getSimpleName())
- .setMaster(parser.get("master"));
-
- conf.set("spark.sql.shuffle.partitions", "4000");
-
- final String sourcePath = parser.get("sourcePath");
- final String index = parser.get("index");
- final String idPath = parser.get("idPath");
- final String cluster = parser.get("cluster");
- final String clusterJson = IOUtils
- .toString(DropAndCreateESIndex.class.getResourceAsStream("/eu/dnetlib/dhp/provision/cluster.json"));
-
- final Map clusterMap = new ObjectMapper().readValue(clusterJson, Map.class);
-
- final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
-
- final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
-
- JavaRDD inputRdd = sc.textFile(sourcePath);
-
- Map esCfg = new HashMap<>();
- esCfg.put("es.nodes", clusterMap.get(cluster));
- esCfg.put("es.mapping.id", idPath);
- esCfg.put("es.batch.write.retry.count", "8");
- esCfg.put("es.batch.write.retry.wait", "60s");
- esCfg.put("es.batch.size.entries", "200");
- esCfg.put("es.nodes.wan.only", "true");
- JavaEsSpark.saveJsonToEs(inputRdd, index, esCfg);
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/Scholix.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/Scholix.java
deleted file mode 100644
index ec3da5cfc..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/Scholix.java
+++ /dev/null
@@ -1,286 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix;
-
-import java.io.Serializable;
-import java.util.*;
-import java.util.stream.Collectors;
-
-import org.apache.commons.lang3.StringUtils;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary;
-import eu.dnetlib.dhp.schema.oaf.Relation;
-import eu.dnetlib.dhp.utils.DHPUtils;
-
-public class Scholix implements Serializable {
- private String publicationDate;
-
- private List publisher;
-
- private List linkprovider;
-
- private ScholixRelationship relationship;
-
- private ScholixResource source;
-
- private ScholixResource target;
-
- private String identifier;
-
- public Scholix clone(final ScholixResource t) {
- final Scholix clone = new Scholix();
- clone.setPublicationDate(publicationDate);
- clone.setPublisher(publisher);
- clone.setLinkprovider(linkprovider);
- clone.setRelationship(relationship);
- clone.setSource(source);
- clone.setTarget(t);
- clone.generatelinkPublisher();
- clone.generateIdentifier();
- return clone;
- }
-
- public static Scholix generateScholixWithSource(
- final String sourceSummaryJson, final String relation) {
- final ObjectMapper mapper = new ObjectMapper();
-
- try {
- ScholixSummary scholixSummary = mapper.readValue(sourceSummaryJson, ScholixSummary.class);
- Relation rel = mapper.readValue(relation, Relation.class);
- final Scholix s = new Scholix();
- if (scholixSummary.getDate() != null && scholixSummary.getDate().size() > 0)
- s.setPublicationDate(scholixSummary.getDate().get(0));
- s
- .setLinkprovider(
- rel
- .getCollectedfrom()
- .stream()
- .map(
- cf -> new ScholixEntityId(
- cf.getValue(),
- Collections
- .singletonList(
- new ScholixIdentifier(cf.getKey(), "dnet_identifier"))))
- .collect(Collectors.toList()));
- s.setRelationship(new ScholixRelationship(rel.getRelType(), rel.getRelClass(), null));
- s.setSource(ScholixResource.fromSummary(scholixSummary));
- return s;
- } catch (Throwable e) {
- throw new RuntimeException(
- String.format("Summary: %s \n relation:%s", sourceSummaryJson, relation), e);
- }
- }
-
- public static Scholix generateScholixWithSource(
- final ScholixSummary scholixSummary, final Relation rel) {
- final Scholix s = new Scholix();
- if (scholixSummary.getDate() != null && scholixSummary.getDate().size() > 0)
- s.setPublicationDate(scholixSummary.getDate().get(0));
- s
- .setLinkprovider(
- rel
- .getCollectedfrom()
- .stream()
- .map(
- cf -> new ScholixEntityId(
- cf.getValue(),
- Collections
- .singletonList(
- new ScholixIdentifier(cf.getKey(), "dnet_identifier"))))
- .collect(Collectors.toList()));
- s.setRelationship(new ScholixRelationship(rel.getRelType(), rel.getRelClass(), null));
- s.setSource(ScholixResource.fromSummary(scholixSummary));
-
- s.setIdentifier(rel.getTarget());
- return s;
- }
-
- private List mergeScholixEntityId(final List a, final List b) {
- final List m = a != null ? new ArrayList<>(a) : new ArrayList<>();
- if (b != null)
- b.forEach(s -> {
- if (s != null) {
- int tt = (int) m
- .stream()
- .filter(t -> t != null && t.getName() != null && t.getName().equalsIgnoreCase(s.getName()))
- .count();
- if (tt == 0) {
- m.add(s);
- }
- }
- });
- return m;
- }
-
- private List mergeScholixIdnetifier(final List a,
- final List b) {
- final List m = a != null ? new ArrayList<>(a) : new ArrayList<>();
- if (b != null)
- b.forEach(s -> {
- int tt = (int) m.stream().filter(t -> t.getIdentifier().equalsIgnoreCase(s.getIdentifier())).count();
- if (tt == 0) {
- m.add(s);
- }
- });
- return m;
- }
-
- private List mergeScholixCollectedFrom(final List a,
- final List b) {
- final List m = a != null ? new ArrayList<>(a) : new ArrayList<>();
- if (b != null)
- b.forEach(s -> {
- int tt = (int) m
- .stream()
- .filter(t -> t.getProvider().getName().equalsIgnoreCase(s.getProvider().getName()))
- .count();
- if (tt == 0) {
- m.add(s);
- }
- });
- return m;
- }
-
- private ScholixRelationship mergeRelationships(final ScholixRelationship a, final ScholixRelationship b) {
- ScholixRelationship result = new ScholixRelationship();
- result.setName(a == null || StringUtils.isEmpty(a.getName()) ? b.getName() : a.getName());
- result.setInverse(a == null || StringUtils.isEmpty(a.getInverse()) ? b.getInverse() : a.getInverse());
- result.setSchema(a == null || StringUtils.isEmpty(a.getSchema()) ? b.getSchema() : a.getSchema());
- return result;
- }
-
- private ScholixResource mergeResource(final ScholixResource a, final ScholixResource b) {
- if (a == null)
- return b;
- final ScholixResource result = new ScholixResource();
- result.setCollectedFrom(mergeScholixCollectedFrom(a.getCollectedFrom(), b.getCollectedFrom()));
- result.setCreator(mergeScholixEntityId(a.getCreator(), b.getCreator()));
- result
- .setDnetIdentifier(
- StringUtils.isBlank(a.getDnetIdentifier()) ? b.getDnetIdentifier() : a.getDnetIdentifier());
- result.setIdentifier(mergeScholixIdnetifier(a.getIdentifier(), b.getIdentifier()));
- result.setObjectType(StringUtils.isNotBlank(a.getObjectType()) ? a.getObjectType() : b.getObjectType());
- result
- .setObjectSubType(
- StringUtils.isNotBlank(a.getObjectSubType()) ? a.getObjectSubType() : b.getObjectSubType());
- result.setPublisher(mergeScholixEntityId(a.getPublisher(), b.getPublisher()));
- result
- .setPublicationDate(
- StringUtils.isNotBlank(a.getPublicationDate()) ? a.getPublicationDate() : b.getPublicationDate());
- result.setTitle(StringUtils.isNotBlank(a.getTitle()) ? a.getTitle() : b.getTitle());
- return result;
-
- }
-
- public void mergeFrom(final Scholix other) {
- linkprovider = mergeScholixEntityId(linkprovider, other.getLinkprovider());
- publisher = mergeScholixEntityId(publisher, other.getPublisher());
- if (StringUtils.isEmpty(publicationDate))
- publicationDate = other.getPublicationDate();
- relationship = mergeRelationships(relationship, other.getRelationship());
- source = mergeResource(source, other.getSource());
- target = mergeResource(target, other.getTarget());
- generateIdentifier();
- }
-
- public void generatelinkPublisher() {
- Set publisher = new HashSet<>();
- if (source.getPublisher() != null)
- publisher
- .addAll(
- source
- .getPublisher()
- .stream()
- .map(ScholixEntityId::getName)
- .collect(Collectors.toList()));
- if (target.getPublisher() != null)
- publisher
- .addAll(
- target
- .getPublisher()
- .stream()
- .map(ScholixEntityId::getName)
- .collect(Collectors.toList()));
- this.publisher = publisher.stream().map(k -> new ScholixEntityId(k, null)).collect(Collectors.toList());
- }
-
- public void generateIdentifier() {
- setIdentifier(
- DHPUtils
- .md5(
- String
- .format(
- "%s::%s::%s",
- source.getDnetIdentifier(), relationship.getName(), target.getDnetIdentifier())));
- }
-
- public Scholix addTarget(final String targetSummaryJson) {
- final ObjectMapper mapper = new ObjectMapper();
-
- try {
- ScholixSummary targetSummary = mapper.readValue(targetSummaryJson, ScholixSummary.class);
- setTarget(ScholixResource.fromSummary(targetSummary));
- generateIdentifier();
- return this;
- } catch (Throwable e) {
- throw new RuntimeException(e);
- }
- }
-
- public String getPublicationDate() {
- return publicationDate;
- }
-
- public void setPublicationDate(String publicationDate) {
- this.publicationDate = publicationDate;
- }
-
- public List getPublisher() {
- return publisher;
- }
-
- public void setPublisher(List publisher) {
- this.publisher = publisher;
- }
-
- public List getLinkprovider() {
- return linkprovider;
- }
-
- public void setLinkprovider(List linkprovider) {
- this.linkprovider = linkprovider;
- }
-
- public ScholixRelationship getRelationship() {
- return relationship;
- }
-
- public void setRelationship(ScholixRelationship relationship) {
- this.relationship = relationship;
- }
-
- public ScholixResource getSource() {
- return source;
- }
-
- public void setSource(ScholixResource source) {
- this.source = source;
- }
-
- public ScholixResource getTarget() {
- return target;
- }
-
- public void setTarget(ScholixResource target) {
- this.target = target;
- }
-
- public String getIdentifier() {
- return identifier;
- }
-
- public void setIdentifier(String identifier) {
- this.identifier = identifier;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixCollectedFrom.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixCollectedFrom.java
deleted file mode 100644
index 9ce071fbc..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixCollectedFrom.java
+++ /dev/null
@@ -1,45 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix;
-
-import java.io.Serializable;
-
-public class ScholixCollectedFrom implements Serializable {
-
- private ScholixEntityId provider;
- private String provisionMode;
- private String completionStatus;
-
- public ScholixCollectedFrom() {
- }
-
- public ScholixCollectedFrom(
- ScholixEntityId provider, String provisionMode, String completionStatus) {
- this.provider = provider;
- this.provisionMode = provisionMode;
- this.completionStatus = completionStatus;
- }
-
- public ScholixEntityId getProvider() {
- return provider;
- }
-
- public void setProvider(ScholixEntityId provider) {
- this.provider = provider;
- }
-
- public String getProvisionMode() {
- return provisionMode;
- }
-
- public void setProvisionMode(String provisionMode) {
- this.provisionMode = provisionMode;
- }
-
- public String getCompletionStatus() {
- return completionStatus;
- }
-
- public void setCompletionStatus(String completionStatus) {
- this.completionStatus = completionStatus;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixEntityId.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixEntityId.java
deleted file mode 100644
index e797017bc..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixEntityId.java
+++ /dev/null
@@ -1,34 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix;
-
-import java.io.Serializable;
-import java.util.List;
-
-public class ScholixEntityId implements Serializable {
- private String name;
- private List identifiers;
-
- public ScholixEntityId() {
- }
-
- public ScholixEntityId(String name, List identifiers) {
- this.name = name;
- this.identifiers = identifiers;
- }
-
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- public List getIdentifiers() {
- return identifiers;
- }
-
- public void setIdentifiers(List identifiers) {
- this.identifiers = identifiers;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixIdentifier.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixIdentifier.java
deleted file mode 100644
index 0dd15336a..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixIdentifier.java
+++ /dev/null
@@ -1,33 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix;
-
-import java.io.Serializable;
-
-public class ScholixIdentifier implements Serializable {
- private String identifier;
- private String schema;
-
- public ScholixIdentifier() {
- }
-
- public ScholixIdentifier(String identifier, String schema) {
- this.identifier = identifier;
- this.schema = schema;
- }
-
- public String getIdentifier() {
- return identifier;
- }
-
- public void setIdentifier(String identifier) {
- this.identifier = identifier;
- }
-
- public String getSchema() {
- return schema;
- }
-
- public void setSchema(String schema) {
- this.schema = schema;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixRelationship.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixRelationship.java
deleted file mode 100644
index 0cbdf43e7..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixRelationship.java
+++ /dev/null
@@ -1,43 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix;
-
-import java.io.Serializable;
-
-public class ScholixRelationship implements Serializable {
- private String name;
- private String schema;
- private String inverse;
-
- public ScholixRelationship() {
- }
-
- public ScholixRelationship(String name, String schema, String inverse) {
- this.name = name;
- this.schema = schema;
- this.inverse = inverse;
- }
-
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- public String getSchema() {
- return schema;
- }
-
- public void setSchema(String schema) {
- this.schema = schema;
- }
-
- public String getInverse() {
- return inverse;
- }
-
- public void setInverse(String inverse) {
- this.inverse = inverse;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java
deleted file mode 100644
index 6de30c748..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java
+++ /dev/null
@@ -1,151 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary;
-
-public class ScholixResource implements Serializable {
-
- private List identifier;
- private String dnetIdentifier;
- private String objectType;
- private String objectSubType;
- private String title;
- private List creator;
- private String publicationDate;
- private List publisher;
- private List collectedFrom;
-
- public static ScholixResource fromSummary(ScholixSummary summary) {
-
- final ScholixResource resource = new ScholixResource();
-
- resource.setDnetIdentifier(summary.getId());
-
- resource
- .setIdentifier(
- summary
- .getLocalIdentifier()
- .stream()
- .map(i -> new ScholixIdentifier(i.getId(), i.getType()))
- .collect(Collectors.toList()));
-
- resource.setObjectType(summary.getTypology().toString());
-
- if (summary.getTitle() != null && summary.getTitle().size() > 0)
- resource.setTitle(summary.getTitle().get(0));
-
- if (summary.getAuthor() != null)
- resource
- .setCreator(
- summary
- .getAuthor()
- .stream()
- .map(c -> new ScholixEntityId(c, null))
- .collect(Collectors.toList()));
-
- if (summary.getDate() != null && summary.getDate().size() > 0)
- resource.setPublicationDate(summary.getDate().get(0));
- if (summary.getPublisher() != null)
- resource
- .setPublisher(
- summary
- .getPublisher()
- .stream()
- .map(p -> new ScholixEntityId(p, null))
- .collect(Collectors.toList()));
- if (summary.getDatasources() != null)
- resource
- .setCollectedFrom(
- summary
- .getDatasources()
- .stream()
- .map(
- d -> new ScholixCollectedFrom(
- new ScholixEntityId(
- d.getDatasourceName(),
- Collections
- .singletonList(
- new ScholixIdentifier(d.getDatasourceId(), "dnet_identifier"))),
- "collected",
- d.getCompletionStatus()))
- .collect(Collectors.toList()));
- return resource;
- }
-
- public List getIdentifier() {
- return identifier;
- }
-
- public void setIdentifier(List identifier) {
- this.identifier = identifier;
- }
-
- public String getDnetIdentifier() {
- return dnetIdentifier;
- }
-
- public void setDnetIdentifier(String dnetIdentifier) {
- this.dnetIdentifier = dnetIdentifier;
- }
-
- public String getObjectType() {
- return objectType;
- }
-
- public void setObjectType(String objectType) {
- this.objectType = objectType;
- }
-
- public String getObjectSubType() {
- return objectSubType;
- }
-
- public void setObjectSubType(String objectSubType) {
- this.objectSubType = objectSubType;
- }
-
- public String getTitle() {
- return title;
- }
-
- public void setTitle(String title) {
- this.title = title;
- }
-
- public List getCreator() {
- return creator;
- }
-
- public void setCreator(List creator) {
- this.creator = creator;
- }
-
- public String getPublicationDate() {
- return publicationDate;
- }
-
- public void setPublicationDate(String publicationDate) {
- this.publicationDate = publicationDate;
- }
-
- public List getPublisher() {
- return publisher;
- }
-
- public void setPublisher(List publisher) {
- this.publisher = publisher;
- }
-
- public List getCollectedFrom() {
- return collectedFrom;
- }
-
- public void setCollectedFrom(List collectedFrom) {
- this.collectedFrom = collectedFrom;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/CollectedFromType.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/CollectedFromType.java
deleted file mode 100644
index 6d6f46f54..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/CollectedFromType.java
+++ /dev/null
@@ -1,44 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix.summary;
-
-import java.io.Serializable;
-
-public class CollectedFromType implements Serializable {
-
- private String datasourceName;
- private String datasourceId;
- private String completionStatus;
-
- public CollectedFromType() {
- }
-
- public CollectedFromType(String datasourceName, String datasourceId, String completionStatus) {
- this.datasourceName = datasourceName;
- this.datasourceId = datasourceId;
- this.completionStatus = completionStatus;
- }
-
- public String getDatasourceName() {
- return datasourceName;
- }
-
- public void setDatasourceName(String datasourceName) {
- this.datasourceName = datasourceName;
- }
-
- public String getDatasourceId() {
- return datasourceId;
- }
-
- public void setDatasourceId(String datasourceId) {
- this.datasourceId = datasourceId;
- }
-
- public String getCompletionStatus() {
- return completionStatus;
- }
-
- public void setCompletionStatus(String completionStatus) {
- this.completionStatus = completionStatus;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/SchemeValue.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/SchemeValue.java
deleted file mode 100644
index e9d94fccf..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/SchemeValue.java
+++ /dev/null
@@ -1,33 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix.summary;
-
-import java.io.Serializable;
-
-public class SchemeValue implements Serializable {
- private String scheme;
- private String value;
-
- public SchemeValue() {
- }
-
- public SchemeValue(String scheme, String value) {
- this.scheme = scheme;
- this.value = value;
- }
-
- public String getScheme() {
- return scheme;
- }
-
- public void setScheme(String scheme) {
- this.scheme = scheme;
- }
-
- public String getValue() {
- return value;
- }
-
- public void setValue(String value) {
- this.value = value;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/ScholixSummary.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/ScholixSummary.java
deleted file mode 100644
index 3b808ba51..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/ScholixSummary.java
+++ /dev/null
@@ -1,321 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix.summary;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.databind.DeserializationFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import eu.dnetlib.dhp.provision.RelatedItemInfo;
-import eu.dnetlib.dhp.schema.oaf.Author;
-import eu.dnetlib.dhp.schema.oaf.Oaf;
-import eu.dnetlib.dhp.schema.oaf.OafEntity;
-import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
-import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
-import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
-import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown;
-
-public class ScholixSummary implements Serializable {
- private String id;
- private List localIdentifier;
- private Typology typology;
- private List title;
- private List author;
- private List date;
- private String description;
- private List subject;
- private List publisher;
- private long relatedPublications;
- private long relatedDatasets;
- private long relatedUnknown;
- private List datasources;
-
- public String getId() {
- return id;
- }
-
- public void setId(String id) {
- this.id = id;
- }
-
- public List getLocalIdentifier() {
- return localIdentifier;
- }
-
- public void setLocalIdentifier(List localIdentifier) {
- this.localIdentifier = localIdentifier;
- }
-
- public Typology getTypology() {
- return typology;
- }
-
- public void setTypology(Typology typology) {
- this.typology = typology;
- }
-
- public List getTitle() {
- return title;
- }
-
- public void setTitle(List title) {
- this.title = title;
- }
-
- public List getAuthor() {
- return author;
- }
-
- public void setAuthor(List author) {
- this.author = author;
- }
-
- public List getDate() {
- return date;
- }
-
- public void setDate(List date) {
- this.date = date;
- }
-
- @JsonProperty("abstract")
- public String getDescription() {
- return description;
- }
-
- @JsonProperty("abstract")
- public void setDescription(String description) {
- this.description = description;
- }
-
- public List getSubject() {
- return subject;
- }
-
- public void setSubject(List subject) {
- this.subject = subject;
- }
-
- public List getPublisher() {
- return publisher;
- }
-
- public void setPublisher(List publisher) {
- this.publisher = publisher;
- }
-
- public long getRelatedPublications() {
- return relatedPublications;
- }
-
- public void setRelatedPublications(long relatedPublications) {
- this.relatedPublications = relatedPublications;
- }
-
- public long getRelatedDatasets() {
- return relatedDatasets;
- }
-
- public void setRelatedDatasets(long relatedDatasets) {
- this.relatedDatasets = relatedDatasets;
- }
-
- public long getRelatedUnknown() {
- return relatedUnknown;
- }
-
- public void setRelatedUnknown(long relatedUnknown) {
- this.relatedUnknown = relatedUnknown;
- }
-
- public List getDatasources() {
- return datasources;
- }
-
- public void setDatasources(List datasources) {
- this.datasources = datasources;
- }
-
- public static ScholixSummary fromOAF(final Oaf oaf) {
- try {
- final RelatedItemInfo relatedItemInfo = new RelatedItemInfo();
-
- if (oaf instanceof DLIPublication)
- return summaryFromPublication((DLIPublication) oaf, relatedItemInfo);
- if (oaf instanceof DLIDataset)
- return summaryFromDataset((DLIDataset) oaf, relatedItemInfo);
- if (oaf instanceof DLIUnknown)
- return summaryFromUnknown((DLIUnknown) oaf, relatedItemInfo);
-
- } catch (Throwable e) {
- throw new RuntimeException(e);
- }
- return null;
- }
-
- private static ScholixSummary summaryFromDataset(
- final DLIDataset item, final RelatedItemInfo relatedItemInfo) {
- ScholixSummary summary = new ScholixSummary();
- summary.setId(item.getId());
-
- if (item.getPid() != null)
- summary
- .setLocalIdentifier(
- item
- .getPid()
- .stream()
- .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid()))
- .collect(Collectors.toList()));
-
- summary.setTypology(Typology.dataset);
- if (item.getTitle() != null)
- summary
- .setTitle(
- item.getTitle().stream().map(StructuredProperty::getValue).collect(Collectors.toList()));
-
- if (item.getAuthor() != null) {
- summary
- .setAuthor(
- item.getAuthor().stream().map(Author::getFullname).collect(Collectors.toList()));
- }
-
- if (item.getRelevantdate() != null)
- summary
- .setDate(
- item
- .getRelevantdate()
- .stream()
- .filter(d -> "date".equalsIgnoreCase(d.getQualifier().getClassname()))
- .map(StructuredProperty::getValue)
- .collect(Collectors.toList()));
-
- if (item.getDescription() != null && item.getDescription().size() > 0)
- summary.setDescription(item.getDescription().get(0).getValue());
-
- if (item.getSubject() != null) {
- summary
- .setSubject(
- item
- .getSubject()
- .stream()
- .map(s -> new SchemeValue(s.getQualifier().getClassid(), s.getValue()))
- .collect(Collectors.toList()));
- }
- if (item.getPublisher() != null)
- summary.setPublisher(Collections.singletonList(item.getPublisher().getValue()));
-
- summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset());
- summary.setRelatedPublications(relatedItemInfo.getRelatedPublication());
- summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown());
-
- if (item.getDlicollectedfrom() != null)
- summary
- .setDatasources(
- item
- .getDlicollectedfrom()
- .stream()
- .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus()))
- .collect(Collectors.toList()));
- return summary;
- }
-
- private static ScholixSummary summaryFromPublication(
- final DLIPublication item, final RelatedItemInfo relatedItemInfo) {
- ScholixSummary summary = new ScholixSummary();
- summary.setId(item.getId());
-
- if (item.getPid() != null)
- summary
- .setLocalIdentifier(
- item
- .getPid()
- .stream()
- .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid()))
- .collect(Collectors.toList()));
-
- summary.setTypology(Typology.publication);
- if (item.getTitle() != null)
- summary
- .setTitle(
- item.getTitle().stream().map(StructuredProperty::getValue).collect(Collectors.toList()));
-
- if (item.getAuthor() != null) {
- summary
- .setAuthor(
- item.getAuthor().stream().map(Author::getFullname).collect(Collectors.toList()));
- }
-
- if (item.getRelevantdate() != null)
- summary
- .setDate(
- item
- .getRelevantdate()
- .stream()
- .filter(d -> "date".equalsIgnoreCase(d.getQualifier().getClassname()))
- .map(StructuredProperty::getValue)
- .collect(Collectors.toList()));
-
- if (item.getDescription() != null && item.getDescription().size() > 0)
- summary.setDescription(item.getDescription().get(0).getValue());
-
- if (item.getSubject() != null) {
- summary
- .setSubject(
- item
- .getSubject()
- .stream()
- .map(s -> new SchemeValue(s.getQualifier().getClassid(), s.getValue()))
- .collect(Collectors.toList()));
- }
-
- if (item.getPublisher() != null)
- summary.setPublisher(Collections.singletonList(item.getPublisher().getValue()));
-
- summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset());
- summary.setRelatedPublications(relatedItemInfo.getRelatedPublication());
- summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown());
-
- if (item.getDlicollectedfrom() != null)
- summary
- .setDatasources(
- item
- .getDlicollectedfrom()
- .stream()
- .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus()))
- .collect(Collectors.toList()));
-
- return summary;
- }
-
- private static ScholixSummary summaryFromUnknown(
- final DLIUnknown item, final RelatedItemInfo relatedItemInfo) {
- ScholixSummary summary = new ScholixSummary();
- summary.setId(item.getId());
- if (item.getPid() != null)
- summary
- .setLocalIdentifier(
- item
- .getPid()
- .stream()
- .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid()))
- .collect(Collectors.toList()));
-
- summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset());
- summary.setRelatedPublications(relatedItemInfo.getRelatedPublication());
- summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown());
- summary.setTypology(Typology.unknown);
- if (item.getDlicollectedfrom() != null)
- summary
- .setDatasources(
- item
- .getDlicollectedfrom()
- .stream()
- .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus()))
- .collect(Collectors.toList()));
- return summary;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/TypedIdentifier.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/TypedIdentifier.java
deleted file mode 100644
index c4148ad24..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/TypedIdentifier.java
+++ /dev/null
@@ -1,33 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix.summary;
-
-import java.io.Serializable;
-
-public class TypedIdentifier implements Serializable {
- private String id;
- private String type;
-
- public TypedIdentifier() {
- }
-
- public TypedIdentifier(String id, String type) {
- this.id = id;
- this.type = type;
- }
-
- public String getId() {
- return id;
- }
-
- public void setId(String id) {
- this.id = id;
- }
-
- public String getType() {
- return type;
- }
-
- public void setType(String type) {
- this.type = type;
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/Typology.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/Typology.java
deleted file mode 100644
index effa32b6b..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/Typology.java
+++ /dev/null
@@ -1,8 +0,0 @@
-
-package eu.dnetlib.dhp.provision.scholix.summary;
-
-import java.io.Serializable;
-
-public enum Typology implements Serializable {
- dataset, publication, unknown
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossRefParserJSON.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossRefParserJSON.java
deleted file mode 100644
index a172ef698..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossRefParserJSON.java
+++ /dev/null
@@ -1,131 +0,0 @@
-
-package eu.dnetlib.dhp.provision.update;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import com.google.gson.JsonArray;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonParser;
-
-import eu.dnetlib.dhp.provision.scholix.ScholixCollectedFrom;
-import eu.dnetlib.dhp.provision.scholix.ScholixEntityId;
-import eu.dnetlib.dhp.provision.scholix.ScholixIdentifier;
-import eu.dnetlib.dhp.provision.scholix.ScholixResource;
-import eu.dnetlib.dhp.utils.DHPUtils;
-
-public class CrossRefParserJSON {
-
- private static final List collectedFrom = generateCrossrefCollectedFrom("complete");
-
- public static ScholixResource parseRecord(final String record) {
- if (record == null)
- return null;
- JsonElement jElement = new JsonParser().parse(record);
- JsonElement source = null;
- if (jElement.getAsJsonObject().has("_source")) {
- source = jElement.getAsJsonObject().get("_source");
- if (source == null || !source.isJsonObject())
- return null;
- } else if (jElement.getAsJsonObject().has("DOI")) {
- source = jElement;
- } else {
- return null;
- }
-
- final JsonObject message = source.getAsJsonObject();
- ScholixResource currentObject = new ScholixResource();
-
- if (message.get("DOI") != null) {
- final String doi = message.get("DOI").getAsString();
- currentObject.setIdentifier(Collections.singletonList(new ScholixIdentifier(doi, "doi")));
- }
-
- if ((!message.get("created").isJsonNull())
- && (message.getAsJsonObject("created").get("date-time") != null)) {
- currentObject
- .setPublicationDate(
- message.getAsJsonObject("created").get("date-time").getAsString());
- }
-
- if (message.get("title") != null
- && !message.get("title").isJsonNull()
- && message.get("title").isJsonArray()) {
-
- JsonArray array = message.get("title").getAsJsonArray();
- currentObject.setTitle(array.get(0).getAsString());
- }
- if (message.get("author") != null && !message.get("author").isJsonNull()) {
- JsonArray author = message.getAsJsonArray("author");
- List authorList = new ArrayList<>();
- for (JsonElement anAuthor : author) {
- JsonObject currentAuth = anAuthor.getAsJsonObject();
-
- String family = "";
- String given = "";
- if (currentAuth != null
- && currentAuth.get("family") != null
- && !currentAuth.get("family").isJsonNull()) {
- family = currentAuth.get("family").getAsString();
- }
- if (currentAuth != null
- && currentAuth.get("given") != null
- && !currentAuth.get("given").isJsonNull()) {
- given = currentAuth.get("given").getAsString();
- }
- authorList.add(new ScholixEntityId(String.format("%s %s", family, given), null));
- }
- currentObject.setCreator(authorList);
- }
- if (message.get("publisher") != null && !message.get("publisher").isJsonNull()) {
- currentObject
- .setPublisher(
- Collections
- .singletonList(
- new ScholixEntityId(message.get("publisher").getAsString(), null)));
- }
- currentObject.setCollectedFrom(collectedFrom);
- currentObject.setObjectType("publication");
- currentObject
- .setDnetIdentifier(
- generateId(message.get("DOI").getAsString(), "doi", "publication"));
-
- return currentObject;
- }
-
- private static List generateCrossrefCollectedFrom(
- final String completionStatus) {
- final ScholixEntityId scholixEntityId = new ScholixEntityId(
- "Crossref",
- Collections
- .singletonList(
- new ScholixIdentifier("dli_________::crossref", "dnet_identifier")));
- return Collections
- .singletonList(
- new ScholixCollectedFrom(scholixEntityId, "resolved", completionStatus));
- }
-
- private static String generateId(
- final String pid, final String pidType, final String entityType) {
- String type;
- switch (entityType) {
- case "publication":
- type = "50|";
- break;
- case "dataset":
- type = "60|";
- break;
- case "unknown":
- type = "70|";
- break;
- default:
- throw new IllegalArgumentException("unexpected value " + entityType);
- }
- return type
- + DHPUtils
- .md5(
- String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim()));
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossrefClient.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossrefClient.java
deleted file mode 100644
index 9ace7b37a..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossrefClient.java
+++ /dev/null
@@ -1,90 +0,0 @@
-
-package eu.dnetlib.dhp.provision.update;
-
-import java.io.ByteArrayOutputStream;
-import java.util.zip.Inflater;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.io.IOUtils;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-
-import com.google.gson.JsonElement;
-import com.google.gson.JsonParser;
-
-import eu.dnetlib.dhp.provision.scholix.ScholixResource;
-
-public class CrossrefClient {
-
- private String host;
- private String index = "crossref";
- private String indexType = "item";
-
- public CrossrefClient(String host) {
- this.host = host;
- }
-
- public String getHost() {
- return host;
- }
-
- public void setHost(String host) {
- this.host = host;
- }
-
- public String getIndex() {
- return index;
- }
-
- public void setIndex(String index) {
- this.index = index;
- }
-
- public String getIndexType() {
- return indexType;
- }
-
- public void setIndexType(String indexType) {
- this.indexType = indexType;
- }
-
- private static String decompressBlob(final String blob) {
- try {
- byte[] byteArray = Base64.decodeBase64(blob.getBytes());
- final Inflater decompresser = new Inflater();
- decompresser.setInput(byteArray);
- final ByteArrayOutputStream bos = new ByteArrayOutputStream(byteArray.length);
- byte[] buffer = new byte[8192];
- while (!decompresser.finished()) {
- int size = decompresser.inflate(buffer);
- bos.write(buffer, 0, size);
- }
- decompresser.end();
- return bos.toString();
- } catch (Throwable e) {
- throw new RuntimeException("Wrong record:" + blob, e);
- }
- }
-
- public ScholixResource getResourceByDOI(final String doi) {
- try (CloseableHttpClient client = HttpClients.createDefault()) {
- HttpGet httpGet = new HttpGet(
- String
- .format(
- "http://%s:9200/%s/%s/%s", host, index, indexType, doi.replaceAll("/", "%2F")));
- CloseableHttpResponse response = client.execute(httpGet);
- String json = IOUtils.toString(response.getEntity().getContent());
- if (json.contains("blob")) {
- JsonParser p = new JsonParser();
- final JsonElement root = p.parse(json);
- json = decompressBlob(
- root.getAsJsonObject().get("_source").getAsJsonObject().get("blob").getAsString());
- }
- return CrossRefParserJSON.parseRecord(json);
- } catch (Throwable e) {
- return null;
- }
- }
-}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java
deleted file mode 100644
index 10426b29c..000000000
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java
+++ /dev/null
@@ -1,229 +0,0 @@
-
-package eu.dnetlib.dhp.provision.update;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.commons.lang3.StringUtils;
-
-import com.jayway.jsonpath.JsonPath;
-
-import eu.dnetlib.dhp.provision.scholix.*;
-import eu.dnetlib.dhp.utils.DHPUtils;
-import eu.dnetlib.scholexplorer.relation.RelInfo;
-import eu.dnetlib.scholexplorer.relation.RelationMapper;
-
-public class Datacite2Scholix {
-
- private String rootPath = "$.attributes";
- final RelationMapper relationMapper;
-
- public Datacite2Scholix(RelationMapper relationMapper) {
- this.relationMapper = relationMapper;
- }
-
- public List generateScholixFromJson(final String dJson) {
- List