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