diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala
index 096217a55..d1f6d8613 100644
--- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala
+++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala
@@ -14,7 +14,7 @@ import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.matching.Regex
-case class CrossrefDT(doi: String, json:String) {}
+case class CrossrefDT(doi: String, json:String, timestamp: Long) {}
case class mappingAffiliation(name: String) {}
@@ -265,18 +265,20 @@ case object Crossref2Oaf {
}
- def generateRelation(sourceId:String, targetId:String, nsPrefix:String) :Relation = {
+ def generateRelation(sourceId:String, targetId:String, relClass:String) :Relation = {
val r = new Relation
r.setSource(sourceId)
- r.setTarget(s"40|$nsPrefix::$targetId")
+ r.setTarget(targetId)
r.setRelType("resultProject")
- r.setRelClass("isProducedBy")
+ r.setRelClass(relClass)
r.setSubRelType("outcome")
r.setCollectedfrom(List(cf).asJava)
r.setDataInfo(di)
r.setLastupdatetimestamp(ts)
r
+
+
}
@@ -284,12 +286,18 @@ case object Crossref2Oaf {
if (funder.award.isDefined && funder.award.get.nonEmpty)
funder.award.get.map(extractField).filter(a => a!= null && a.nonEmpty).foreach(
award => {
- val targetId = DHPUtils.md5(award)
- queue += generateRelation(sourceId, targetId, nsPrefix)
+ val targetId = getProjectId(nsPrefix, DHPUtils.md5(award))
+ queue += generateRelation(sourceId, targetId , "isProducedBy")
+ queue += generateRelation(targetId , sourceId, "produces")
}
)
}
+ def getProjectId (nsPrefix:String, targetId:String):String = {
+ "40|$nsPrefix::$targetId"
+ }
+
+
if (funders != null)
funders.foreach(funder => {
if (funder.DOI.isDefined && funder.DOI.get.nonEmpty) {
@@ -310,22 +318,33 @@ case object Crossref2Oaf {
case "10.13039/501100002341" => generateSimpleRelationFromAward(funder, "aka_________", a => a)
case "10.13039/501100001602" => generateSimpleRelationFromAward(funder, "aka_________", a => a.replace("SFI", ""))
case "10.13039/501100000923" => generateSimpleRelationFromAward(funder, "arc_________", a => a)
- case "10.13039/501100000038"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "nserc_______" )
- case "10.13039/501100000155"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "sshrc_______" )
- case "10.13039/501100000024"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "cihr________" )
+ case "10.13039/501100000038"=> val targetId = getProjectId("nserc_______" , "1e5e62235d094afd01cd56e65112fc63")
+ queue += generateRelation(sourceId, targetId, "isProducedBy" )
+ queue += generateRelation(targetId, sourceId, "produces" )
+ case "10.13039/501100000155"=> val targetId = getProjectId("sshrc_______" , "1e5e62235d094afd01cd56e65112fc63")
+ queue += generateRelation(sourceId,targetId, "isProducedBy" )
+ queue += generateRelation(targetId,sourceId, "produces" )
+ case "10.13039/501100000024"=> val targetId = getProjectId("cihr________" , "1e5e62235d094afd01cd56e65112fc63")
+ queue += generateRelation(sourceId,targetId, "isProducedBy" )
+ queue += generateRelation(targetId,sourceId, "produces" )
case "10.13039/501100002848" => generateSimpleRelationFromAward(funder, "conicytf____", a => a)
case "10.13039/501100003448" => generateSimpleRelationFromAward(funder, "gsrt________", extractECAward)
case "10.13039/501100010198" => generateSimpleRelationFromAward(funder, "sgov________", a=>a)
case "10.13039/501100004564" => generateSimpleRelationFromAward(funder, "mestd_______", extractECAward)
case "10.13039/501100003407" => generateSimpleRelationFromAward(funder, "miur________", a=>a)
- queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "miur________" )
+ val targetId = getProjectId("miur________" , "1e5e62235d094afd01cd56e65112fc63")
+ queue += generateRelation(sourceId,targetId, "isProducedBy" )
+ queue += generateRelation(targetId,sourceId, "produces" )
case "10.13039/501100006588" |
"10.13039/501100004488" => generateSimpleRelationFromAward(funder, "irb_hr______", a=>a.replaceAll("Project No.", "").replaceAll("HRZZ-","") )
case "10.13039/501100006769"=> generateSimpleRelationFromAward(funder, "rsf_________", a=>a)
case "10.13039/501100001711"=> generateSimpleRelationFromAward(funder, "snsf________", snsfRule)
case "10.13039/501100004410"=> generateSimpleRelationFromAward(funder, "tubitakf____", a =>a)
case "10.10.13039/100004440"=> generateSimpleRelationFromAward(funder, "wt__________", a =>a)
- case "10.13039/100004440"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" )
+ case "10.13039/100004440"=> val targetId = getProjectId("wt__________" , "1e5e62235d094afd01cd56e65112fc63")
+ queue += generateRelation(sourceId,targetId, "isProducedBy" )
+ queue += generateRelation(targetId,sourceId, "produces" )
+
case _ => logger.debug("no match for "+funder.DOI.get )
@@ -341,7 +360,9 @@ case object Crossref2Oaf {
case "The French National Research Agency (ANR)" |
"The French National Research Agency" => generateSimpleRelationFromAward(funder, "anr_________", a => a)
case "CONICYT, Programa de FormaciĆ³n de Capital Humano Avanzado" => generateSimpleRelationFromAward(funder, "conicytf____", extractECAward)
- case "Wellcome Trust Masters Fellowship" => queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" )
+ case "Wellcome Trust Masters Fellowship" => val targetId = getProjectId("wt__________", "1e5e62235d094afd01cd56e65112fc63")
+ queue += generateRelation(sourceId, targetId, "isProducedBy" )
+ queue += generateRelation(targetId, sourceId, "produces" )
case _ => logger.debug("no match for "+funder.name )
}
diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala
index fac4c90b4..08319058c 100644
--- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala
+++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala
@@ -29,69 +29,90 @@ object SparkMapDumpIntoOAF {
.appName(SparkMapDumpIntoOAF.getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
+ implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
implicit val mapEncoderRelatons: Encoder[Relation] = Encoders.kryo[Relation]
implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo[OafDataset]
val sc = spark.sparkContext
val targetPath = parser.get("targetPath")
+ import spark.implicits._
- sc.sequenceFile(parser.get("sourcePath"), classOf[IntWritable], classOf[Text])
- .map(k => k._2.toString).map(CrossrefImporter.decompressBlob)
- .flatMap(k => Crossref2Oaf.convert(k)).saveAsObjectFile(s"${targetPath}/mixObject")
-
- val inputRDD = sc.objectFile[Oaf](s"${targetPath}/mixObject").filter(p=> p!= null)
-
- val distinctPubs:RDD[Publication] = inputRDD.filter(k => k != null && k.isInstanceOf[Publication])
- .map(k => k.asInstanceOf[Publication]).map { p: Publication => Tuple2(p.getId, p) }.reduceByKey { case (p1: Publication, p2: Publication) =>
- var r = if (p1 == null) p2 else p1
- if (p1 != null && p2 != null) {
- if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
- if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
- r = p2
- else
- r = p1
- } else {
- r = if (p1.getLastupdatetimestamp == null) p2 else p1
- }
- }
- r
- }.map(_._2)
-
- val pubs:Dataset[Publication] = spark.createDataset(distinctPubs)
- pubs.write.mode(SaveMode.Overwrite).save(s"${targetPath}/publication")
+ spark.read.load(parser.get("sourcePath")).as[CrossrefDT]
+ .flatMap(k => Crossref2Oaf.convert(k.json))
+ .filter(o => o != null)
+ .write.mode(SaveMode.Overwrite).save(s"$targetPath/mixObject")
- val distincDatasets:RDD[OafDataset] = inputRDD.filter(k => k != null && k.isInstanceOf[OafDataset])
- .map(k => k.asInstanceOf[OafDataset]).map(p => Tuple2(p.getId, p)).reduceByKey { case (p1: OafDataset, p2: OafDataset) =>
- var r = if (p1 == null) p2 else p1
- if (p1 != null && p2 != null) {
- if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
- if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
- r = p2
- else
- r = p1
- } else {
- r = if (p1.getLastupdatetimestamp == null) p2 else p1
- }
- }
- r
- }.map(_._2)
+ val ds:Dataset[Oaf] = spark.read.load(s"$targetPath/mixObject").as[Oaf]
- spark.createDataset(distincDatasets).write.mode(SaveMode.Overwrite).save(s"${targetPath}/dataset")
+ ds.filter(o => o.isInstanceOf[Publication]).map(o => o.asInstanceOf[Publication]).write.save(s"$targetPath/publication")
+
+ ds.filter(o => o.isInstanceOf[Relation]).map(o => o.asInstanceOf[Relation]).write.save(s"$targetPath/relation")
+
+ ds.filter(o => o.isInstanceOf[OafDataset]).map(o => o.asInstanceOf[OafDataset]).write.save(s"$targetPath/dataset")
- val distinctRels =inputRDD.filter(k => k != null && k.isInstanceOf[Relation])
- .map(k => k.asInstanceOf[Relation]).map(r=> (s"${r.getSource}::${r.getTarget}",r))
- .reduceByKey { case (p1: Relation, p2: Relation) =>
- if (p1 == null) p2 else p1
- }.map(_._2)
-
- val rels: Dataset[Relation] = spark.createDataset(distinctRels)
-
- rels.write.mode(SaveMode.Overwrite).save(s"${targetPath}/relations")
+//
+//
+//
+// sc.sequenceFile(parser.get("sourcePath"), classOf[IntWritable], classOf[Text])
+// .map(k => k._2.toString).map(CrossrefImporter.decompressBlob)
+// .flatMap(k => Crossref2Oaf.convert(k)).saveAsObjectFile(s"${targetPath}/mixObject")
+//
+// val inputRDD = sc.objectFile[Oaf](s"${targetPath}/mixObject").filter(p=> p!= null)
+//
+// val distinctPubs:RDD[Publication] = inputRDD.filter(k => k != null && k.isInstanceOf[Publication])
+// .map(k => k.asInstanceOf[Publication]).map { p: Publication => Tuple2(p.getId, p) }.reduceByKey { case (p1: Publication, p2: Publication) =>
+// var r = if (p1 == null) p2 else p1
+// if (p1 != null && p2 != null) {
+// if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
+// if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
+// r = p2
+// else
+// r = p1
+// } else {
+// r = if (p1.getLastupdatetimestamp == null) p2 else p1
+// }
+// }
+// r
+// }.map(_._2)
+//
+// val pubs:Dataset[Publication] = spark.createDataset(distinctPubs)
+// pubs.write.mode(SaveMode.Overwrite).save(s"${targetPath}/publication")
+//
+//
+// val distincDatasets:RDD[OafDataset] = inputRDD.filter(k => k != null && k.isInstanceOf[OafDataset])
+// .map(k => k.asInstanceOf[OafDataset]).map(p => Tuple2(p.getId, p)).reduceByKey { case (p1: OafDataset, p2: OafDataset) =>
+// var r = if (p1 == null) p2 else p1
+// if (p1 != null && p2 != null) {
+// if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
+// if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
+// r = p2
+// else
+// r = p1
+// } else {
+// r = if (p1.getLastupdatetimestamp == null) p2 else p1
+// }
+// }
+// r
+// }.map(_._2)
+//
+// spark.createDataset(distincDatasets).write.mode(SaveMode.Overwrite).save(s"${targetPath}/dataset")
+//
+//
+//
+// val distinctRels =inputRDD.filter(k => k != null && k.isInstanceOf[Relation])
+// .map(k => k.asInstanceOf[Relation]).map(r=> (s"${r.getSource}::${r.getTarget}",r))
+// .reduceByKey { case (p1: Relation, p2: Relation) =>
+// if (p1 == null) p2 else p1
+// }.map(_._2)
+//
+// val rels: Dataset[Relation] = spark.createDataset(distinctRels)
+//
+// rels.write.mode(SaveMode.Overwrite).save(s"${targetPath}/relations")
}
diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml
index be4a45afe..a9cc9ea3c 100644
--- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml
+++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml
@@ -16,10 +16,10 @@
sparkExecutorCores
number of cores used by single executor
-
- timestamp
- Timestamp for incremental Harvesting
-
+
+
+
+
@@ -30,29 +30,29 @@
Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
-
-
-
-
-
-
-
-
+
+
+
+
+
+
+
+
-
-
- ${jobTracker}
- ${nameNode}
- eu.dnetlib.doiboost.crossref.CrossrefImporter
- -t${workingPath}/input/crossref/index_dump_1
- -n${nameNode}
- -ts${timestamp}
-
-
-
-
+
+
+
+
+
+
+
+
+
+
+
+
@@ -68,7 +68,7 @@
--driver-memory=${sparkDriverMemory}
${sparkExtraOPT}
- --sourcePath${workingPath}/input/crossref/index_dump,${workingPath}/input/crossref/index_dump_1,${workingPath}/crossref/index_dump
+ --sourcePath${workingPath}/input/crossref/crossref_ds
--targetPath${workingPath}/input/crossref
--masteryarn-cluster
@@ -78,26 +78,26 @@
-
-
- yarn-cluster
- cluster
- ExtractCrossrefToOAF
- eu.dnetlib.doiboost.crossref.CrossrefDataset
- dhp-doiboost-${projectVersion}.jar
-
- --executor-memory=${sparkExecutorMemory}
- --executor-cores=${sparkExecutorCores}
- --driver-memory=${sparkDriverMemory}
- ${sparkExtraOPT}
-
- --sourcePath/data/doiboost/crossref/cr_dataset
- --targetPath/data/doiboost/crossref/crossrefDataset
- --masteryarn-cluster
-
-
-
-
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/dhp/doiboost/QueryTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/dhp/doiboost/QueryTest.scala
deleted file mode 100644
index 61c1f5111..000000000
--- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/dhp/doiboost/QueryTest.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-package eu.dnetlib.dhp.doiboost
-
-import eu.dnetlib.dhp.schema.oaf.{Publication, Relation}
-import org.apache.spark.SparkContext
-import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
-import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
-import org.json4s
-import org.json4s.DefaultFormats
-import org.json4s.jackson.JsonMethods._
-
-import scala.collection.JavaConverters._
-class QueryTest {
-
- def extract_payload(input:String) :String = {
-
- implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
- lazy val json: json4s.JValue = parse(input)
-
-
- compact(render((json \ "payload")))
-
-
-
- }
-
-
- def has_ands(r:Relation) :Boolean = {
-
- r.getCollectedfrom!= null && r.getCollectedfrom.asScala.count(k => k.getValue.contains("Australian")) > 0
-
- }
-
- def hasInstanceWithUrl(p:Publication):Boolean = {
- val c = p.getInstance.asScala.map(i => i.getUrl!= null && !i.getUrl.isEmpty).size
- !(!p.getInstance.isEmpty && c == p.getInstance().size)
- }
-
-
- def hasNullAccessRights(p:Publication):Boolean = {
- val c = p.getInstance.asScala.map(i => i.getAccessright!= null && i.getAccessright.getClassname.nonEmpty).size
- !p.getInstance.isEmpty && c == p.getInstance().size()
- }
-
-
- def myQuery(spark:SparkSession, sc:SparkContext): Unit = {
- implicit val mapEncoderPub: Encoder[Publication] = Encoders.kryo[Publication]
-
-
-
- val mapper = new ObjectMapper()
- mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
-
-
- val ds:Dataset[Publication] = spark.read.load("/tmp/p").as[Publication]
-
-
-
- ds.filter(p =>p.getBestaccessright!= null && p.getBestaccessright.getClassname.nonEmpty).count()
-
-
- }
-
-}
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java
index e9f783670..5155d0242 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java
+++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java
@@ -1,10 +1,7 @@
package eu.dnetlib.dhp.oa.graph.clean;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.Objects;
-import java.util.Set;
+import java.util.*;
import java.util.function.Function;
import java.util.stream.Collectors;
@@ -18,7 +15,9 @@ import eu.dnetlib.dhp.schema.oaf.*;
public class CleaningFunctions {
+ public static final String DOI_URL_PREFIX_REGEX = "(^http(s?):\\/\\/)(((dx\\.)?doi\\.org)|(handle\\.test\\.datacite\\.org))\\/";
public static final String ORCID_PREFIX_REGEX = "^http(s?):\\/\\/orcid\\.org\\/";
+ public static final String CLEANING_REGEX = "(?:\\n|\\r|\\t)";
public static final Set PID_BLACKLIST = new HashSet<>();
@@ -79,7 +78,7 @@ public class CleaningFunctions {
return value;
}
- protected static T fixDefaults(T value) {
+ public static T fixDefaults(T value) {
if (value instanceof Datasource) {
// nothing to clean here
} else if (value instanceof Project) {
@@ -112,6 +111,29 @@ public class CleaningFunctions {
.filter(sp -> StringUtils.isNotBlank(sp.getValue()))
.filter(sp -> Objects.nonNull(sp.getQualifier()))
.filter(sp -> StringUtils.isNotBlank(sp.getQualifier().getClassid()))
+ .map(CleaningFunctions::cleanValue)
+ .collect(Collectors.toList()));
+ }
+ if (Objects.nonNull(r.getTitle())) {
+ r
+ .setTitle(
+ r
+ .getTitle()
+ .stream()
+ .filter(Objects::nonNull)
+ .filter(sp -> StringUtils.isNotBlank(sp.getValue()))
+ .map(CleaningFunctions::cleanValue)
+ .collect(Collectors.toList()));
+ }
+ if (Objects.nonNull(r.getDescription())) {
+ r
+ .setDescription(
+ r
+ .getDescription()
+ .stream()
+ .filter(Objects::nonNull)
+ .filter(sp -> StringUtils.isNotBlank(sp.getValue()))
+ .map(CleaningFunctions::cleanValue)
.collect(Collectors.toList()));
}
if (Objects.nonNull(r.getPid())) {
@@ -125,10 +147,7 @@ public class CleaningFunctions {
.filter(sp -> !PID_BLACKLIST.contains(sp.getValue().trim().toLowerCase()))
.filter(sp -> Objects.nonNull(sp.getQualifier()))
.filter(sp -> StringUtils.isNotBlank(sp.getQualifier().getClassid()))
- .map(sp -> {
- sp.setValue(StringUtils.trim(sp.getValue()));
- return sp;
- })
+ .map(CleaningFunctions::normalizePidValue)
.collect(Collectors.toList()));
}
if (Objects.isNull(r.getResourcetype()) || StringUtils.isBlank(r.getResourcetype().getClassid())) {
@@ -211,6 +230,16 @@ public class CleaningFunctions {
return value;
}
+ protected static StructuredProperty cleanValue(StructuredProperty s) {
+ s.setValue(s.getValue().replaceAll(CLEANING_REGEX, " "));
+ return s;
+ }
+
+ protected static Field cleanValue(Field s) {
+ s.setValue(s.getValue().replaceAll(CLEANING_REGEX, " "));
+ return s;
+ }
+
// HELPERS
private static void fixVocabName(Qualifier q, String vocabularyName) {
@@ -226,4 +255,24 @@ public class CleaningFunctions {
classid, classname, scheme, scheme);
}
+ /**
+ * Utility method that normalises PID values on a per-type basis.
+ * @param pid the PID whose value will be normalised.
+ * @return the PID containing the normalised value.
+ */
+ public static StructuredProperty normalizePidValue(StructuredProperty pid) {
+ String value = Optional
+ .ofNullable(pid.getValue())
+ .map(String::trim)
+ .orElseThrow(() -> new IllegalArgumentException("PID value cannot be empty"));
+ switch (pid.getQualifier().getClassid()) {
+
+ // TODO add cleaning for more PID types as needed
+ case "doi":
+ pid.setValue(value.toLowerCase().replaceAll(DOI_URL_PREFIX_REGEX, ""));
+ break;
+ }
+ return pid;
+ }
+
}
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
index 95bea74a2..e19432f29 100644
--- 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
@@ -12,6 +12,7 @@ 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 {
@@ -41,71 +42,71 @@ public class MakeTar implements Serializable {
FileSystem fileSystem = FileSystem.get(conf);
- makeTArArchive(fileSystem, inputPath, outputPath);
+ 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();
- }
+// 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
index 1dcbf6ccc..2e2b7bc26 100644
--- 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
@@ -73,7 +73,7 @@ public class SendToZenodoHDFS implements Serializable {
}
zenodoApiClient.sendMretadata(metadata);
- zenodoApiClient.publish();
+// zenodoApiClient.publish();
}
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml
index 6d7056503..fd8c773c9 100644
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml
+++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml
@@ -8,10 +8,10 @@
targetPath
the target path
-
- metadata
- the metadata
-
+
+
+
+
@@ -34,20 +34,20 @@
-
-
- eu.dnetlib.dhp.export.zenodo.SendToZenodoHDFS
- --hdfsPath/user/dnet.scholexplorer/scholix/provision/scholix.tar/scholix-2020-10-16.tar
- --nameNode${nameNode}
- --accessTokenb6ddrY6b77WxcDEevn9gqVE5sL5sDNjdUijt75W3o7cQo5vpFFI48dMiu8Gv
- --connectionUrlhttps://zenodo.org/api/deposit/depositions
- --metadata${metadata}
- --conceptRecordId1200252
- --newDepositionfalse
-
-
-
-
+
+
+
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file