Merge branch 'beta' into peer_reviewed

This commit is contained in:
Claudio Atzori 2023-10-03 10:52:23 +02:00
commit b446a9ed98
7 changed files with 110 additions and 79 deletions

View File

@ -12,6 +12,7 @@ import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
@ -30,12 +31,16 @@ import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import eu.dnetlib.dhp.utils.DHPUtils;
import scala.Tuple2;
public class CreateActionSetSparkJob implements Serializable {
public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations";
public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations";
private static final String ID_PREFIX = "50|doi_________::";
private static final String DOI_PREFIX = "50|doi_________::";
private static final String PMID_PREFIX = "50|pmid________::";
private static final String TRUST = "0.91";
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
@ -84,33 +89,55 @@ public class CreateActionSetSparkJob implements Serializable {
private static void extractContent(SparkSession spark, String inputPath, String outputPath,
boolean shouldDuplicateRels) {
spark
getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, "COCI")
.union(getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, "POCI"))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);
}
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath,
boolean shouldDuplicateRels, String prefix) {
return spark
.read()
.textFile(inputPath + "/*")
.textFile(inputPath + "/" + prefix + "/" + prefix + "_JSON/*")
.map(
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
Encoders.bean(COCI.class))
.flatMap(
(FlatMapFunction<COCI, Relation>) value -> createRelation(value, shouldDuplicateRels).iterator(),
(FlatMapFunction<COCI, Relation>) value -> createRelation(
value, shouldDuplicateRels, prefix)
.iterator(),
Encoders.bean(Relation.class))
.filter((FilterFunction<Relation>) value -> value != null)
.toJavaRDD()
.map(p -> new AtomicAction(p.getClass(), p))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
}
private static List<Relation> createRelation(COCI value, boolean duplicate) {
private static List<Relation> createRelation(COCI value, boolean duplicate, String p) {
List<Relation> relationList = new ArrayList<>();
String prefix;
String citing;
String cited;
if (p.equals("COCI")) {
prefix = DOI_PREFIX;
citing = prefix
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCiting()));
cited = prefix
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCited()));
String citing = ID_PREFIX
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCiting()));
final String cited = ID_PREFIX
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCited()));
} else {
prefix = PMID_PREFIX;
citing = prefix
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("pmid", value.getCiting()));
cited = prefix
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("pmid", value.getCited()));
}
if (!citing.equals(cited)) {
relationList
@ -120,7 +147,7 @@ public class CreateActionSetSparkJob implements Serializable {
cited, ModelConstants.CITES));
if (duplicate && value.getCiting().endsWith(".refs")) {
citing = ID_PREFIX + IdentifierFactory
citing = prefix + IdentifierFactory
.md5(
CleaningFunctions
.normalizePidValue(
@ -132,59 +159,30 @@ public class CreateActionSetSparkJob implements Serializable {
return relationList;
}
private static Collection<Relation> getRelations(String citing, String cited) {
return Arrays
.asList(
getRelation(citing, cited, ModelConstants.CITES),
getRelation(cited, citing, ModelConstants.IS_CITED_BY));
}
public static Relation getRelation(
String source,
String target,
String relclass) {
Relation r = new Relation();
r.setCollectedfrom(getCollectedFrom());
r.setSource(source);
r.setTarget(target);
r.setRelClass(relclass);
r.setRelType(ModelConstants.RESULT_RESULT);
r.setSubRelType(ModelConstants.CITATION);
r
.setDataInfo(
getDataInfo());
return r;
return OafMapperUtils
.getRelation(
source,
target,
ModelConstants.RESULT_RESULT,
ModelConstants.CITATION,
relclass,
Arrays
.asList(
OafMapperUtils.keyValue(ModelConstants.OPENOCITATIONS_ID, ModelConstants.OPENOCITATIONS_NAME)),
OafMapperUtils
.dataInfo(
false, null, false, false,
OafMapperUtils
.qualifier(
OPENCITATIONS_CLASSID, OPENCITATIONS_CLASSNAME,
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
TRUST),
null);
}
public static List<KeyValue> getCollectedFrom() {
KeyValue kv = new KeyValue();
kv.setKey(ModelConstants.OPENOCITATIONS_ID);
kv.setValue(ModelConstants.OPENOCITATIONS_NAME);
return Arrays.asList(kv);
}
public static DataInfo getDataInfo() {
DataInfo di = new DataInfo();
di.setInferred(false);
di.setDeletedbyinference(false);
di.setTrust(TRUST);
di
.setProvenanceaction(
getQualifier(OPENCITATIONS_CLASSID, OPENCITATIONS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS));
return di;
}
public static Qualifier getQualifier(String class_id, String class_name,
String qualifierSchema) {
Qualifier pa = new Qualifier();
pa.setClassid(class_id);
pa.setClassname(class_name);
pa.setSchemeid(qualifierSchema);
pa.setSchemename(qualifierSchema);
return pa;
}
}

View File

@ -45,6 +45,9 @@ public class GetOpenCitationsRefs implements Serializable {
final String hdfsNameNode = parser.get("hdfsNameNode");
log.info("hdfsNameNode {}", hdfsNameNode);
final String prefix = parser.get("prefix");
log.info("prefix {}", prefix);
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
@ -53,30 +56,31 @@ public class GetOpenCitationsRefs implements Serializable {
GetOpenCitationsRefs ocr = new GetOpenCitationsRefs();
for (String file : inputFile) {
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem);
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem, prefix);
}
}
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem)
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem, String prefix)
throws IOException {
final Path path = new Path(inputFile);
FSDataInputStream oc_zip = fileSystem.open(path);
int count = 1;
// 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;
count++;
// fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
fileName = fileName.substring(0, fileName.lastIndexOf("."));
// count++;
try (
FSDataOutputStream out = fileSystem
.create(new Path(workingPath + "/COCI/" + fileName + ".gz"));
.create(new Path(workingPath + "/" + prefix + "/" + fileName + ".gz"));
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
IOUtils.copy(zis, gzipOs);

View File

@ -49,6 +49,9 @@ public class ReadCOCI implements Serializable {
final String workingPath = parser.get("workingPath");
log.info("workingPath {}", workingPath);
final String format = parser.get("format");
log.info("format {}", format);
SparkConf sconf = new SparkConf();
final String delimiter = Optional
@ -64,13 +67,14 @@ public class ReadCOCI implements Serializable {
workingPath,
inputFile,
outputPath,
delimiter);
delimiter,
format);
});
}
private static void doRead(SparkSession spark, String workingPath, String[] inputFiles,
String outputPath,
String delimiter) throws IOException {
String delimiter, String format) throws IOException {
for (String inputFile : inputFiles) {
String p_string = workingPath + "/" + inputFile + ".gz";
@ -87,9 +91,15 @@ public class ReadCOCI implements Serializable {
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.setOci(row.getString(0));
coci.setCiting(row.getString(1));
coci.setCited(row.getString(2));
return coci;
}, Encoders.bean(COCI.class))
.write()

View File

@ -21,5 +21,10 @@
"paramLongName": "shouldDuplicateRels",
"paramDescription": "the hdfs name node",
"paramRequired": false
},{
"paramName": "p",
"paramLongName": "prefix",
"paramDescription": "the hdfs name node",
"paramRequired": true
}
]

View File

@ -16,5 +16,11 @@
"paramLongName": "hdfsNameNode",
"paramDescription": "the hdfs name node",
"paramRequired": true
},
{
"paramName": "p",
"paramLongName": "prefix",
"paramDescription": "COCI or POCI",
"paramRequired": true
}
]

View File

@ -30,7 +30,12 @@
"paramLongName": "inputFile",
"paramDescription": "the hdfs name node",
"paramRequired": true
}
}, {
"paramName": "f",
"paramLongName": "format",
"paramDescription": "the hdfs name node",
"paramRequired": true
}
]

View File

@ -46,7 +46,7 @@
</configuration>
<exec>download.sh</exec>
<argument>${filelist}</argument>
<argument>${workingPath}/Original</argument>
<argument>${workingPath}/${prefix}/Original</argument>
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
<file>download.sh</file>
<capture-output/>
@ -59,7 +59,8 @@
<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}</arg>
<arg>--workingPath</arg><arg>${workingPath}/${prefix}</arg>
<arg>--prefix</arg><arg>${prefix}</arg>
</java>
<ok to="read"/>
<error to="Kill"/>
@ -82,10 +83,11 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}/COCI</arg>
<arg>--outputPath</arg><arg>${workingPath}/COCI_JSON/</arg>
<arg>--workingPath</arg><arg>${workingPath}/${prefix}/${prefix}</arg>
<arg>--outputPath</arg><arg>${workingPath}/${prefix}/${prefix}_JSON/</arg>
<arg>--delimiter</arg><arg>${delimiter}</arg>
<arg>--inputFile</arg><arg>${inputFileCoci}</arg>
<arg>--format</arg><arg>${prefix}</arg>
</spark>
<ok to="create_actionset"/>
<error to="Kill"/>
@ -108,8 +110,9 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/COCI_JSON</arg>
<arg>--inputPath</arg><arg>${workingPath}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--prefix</arg><arg>${prefix}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>