forked from D-Net/dnet-hadoop
[OCNEW] first implementation
This commit is contained in:
parent
3cd5590f3b
commit
fd34372c40
|
@ -28,6 +28,7 @@ import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
|
|||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.*;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
|
@ -37,12 +38,6 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations";
|
||||
public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations";
|
||||
|
||||
// DOI-to-DOI citations
|
||||
public static final String COCI = "COCI";
|
||||
|
||||
// PMID-to-PMID citations
|
||||
public static final String POCI = "POCI";
|
||||
|
||||
private static final String DOI_PREFIX = "50|doi_________::";
|
||||
|
||||
private static final String PMID_PREFIX = "50|pmid________::";
|
||||
|
@ -95,22 +90,21 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
private static void extractContent(SparkSession spark, String inputPath, String outputPath,
|
||||
boolean shouldDuplicateRels) {
|
||||
|
||||
getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, COCI)
|
||||
.union(getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, POCI))
|
||||
getTextTextJavaPairRDD(spark, inputPath)
|
||||
// .union(getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, POCI))
|
||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
}
|
||||
|
||||
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath,
|
||||
boolean shouldDuplicateRels, String prefix) {
|
||||
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath) {
|
||||
return spark
|
||||
.read()
|
||||
.textFile(inputPath + "/" + prefix + "/" + prefix + "_JSON/*")
|
||||
.textFile(inputPath)
|
||||
.map(
|
||||
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
|
||||
Encoders.bean(COCI.class))
|
||||
.flatMap(
|
||||
(FlatMapFunction<COCI, Relation>) value -> createRelation(
|
||||
value, shouldDuplicateRels, prefix)
|
||||
value)
|
||||
.iterator(),
|
||||
Encoders.bean(Relation.class))
|
||||
.filter((FilterFunction<Relation>) Objects::nonNull)
|
||||
|
@ -121,34 +115,41 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
|
||||
}
|
||||
|
||||
private static List<Relation> createRelation(COCI value, boolean duplicate, String p) {
|
||||
private static List<Relation> createRelation(COCI value) {
|
||||
|
||||
List<Relation> relationList = new ArrayList<>();
|
||||
String prefix;
|
||||
|
||||
String citing;
|
||||
String cited;
|
||||
|
||||
switch (p) {
|
||||
case COCI:
|
||||
prefix = DOI_PREFIX;
|
||||
citing = prefix
|
||||
switch (value.getCiting_pid()) {
|
||||
case "doi":
|
||||
citing = DOI_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCiting()));
|
||||
cited = prefix
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCited()));
|
||||
break;
|
||||
case POCI:
|
||||
prefix = PMID_PREFIX;
|
||||
citing = prefix
|
||||
case "pmid":
|
||||
citing = PMID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCiting()));
|
||||
cited = prefix
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCited()));
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCiting()));
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Invalid prefix: " + p);
|
||||
throw new IllegalStateException("Invalid prefix: " + value.getCiting_pid());
|
||||
}
|
||||
|
||||
switch (value.getCited_pid()) {
|
||||
case "doi":
|
||||
cited = DOI_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCiting()));
|
||||
break;
|
||||
case "pmid":
|
||||
cited = PMID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCiting()));
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Invalid prefix: " + value.getCited_pid());
|
||||
}
|
||||
|
||||
if (!citing.equals(cited)) {
|
||||
|
@ -157,15 +158,6 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
getRelation(
|
||||
citing,
|
||||
cited, ModelConstants.CITES));
|
||||
|
||||
if (duplicate && value.getCiting().endsWith(".refs")) {
|
||||
citing = prefix + IdentifierFactory
|
||||
.md5(
|
||||
CleaningFunctions
|
||||
.normalizePidValue(
|
||||
"doi", value.getCiting().substring(0, value.getCiting().indexOf(".refs"))));
|
||||
relationList.add(getRelation(citing, cited, ModelConstants.CITES));
|
||||
}
|
||||
}
|
||||
|
||||
return relationList;
|
||||
|
|
|
@ -12,10 +12,7 @@ import java.util.zip.ZipInputStream;
|
|||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -37,17 +34,17 @@ public class GetOpenCitationsRefs implements Serializable {
|
|||
|
||||
parser.parseArgument(args);
|
||||
|
||||
final String[] inputFile = parser.get("inputFile").split(";");
|
||||
log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
// final String[] inputFile = parser.get("inputFile").split(";");
|
||||
// log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
|
||||
final String workingPath = parser.get("workingPath");
|
||||
log.info("workingPath {}", workingPath);
|
||||
final String inputPath = parser.get("inputPath");
|
||||
log.info("inputPath {}", inputPath);
|
||||
|
||||
final String hdfsNameNode = parser.get("hdfsNameNode");
|
||||
log.info("hdfsNameNode {}", hdfsNameNode);
|
||||
|
||||
final String prefix = parser.get("prefix");
|
||||
log.info("prefix {}", prefix);
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.defaultFS", hdfsNameNode);
|
||||
|
@ -56,41 +53,42 @@ public class GetOpenCitationsRefs implements Serializable {
|
|||
|
||||
GetOpenCitationsRefs ocr = new GetOpenCitationsRefs();
|
||||
|
||||
for (String file : inputFile) {
|
||||
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem, prefix);
|
||||
}
|
||||
ocr.doExtract(inputPath, outputPath, fileSystem);
|
||||
|
||||
}
|
||||
|
||||
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem, String prefix)
|
||||
private void doExtract(String inputPath, String outputPath, FileSystem fileSystem)
|
||||
throws IOException {
|
||||
|
||||
final Path path = new Path(inputFile);
|
||||
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
|
||||
.listFiles(
|
||||
new Path(inputPath), true);
|
||||
while (fileStatusListIterator.hasNext()) {
|
||||
LocatedFileStatus fileStatus = fileStatusListIterator.next();
|
||||
// do stuff with the file like ...
|
||||
FSDataInputStream oc_zip = fileSystem.open(fileStatus.getPath());
|
||||
try (ZipInputStream zis = new ZipInputStream(oc_zip)) {
|
||||
ZipEntry entry = null;
|
||||
while ((entry = zis.getNextEntry()) != null) {
|
||||
|
||||
FSDataInputStream oc_zip = fileSystem.open(path);
|
||||
if (!entry.isDirectory()) {
|
||||
String fileName = entry.getName();
|
||||
// fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
|
||||
fileName = fileName.substring(0, fileName.lastIndexOf("."));
|
||||
// count++;
|
||||
try (
|
||||
FSDataOutputStream out = fileSystem
|
||||
.create(new Path(outputPath + "/" + fileName + ".gz"));
|
||||
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
|
||||
|
||||
// int count = 1;
|
||||
try (ZipInputStream zis = new ZipInputStream(oc_zip)) {
|
||||
ZipEntry entry = null;
|
||||
while ((entry = zis.getNextEntry()) != null) {
|
||||
|
||||
if (!entry.isDirectory()) {
|
||||
String fileName = entry.getName();
|
||||
// fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
|
||||
fileName = fileName.substring(0, fileName.lastIndexOf("."));
|
||||
// count++;
|
||||
try (
|
||||
FSDataOutputStream out = fileSystem
|
||||
.create(new Path(workingPath + "/" + prefix + "/" + fileName + ".gz"));
|
||||
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
|
||||
|
||||
IOUtils.copy(zis, gzipOs);
|
||||
IOUtils.copy(zis, gzipOs);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,123 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.opencitations;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 29/02/24
|
||||
*/
|
||||
public class MapOCIdsInPids implements Serializable {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
|
||||
private static final String DELIMITER = ",";
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static void main(final String[] args) throws IOException, ParseException {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
MapOCIdsInPids.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/remap_parameters.json"))));
|
||||
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String inputPath = parser.get("inputPath");
|
||||
log.info("inputPath {}", inputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> mapIdentifiers(spark, inputPath, outputPath));
|
||||
|
||||
}
|
||||
|
||||
private static void mapIdentifiers(SparkSession spark, String inputPath, String outputPath) {
|
||||
Dataset<COCI> coci = spark
|
||||
.read()
|
||||
.textFile(inputPath + "/JSON")
|
||||
.map(
|
||||
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
|
||||
Encoders.bean(COCI.class));
|
||||
|
||||
Dataset<Tuple2<String, String>> correspondenceData = spark
|
||||
.read()
|
||||
.format("csv")
|
||||
.option("sep", DELIMITER)
|
||||
.option("inferSchema", "true")
|
||||
.option("header", "true")
|
||||
.option("quotes", "\"")
|
||||
.load(inputPath + "/correspondence/omid.zip")
|
||||
.repartition(5000)
|
||||
.flatMap((FlatMapFunction<Row, Tuple2<String, String>>) r -> {
|
||||
String ocIdentifier = r.getAs("omid");
|
||||
String[] correspondentIdentifiers = ((String) r.getAs("id")).split(" ");
|
||||
return Arrays
|
||||
.stream(correspondentIdentifiers)
|
||||
.map(ci -> new Tuple2<String, String>(ocIdentifier, ci))
|
||||
.collect(Collectors.toList())
|
||||
.iterator();
|
||||
}, Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||
|
||||
Dataset<COCI> mappedCitingDataset = coci
|
||||
.joinWith(correspondenceData, coci.col("citing").equalTo(correspondenceData.col("_1")), "left")
|
||||
.map((MapFunction<Tuple2<COCI, Tuple2<String, String>>, COCI>) t2 -> {
|
||||
String correspondent = t2._2()._2();
|
||||
t2._1().setCiting_pid(correspondent.substring(0, correspondent.indexOf(":")));
|
||||
t2._1().setCiting(correspondent.substring(correspondent.indexOf(":") + 1));
|
||||
return t2._1();
|
||||
}, Encoders.bean(COCI.class));
|
||||
|
||||
mappedCitingDataset
|
||||
.joinWith(correspondenceData, mappedCitingDataset.col("cited").equalTo(correspondenceData.col("_1")))
|
||||
.map((MapFunction<Tuple2<COCI, Tuple2<String, String>>, COCI>) t2 -> {
|
||||
String correspondent = t2._2()._2();
|
||||
t2._1().setCited_pid(correspondent.substring(0, correspondent.indexOf(":")));
|
||||
t2._1().setCited(correspondent.substring(correspondent.indexOf(":") + 1));
|
||||
return t2._1();
|
||||
}, Encoders.bean(COCI.class))
|
||||
.write()
|
||||
.mode(SaveMode.Append)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath);
|
||||
}
|
||||
|
||||
}
|
|
@ -12,10 +12,7 @@ import java.util.Optional;
|
|||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
|
@ -42,19 +39,21 @@ public class ReadCOCI implements Serializable {
|
|||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
final String[] inputFile = parser.get("inputFile").split(";");
|
||||
log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
final String hdfsNameNode = parser.get("hdfsNameNode");
|
||||
log.info("hdfsNameNode {}", hdfsNameNode);
|
||||
|
||||
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String workingPath = parser.get("workingPath");
|
||||
final String workingPath = parser.get("inputPath");
|
||||
log.info("workingPath {}", workingPath);
|
||||
|
||||
final String format = parser.get("format");
|
||||
log.info("format {}", format);
|
||||
|
||||
SparkConf sconf = new SparkConf();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.defaultFS", hdfsNameNode);
|
||||
|
||||
FileSystem fileSystem = FileSystem.get(conf);
|
||||
final String delimiter = Optional
|
||||
.ofNullable(parser.get("delimiter"))
|
||||
.orElse(DEFAULT_DELIMITER);
|
||||
|
@ -66,19 +65,20 @@ public class ReadCOCI implements Serializable {
|
|||
doRead(
|
||||
spark,
|
||||
workingPath,
|
||||
inputFile,
|
||||
fileSystem,
|
||||
outputPath,
|
||||
delimiter,
|
||||
format);
|
||||
delimiter);
|
||||
});
|
||||
}
|
||||
|
||||
private static void doRead(SparkSession spark, String workingPath, String[] inputFiles,
|
||||
private static void doRead(SparkSession spark, String workingPath, FileSystem fileSystem,
|
||||
String outputPath,
|
||||
String delimiter, String format) {
|
||||
|
||||
for (String inputFile : inputFiles) {
|
||||
String pString = workingPath + "/" + inputFile + ".gz";
|
||||
String delimiter) throws IOException {
|
||||
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
|
||||
.listFiles(
|
||||
new Path(workingPath), true);
|
||||
while (fileStatusListIterator.hasNext()) {
|
||||
LocatedFileStatus fileStatus = fileStatusListIterator.next();
|
||||
|
||||
Dataset<Row> cociData = spark
|
||||
.read()
|
||||
|
@ -87,26 +87,23 @@ public class ReadCOCI implements Serializable {
|
|||
.option("inferSchema", "true")
|
||||
.option("header", "true")
|
||||
.option("quotes", "\"")
|
||||
.load(pString)
|
||||
.load(fileStatus.getPath().toString())
|
||||
.repartition(100);
|
||||
|
||||
cociData.map((MapFunction<Row, COCI>) row -> {
|
||||
COCI coci = new COCI();
|
||||
if (format.equals("COCI")) {
|
||||
coci.setCiting(row.getString(1));
|
||||
coci.setCited(row.getString(2));
|
||||
} else {
|
||||
coci.setCiting(String.valueOf(row.getInt(1)));
|
||||
coci.setCited(String.valueOf(row.getInt(2)));
|
||||
}
|
||||
|
||||
coci.setCiting(row.getString(1));
|
||||
coci.setCited(row.getString(2));
|
||||
|
||||
coci.setOci(row.getString(0));
|
||||
|
||||
return coci;
|
||||
}, Encoders.bean(COCI.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.mode(SaveMode.Append)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath + inputFile);
|
||||
.json(outputPath);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -9,8 +9,10 @@ public class COCI implements Serializable {
|
|||
private String oci;
|
||||
|
||||
private String citing;
|
||||
private String citing_pid;
|
||||
|
||||
private String cited;
|
||||
private String cited_pid;
|
||||
|
||||
public String getOci() {
|
||||
return oci;
|
||||
|
@ -36,4 +38,19 @@ public class COCI implements Serializable {
|
|||
this.cited = cited;
|
||||
}
|
||||
|
||||
public String getCiting_pid() {
|
||||
return citing_pid;
|
||||
}
|
||||
|
||||
public void setCiting_pid(String citing_pid) {
|
||||
this.citing_pid = citing_pid;
|
||||
}
|
||||
|
||||
public String getCited_pid() {
|
||||
return cited_pid;
|
||||
}
|
||||
|
||||
public void setCited_pid(String cited_pid) {
|
||||
this.cited_pid = cited_pid;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,13 +1,13 @@
|
|||
[
|
||||
{
|
||||
"paramName": "if",
|
||||
"paramLongName": "inputFile",
|
||||
"paramName": "ip",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "wp",
|
||||
"paramLongName": "workingPath",
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
},
|
||||
|
@ -16,11 +16,5 @@
|
|||
"paramLongName": "hdfsNameNode",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "p",
|
||||
"paramLongName": "prefix",
|
||||
"paramDescription": "COCI or POCI",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
[
|
||||
{
|
||||
"paramName": "wp",
|
||||
"paramLongName": "workingPath",
|
||||
"paramName": "ip",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
|
@ -24,15 +24,9 @@
|
|||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "if",
|
||||
"paramLongName": "inputFile",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}, {
|
||||
"paramName": "f",
|
||||
"paramLongName": "format",
|
||||
}, {
|
||||
"paramName": "nn",
|
||||
"paramLongName": "hdfsNameNode",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}
|
||||
|
|
|
@ -27,7 +27,8 @@
|
|||
<case to="download">${wf:conf('resumeFrom') eq 'DownloadDump'}</case>
|
||||
<case to="extract">${wf:conf('resumeFrom') eq 'ExtractContent'}</case>
|
||||
<case to="read">${wf:conf('resumeFrom') eq 'ReadContent'}</case>
|
||||
<default to="create_actionset"/> <!-- first action to be done when downloadDump is to be performed -->
|
||||
<case to="create_actionset">${wf:conf('resumeFrom') eq 'CreateAS'}</case>
|
||||
<default to="deleteoutputpath"/> <!-- first action to be done when downloadDump is to be performed -->
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
|
@ -35,6 +36,15 @@
|
|||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="deleteoutputpath">
|
||||
<fs>
|
||||
<delete path='${inputPath}'/>
|
||||
<mkdir path='${inputPath}'/>
|
||||
</fs>
|
||||
<ok to="download"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="download">
|
||||
<shell xmlns="uri:oozie:shell-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
|
@ -47,7 +57,28 @@
|
|||
</configuration>
|
||||
<exec>download.sh</exec>
|
||||
<argument>${filelist}</argument>
|
||||
<argument>${workingPath}/${prefix}/Original</argument>
|
||||
<argument>${inputPath}/Original</argument>
|
||||
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
|
||||
<file>download.sh</file>
|
||||
<capture-output/>
|
||||
</shell>
|
||||
<ok to="download_correspondence"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<!-- downloads the correspondence from the omid and the pid (doi, pmid etc)-->
|
||||
<action name="download_correspondence">
|
||||
<shell xmlns="uri:oozie:shell-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>mapred.job.queue.name</name>
|
||||
<value>${queueName}</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<exec>download_corr.sh</exec>
|
||||
<argument>${filecorrespondence}</argument>
|
||||
<argument>${inputPath}/correspondence</argument>
|
||||
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
|
||||
<file>download.sh</file>
|
||||
<capture-output/>
|
||||
|
@ -60,9 +91,19 @@
|
|||
<java>
|
||||
<main-class>eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs</main-class>
|
||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||
<arg>--inputFile</arg><arg>${inputFile}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/${prefix}</arg>
|
||||
<arg>--prefix</arg><arg>${prefix}</arg>
|
||||
<arg>--inputPath</arg><arg>${inputPath}/Original</arg>
|
||||
<arg>--outputPath</arg><arg>${inputPath}/Extracted</arg>
|
||||
</java>
|
||||
<ok to="read"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="extract_correspondence">
|
||||
<java>
|
||||
<main-class>eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs</main-class>
|
||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||
<arg>--inputPath</arg><arg>${inputPath}/correspondence</arg>
|
||||
<arg>--outputPath</arg><arg>${inputPath}/correspondence_extracted</arg>
|
||||
</java>
|
||||
<ok to="read"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -85,11 +126,34 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/${prefix}/${prefix}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingPath}/${prefix}/${prefix}_JSON/</arg>
|
||||
<arg>--inputPath</arg><arg>${inputPath}/Extracted</arg>
|
||||
<arg>--outputPath</arg><arg>${inputPath}/JSON</arg>
|
||||
<arg>--delimiter</arg><arg>${delimiter}</arg>
|
||||
<arg>--inputFile</arg><arg>${inputFileCoci}</arg>
|
||||
<arg>--format</arg><arg>${prefix}</arg>
|
||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||
</spark>
|
||||
<ok to="remap"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="remap">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Produces the AS for OC</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.opencitations.MapOCIdsInPids</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${inputPath}</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPathExtraction}</arg>
|
||||
</spark>
|
||||
<ok to="create_actionset"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -112,7 +176,7 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--inputPath</arg><arg>${outputPathExtraction}</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
|
|
|
@ -0,0 +1,20 @@
|
|||
[
|
||||
{
|
||||
"paramName": "ip",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManged",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
Loading…
Reference in New Issue