Compare commits

..

38 Commits

Author SHA1 Message Date
Claudio Atzori 3c23d5f9bc Merge branch 'beta' into SWH_integration 2023-10-06 14:15:38 +02:00
Claudio Atzori 858931ccb6 [SWH] compress the output actionset 2023-10-06 14:03:33 +02:00
Claudio Atzori f759b18bca [SWH] aligned parameter name 2023-10-06 13:43:20 +02:00
Claudio Atzori eed9fe0902 code formatting 2023-10-06 12:31:17 +02:00
Claudio Atzori 7f27111b1f Merge branch 'importpoci' into beta 2023-10-06 12:23:28 +02:00
Claudio Atzori 73c49b8d26 Merge branch 'beta' into SWH_integration 2023-10-06 12:21:51 +02:00
Sandro La Bruzzo 13f332ce77 ignored jenv prop 2023-10-06 10:40:05 +02:00
Serafeim Chatzopoulos 1bb83b9188 Add prefix in SWH ID 2023-10-04 20:31:45 +03:00
Claudio Atzori ee8a39e7d2 cleanup and refinements 2023-10-04 12:32:05 +02:00
Serafeim Chatzopoulos e9f24df21c Move SWH API Key from constants to workflow param 2023-10-03 20:57:57 +03:00
Serafeim Chatzopoulos cae75fc75d Add SWH in the collectedFrom field 2023-10-03 16:55:10 +03:00
Serafeim Chatzopoulos b49a3ac9b2 Add actionsetsPath as a global WF param 2023-10-03 15:43:38 +03:00
Serafeim Chatzopoulos 24c43e0c60 Restructure workflow parameters 2023-10-03 15:11:58 +03:00
Serafeim Chatzopoulos 9f73d93e62 Add param for limiting repo Urls 2023-10-03 14:39:08 +03:00
Claudio Atzori f344ad76d0 Merge pull request 'extended existing code to import of POCI from open citation' (#340) from importpoci into beta
Reviewed-on: D-Net/dnet-hadoop#340
2023-10-03 10:52:11 +02:00
Claudio Atzori 5919e488dd Merge branch 'beta' into importpoci 2023-10-03 10:43:53 +02:00
Serafeim Chatzopoulos 839a8524e7 Add action for creating actionsets 2023-10-02 23:50:38 +03:00
Miriam Baglioni d7fccdc64b fixed paths in wf to match the req of the pathname 2023-10-02 14:10:57 +02:00
Miriam Baglioni 9898470b0e Addressing comments in D-Net/dnet-hadoop#340\#issuecomment-10592 2023-10-02 12:54:16 +02:00
Giambattista Bloisi c412dc162b Fix bug in conversion from dedup json model to Spark Dataset of Rows: list of strings contained the json escaped representation of the value instead of the plain value, this caused instanceTypeMatch failures because of the leading and trailing double quotes 2023-10-02 11:34:51 +02:00
Claudio Atzori 5d09b7db8b Merge pull request 'SparkPropagateRelation relations do not propagate deletedByInference and invisible' (#333) from consistency_keep_mergerels into beta
Reviewed-on: D-Net/dnet-hadoop#333
2023-10-02 11:27:57 +02:00
Claudio Atzori 7b403a920f Merge branch 'beta' into consistency_keep_mergerels 2023-10-02 11:26:00 +02:00
Claudio Atzori dc86018a5f Merge branch 'merge_entities_job' into beta 2023-10-02 11:24:48 +02:00
Giambattista Bloisi 3c47920c78 Use asScala to convert java List to Scala Sequence 2023-10-02 11:04:47 +02:00
Claudio Atzori 7f244d9a7a code formatting 2023-10-02 11:04:36 +02:00
Giambattista Bloisi e239b81740 Fix defect #8997: GenerateEventsJob is generating huge amounts of logs because broker entity similarity calculation consistently failed 2023-10-02 11:04:18 +02:00
Miriam Baglioni e84f5b5e64 extended existing codo to accomodate import of POCI from open citation 2023-10-02 09:25:16 +02:00
Serafeim Chatzopoulos ab0d70691c Add step for archiving repoUrls to SWH 2023-09-28 20:56:18 +03:00
Serafeim Chatzopoulos ed9c81a0b7 Add steps to collect last visit data && archive not found repository URLs 2023-09-27 19:00:54 +03:00
Alessia Bardi 0935d7757c Use v5 of the UNIBI Gold ISSN list in test 2023-09-20 15:41:35 +02:00
Alessia Bardi cc7204a089 tests for d4science catalog 2023-09-20 15:38:32 +02:00
Sandro La Bruzzo 76476cdfb6 Added maven repo for dependencies that are not in maven central 2023-09-20 10:33:14 +02:00
Serafeim Chatzopoulos 9d44418d38 Add collecting software code repository URLs 2023-09-14 18:43:25 +03:00
Serafeim Chatzopoulos 395a4af020 Run CC and RAM sequentieally in dhp-impact-indicators WF 2023-09-13 08:59:40 +02:00
Claudio Atzori 8a6892cc63 [graph dedup] consistency wf should not remove the relations while dispatching the entities 2023-09-12 21:27:05 +02:00
Claudio Atzori 4786aa0e09 added Archive ouverte UNIGE (ETHZ.UNIGENF, opendoar____::1400) to the Datacite hostedBy_map 2023-09-07 11:21:07 +02:00
Giambattista Bloisi 2caaaec42d Include SparkCleanRelation logic in SparkPropagateRelation
SparkPropagateRelation includes merge relations
Revised tests for SparkPropagateRelation
2023-09-04 11:33:20 +02:00
Giambattista Bloisi 6cc7d8ca7b GroupEntities and DispatchEntites are now merged in GroupEntitiesSparkJob 2023-08-30 10:43:31 +02:00
68 changed files with 2254 additions and 3884 deletions

128
README.md
View File

@ -1,128 +1,2 @@
# dnet-hadoop
Dnet-hadoop is the project that defined all the [OOZIE workflows](https://oozie.apache.org/) for the OpenAIRE Graph construction, processing, provisioning.
How to build, package and run oozie workflows
====================
Oozie-installer is a utility allowing building, uploading and running oozie workflows. In practice, it creates a `*.tar.gz`
package that contains resources that define a workflow and some helper scripts.
This module is automatically executed when running:
`mvn package -Poozie-package -Dworkflow.source.dir=classpath/to/parent/directory/of/oozie_app`
on module having set:
```
<parent>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-workflows</artifactId>
</parent>
```
in `pom.xml` file. `oozie-package` profile initializes oozie workflow packaging, `workflow.source.dir` property points to
a workflow (notice: this is not a relative path but a classpath to directory usually holding `oozie_app` subdirectory).
The outcome of this packaging is `oozie-package.tar.gz` file containing inside all the resources required to run Oozie workflow:
- jar packages
- workflow definitions
- job properties
- maintenance scripts
Required properties
====================
In order to include proper workflow within package, `workflow.source.dir` property has to be set. It could be provided
by setting `-Dworkflow.source.dir=some/job/dir` maven parameter.
In oder to define full set of cluster environment properties one should create `~/.dhp/application.properties` file with
the following properties:
- `dhp.hadoop.frontend.user.name` - your user name on hadoop cluster and frontend machine
- `dhp.hadoop.frontend.host.name` - frontend host name
- `dhp.hadoop.frontend.temp.dir` - frontend directory for temporary files
- `dhp.hadoop.frontend.port.ssh` - frontend machine ssh port
- `oozieServiceLoc` - oozie service location required by run_workflow.sh script executing oozie job
- `nameNode` - name node address
- `jobTracker` - job tracker address
- `oozie.execution.log.file.location` - location of file that will be created when executing oozie job, it contains output
produced by `run_workflow.sh` script (needed to obtain oozie job id)
- `maven.executable` - mvn command location, requires parameterization due to a different setup of CI cluster
- `sparkDriverMemory` - amount of memory assigned to spark jobs driver
- `sparkExecutorMemory` - amount of memory assigned to spark jobs executors
- `sparkExecutorCores` - number of cores assigned to spark jobs executors
All values will be overriden with the ones from `job.properties` and eventually `job-override.properties` stored in module's
main folder.
When overriding properties from `job.properties`, `job-override.properties` file can be created in main module directory
(the one containing `pom.xml` file) and define all new properties which will override existing properties.
One can provide those properties one by one as command line `-D` arguments.
Properties overriding order is the following:
1. `pom.xml` defined properties (located in the project root dir)
2. `~/.dhp/application.properties` defined properties
3. `${workflow.source.dir}/job.properties`
4. `job-override.properties` (located in the project root dir)
5. `maven -Dparam=value`
where the maven `-Dparam` property is overriding all the other ones.
Workflow definition requirements
====================
`workflow.source.dir` property should point to the following directory structure:
[${workflow.source.dir}]
|
|-job.properties (optional)
|
\-[oozie_app]
|
\-workflow.xml
This property can be set using maven `-D` switch.
`[oozie_app]` is the default directory name however it can be set to any value as soon as `oozieAppDir` property is
provided with directory name as value.
Sub-workflows are supported as well and sub-workflow directories should be nested within `[oozie_app]` directory.
Creating oozie installer step-by-step
=====================================
Automated oozie-installer steps are the following:
1. creating jar packages: `*.jar` and `*tests.jar` along with copying all dependencies in `target/dependencies`
2. reading properties from maven, `~/.dhp/application.properties`, `job.properties`, `job-override.properties`
3. invoking priming mechanism linking resources from import.txt file (currently resolving subworkflow resources)
4. assembling shell scripts for preparing Hadoop filesystem, uploading Oozie application and starting workflow
5. copying whole `${workflow.source.dir}` content to `target/${oozie.package.file.name}`
6. generating updated `job.properties` file in `target/${oozie.package.file.name}` based on maven,
`~/.dhp/application.properties`, `job.properties` and `job-override.properties`
7. creating `lib` directory (or multiple directories for sub-workflows for each nested directory) and copying jar packages
created at step (1) to each one of them
8. bundling whole `${oozie.package.file.name}` directory into single tar.gz package
Uploading oozie package and running workflow on cluster
=======================================================
In order to simplify deployment and execution process two dedicated profiles were introduced:
- `deploy`
- `run`
to be used along with `oozie-package` profile e.g. by providing `-Poozie-package,deploy,run` maven parameters.
The `deploy` profile supplements packaging process with:
1) uploading oozie-package via scp to `/home/${user.name}/oozie-packages` directory on `${dhp.hadoop.frontend.host.name}` machine
2) extracting uploaded package
3) uploading oozie content to hadoop cluster HDFS location defined in `oozie.wf.application.path` property (generated dynamically by maven build process, based on `${dhp.hadoop.frontend.user.name}` and `workflow.source.dir` properties)
The `run` profile introduces:
1) executing oozie application uploaded to HDFS cluster using `deploy` command. Triggers `run_workflow.sh` script providing runtime properties defined in `job.properties` file.
Notice: ssh access to frontend machine has to be configured on system level and it is preferable to set key-based authentication in order to simplify remote operations.
Dnet-hadoop is the project that defined all the OOZIE workflows for the OpenAIRE Graph construction, processing, provisioning.

View File

@ -51,6 +51,7 @@ public class Constants {
public static final String RETRY_DELAY = "retryDelay";
public static final String CONNECT_TIMEOUT = "connectTimeOut";
public static final String READ_TIMEOUT = "readTimeOut";
public static final String REQUEST_METHOD = "requestMethod";
public static final String FROM_DATE_OVERRIDE = "fromDateOverride";
public static final String UNTIL_DATE_OVERRIDE = "untilDateOverride";

View File

@ -1,6 +1,9 @@
package eu.dnetlib.dhp.common.collection;
import java.util.HashMap;
import java.util.Map;
/**
* Bundles the http connection parameters driving the client behaviour.
*/
@ -13,6 +16,8 @@ public class HttpClientParams {
public static int _connectTimeOut = 10; // seconds
public static int _readTimeOut = 30; // seconds
public static String _requestMethod = "GET";
/**
* Maximum number of allowed retires before failing
*/
@ -38,17 +43,30 @@ public class HttpClientParams {
*/
private int readTimeOut;
/**
* Custom http headers
*/
private Map<String, String> headers;
/**
* Request method (i.e., GET, POST etc)
*/
private String requestMethod;
public HttpClientParams() {
this(_maxNumberOfRetry, _requestDelay, _retryDelay, _connectTimeOut, _readTimeOut);
this(_maxNumberOfRetry, _requestDelay, _retryDelay, _connectTimeOut, _readTimeOut, new HashMap<>(),
_requestMethod);
}
public HttpClientParams(int maxNumberOfRetry, int requestDelay, int retryDelay, int connectTimeOut,
int readTimeOut) {
int readTimeOut, Map<String, String> headers, String requestMethod) {
this.maxNumberOfRetry = maxNumberOfRetry;
this.requestDelay = requestDelay;
this.retryDelay = retryDelay;
this.connectTimeOut = connectTimeOut;
this.readTimeOut = readTimeOut;
this.headers = headers;
this.requestMethod = requestMethod;
}
public int getMaxNumberOfRetry() {
@ -91,4 +109,19 @@ public class HttpClientParams {
this.readTimeOut = readTimeOut;
}
public Map<String, String> getHeaders() {
return headers;
}
public void setHeaders(Map<String, String> headers) {
this.headers = headers;
}
public String getRequestMethod() {
return requestMethod;
}
public void setRequestMethod(String requestMethod) {
this.requestMethod = requestMethod;
}
}

View File

@ -107,7 +107,14 @@ public class HttpConnector2 {
urlConn.setReadTimeout(getClientParams().getReadTimeOut() * 1000);
urlConn.setConnectTimeout(getClientParams().getConnectTimeOut() * 1000);
urlConn.addRequestProperty(HttpHeaders.USER_AGENT, userAgent);
urlConn.setRequestMethod(getClientParams().getRequestMethod());
// if provided, add custom headers
if (!getClientParams().getHeaders().isEmpty()) {
for (Map.Entry<String, String> headerEntry : getClientParams().getHeaders().entrySet()) {
urlConn.addRequestProperty(headerEntry.getKey(), headerEntry.getValue());
}
}
if (log.isDebugEnabled()) {
logHeaderFields(urlConn);
}

View File

@ -1,97 +0,0 @@
package eu.dnetlib.dhp.oa.merge;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Objects;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.common.ModelSupport;
public class DispatchEntitiesSparkJob {
private static final Logger log = LoggerFactory.getLogger(DispatchEntitiesSparkJob.class);
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
Objects
.requireNonNull(
DispatchEntitiesSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/merge/dispatch_entities_parameters.json")));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String inputPath = parser.get("inputPath");
log.info("inputPath: {}", inputPath);
String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
boolean filterInvisible = Boolean.parseBoolean(parser.get("filterInvisible"));
log.info("filterInvisible: {}", filterInvisible);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> dispatchEntities(spark, inputPath, outputPath, filterInvisible));
}
private static void dispatchEntities(
SparkSession spark,
String inputPath,
String outputPath,
boolean filterInvisible) {
Dataset<String> df = spark.read().textFile(inputPath);
ModelSupport.oafTypes.entrySet().parallelStream().forEach(entry -> {
String entityType = entry.getKey();
Class<?> clazz = entry.getValue();
final String entityPath = outputPath + "/" + entityType;
if (!entityType.equalsIgnoreCase("relation")) {
HdfsSupport.remove(entityPath, spark.sparkContext().hadoopConfiguration());
Dataset<Row> entityDF = spark
.read()
.schema(Encoders.bean(clazz).schema())
.json(
df
.filter((FilterFunction<String>) s -> s.startsWith(clazz.getName()))
.map(
(MapFunction<String, String>) s -> StringUtils.substringAfter(s, "|"),
Encoders.STRING()));
if (filterInvisible) {
entityDF = entityDF.filter("dataInfo.invisible != true");
}
entityDF
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(entityPath);
}
});
}
}

View File

@ -2,36 +2,28 @@
package eu.dnetlib.dhp.oa.merge;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static eu.dnetlib.dhp.utils.DHPUtils.toSeq;
import static org.apache.spark.sql.functions.col;
import static org.apache.spark.sql.functions.when;
import java.io.IOException;
import java.util.List;
import java.util.Objects;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ForkJoinPool;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.ReduceFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.expressions.Aggregator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.jayway.jsonpath.Configuration;
import com.jayway.jsonpath.DocumentContext;
import com.jayway.jsonpath.JsonPath;
import com.jayway.jsonpath.Option;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import scala.Tuple2;
@ -39,13 +31,9 @@ import scala.Tuple2;
* Groups the graph content by entity identifier to ensure ID uniqueness
*/
public class GroupEntitiesSparkJob {
private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class);
private static final String ID_JPATH = "$.id";
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
private static final Encoder<OafEntity> OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class);
public static void main(String[] args) throws Exception {
@ -66,9 +54,15 @@ public class GroupEntitiesSparkJob {
String graphInputPath = parser.get("graphInputPath");
log.info("graphInputPath: {}", graphInputPath);
String checkpointPath = parser.get("checkpointPath");
log.info("checkpointPath: {}", checkpointPath);
String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
boolean filterInvisible = Boolean.valueOf(parser.get("filterInvisible"));
log.info("filterInvisible: {}", filterInvisible);
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
@ -78,126 +72,95 @@ public class GroupEntitiesSparkJob {
isSparkSessionManaged,
spark -> {
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
groupEntities(spark, graphInputPath, outputPath);
groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible);
});
}
private static void groupEntities(
SparkSession spark,
String inputPath,
String outputPath) {
String checkpointPath,
String outputPath,
boolean filterInvisible) {
final TypedColumn<OafEntity, OafEntity> aggregator = new GroupingAggregator().toColumn();
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
spark
.read()
.textFile(toSeq(listEntityPaths(inputPath, sc)))
.map((MapFunction<String, OafEntity>) GroupEntitiesSparkJob::parseOaf, Encoders.kryo(OafEntity.class))
.filter((FilterFunction<OafEntity>) e -> StringUtils.isNotBlank(ModelSupport.idFn().apply(e)))
.groupByKey((MapFunction<OafEntity, String>) oaf -> ModelSupport.idFn().apply(oaf), Encoders.STRING())
.agg(aggregator)
Dataset<OafEntity> allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC);
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
String entity = e.getKey().name();
Class<? extends OafEntity> entityClass = e.getValue();
String entityInputPath = inputPath + "/" + entity;
if (!HdfsSupport.exists(entityInputPath, spark.sparkContext().hadoopConfiguration())) {
continue;
}
allEntities = allEntities
.union(
((Dataset<OafEntity>) spark
.read()
.schema(Encoders.bean(entityClass).schema())
.json(entityInputPath)
.filter("length(id) > 0")
.as(Encoders.bean(entityClass)))
.map((MapFunction<OafEntity, OafEntity>) r -> r, OAFENTITY_KRYO_ENC));
}
Dataset<?> groupedEntities = allEntities
.groupByKey((MapFunction<OafEntity, String>) OafEntity::getId, Encoders.STRING())
.reduceGroups((ReduceFunction<OafEntity>) (b, a) -> OafMapperUtils.mergeEntities(b, a))
.map(
(MapFunction<Tuple2<String, OafEntity>, String>) t -> t._2().getClass().getName() +
"|" + OBJECT_MAPPER.writeValueAsString(t._2()),
Encoders.STRING())
(MapFunction<Tuple2<String, OafEntity>, Tuple2<String, OafEntity>>) t -> new Tuple2(
t._2().getClass().getName(), t._2()),
Encoders.tuple(Encoders.STRING(), OAFENTITY_KRYO_ENC));
// pivot on "_1" (classname of the entity)
// created columns containing only entities of the same class
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
String entity = e.getKey().name();
Class<? extends OafEntity> entityClass = e.getValue();
groupedEntities = groupedEntities
.withColumn(
entity,
when(col("_1").equalTo(entityClass.getName()), col("_2")));
}
groupedEntities
.drop("_1", "_2")
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.text(outputPath);
}
.option("compression", "gzip")
.save(checkpointPath);
public static class GroupingAggregator extends Aggregator<OafEntity, OafEntity, OafEntity> {
ForkJoinPool parPool = new ForkJoinPool(ModelSupport.entityTypes.size());
@Override
public OafEntity zero() {
return null;
}
@Override
public OafEntity reduce(OafEntity b, OafEntity a) {
return mergeAndGet(b, a);
}
private OafEntity mergeAndGet(OafEntity b, OafEntity a) {
if (Objects.nonNull(a) && Objects.nonNull(b)) {
return OafMapperUtils.mergeEntities(b, a);
}
return Objects.isNull(a) ? b : a;
}
@Override
public OafEntity merge(OafEntity b, OafEntity a) {
return mergeAndGet(b, a);
}
@Override
public OafEntity finish(OafEntity j) {
return j;
}
@Override
public Encoder<OafEntity> bufferEncoder() {
return Encoders.kryo(OafEntity.class);
}
@Override
public Encoder<OafEntity> outputEncoder() {
return Encoders.kryo(OafEntity.class);
}
}
private static OafEntity parseOaf(String s) {
DocumentContext dc = JsonPath
.parse(s, Configuration.defaultConfiguration().addOptions(Option.SUPPRESS_EXCEPTIONS));
final String id = dc.read(ID_JPATH);
if (StringUtils.isNotBlank(id)) {
String prefix = StringUtils.substringBefore(id, "|");
switch (prefix) {
case "10":
return parse(s, Datasource.class);
case "20":
return parse(s, Organization.class);
case "40":
return parse(s, Project.class);
case "50":
String resultType = dc.read("$.resulttype.classid");
switch (resultType) {
case "publication":
return parse(s, Publication.class);
case "dataset":
return parse(s, eu.dnetlib.dhp.schema.oaf.Dataset.class);
case "software":
return parse(s, Software.class);
case "other":
return parse(s, OtherResearchProduct.class);
default:
throw new IllegalArgumentException(String.format("invalid resultType: '%s'", resultType));
}
default:
throw new IllegalArgumentException(String.format("invalid id prefix: '%s'", prefix));
}
} else {
throw new IllegalArgumentException(String.format("invalid oaf: '%s'", s));
}
}
private static <T extends OafEntity> OafEntity parse(String s, Class<T> clazz) {
try {
return OBJECT_MAPPER.readValue(s, clazz);
} catch (IOException e) {
throw new IllegalArgumentException(e);
}
}
private static List<String> listEntityPaths(String inputPath, JavaSparkContext sc) {
return HdfsSupport
.listFiles(inputPath, sc.hadoopConfiguration())
ModelSupport.entityTypes
.entrySet()
.stream()
.filter(f -> !f.toLowerCase().contains("relation"))
.collect(Collectors.toList());
}
.map(e -> parPool.submit(() -> {
String entity = e.getKey().name();
Class<? extends OafEntity> entityClass = e.getValue();
spark
.read()
.load(checkpointPath)
.select(col(entity).as("value"))
.filter("value IS NOT NULL")
.as(OAFENTITY_KRYO_ENC)
.map((MapFunction<OafEntity, OafEntity>) r -> r, (Encoder<OafEntity>) Encoders.bean(entityClass))
.filter(filterInvisible ? "dataInfo.invisible != TRUE" : "TRUE")
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath + "/" + entity);
}))
.collect(Collectors.toList())
.forEach(t -> {
try {
t.get();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
});
}
}

View File

@ -7,7 +7,7 @@ import java.util.regex.Pattern;
// https://researchguides.stevens.edu/c.php?g=442331&p=6577176
public class PmidCleaningRule {
public static final Pattern PATTERN = Pattern.compile("0*(\\d{1,8})");
public static final Pattern PATTERN = Pattern.compile("[1-9]{1,8}");
public static String clean(String pmid) {
String s = pmid
@ -17,7 +17,7 @@ public class PmidCleaningRule {
final Matcher m = PATTERN.matcher(s);
if (m.find()) {
return m.group(1);
return m.group();
}
return "";
}

View File

@ -1,26 +0,0 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "i",
"paramLongName": "inputPath",
"paramDescription": "the source path",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "path of the output graph",
"paramRequired": true
},
{
"paramName": "fi",
"paramLongName": "filterInvisible",
"paramDescription": "if true filters out invisible entities",
"paramRequired": true
}
]

View File

@ -8,13 +8,25 @@
{
"paramName": "gin",
"paramLongName": "graphInputPath",
"paramDescription": "the graph root path",
"paramDescription": "the input graph root path",
"paramRequired": true
},
{
"paramName": "cp",
"paramLongName": "checkpointPath",
"paramDescription": "checkpoint directory",
"paramRequired": true
},
{
"paramName": "out",
"paramLongName": "outputPath",
"paramDescription": "the output merged graph root path",
"paramDescription": "the output graph root path",
"paramRequired": true
},
{
"paramName": "fi",
"paramLongName": "filterInvisible",
"paramDescription": "if true filters out invisible entities",
"paramRequired": true
}
]

View File

@ -9,16 +9,10 @@ class PmidCleaningRuleTest {
@Test
void testCleaning() {
// leading zeros are removed
assertEquals("1234", PmidCleaningRule.clean("01234"));
// tolerant to spaces in the middle
assertEquals("1234567", PmidCleaningRule.clean("0123 4567"));
// stop parsing at first not numerical char
assertEquals("123", PmidCleaningRule.clean("0123x4567"));
// invalid id leading to empty result
assertEquals("", PmidCleaningRule.clean("abc"));
// valid id with zeroes in the number
assertEquals("20794075", PmidCleaningRule.clean("20794075"));
}
}

View File

@ -18,7 +18,6 @@ package eu.dnetlib.pace.util;
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/*
* Diff Match and Patch
* Copyright 2018 The diff-match-patch Authors.

View File

@ -1,72 +0,0 @@
# Action Management Framework
This module implements the oozie workflow for the integration of pre-built contents into the OpenAIRE Graph.
Such contents can be
* brand new, non-existing records to be introduced as nodes of the graph
* updates (or enrichment) for records that does exist in the graph (e.g. a new subject term for a publication)
* relations among existing nodes
The actionset contents are organised into logical containers, each of them can contain multiple versions contents and is characterised by
* a name
* an identifier
* the paths on HDFS where each version of the contents is stored
Each version is then characterised by
* the creation date
* the last update date
* the indication where it is the latest one or it is an expired version, candidate for garbage collection
## ActionSet serialization
Each actionset version contains records compliant to the graph internal data model, i.e. subclasses of `eu.dnetlib.dhp.schema.oaf.Oaf`,
defined in the external schemas module
```
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>${dhp-schemas.artifact}</artifactId>
<version>${dhp-schemas.version}</version>
</dependency>
```
When the actionset contains a relationship, the model class to use is `eu.dnetlib.dhp.schema.oaf.Relation`, otherwise
when the actionset contains an entity, it is a `eu.dnetlib.dhp.schema.oaf.OafEntity` or one of its subclasses
`Datasource`, `Organization`, `Project`, `Result` (or one of its subclasses `Publication`, `Dataset`, etc...).
Then, each OpenAIRE Graph model class instance must be wrapped using the class `eu.dnetlib.dhp.schema.action.AtomicAction`, a generic
container that defines two attributes
* `T payload` the OpenAIRE Graph class instance containing the data;
* `Class<T> clazz` must contain the class whose instance is contained in the payload.
Each AtomicAction can be then serialised in JSON format using `com.fasterxml.jackson.databind.ObjectMapper` from
```
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<version>${dhp.jackson.version}</version>
</dependency>
```
Then, the JSON serialization must be stored as a GZip compressed sequence file (`org.apache.hadoop.mapred.SequenceFileOutputFormat`).
As such, it contains a set of tuples, a key and a value defined as `org.apache.hadoop.io.Text` where
* the `key` must be set to the class canonical name contained in the `AtomicAction`;
* the `value` must be set to the AtomicAction JSON serialization.
The following snippet provides an example of how create an actionset version of Relation records:
```
rels // JavaRDD<Relation>
.map(relation -> new AtomicAction<Relation>(Relation.class, relation))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
```

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,15 +31,29 @@ 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.schema.oaf.utils.PidType;
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_________::";
// 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________::";
private static final String TRUST = "0.91";
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws IOException, ParseException {
@ -62,7 +77,7 @@ public class CreateActionSetSparkJob implements Serializable {
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("inputPath");
log.info("inputPath {}", inputPath.toString());
log.info("inputPath {}", inputPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}", outputPath);
@ -76,41 +91,68 @@ public class CreateActionSetSparkJob implements Serializable {
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
extractContent(spark, inputPath, outputPath, shouldDuplicateRels);
});
spark -> extractContent(spark, inputPath, outputPath, shouldDuplicateRels));
}
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)
.filter((FilterFunction<Relation>) Objects::nonNull)
.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;
String citing = ID_PREFIX
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCiting()));
final String cited = ID_PREFIX
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCited()));
switch (p) {
case COCI:
prefix = DOI_PREFIX;
citing = prefix
+ IdentifierFactory
.md5(CleaningFunctions.normalizePidValue(PidType.doi.toString(), value.getCiting()));
cited = prefix
+ IdentifierFactory
.md5(CleaningFunctions.normalizePidValue(PidType.doi.toString(), value.getCited()));
break;
case POCI:
prefix = PMID_PREFIX;
citing = prefix
+ IdentifierFactory
.md5(CleaningFunctions.normalizePidValue(PidType.pmid.toString(), value.getCiting()));
cited = prefix
+ IdentifierFactory
.md5(CleaningFunctions.normalizePidValue(PidType.pmid.toString(), value.getCited()));
break;
default:
throw new IllegalStateException("Invalid prefix: " + p);
}
if (!citing.equals(cited)) {
relationList
@ -120,7 +162,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 +174,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;
}
String relClass) {
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;
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);
}
}

View File

@ -3,6 +3,7 @@ package eu.dnetlib.dhp.actionmanager.opencitations;
import java.io.*;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Objects;
import java.util.zip.GZIPOutputStream;
import java.util.zip.ZipEntry;
@ -37,7 +38,7 @@ public class GetOpenCitationsRefs implements Serializable {
parser.parseArgument(args);
final String[] inputFile = parser.get("inputFile").split(";");
log.info("inputFile {}", inputFile.toString());
log.info("inputFile {}", Arrays.asList(inputFile));
final String workingPath = parser.get("workingPath");
log.info("workingPath {}", workingPath);
@ -45,6 +46,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 +57,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

@ -7,6 +7,7 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
@ -42,13 +43,16 @@ public class ReadCOCI implements Serializable {
log.info("outputPath: {}", outputPath);
final String[] inputFile = parser.get("inputFile").split(";");
log.info("inputFile {}", inputFile.toString());
log.info("inputFile {}", Arrays.asList(inputFile));
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
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,16 +68,17 @@ 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) {
for (String inputFile : inputFiles) {
String p_string = workingPath + "/" + inputFile + ".gz";
String pString = workingPath + "/" + inputFile + ".gz";
Dataset<Row> cociData = spark
.read()
@ -82,14 +87,20 @@ public class ReadCOCI implements Serializable {
.option("inferSchema", "true")
.option("header", "true")
.option("quotes", "\"")
.load(p_string)
.load(pString)
.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.setOci(row.getString(0));
coci.setCiting(row.getString(1));
coci.setCited(row.getString(2));
return coci;
}, Encoders.bean(COCI.class))
.write()

View File

@ -16,10 +16,11 @@
"paramLongName": "isSparkSessionManaged",
"paramDescription": "the hdfs name node",
"paramRequired": false
}, {
"paramName": "sdr",
"paramLongName": "shouldDuplicateRels",
"paramDescription": "the hdfs name node",
"paramRequired": false
}
},
{
"paramName": "sdr",
"paramLongName": "shouldDuplicateRels",
"paramDescription": "activates/deactivates the construction of bidirectional relations Cites/IsCitedBy",
"paramRequired": false
}
]

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

@ -34,6 +34,7 @@
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="download">
<shell xmlns="uri:oozie:shell-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
@ -46,7 +47,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/>
@ -54,12 +55,14 @@
<ok to="extract"/>
<error to="Kill"/>
</action>
<action name="extract">
<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}</arg>
<arg>--workingPath</arg><arg>${workingPath}/${prefix}</arg>
<arg>--prefix</arg><arg>${prefix}</arg>
</java>
<ok to="read"/>
<error to="Kill"/>
@ -82,10 +85,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,7 +112,7 @@
--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>
</spark>
<ok to="End"/>

View File

@ -222,7 +222,7 @@ object BioDBToOAF {
def uniprotToOAF(input: String): List[Oaf] = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json = parse(input)
val pid = (json \ "pid").extract[String].trim()
val pid = (json \ "pid").extract[String]
val d = new Dataset

View File

@ -1,44 +1,15 @@
{"classification": "Signaling protein", "pdb": "5NM4", "deposition_date": "2017-04-05", "title": "A2a adenosine receptor room-temperature structure determined by serial Femtosecond crystallography", "Keywords": ["Oom-temperature", " serial crystallography", " signaling protein"], "authors": ["T.weinert", "R.cheng", "D.james", "D.gashi", "P.nogly", "K.jaeger", "M.hennig", "", "J.standfuss"], "pmid": "28912485", "doi": "10.1038/S41467-017-00630-4"}
{"classification": "Oxidoreductase/oxidoreductase inhibitor", "pdb": "4KN3", "deposition_date": "2013-05-08", "title": "Structure of the y34ns91g double mutant of dehaloperoxidase from Amphitrite ornata with 2,4,6-trichlorophenol", "Keywords": ["Lobin", " oxygen storage", " peroxidase", " oxidoreductase", " oxidoreductase-", "Oxidoreductase inhibitor complex"], "authors": ["C.wang", "L.lovelace", "L.lebioda"], "pmid": "23952341", "doi": "10.1021/BI400627W"}
{"classification": "Transport protein", "pdb": "8HKM", "deposition_date": "2022-11-27", "title": "Ion channel", "Keywords": ["On channel", " transport protein"], "authors": ["D.h.jiang", "J.t.zhang"], "pmid": "37494189", "doi": "10.1016/J.CELREP.2023.112858"}
{"classification": "Signaling protein", "pdb": "6JT1", "deposition_date": "2019-04-08", "title": "Structure of human soluble guanylate cyclase in the heme oxidised State", "Keywords": ["Oluble guanylate cyclase", " signaling protein"], "authors": ["L.chen", "Y.kang", "R.liu", "J.-x.wu"], "pmid": "31514202", "doi": "10.1038/S41586-019-1584-6"}
{"classification": "Immune system", "pdb": "7OW6", "deposition_date": "2021-06-16", "title": "Crystal structure of a tcr in complex with hla-a*11:01 bound to kras G12d peptide (vvvgadgvgk)", "Keywords": ["La", " kras", " tcr", " immune system"], "authors": ["V.karuppiah", "R.a.robinson"], "doi": "10.1038/S41467-022-32811-1"}
{"classification": "Biosynthetic protein", "pdb": "5EQ8", "deposition_date": "2015-11-12", "title": "Crystal structure of medicago truncatula histidinol-phosphate Phosphatase (mthpp) in complex with l-histidinol", "Keywords": ["Istidine biosynthesis", " metabolic pathways", " dimer", " plant", "", "Biosynthetic protein"], "authors": ["M.ruszkowski", "Z.dauter"], "pmid": "26994138", "doi": "10.1074/JBC.M115.708727"}
{"classification": "De novo protein", "pdb": "8CWA", "deposition_date": "2022-05-18", "title": "Solution nmr structure of 8-residue rosetta-designed cyclic peptide D8.21 in cdcl3 with cis/trans switching (tc conformation, 53%)", "Keywords": ["Yclic peptide", " non natural amino acids", " cis/trans", " switch peptides", "", "De novo design", "Membrane permeability", "De novo protein"], "authors": ["T.a.ramelot", "R.tejero", "G.t.montelione"], "pmid": "36041435", "doi": "10.1016/J.CELL.2022.07.019"}
{"classification": "Hydrolase", "pdb": "3R6M", "deposition_date": "2011-03-21", "title": "Crystal structure of vibrio parahaemolyticus yeaz", "Keywords": ["Ctin/hsp70 nucleotide-binding fold", " bacterial resuscitation", " viable", "But non-culturable state", "Resuscitation promoting factor", "Ygjd", "", "Yjee", "Vibrio parahaemolyticus", "Hydrolase"], "authors": ["A.roujeinikova", "I.aydin"], "pmid": "21858042", "doi": "10.1371/JOURNAL.PONE.0023245"}
{"classification": "Hydrolase", "pdb": "2W5J", "deposition_date": "2008-12-10", "title": "Structure of the c14-rotor ring of the proton translocating Chloroplast atp synthase", "Keywords": ["Ydrolase", " chloroplast", " atp synthase", " lipid-binding", " cf(0)", " membrane", "", "Transport", "Formylation", "Energy transduction", "Hydrogen ion transport", "", "Ion transport", "Transmembrane", "Membrane protein"], "authors": ["M.vollmar", "D.schlieper", "M.winn", "C.buechner", "G.groth"], "pmid": "19423706", "doi": "10.1074/JBC.M109.006916"}
{"classification": "De novo protein", "pdb": "4GLU", "deposition_date": "2012-08-14", "title": "Crystal structure of the mirror image form of vegf-a", "Keywords": ["-protein", " covalent dimer", " cysteine knot protein", " growth factor", " de", "Novo protein"], "authors": ["K.mandal", "M.uppalapati", "D.ault-riche", "J.kenney", "J.lowitz", "S.sidhu", "", "S.b.h.kent"], "pmid": "22927390", "doi": "10.1073/PNAS.1210483109"}
{"classification": "Hydrolase/hydrolase inhibitor", "pdb": "3WYL", "deposition_date": "2014-09-01", "title": "Crystal structure of the catalytic domain of pde10a complexed with 5- Methoxy-3-(1-phenyl-1h-pyrazol-5-yl)-1-(3-(trifluoromethyl)phenyl) Pyridazin-4(1h)-one", "Keywords": ["Ydrolase-hydrolase inhibitor complex"], "authors": ["H.oki", "Y.hayano"], "pmid": "25384088", "doi": "10.1021/JM5013648"}
{"classification": "Isomerase", "pdb": "5BOR", "deposition_date": "2015-05-27", "title": "Structure of acetobacter aceti pure-s57c, sulfonate form", "Keywords": ["Cidophile", " pure", " purine biosynthesis", " isomerase"], "authors": ["K.l.sullivan", "T.j.kappock"]}
{"classification": "Hydrolase", "pdb": "1X0C", "deposition_date": "2005-03-17", "title": "Improved crystal structure of isopullulanase from aspergillus niger Atcc 9642", "Keywords": ["Ullulan", " glycoside hydrolase family 49", " glycoprotein", " hydrolase"], "authors": ["M.mizuno", "T.tonozuka", "A.yamamura", "Y.miyasaka", "H.akeboshi", "S.kamitori", "", "A.nishikawa", "Y.sakano"], "pmid": "18155243", "doi": "10.1016/J.JMB.2007.11.098"}
{"classification": "Oxidoreductase", "pdb": "7CUP", "deposition_date": "2020-08-23", "title": "Structure of 2,5-dihydroxypridine dioxygenase from pseudomonas putida Kt2440", "Keywords": ["On-heme dioxygenase", " oxidoreductase"], "authors": ["G.q.liu", "H.z.tang"]}
{"classification": "Ligase", "pdb": "1VCN", "deposition_date": "2004-03-10", "title": "Crystal structure of t.th. hb8 ctp synthetase complex with sulfate Anion", "Keywords": ["Etramer", " riken structural genomics/proteomics initiative", " rsgi", "", "Structural genomics", "Ligase"], "authors": ["M.goto", "Riken structural genomics/proteomics initiative (rsgi)"], "pmid": "15296735", "doi": "10.1016/J.STR.2004.05.013"}
{"classification": "Transferase/transferase inhibitor", "pdb": "6C9V", "deposition_date": "2018-01-28", "title": "Mycobacterium tuberculosis adenosine kinase bound to (2r,3s,4r,5r)-2- (hydroxymethyl)-5-(6-(4-phenylpiperazin-1-yl)-9h-purin-9-yl) Tetrahydrofuran-3,4-diol", "Keywords": ["Ucleoside analog", " complex", " inhibitor", " structural genomics", " psi-2", "", "Protein structure initiative", "Tb structural genomics consortium", "", "Tbsgc", "Transferase-transferase inhibitor complex"], "authors": ["R.a.crespo", "Tb structural genomics consortium (tbsgc)"], "pmid": "31002508", "doi": "10.1021/ACS.JMEDCHEM.9B00020"}
{"classification": "De novo protein", "pdb": "4LPY", "deposition_date": "2013-07-16", "title": "Crystal structure of tencon variant g10", "Keywords": ["Ibronectin type iii fold", " alternate scaffold", " de novo protein"], "authors": ["A.teplyakov", "G.obmolova", "G.l.gilliland"], "pmid": "24375666", "doi": "10.1002/PROT.24502"}
{"classification": "Isomerase", "pdb": "2Y88", "deposition_date": "2011-02-03", "title": "Crystal structure of mycobacterium tuberculosis phosphoribosyl Isomerase (variant d11n) with bound prfar", "Keywords": ["Romatic amino acid biosynthesis", " isomerase", " tim-barrel", " histidine", "Biosynthesis", "Tryptophan biosynthesis"], "authors": ["J.kuper", "A.v.due", "A.geerlof", "M.wilmanns"], "pmid": "21321225", "doi": "10.1073/PNAS.1015996108"}
{"classification": "Unknown function", "pdb": "1SR0", "deposition_date": "2004-03-22", "title": "Crystal structure of signalling protein from sheep(sps-40) at 3.0a Resolution using crystal grown in the presence of polysaccharides", "Keywords": ["Ignalling protein", " involution", " unknown function"], "authors": ["D.b.srivastava", "A.s.ethayathulla", "N.singh", "J.kumar", "S.sharma", "T.p.singh"]}
{"classification": "Dna binding protein", "pdb": "3RH2", "deposition_date": "2011-04-11", "title": "Crystal structure of a tetr-like transcriptional regulator (sama_0099) From shewanella amazonensis sb2b at 2.42 a resolution", "Keywords": ["Na/rna-binding 3-helical bundle", " structural genomics", " joint center", "For structural genomics", "Jcsg", "Protein structure initiative", "Psi-", "Biology", "Dna binding protein"], "authors": ["Joint center for structural genomics (jcsg)"]}
{"classification": "Transferase", "pdb": "2WK5", "deposition_date": "2009-06-05", "title": "Structural features of native human thymidine phosphorylase And in complex with 5-iodouracil", "Keywords": ["Lycosyltransferase", " developmental protein", " angiogenesis", "", "5-iodouracil", "Growth factor", "Enzyme kinetics", "", "Differentiation", "Disease mutation", "Thymidine", "Phosphorylase", "Chemotaxis", "Transferase", "Mutagenesis", "", "Polymorphism"], "authors": ["E.mitsiki", "A.c.papageorgiou", "S.iyer", "N.thiyagarajan", "S.h.prior", "", "D.sleep", "C.finnis", "K.r.acharya"], "pmid": "19555658", "doi": "10.1016/J.BBRC.2009.06.104"}
{"classification": "Hydrolase", "pdb": "3P9Y", "deposition_date": "2010-10-18", "title": "Crystal structure of the drosophila melanogaster ssu72-pctd complex", "Keywords": ["Hosphatase", " cis proline", " lmw ptp-like fold", " rna polymerase ii ctd", "", "Hydrolase"], "authors": ["J.w.werner-allen", "P.zhou"], "pmid": "21159777", "doi": "10.1074/JBC.M110.197129"}
{"classification": "Recombination/dna", "pdb": "6OEO", "deposition_date": "2019-03-27", "title": "Cryo-em structure of mouse rag1/2 nfc complex (dna1)", "Keywords": ["(d)j recombination", " dna transposition", " rag", " scid", " recombination", "", "Recombination-dna complex"], "authors": ["X.chen", "Y.cui", "Z.h.zhou", "W.yang", "M.gellert"], "pmid": "32015552", "doi": "10.1038/S41594-019-0363-2"}
{"classification": "Hydrolase", "pdb": "4ECA", "deposition_date": "1997-02-21", "title": "Asparaginase from e. coli, mutant t89v with covalently bound aspartate", "Keywords": ["Ydrolase", " acyl-enzyme intermediate", " threonine amidohydrolase"], "authors": ["G.j.palm", "J.lubkowski", "A.wlodawer"], "pmid": "8706862", "doi": "10.1016/0014-5793(96)00660-6"}
{"classification": "Transcription/protein binding", "pdb": "3UVX", "deposition_date": "2011-11-30", "title": "Crystal structure of the first bromodomain of human brd4 in complex With a diacetylated histone 4 peptide (h4k12ack16ac)", "Keywords": ["Romodomain", " bromodomain containing protein 4", " cap", " hunk1", " mcap", "", "Mitotic chromosome associated protein", "Peptide complex", "Structural", "Genomics consortium", "Sgc", "Transcription-protein binding complex"], "authors": ["P.filippakopoulos", "S.picaud", "T.keates", "E.ugochukwu", "F.von delft", "", "C.h.arrowsmith", "A.m.edwards", "J.weigelt", "C.bountra", "S.knapp", "Structural", "Genomics consortium (sgc)"], "pmid": "22464331", "doi": "10.1016/J.CELL.2012.02.013"}
{"classification": "Membrane protein", "pdb": "1TLZ", "deposition_date": "2004-06-10", "title": "Tsx structure complexed with uridine", "Keywords": ["Ucleoside transporter", " beta barrel", " uridine", " membrane", "Protein"], "authors": ["J.ye", "B.van den berg"], "pmid": "15272310", "doi": "10.1038/SJ.EMBOJ.7600330"}
{"classification": "Dna binding protein", "pdb": "7AZD", "deposition_date": "2020-11-16", "title": "Dna polymerase sliding clamp from escherichia coli with peptide 20 Bound", "Keywords": ["Ntibacterial drug", " dna binding protein"], "authors": ["C.monsarrat", "G.compain", "C.andre", "I.martiel", "S.engilberge", "V.olieric", "", "P.wolff", "K.brillet", "M.landolfo", "C.silva da veiga", "J.wagner", "G.guichard", "", "D.y.burnouf"], "pmid": "34806883", "doi": "10.1021/ACS.JMEDCHEM.1C00918"}
{"classification": "Transferase", "pdb": "5N3K", "deposition_date": "2017-02-08", "title": "Camp-dependent protein kinase a from cricetulus griseus in complex With fragment like molecule o-guanidino-l-homoserine", "Keywords": ["Ragment", " complex", " transferase", " serine threonine kinase", " camp", "", "Kinase", "Pka"], "authors": ["C.siefker", "A.heine", "G.klebe"]}
{"classification": "Biosynthetic protein", "pdb": "8H52", "deposition_date": "2022-10-11", "title": "Crystal structure of helicobacter pylori carboxyspermidine Dehydrogenase in complex with nadp", "Keywords": ["Arboxyspermidine dehydrogenase", " biosynthetic protein"], "authors": ["K.y.ko", "S.c.park", "S.y.cho", "S.i.yoon"], "pmid": "36283333", "doi": "10.1016/J.BBRC.2022.10.049"}
{"classification": "Metal binding protein", "pdb": "6DYC", "deposition_date": "2018-07-01", "title": "Co(ii)-bound structure of the engineered cyt cb562 variant, ch3", "Keywords": ["Esigned protein", " 4-helix bundle", " electron transport", " metal binding", "Protein"], "authors": ["F.a.tezcan", "J.rittle"], "pmid": "30778140", "doi": "10.1038/S41557-019-0218-9"}
{"classification": "Protein fibril", "pdb": "6A6B", "deposition_date": "2018-06-27", "title": "Cryo-em structure of alpha-synuclein fiber", "Keywords": ["Lpha-syn fiber", " parkinson disease", " protein fibril"], "authors": ["Y.w.li", "C.y.zhao", "F.luo", "Z.liu", "X.gui", "Z.luo", "X.zhang", "D.li", "C.liu", "X.li"], "pmid": "30065316", "doi": "10.1038/S41422-018-0075-X"}
{"classification": "Dna", "pdb": "7D5E", "deposition_date": "2020-09-25", "title": "Left-handed g-quadruplex containing two bulges", "Keywords": ["-quadruplex", " bulge", " dna", " left-handed"], "authors": ["P.das", "A.maity", "K.h.ngo", "F.r.winnerdy", "B.bakalar", "Y.mechulam", "E.schmitt", "", "A.t.phan"], "pmid": "33503265", "doi": "10.1093/NAR/GKAA1259"}
{"classification": "Transferase", "pdb": "3RSY", "deposition_date": "2011-05-02", "title": "Cellobiose phosphorylase from cellulomonas uda in complex with sulfate And glycerol", "Keywords": ["H94", " alpha barrel", " cellobiose phosphorylase", " disaccharide", "Phosphorylase", "Transferase"], "authors": ["A.van hoorebeke", "J.stout", "W.soetaert", "J.van beeumen", "T.desmet", "S.savvides"]}
{"classification": "Oxidoreductase", "pdb": "7MCI", "deposition_date": "2021-04-02", "title": "Mofe protein from azotobacter vinelandii with a sulfur-replenished Cofactor", "Keywords": ["Zotobacter vinelandii", " mofe-protein", " nitrogenase", " oxidoreductase"], "authors": ["W.kang", "C.lee", "Y.hu", "M.w.ribbe"], "doi": "10.1038/S41929-022-00782-7"}
{"classification": "Dna", "pdb": "1XUW", "deposition_date": "2004-10-26", "title": "Structural rationalization of a large difference in rna affinity Despite a small difference in chemistry between two 2'-o-modified Nucleic acid analogs", "Keywords": ["Na mimetic methylcarbamate amide analog", " dna"], "authors": ["R.pattanayek", "L.sethaphong", "C.pan", "M.prhavc", "T.p.prakash", "M.manoharan", "", "M.egli"], "pmid": "15547979", "doi": "10.1021/JA044637K"}
{"classification": "Lyase", "pdb": "7C0D", "deposition_date": "2020-05-01", "title": "Crystal structure of azospirillum brasilense l-2-keto-3-deoxyarabonate Dehydratase (hydroxypyruvate-bound form)", "Keywords": ["-2-keto-3-deoxyarabonate dehydratase", " lyase"], "authors": ["Y.watanabe", "S.watanabe"], "pmid": "32697085", "doi": "10.1021/ACS.BIOCHEM.0C00515"}
{"classification": "Signaling protein", "pdb": "5LYK", "deposition_date": "2016-09-28", "title": "Crystal structure of intracellular b30.2 domain of btn3a1 bound to Citrate", "Keywords": ["30.2", " butyrophilin", " signaling protein"], "authors": ["F.mohammed", "A.t.baker", "M.salim", "B.e.willcox"], "pmid": "28862425", "doi": "10.1021/ACSCHEMBIO.7B00694"}
{"classification": "Toxin", "pdb": "4IZL", "deposition_date": "2013-01-30", "title": "Structure of the n248a mutant of the panton-valentine leucocidin s Component from staphylococcus aureus", "Keywords": ["I-component leucotoxin", " staphylococcus aureus", " s component", "Leucocidin", "Beta-barrel pore forming toxin", "Toxin"], "authors": ["L.maveyraud", "B.j.laventie", "G.prevost", "L.mourey"], "pmid": "24643034", "doi": "10.1371/JOURNAL.PONE.0092094"}
{"classification": "Dna", "pdb": "6F3C", "deposition_date": "2017-11-28", "title": "The cytotoxic [pt(h2bapbpy)] platinum complex interacting with the Cgtacg hexamer", "Keywords": ["Rug-dna complex", " four-way junction", " dna"], "authors": ["M.ferraroni", "C.bazzicalupi", "P.gratteri", "F.papi"], "pmid": "31046177", "doi": "10.1002/ANIE.201814532"}
{"classification": "Signaling protein/inhibitor", "pdb": "4L5M", "deposition_date": "2013-06-11", "title": "Complexe of arno sec7 domain with the protein-protein interaction Inhibitor n-(4-hydroxy-2,6-dimethylphenyl)benzenesulfonamide at ph6.5", "Keywords": ["Ec-7domain", " signaling protein-inhibitor complex"], "authors": ["F.hoh", "J.rouhana"], "pmid": "24112024", "doi": "10.1021/JM4009357"}
{"classification": "Signaling protein", "pdb": "5I6J", "deposition_date": "2016-02-16", "title": "Crystal structure of srgap2 f-barx", "Keywords": ["Rgap2", " f-bar", " fx", " signaling protein"], "authors": ["M.sporny", "J.guez-haddad", "M.n.isupov", "Y.opatowsky"], "pmid": "28333212", "doi": "10.1093/MOLBEV/MSX094"}
{"classification": "Metal binding protein", "pdb": "1Q80", "deposition_date": "2003-08-20", "title": "Solution structure and dynamics of nereis sarcoplasmic calcium binding Protein", "Keywords": ["Ll-alpha", " metal binding protein"], "authors": ["G.rabah", "R.popescu", "J.a.cox", "Y.engelborghs", "C.t.craescu"], "pmid": "15819893", "doi": "10.1111/J.1742-4658.2005.04629.X"}
{"classification": "Transferase", "pdb": "1TW1", "deposition_date": "2004-06-30", "title": "Beta-1,4-galactosyltransferase mutant met344his (m344h-gal-t1) complex With udp-galactose and magnesium", "Keywords": ["Et344his mutation; closed conformation; mn binding", " transferase"], "authors": ["B.ramakrishnan", "E.boeggeman", "P.k.qasba"], "pmid": "15449940", "doi": "10.1021/BI049007+"}
{"classification": "Rna", "pdb": "2PN4", "deposition_date": "2007-04-23", "title": "Crystal structure of hepatitis c virus ires subdomain iia", "Keywords": ["Cv", " ires", " subdoamin iia", " rna", " strontium", " hepatitis"], "authors": ["Q.zhao", "Q.han", "C.r.kissinger", "P.a.thompson"], "pmid": "18391410", "doi": "10.1107/S0907444908002011"}
{"pdb": "1CW0", "title": "crystal structure analysis of very short patch repair (vsr) endonuclease in complex with a duplex dna", "authors": ["S.E.Tsutakawa", "H.Jingami", "K.Morikawa"], "doi": "10.1016/S0092-8674(00)81550-0", "pmid": "10612397"}
{"pdb": "2CWW", "title": "crystal structure of thermus thermophilus ttha1280, a putative sam- dependent rna methyltransferase, in complex with s-adenosyl-l- homocysteine", "authors": ["A.A.Pioszak", "K.Murayama", "N.Nakagawa", "A.Ebihara", "S.Kuramitsu", "M.Shirouzu", "S.Yokoyama", "Riken Structural Genomics/proteomics Initiative (Rsgi)"], "doi": "10.1107/S1744309105029842", "pmid": "16511182"}
{"pdb": "6CWE", "title": "structure of alpha-gsa[8,6p] bound by cd1d and in complex with the va14vb8.2 tcr", "authors": ["J.Wang", "D.Zajonc"], "doi": null, "pmid": null}
{"pdb": "5CWS", "title": "crystal structure of the intact chaetomium thermophilum nsp1-nup49- nup57 channel nucleoporin heterotrimer bound to its nic96 nuclear pore complex attachment site", "authors": ["C.J.Bley", "S.Petrovic", "M.Paduch", "V.Lu", "A.A.Kossiakoff", "A.Hoelz"], "doi": "10.1126/SCIENCE.AAC9176", "pmid": "26316600"}
{"pdb": "5CWE", "title": "structure of cyp107l2 from streptomyces avermitilis with lauric acid", "authors": ["T.-V.Pham", "S.-H.Han", "J.-H.Kim", "D.-H.Kim", "L.-W.Kang"], "doi": null, "pmid": null}
{"pdb": "7CW4", "title": "acetyl-coa acetyltransferase from bacillus cereus atcc 14579", "authors": ["J.Hong", "K.J.Kim"], "doi": "10.1016/J.BBRC.2020.09.048", "pmid": "32972748"}
{"pdb": "2CWP", "title": "crystal structure of metrs related protein from pyrococcus horikoshii", "authors": ["K.Murayama", "M.Kato-Murayama", "M.Shirouzu", "S.Yokoyama", "Riken StructuralGenomics/proteomics Initiative (Rsgi)"], "doi": null, "pmid": null}
{"pdb": "2CW7", "title": "crystal structure of intein homing endonuclease ii", "authors": ["H.Matsumura", "H.Takahashi", "T.Inoue", "H.Hashimoto", "M.Nishioka", "S.Fujiwara", "M.Takagi", "T.Imanaka", "Y.Kai"], "doi": "10.1002/PROT.20858", "pmid": "16493661"}
{"pdb": "1CWU", "title": "brassica napus enoyl acp reductase a138g mutant complexed with nad+ and thienodiazaborine", "authors": ["A.Roujeinikova", "J.B.Rafferty", "D.W.Rice"], "doi": "10.1074/JBC.274.43.30811", "pmid": "10521472"}
{"pdb": "3CWN", "title": "escherichia coli transaldolase b mutant f178y", "authors": ["T.Sandalova", "G.Schneider", "A.Samland"], "doi": "10.1074/JBC.M803184200", "pmid": "18687684"}
{"pdb": "1CWL", "title": "human cyclophilin a complexed with 4 4-hydroxy-meleu cyclosporin", "authors": ["V.Mikol", "J.Kallen", "P.Taylor", "M.D.Walkinshaw"], "doi": "10.1006/JMBI.1998.2108", "pmid": "9769216"}
{"pdb": "3CW2", "title": "crystal structure of the intact archaeal translation initiation factor 2 from sulfolobus solfataricus .", "authors": ["E.A.Stolboushkina", "S.V.Nikonov", "A.D.Nikulin", "U.Blaesi", "D.J.Manstein", "R.V.Fedorov", "M.B.Garber", "O.S.Nikonov"], "doi": "10.1016/J.JMB.2008.07.039", "pmid": "18675278"}
{"pdb": "3CW9", "title": "4-chlorobenzoyl-coa ligase/synthetase in the thioester-forming conformation, bound to 4-chlorophenacyl-coa", "authors": ["A.S.Reger", "J.Cao", "R.Wu", "D.Dunaway-Mariano", "A.M.Gulick"], "doi": "10.1021/BI800696Y", "pmid": "18620418"}
{"pdb": "3CWU", "title": "crystal structure of an alka host/guest complex 2'-fluoro-2'-deoxy-1, n6-ethenoadenine:thymine base pair", "authors": ["B.R.Bowman", "S.Lee", "S.Wang", "G.L.Verdine"], "doi": "10.1016/J.STR.2008.04.012", "pmid": "18682218"}
{"pdb": "5CWF", "title": "crystal structure of de novo designed helical repeat protein dhr8", "authors": ["G.Bhabha", "D.C.Ekiert"], "doi": "10.1038/NATURE16162", "pmid": "26675729"}

View File

@ -1,36 +1,6 @@
{"pid": " Q6GZX4", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 43"}], "title": "Putative transcription factor 001R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZX3", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 45"}], "title": "Uncharacterized protein 002L", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197F8", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2022-02-23", "date_info": "entry version 29"}], "title": "Uncharacterized protein 002R", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q197F7", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2020-08-12", "date_info": "entry version 23"}], "title": "Uncharacterized protein 003L", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q6GZX2", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 37"}], "title": "Uncharacterized protein 3R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZX1", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 38"}], "title": "Uncharacterized protein 004R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197F5", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2022-10-12", "date_info": "entry version 32"}], "title": "Uncharacterized protein 005L", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q6GZX0", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 47"}], "title": "Uncharacterized protein 005R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q91G88", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2001-12-01", "date_info": "sequence version 1"}, {"date": "2023-06-28", "date_info": "entry version 53"}], "title": "Putative KilA-N domain-containing protein 006L", "organism_species": "Invertebrate iridescent virus 6 (IIV-6) (Chilo iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Iridovirus"], "references": [{"PubMed": "17239238"}, {"DOI": "10.1186/1743-422x-4-11"}]}
{"pid": " Q6GZW9", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 34"}], "title": "Uncharacterized protein 006R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZW8", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 32"}], "title": "Uncharacterized protein 007R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197F3", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2023-02-22", "date_info": "entry version 28"}], "title": "Uncharacterized protein 007R", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q197F2", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2022-02-23", "date_info": "entry version 22"}], "title": "Uncharacterized protein 008L", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q6GZW6", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 67"}], "title": "Putative helicase 009L", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q91G85", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2001-12-01", "date_info": "sequence version 1"}, {"date": "2023-02-22", "date_info": "entry version 38"}], "title": "Uncharacterized protein 009R", "organism_species": "Invertebrate iridescent virus 6 (IIV-6) (Chilo iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Iridovirus"], "references": [{"PubMed": "17239238"}, {"DOI": "10.1186/1743-422x-4-11"}]}
{"pid": " Q6GZW5", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 37"}], "title": "Uncharacterized protein 010R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197E9", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2023-02-22", "date_info": "entry version 28"}], "title": "Uncharacterized protein 011L", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q6GZW4", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 37"}], "title": "Uncharacterized protein 011R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZW3", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 35"}], "title": "Uncharacterized protein 012L", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197E7", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2023-02-22", "date_info": "entry version 37"}], "title": "Uncharacterized protein IIV3-013L", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q6GZW2", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 30"}], "title": "Uncharacterized protein 013R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZW1", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 35"}], "title": "Uncharacterized protein 014R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZW0", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 50"}], "title": "Uncharacterized protein 015R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZV8", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 35"}], "title": "Uncharacterized protein 017L", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZV7", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 33"}], "title": "Uncharacterized protein 018L", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZV6", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 87"}], "title": "Putative serine/threonine-protein kinase 019R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZV5", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 40"}], "title": "Uncharacterized protein 020R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZV4", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 35"}], "title": "Uncharacterized protein 021L", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197D8", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2022-12-14", "date_info": "entry version 35"}], "title": "Transmembrane protein 022L", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q6GZV2", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 33"}], "title": "Uncharacterized protein 023R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197D7", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2023-02-22", "date_info": "entry version 25"}], "title": "Uncharacterized protein 023R", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q6GZV1", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 37"}], "title": "Uncharacterized protein 024R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q197D5", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2006-07-11", "date_info": "sequence version 1"}, {"date": "2022-10-12", "date_info": "entry version 24"}], "title": "Uncharacterized protein 025R", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus"], "references": [{"PubMed": "16912294"}, {"DOI": "10.1128/jvi.00464-06"}]}
{"pid": " Q91G70", "dates": [{"date": "2009-06-16", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2001-12-01", "date_info": "sequence version 1"}, {"date": "2020-08-12", "date_info": "entry version 32"}], "title": "Uncharacterized protein 026R", "organism_species": "Invertebrate iridescent virus 6 (IIV-6) (Chilo iridescent virus)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Iridovirus"], "references": [{"PubMed": "17239238"}, {"DOI": "10.1186/1743-422x-4-11"}]}
{"pid": " Q6GZU9", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 49"}], "title": "Uncharacterized protein 027R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": " Q6GZU8", "dates": [{"date": "2011-06-28", "date_info": "integrated into UniProtKB/Swiss-Prot"}, {"date": "2004-07-19", "date_info": "sequence version 1"}, {"date": "2023-09-13", "date_info": "entry version 55"}], "title": "Uncharacterized protein 028R", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3)", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus"], "references": [{"PubMed": "15165820"}, {"DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": "Q6GZX4", "dates": [{"date": "28-JUN-2011", "date_info": " integrated into UniProtKB/Swiss-Prot."}, {"date": "19-JUL-2004", "date_info": " sequence version 1."}, {"date": "12-AUG-2020", "date_info": " entry version 41."}], "title": "Putative transcription factor 001R;", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3).", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus."], "references": [{"PubMed": "15165820"}, {" DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": "Q6GZX3", "dates": [{"date": "28-JUN-2011", "date_info": " integrated into UniProtKB/Swiss-Prot."}, {"date": "19-JUL-2004", "date_info": " sequence version 1."}, {"date": "12-AUG-2020", "date_info": " entry version 42."}], "title": "Uncharacterized protein 002L;", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3).", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus."], "references": [{"PubMed": "15165820"}, {" DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": "Q197F8", "dates": [{"date": "16-JUN-2009", "date_info": " integrated into UniProtKB/Swiss-Prot."}, {"date": "11-JUL-2006", "date_info": " sequence version 1."}, {"date": "12-AUG-2020", "date_info": " entry version 27."}], "title": "Uncharacterized protein 002R;", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus).", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus."], "references": [{"PubMed": "16912294"}, {" DOI": "10.1128/jvi.00464-06"}]}
{"pid": "Q197F7", "dates": [{"date": "16-JUN-2009", "date_info": " integrated into UniProtKB/Swiss-Prot."}, {"date": "11-JUL-2006", "date_info": " sequence version 1."}, {"date": "12-AUG-2020", "date_info": " entry version 23."}], "title": "Uncharacterized protein 003L;", "organism_species": "Invertebrate iridescent virus 3 (IIV-3) (Mosquito iridescent virus).", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Betairidovirinae", "Chloriridovirus."], "references": [{"PubMed": "16912294"}, {" DOI": "10.1128/jvi.00464-06"}]}
{"pid": "Q6GZX2", "dates": [{"date": "28-JUN-2011", "date_info": " integrated into UniProtKB/Swiss-Prot."}, {"date": "19-JUL-2004", "date_info": " sequence version 1."}, {"date": "12-AUG-2020", "date_info": " entry version 36."}], "title": "Uncharacterized protein 3R;", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3).", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus."], "references": [{"PubMed": "15165820"}, {" DOI": "10.1016/j.virol.2004.02.019"}]}
{"pid": "Q6GZX1", "dates": [{"date": "28-JUN-2011", "date_info": " integrated into UniProtKB/Swiss-Prot."}, {"date": "19-JUL-2004", "date_info": " sequence version 1."}, {"date": "12-AUG-2020", "date_info": " entry version 34."}], "title": "Uncharacterized protein 004R;", "organism_species": "Frog virus 3 (isolate Goorha) (FV-3).", "subjects": ["Viruses", "Varidnaviria", "Bamfordvirae", "Nucleocytoviricota", "Megaviricetes", "Pimascovirales", "Iridoviridae", "Alphairidovirinae", "Ranavirus."], "references": [{"PubMed": "15165820"}, {" DOI": "10.1016/j.virol.2004.02.019"}]}

View File

@ -1,57 +0,0 @@
package eu.dnetlib.dhp.oa.dedup;
import java.util.Objects;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class RelationAggregator extends Aggregator<Relation, Relation, Relation> {
private static final Relation ZERO = new Relation();
@Override
public Relation zero() {
return ZERO;
}
@Override
public Relation reduce(Relation b, Relation a) {
return mergeRel(b, a);
}
@Override
public Relation merge(Relation b, Relation a) {
return mergeRel(b, a);
}
@Override
public Relation finish(Relation r) {
return r;
}
private Relation mergeRel(Relation b, Relation a) {
if (Objects.equals(b, ZERO)) {
return a;
}
if (Objects.equals(a, ZERO)) {
return b;
}
b.mergeFrom(a);
return b;
}
@Override
public Encoder<Relation> bufferEncoder() {
return Encoders.kryo(Relation.class);
}
@Override
public Encoder<Relation> outputEncoder() {
return Encoders.kryo(Relation.class);
}
}

View File

@ -1,78 +0,0 @@
package eu.dnetlib.dhp.oa.dedup
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.HdfsSupport
import eu.dnetlib.dhp.schema.oaf.Relation
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql._
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.types.{DataTypes, StructField, StructType}
import org.slf4j.LoggerFactory
object SparkCleanRelation {
private val log = LoggerFactory.getLogger(classOf[SparkCleanRelation])
@throws[Exception]
def main(args: Array[String]): Unit = {
val parser = new ArgumentApplicationParser(
IOUtils.toString(
classOf[SparkCleanRelation].getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json")
)
)
parser.parseArgument(args)
val conf = new SparkConf
new SparkCleanRelation(parser, AbstractSparkAction.getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")))
}
}
class SparkCleanRelation(parser: ArgumentApplicationParser, spark: SparkSession)
extends AbstractSparkAction(parser, spark) {
override def run(isLookUpService: ISLookUpService): Unit = {
val graphBasePath = parser.get("graphBasePath")
val inputPath = parser.get("inputPath")
val outputPath = parser.get("outputPath")
SparkCleanRelation.log.info("graphBasePath: '{}'", graphBasePath)
SparkCleanRelation.log.info("inputPath: '{}'", inputPath)
SparkCleanRelation.log.info("outputPath: '{}'", outputPath)
AbstractSparkAction.removeOutputDir(spark, outputPath)
val entities =
Seq("datasource", "project", "organization", "publication", "dataset", "software", "otherresearchproduct")
val idsSchema = StructType.fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>")
val emptyIds = spark.createDataFrame(spark.sparkContext.emptyRDD[Row].setName("empty"),
idsSchema)
val ids = entities
.foldLeft(emptyIds)((ds, entity) => {
val entityPath = graphBasePath + '/' + entity
if (HdfsSupport.exists(entityPath, spark.sparkContext.hadoopConfiguration)) {
ds.union(spark.read.schema(idsSchema).json(entityPath))
} else {
ds
}
})
.filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true")
.select("id")
.distinct()
val relations = spark.read.schema(Encoders.bean(classOf[Relation]).schema).json(inputPath)
.filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true")
AbstractSparkAction.save(
relations
.join(ids, col("source") === ids("id"), "leftsemi")
.join(ids, col("target") === ids("id"), "leftsemi"),
outputPath,
SaveMode.Overwrite
)
}
}

View File

@ -3,23 +3,19 @@ package eu.dnetlib.dhp.oa.dedup;
import static org.apache.spark.sql.functions.col;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.Objects;
import org.apache.commons.beanutils.BeanUtils;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.ReduceFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
import org.apache.spark.sql.types.StructType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
@ -70,73 +66,63 @@ public class SparkPropagateRelation extends AbstractSparkAction {
log.info("workingPath: '{}'", workingPath);
log.info("graphOutputPath: '{}'", graphOutputPath);
final String outputRelationPath = DedupUtility.createEntityPath(graphOutputPath, "relation");
removeOutputDir(spark, outputRelationPath);
Dataset<Relation> mergeRels = spark
.read()
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
.as(REL_BEAN_ENC);
// <mergedObjectID, dedupID>
Dataset<Row> mergedIds = mergeRels
Dataset<Row> idsToMerge = mergeRels
.where(col("relClass").equalTo(ModelConstants.MERGES))
.select(col("source").as("dedupID"), col("target").as("mergedObjectID"))
.distinct()
.cache();
.distinct();
Dataset<Row> allRels = spark
.read()
.schema(REL_BEAN_ENC.schema())
.json(DedupUtility.createEntityPath(graphBasePath, "relation"));
.json(graphBasePath + "/relation");
Dataset<Relation> dedupedRels = allRels
.joinWith(mergedIds, allRels.col("source").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
.joinWith(mergedIds, col("_1.target").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
.joinWith(idsToMerge, allRels.col("source").equalTo(idsToMerge.col("mergedObjectID")), "left_outer")
.joinWith(idsToMerge, col("_1.target").equalTo(idsToMerge.col("mergedObjectID")), "left_outer")
.select("_1._1", "_1._2.dedupID", "_2.dedupID")
.as(Encoders.tuple(REL_BEAN_ENC, Encoders.STRING(), Encoders.STRING()))
.flatMap(SparkPropagateRelation::addInferredRelations, REL_KRYO_ENC);
.map((MapFunction<Tuple3<Relation, String, String>, Relation>) t -> {
Relation rel = t._1();
String newSource = t._2();
String newTarget = t._3();
Dataset<Relation> processedRelations = distinctRelations(
dedupedRels.union(mergeRels.map((MapFunction<Relation, Relation>) r -> r, REL_KRYO_ENC)))
.filter((FilterFunction<Relation>) r -> !Objects.equals(r.getSource(), r.getTarget()));
if (rel.getDataInfo() == null) {
rel.setDataInfo(new DataInfo());
}
save(processedRelations, outputRelationPath, SaveMode.Overwrite);
}
if (newSource != null || newTarget != null) {
rel.getDataInfo().setDeletedbyinference(false);
private static Iterator<Relation> addInferredRelations(Tuple3<Relation, String, String> t) throws Exception {
Relation existingRel = t._1();
String newSource = t._2();
String newTarget = t._3();
if (newSource != null)
rel.setSource(newSource);
if (newSource == null && newTarget == null) {
return Collections.singleton(t._1()).iterator();
}
if (newTarget != null)
rel.setTarget(newTarget);
}
// update existing relation
if (existingRel.getDataInfo() == null) {
existingRel.setDataInfo(new DataInfo());
}
existingRel.getDataInfo().setDeletedbyinference(true);
return rel;
}, REL_BEAN_ENC);
// Create new relation inferred by dedupIDs
Relation inferredRel = (Relation) BeanUtils.cloneBean(existingRel);
// ids of records that are both not deletedbyinference and not invisible
Dataset<Row> ids = validIds(spark, graphBasePath);
inferredRel.setDataInfo((DataInfo) BeanUtils.cloneBean(existingRel.getDataInfo()));
inferredRel.getDataInfo().setDeletedbyinference(false);
// filter relations that point to valid records, can force them to be visible
Dataset<Relation> cleanedRels = dedupedRels
.join(ids, col("source").equalTo(ids.col("id")), "leftsemi")
.join(ids, col("target").equalTo(ids.col("id")), "leftsemi")
.as(REL_BEAN_ENC)
.map((MapFunction<Relation, Relation>) r -> {
r.getDataInfo().setInvisible(false);
return r;
}, REL_KRYO_ENC);
if (newSource != null)
inferredRel.setSource(newSource);
if (newTarget != null)
inferredRel.setTarget(newTarget);
return Arrays.asList(existingRel, inferredRel).iterator();
}
private Dataset<Relation> distinctRelations(Dataset<Relation> rels) {
return rels
.filter(getRelationFilterFunction())
Dataset<Relation> distinctRels = cleanedRels
.groupByKey(
(MapFunction<Relation, String>) r -> String
.join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
@ -146,13 +132,33 @@ public class SparkPropagateRelation extends AbstractSparkAction {
return b;
})
.map((MapFunction<Tuple2<String, Relation>, Relation>) Tuple2::_2, REL_BEAN_ENC);
final String outputRelationPath = graphOutputPath + "/relation";
removeOutputDir(spark, outputRelationPath);
save(
distinctRels
.union(mergeRels)
.filter("source != target AND dataInfo.deletedbyinference != true AND dataInfo.invisible != true"),
outputRelationPath,
SaveMode.Overwrite);
}
private FilterFunction<Relation> getRelationFilterFunction() {
return r -> StringUtils.isNotBlank(r.getSource()) ||
StringUtils.isNotBlank(r.getTarget()) ||
StringUtils.isNotBlank(r.getRelType()) ||
StringUtils.isNotBlank(r.getSubRelType()) ||
StringUtils.isNotBlank(r.getRelClass());
static Dataset<Row> validIds(SparkSession spark, String graphBasePath) {
StructType idsSchema = StructType
.fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>");
Dataset<Row> allIds = spark.emptyDataset(RowEncoder.apply(idsSchema));
for (EntityType entityType : ModelSupport.entityTypes.keySet()) {
String entityPath = graphBasePath + '/' + entityType.name();
if (HdfsSupport.exists(entityPath, spark.sparkContext().hadoopConfiguration())) {
allIds = allIds.union(spark.read().schema(idsSchema).json(entityPath));
}
}
return allIds
.filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true")
.select("id")
.distinct();
}
}

View File

@ -1,20 +0,0 @@
[
{
"paramName": "i",
"paramLongName": "graphBasePath",
"paramDescription": "the base path of raw graph",
"paramRequired": true
},
{
"paramName": "w",
"paramLongName": "inputPath",
"paramDescription": "the path to the input relation to cleanup",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the output relation cleaned",
"paramRequired": true
}
]

View File

@ -100,35 +100,9 @@
--conf spark.sql.shuffle.partitions=15000
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--graphOutputPath</arg><arg>${workingPath}/propagaterelation/</arg>
<arg>--graphOutputPath</arg><arg>${graphOutputPath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
</spark>
<ok to="CleanRelation"/>
<error to="Kill"/>
</action>
<action name="CleanRelation">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Clean Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCleanRelation</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryOverhead}
--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.shuffle.partitions=15000
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--inputPath</arg><arg>${workingPath}/propagaterelation/relation</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/relation</arg>
</spark>
<ok to="group_entities"/>
<error to="Kill"/>
</action>
@ -152,31 +126,7 @@
--conf spark.sql.shuffle.partitions=15000
</spark-opts>
<arg>--graphInputPath</arg><arg>${graphBasePath}</arg>
<arg>--outputPath</arg><arg>${workingPath}/grouped_entities</arg>
</spark>
<ok to="dispatch_entities"/>
<error to="Kill"/>
</action>
<action name="dispatch_entities">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch grouped entitities</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryOverhead}
--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.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--checkpointPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}</arg>
<arg>--filterInvisible</arg><arg>${filterInvisible}</arg>
</spark>

View File

@ -3,7 +3,6 @@ package eu.dnetlib.dhp.oa.dedup;
import static java.nio.file.Files.createTempDirectory;
import static org.apache.spark.sql.functions.col;
import static org.apache.spark.sql.functions.count;
import static org.junit.jupiter.api.Assertions.*;
import static org.mockito.Mockito.lenient;
@ -23,14 +22,13 @@ import java.util.stream.Collectors;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
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.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.*;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
@ -46,8 +44,6 @@ import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.util.MapDocumentUtil;
import scala.Tuple2;
@ExtendWith(MockitoExtension.class)
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
@ -62,6 +58,8 @@ public class SparkDedupTest implements Serializable {
private static String testGraphBasePath;
private static String testOutputBasePath;
private static String testDedupGraphBasePath;
private static String testConsistencyGraphBasePath;
private static final String testActionSetId = "test-orchestrator";
private static String whitelistPath;
private static List<String> whiteList;
@ -75,6 +73,7 @@ public class SparkDedupTest implements Serializable {
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI())
.toFile()
.getAbsolutePath();
testOutputBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-")
.toAbsolutePath()
.toString();
@ -83,6 +82,10 @@ public class SparkDedupTest implements Serializable {
.toAbsolutePath()
.toString();
testConsistencyGraphBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-")
.toAbsolutePath()
.toString();
whitelistPath = Paths
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/whitelist.simrels.txt").toURI())
.toFile()
@ -674,22 +677,45 @@ public class SparkDedupTest implements Serializable {
assertEquals(mergedOrp, deletedOrp);
}
@Test
@Order(6)
void copyRelationsNoOpenorgsTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkCopyRelationsNoOpenorgs.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
parser
.parseArgument(
new String[] {
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
});
new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService);
final Dataset<Row> outputRels = spark.read().text(testDedupGraphBasePath + "/relation");
System.out.println(outputRels.count());
// assertEquals(2382, outputRels.count());
}
@Test
@Order(7)
void propagateRelationTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json"));
String outputRelPath = testDedupGraphBasePath + "/propagaterelation";
parser
.parseArgument(
new String[] {
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", outputRelPath
"-i", testDedupGraphBasePath, "-w", testOutputBasePath, "-o", testConsistencyGraphBasePath
});
new SparkPropagateRelation(parser, spark).run(isLookUpService);
long relations = jsc.textFile(outputRelPath + "/relation").count();
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
// assertEquals(4860, relations);
System.out.println("relations = " + relations);
@ -699,95 +725,52 @@ public class SparkDedupTest implements Serializable {
.read()
.load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*"))
.as(Encoders.bean(Relation.class));
final JavaPairRDD<String, String> mergedIds = mergeRels
.where("relClass == 'merges'")
.select(mergeRels.col("target"))
.distinct()
.toJavaRDD()
.mapToPair(
(PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(0), "d"));
JavaRDD<String> toCheck = jsc
.textFile(outputRelPath + "/relation")
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json))
.join(mergedIds)
.map(t -> t._2()._1())
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json))
.join(mergedIds)
.map(t -> t._2()._1());
Dataset<Row> inputRels = spark
.read()
.json(testDedupGraphBasePath + "/relation");
long deletedbyinference = toCheck.filter(this::isDeletedByInference).count();
long updated = toCheck.count();
Dataset<Row> outputRels = spark
.read()
.json(testConsistencyGraphBasePath + "/relation");
assertEquals(updated, deletedbyinference);
assertEquals(
0, outputRels
.filter("dataInfo.deletedbyinference == true OR dataInfo.invisible == true")
.count());
assertEquals(
5, outputRels
.filter("relClass NOT IN ('merges', 'isMergedIn')")
.count());
assertEquals(5 + mergeRels.count(), outputRels.count());
}
@Test
@Order(8)
void testCleanBaseRelations() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json"));
// append dangling relations to be cleaned up
void testCleanedPropagatedRelations() throws Exception {
Dataset<Row> df_before = spark
.read()
.schema(Encoders.bean(Relation.class).schema())
.json(testGraphBasePath + "/relation");
Dataset<Row> df_input = df_before
.unionByName(df_before.drop("source").withColumn("source", functions.lit("n/a")))
.unionByName(df_before.drop("target").withColumn("target", functions.lit("n/a")));
df_input.write().mode(SaveMode.Overwrite).json(testOutputBasePath + "_tmp");
parser
.parseArgument(
new String[] {
"--graphBasePath", testGraphBasePath,
"--inputPath", testGraphBasePath + "/relation",
"--outputPath", testDedupGraphBasePath + "/relation"
});
new SparkCleanRelation(parser, spark).run(isLookUpService);
.json(testDedupGraphBasePath + "/relation");
Dataset<Row> df_after = spark
.read()
.schema(Encoders.bean(Relation.class).schema())
.json(testDedupGraphBasePath + "/relation");
assertNotEquals(df_before.count(), df_input.count());
assertNotEquals(df_input.count(), df_after.count());
assertEquals(5, df_after.count());
}
@Test
@Order(9)
void testCleanDedupedRelations() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json"));
String inputRelPath = testDedupGraphBasePath + "/propagaterelation/relation";
// append dangling relations to be cleaned up
Dataset<Row> df_before = spark.read().schema(Encoders.bean(Relation.class).schema()).json(inputRelPath);
df_before.filter(col("dataInfo.deletedbyinference").notEqual(true)).show(50, false);
parser
.parseArgument(
new String[] {
"--graphBasePath", testGraphBasePath,
"--inputPath", inputRelPath,
"--outputPath", testDedupGraphBasePath + "/relation"
});
new SparkCleanRelation(parser, spark).run(isLookUpService);
Dataset<Row> df_after = spark
.read()
.schema(Encoders.bean(Relation.class).schema())
.json(testDedupGraphBasePath + "/relation");
.json(testConsistencyGraphBasePath + "/relation");
assertNotEquals(df_before.count(), df_after.count());
assertEquals(0, df_after.count());
assertEquals(
0, df_after
.filter("dataInfo.deletedbyinference == true OR dataInfo.invisible == true")
.count());
assertEquals(
5, df_after
.filter("relClass NOT IN ('merges', 'isMergedIn')")
.count());
}
@Test
@ -813,6 +796,7 @@ public class SparkDedupTest implements Serializable {
public static void finalCleanUp() throws IOException {
FileUtils.deleteDirectory(new File(testOutputBasePath));
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
FileUtils.deleteDirectory(new File(testConsistencyGraphBasePath));
}
public boolean isDeletedByInference(String s) {

View File

@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup;
import static java.nio.file.Files.createTempDirectory;
import static org.apache.spark.sql.functions.col;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.lenient;
@ -15,10 +16,6 @@ import java.nio.file.Paths;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
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.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
@ -33,8 +30,6 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.util.MapDocumentUtil;
import scala.Tuple2;
@ExtendWith(MockitoExtension.class)
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
@ -44,11 +39,11 @@ public class SparkOpenorgsProvisionTest implements Serializable {
ISLookUpService isLookUpService;
private static SparkSession spark;
private static JavaSparkContext jsc;
private static String testGraphBasePath;
private static String testOutputBasePath;
private static String testDedupGraphBasePath;
private static String testConsistencyGraphBasePath;
private static final String testActionSetId = "test-orchestrator";
@BeforeAll
@ -64,6 +59,9 @@ public class SparkOpenorgsProvisionTest implements Serializable {
testDedupGraphBasePath = createTempDirectory(SparkOpenorgsProvisionTest.class.getSimpleName() + "-")
.toAbsolutePath()
.toString();
testConsistencyGraphBasePath = createTempDirectory(SparkOpenorgsProvisionTest.class.getSimpleName() + "-")
.toAbsolutePath()
.toString();
FileUtils.deleteDirectory(new File(testOutputBasePath));
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
@ -76,8 +74,13 @@ public class SparkOpenorgsProvisionTest implements Serializable {
.master("local[*]")
.config(conf)
.getOrCreate();
}
jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
@AfterAll
public static void finalCleanUp() throws IOException {
FileUtils.deleteDirectory(new File(testOutputBasePath));
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
FileUtils.deleteDirectory(new File(testConsistencyGraphBasePath));
}
@BeforeEach
@ -186,26 +189,21 @@ public class SparkOpenorgsProvisionTest implements Serializable {
new SparkUpdateEntity(parser, spark).run(isLookUpService);
long organizations = jsc.textFile(testDedupGraphBasePath + "/organization").count();
Dataset<Row> organizations = spark.read().json(testDedupGraphBasePath + "/organization");
long mergedOrgs = spark
Dataset<Row> mergedOrgs = spark
.read()
.load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel")
.as(Encoders.bean(Relation.class))
.where("relClass=='merges'")
.javaRDD()
.map(Relation::getTarget)
.distinct()
.count();
.select("target")
.distinct();
assertEquals(80, organizations);
assertEquals(80, organizations.count());
long deletedOrgs = jsc
.textFile(testDedupGraphBasePath + "/organization")
.filter(this::isDeletedByInference)
.count();
Dataset<Row> deletedOrgs = organizations
.filter("dataInfo.deletedbyinference = TRUE");
assertEquals(mergedOrgs, deletedOrgs);
assertEquals(mergedOrgs.count(), deletedOrgs.count());
}
@Test
@ -226,10 +224,9 @@ public class SparkOpenorgsProvisionTest implements Serializable {
new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService);
final JavaRDD<String> rels = jsc.textFile(testDedupGraphBasePath + "/relation");
assertEquals(2382, rels.count());
final Dataset<Row> outputRels = spark.read().text(testDedupGraphBasePath + "/relation");
assertEquals(2382, outputRels.count());
}
@Test
@ -244,51 +241,41 @@ public class SparkOpenorgsProvisionTest implements Serializable {
parser
.parseArgument(
new String[] {
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
"-i", testDedupGraphBasePath, "-w", testOutputBasePath, "-o", testConsistencyGraphBasePath
});
new SparkPropagateRelation(parser, spark).run(isLookUpService);
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
assertEquals(4896, relations);
// check deletedbyinference
final Dataset<Relation> mergeRels = spark
.read()
.load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*"))
.as(Encoders.bean(Relation.class));
final JavaPairRDD<String, String> mergedIds = mergeRels
Dataset<Row> inputRels = spark
.read()
.json(testDedupGraphBasePath + "/relation");
Dataset<Row> outputRels = spark
.read()
.json(testConsistencyGraphBasePath + "/relation");
final Dataset<Row> mergedIds = mergeRels
.where("relClass == 'merges'")
.select(mergeRels.col("target"))
.distinct()
.toJavaRDD()
.mapToPair(
(PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(0), "d"));
.select(col("target").as("id"))
.distinct();
JavaRDD<String> toCheck = jsc
.textFile(testDedupGraphBasePath + "/relation")
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json))
.join(mergedIds)
.map(t -> t._2()._1())
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json))
.join(mergedIds)
.map(t -> t._2()._1());
Dataset<Row> toUpdateRels = inputRels
.as("rel")
.join(mergedIds.as("s"), col("rel.source").equalTo(col("s.id")), "left_outer")
.join(mergedIds.as("t"), col("rel.target").equalTo(col("t.id")), "left_outer")
.filter("s.id IS NOT NULL OR t.id IS NOT NULL")
.distinct();
long deletedbyinference = toCheck.filter(this::isDeletedByInference).count();
long updated = toCheck.count();
Dataset<Row> updatedRels = inputRels
.select("source", "target", "relClass")
.except(outputRels.select("source", "target", "relClass"));
assertEquals(updated, deletedbyinference);
assertEquals(toUpdateRels.count(), updatedRels.count());
assertEquals(140, outputRels.count());
}
@AfterAll
public static void finalCleanUp() throws IOException {
FileUtils.deleteDirectory(new File(testOutputBasePath));
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
}
public boolean isDeletedByInference(String s) {
return s.contains("\"deletedbyinference\":true");
}
}

View File

@ -0,0 +1,13 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>dhp-workflows</artifactId>
<groupId>eu.dnetlib.dhp</groupId>
<version>1.2.5-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>dhp-distcp</artifactId>
</project>

View File

@ -0,0 +1,18 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>sourceNN</name>
<value>webhdfs://namenode2.hadoop.dm.openaire.eu:50071</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
</configuration>

View File

@ -0,0 +1,46 @@
<workflow-app name="distcp" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourceNN</name>
<description>the source name node</description>
</property>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>targetPath</name>
<description>the target path</description>
</property>
<property>
<name>hbase_dump_distcp_memory_mb</name>
<value>6144</value>
<description>memory for distcp action copying InfoSpace dump from remote cluster</description>
</property>
<property>
<name>hbase_dump_distcp_num_maps</name>
<value>1</value>
<description>maximum number of simultaneous copies of InfoSpace dump from remote location</description>
</property>
</parameters>
<start to="distcp"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="distcp">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>-Dmapreduce.map.memory.mb=${hbase_dump_distcp_memory_mb}</arg>
<arg>-pb</arg>
<arg>-m ${hbase_dump_distcp_num_maps}</arg>
<arg>${sourceNN}/${sourcePath}</arg>
<arg>${nameNode}/${targetPath}</arg>
</distcp>
<ok to="End" />
<error to="Kill" />
</action>
<end name="End"/>
</workflow-app>

View File

@ -5,7 +5,6 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;

View File

@ -14,7 +14,6 @@ import org.apache.spark.SparkConf;
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.Row;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
@ -85,26 +84,19 @@ public class SparkCountryPropagationJob {
Dataset<R> res = readPath(spark, sourcePath, resultClazz);
log.info("Reading prepared info: {}", preparedInfoPath);
final Dataset<Row> preparedInfoRaw = spark
Dataset<ResultCountrySet> prepared = spark
.read()
.json(preparedInfoPath);
.json(preparedInfoPath)
.as(Encoders.bean(ResultCountrySet.class));
res
.joinWith(prepared, res.col("id").equalTo(prepared.col("resultId")), "left_outer")
.map(getCountryMergeFn(), Encoders.bean(resultClazz))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
if (!preparedInfoRaw.isEmpty()) {
final Dataset<ResultCountrySet> prepared = preparedInfoRaw.as(Encoders.bean(ResultCountrySet.class));
res
.joinWith(prepared, res.col("id").equalTo(prepared.col("resultId")), "left_outer")
.map(getCountryMergeFn(), Encoders.bean(resultClazz))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
} else {
res
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
}
}
private static <R extends Result> MapFunction<Tuple2<R, ResultCountrySet>, R> getCountryMergeFn() {

View File

@ -260,7 +260,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.dynamicAllocation.enabled=true
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
--conf spark.sql.shuffle.partitions=10000
</spark-opts>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo/mergedCommunityAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/publication</arg>
@ -290,7 +289,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.dynamicAllocation.enabled=true
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
--conf spark.sql.shuffle.partitions=5000
</spark-opts>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo/mergedCommunityAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/dataset</arg>
@ -320,7 +318,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.dynamicAllocation.enabled=true
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
--conf spark.sql.shuffle.partitions=2000
</spark-opts>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo/mergedCommunityAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/otherresearchproduct</arg>
@ -350,7 +347,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.dynamicAllocation.enabled=true
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
--conf spark.sql.shuffle.partitions=1000
</spark-opts>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo/mergedCommunityAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/software</arg>

View File

@ -96,30 +96,7 @@
--conf spark.sql.shuffle.partitions=15000
</spark-opts>
<arg>--graphInputPath</arg><arg>${graphBasePath}</arg>
<arg>--outputPath</arg><arg>${workingPath}/grouped_entities</arg>
</spark>
<ok to="dispatch_entities"/>
<error to="Kill"/>
</action>
<action name="dispatch_entities">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch grouped entities</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--checkpointPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}</arg>
<arg>--filterInvisible</arg><arg>${filterInvisible}</arg>
</spark>

View File

@ -10,7 +10,6 @@ import java.nio.file.Path;
import java.nio.file.Paths;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
@ -23,9 +22,9 @@ import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob;
import eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.utils.DHPUtils;
@ -40,8 +39,9 @@ public class GroupEntitiesSparkJobTest {
private static Path workingDir;
private Path dataInputPath;
private Path groupEntityPath;
private Path dispatchEntityPath;
private Path checkpointPath;
private Path outputPath;
@BeforeAll
public static void beforeAll() throws IOException {
@ -58,8 +58,8 @@ public class GroupEntitiesSparkJobTest {
@BeforeEach
public void beforeEach() throws IOException, URISyntaxException {
dataInputPath = Paths.get(ClassLoader.getSystemResource("eu/dnetlib/dhp/oa/graph/group").toURI());
groupEntityPath = workingDir.resolve("grouped_entity");
dispatchEntityPath = workingDir.resolve("dispatched_entity");
checkpointPath = workingDir.resolve("grouped_entity");
outputPath = workingDir.resolve("dispatched_entity");
}
@AfterAll
@ -76,39 +76,28 @@ public class GroupEntitiesSparkJobTest {
Boolean.FALSE.toString(),
"-graphInputPath",
dataInputPath.toString(),
"-checkpointPath",
checkpointPath.toString(),
"-outputPath",
groupEntityPath.toString()
outputPath.toString(),
"-filterInvisible",
Boolean.FALSE.toString()
});
Dataset<Result> output = spark
Dataset<OafEntity> checkpointTable = spark
.read()
.textFile(groupEntityPath.toString())
.map((MapFunction<String, String>) s -> StringUtils.substringAfter(s, "|"), Encoders.STRING())
.map((MapFunction<String, Result>) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class));
.load(checkpointPath.toString())
.selectExpr("COALESCE(*)")
.as(Encoders.kryo(OafEntity.class));
assertEquals(
1,
output
checkpointTable
.filter(
(FilterFunction<Result>) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9"
(FilterFunction<OafEntity>) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9"
.equals(r.getId()) &&
r.getCollectedfrom().stream().anyMatch(kv -> kv.getValue().equalsIgnoreCase("zenodo")))
.count());
}
@Test
@Order(2)
void testDispatchEntities() throws Exception {
DispatchEntitiesSparkJob.main(new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
groupEntityPath.toString(),
"-outputPath",
dispatchEntityPath.resolve(".").toString(),
"-filterInvisible",
Boolean.TRUE.toString()
});
Dataset<Result> output = spark
.read()
@ -116,7 +105,7 @@ public class GroupEntitiesSparkJobTest {
DHPUtils
.toSeq(
HdfsSupport
.listFiles(dispatchEntityPath.toString(), spark.sparkContext().hadoopConfiguration())))
.listFiles(outputPath.toString(), spark.sparkContext().hadoopConfiguration())))
.map((MapFunction<String, Result>) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class));
assertEquals(3, output.count());

View File

@ -1,586 +0,0 @@
<record>
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<header>
<dri:objIdentifier>doi_dedup___::e225555a08a082ad8f53f179bc59c5d0</dri:objIdentifier>
<dri:dateOfCollection>2023-01-27T05:32:10Z</dri:dateOfCollection>
</header>
<metadata>
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
<oaf:result>
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2"/>
<collectedfrom name="UnpayWall" id="openaire____::8ac8380272269217cb09a928c8caa993"/>
<collectedfrom name="ORCID" id="openaire____::806360c771262b4d6770e7cdf04b5c5a"/>
<collectedfrom name="Microsoft Academic Graph" id="openaire____::5f532a3fc4f1ea403f37070f59a7a53a"/>
<collectedfrom name="Oxford University Research Archive"
id="opendoar____::2290a7385ed77cc5592dc2153229f082"/>
<collectedfrom name="PubMed Central" id="opendoar____::eda80a3d5b344bc40f3bc04f65b7a357"/>
<collectedfrom name="Europe PubMed Central" id="opendoar____::8b6dd7db9af49e67306feb59a8bdc52c"/>
<originalId>10.1098/rsta.2020.0257</originalId>
<originalId>50|doiboost____::e225555a08a082ad8f53f179bc59c5d0</originalId>
<originalId>3211056089</originalId>
<originalId>50|od______1064::83eb0f76b60445d72bb7428a1b68ef1a</originalId>
<originalId>oai:ora.ox.ac.uk:uuid:9fc4563a-07e1-41d1-8b99-31ce2f8ac027</originalId>
<originalId>50|od_______267::6d978e42c57dfc79d61a84ab5be28cb8</originalId>
<originalId>oai:pubmedcentral.nih.gov:8543046</originalId>
<originalId>od_______267::6d978e42c57dfc79d61a84ab5be28cb8</originalId>
<originalId>34689630</originalId>
<originalId>PMC8543046</originalId>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types">10.1098/rsta.2020.0257
</pid>
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:crosswalk:repository"
trust="0.9">PMC8543046
</pid>
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types" schemename="dnet:pid_types"
inferred="false" provenanceaction="sysimport:crosswalk:repository" trust="0.9">34689630
</pid>
<measure id="influence" score="5.534974E-9" class="C"/>
<measure id="popularity" score="2.0811036E-8" class="C"/>
<measure id="influence_alt" score="8" class="C"/>
<measure id="popularity_alt" score="2.736" class="C"/>
<measure id="impulse" score="8" class="C"/>
<measure id="downloads" count="1"/>
<measure id="views" count="4"/>
<title classid="alternative title" classname="alternative title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title">A completely automated pipeline for 3D reconstruction of
human heart from 2D cine magnetic resonance slices.
</title>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title">A completely automated pipeline for 3D reconstruction of
human heart from 2D cine magnetic resonance slices
</title>
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<creator rank="8" URL="https://academic.microsoft.com/#/detail/2103042895">Vicente Grau</creator>
<creator rank="1" orcid="0000-0001-8198-5128"
URL="https://academic.microsoft.com/#/detail/2693349397">Abhirup Banerjee
</creator>
<creator rank="3" URL="https://academic.microsoft.com/#/detail/1425738153">Ernesto Zacur</creator>
<creator rank="6" URL="https://academic.microsoft.com/#/detail/2043191934">Robin P. Choudhury
</creator>
<creator rank="7" URL="https://academic.microsoft.com/#/detail/2166186014">Blanca Rodriguez
</creator>
<creator rank="2" URL="https://academic.microsoft.com/#/detail/2889449215">Julia Camps</creator>
<creator rank="5" URL="https://academic.microsoft.com/#/detail/264390263">Yoram Rudy</creator>
<creator rank="4" URL="https://academic.microsoft.com/#/detail/2713879776">Christopher M. Andrews
</creator>
<country classid="GB" classname="United Kingdom" schemeid="dnet:countries"
schemename="dnet:countries"/>
<dateofacceptance>2021-10-01</dateofacceptance>
<description>Cardiac magnetic resonance (CMR) imaging is a valuable modality in the diagnosis and
characterization of cardiovascular diseases, since it can identify abnormalities in structure
and function of the myocardium non-invasively and without the need for ionizing radiation.
However, in clinical practice, it is commonly acquired as a collection of separated and
independent 2D image planes, which limits its accuracy in 3D analysis. This paper presents a
completely automated pipeline for generating patient-specific 3D biventricular heart models from
cine magnetic resonance (MR) slices. Our pipeline automatically selects the relevant cine MR
images, segments them using a deep learning-based method to extract the heart contours, and
aligns the contours in 3D space correcting possible misalignments due to breathing or subject
motion first using the intensity and contours information from the cine data and next with the
help of a statistical shape model. Finally, the sparse 3D representation of the contours is used
to generate a smooth 3D biventricular mesh. The computational pipeline is applied and evaluated
in a CMR dataset of 20 healthy subjects. Our results show an average reduction of misalignment
artefacts from 1.82±1.60mm to 0.72±0.73mm over 20 subjects, in terms of distance from the
final reconstructed mesh. The high-resolution 3D biventricular meshes obtained with our
computational pipeline are used for simulations of electrical activation patterns, showing
agreement with non-invasive electrocardiographic imaging. The automatic methodologies presented
here for patient-specific MR imaging-based 3D biventricular representations contribute to the
efficient realization of precision medicine, enabling the enhanced interpretability of clinical
data, the digital twin vision through patient-specific image-based modelling and simulation, and
augmented reality applications.
This article is part of the theme issue Advanced computation in cardiovascular physiology: new
challenges and opportunities.
</description>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">General Physics and Astronomy
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">General Engineering
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">General Mathematics
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Pipeline (computing)
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Cine mri
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Structure and function
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Cardiac magnetic resonance
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Magnetic resonance imaging
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.5338496">medicine.diagnostic_test
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.5338496">medicine
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Human heart
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Modality (humancomputer interaction)
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">3D reconstruction
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Computer science
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Nuclear magnetic resonance
</subject>
<subject classid="SDG" classname="Sustainable Development Goals"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:sdg">3. Good health
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03 medical and health sciences
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0302 clinical medicine
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030218 Nuclear Medicine &amp;
Medical Imaging
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03021801 Radiology/Image
segmentation
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03021801 Radiology/Image
segmentation - deep learning/datum
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030204 Cardiovascular System
&amp; Hematology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03020401 Aging-associated
diseases/Heart diseases
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030217 Neurology &amp;
Neurosurgery
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03021701 Brain/Neural circuits
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">Articles
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">Research Articles
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">cardiac mesh reconstruction
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">cine MRI
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">misalignment correction
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">electrophysiological
simulation
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">ECGI
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.9">Heart
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.9">Humans
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.9">Imaging, Three-Dimensional
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.9">Magnetic Resonance Imaging
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.9">Magnetic Resonance Imaging, Cine
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.9">Magnetic Resonance Spectroscopy
</subject>
<language classid="und" classname="Undetermined" schemeid="dnet:languages"
schemename="dnet:languages"/>
<relevantdate classid="created" classname="created" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date">2021-10-25
</relevantdate>
<relevantdate classid="published-online" classname="published-online" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date">2021-10-25
</relevantdate>
<relevantdate classid="published-print" classname="published-print" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date">2021-12-13
</relevantdate>
<relevantdate classid="UNKNOWN" classname="UNKNOWN" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">2021-01-01
</relevantdate>
<relevantdate classid="available" classname="available" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">2023-01-05
</relevantdate>
<relevantdate classid="Accepted" classname="Accepted" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">2021-05-28
</relevantdate>
<relevantdate classid="issued" classname="issued" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">2023-01-05
</relevantdate>
<publisher>The Royal Society</publisher>
<source>Crossref</source>
<source/>
<source>Philosophical transactions. Series A, Mathematical, physical, and engineering sciences
</source>
<resulttype classid="publication" classname="publication" schemeid="dnet:result_typologies"
schemename="dnet:result_typologies"/>
<resourcetype classid="UNKNOWN" classname="UNKNOWN" schemeid="dnet:dataCite_resource"
schemename="dnet:dataCite_resource"/>
<journal issn="1364-503X" eissn="1471-2962" vol="379">Philosophical Transactions of the Royal
Society A: Mathematical, Physical and Engineering Sciences
</journal>
<context id="dth" label="Digital Twins in Health" type="community"></context>
<context id="EC" label="European Commission" type="funding">
<category id="EC::H2020" label="Horizon 2020 Framework Programme">
<concept id="EC::H2020::RIA" label="Research and Innovation action"/>
</category>
</context>
<datainfo>
<inferred>true</inferred>
<deletedbyinference>false</deletedbyinference>
<trust>0.8</trust>
<inferenceprovenance>dedup-result-decisiontree-v3</inferenceprovenance>
<provenanceaction classid="sysimport:dedup" classname="Inferred by OpenAIRE"
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
</datainfo>
<rels>
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="sysimport:actionset">
<to class="hasAuthorInstitution" scheme="dnet:result_organization_relations"
type="organization">openorgs____::6a7b1b4c40a067a1f209de6867fe094d
</to>
<country classid="GB" classname="United Kingdom" schemeid="dnet:countries"
schemename="dnet:countries"/>
<legalname>University of Oxford</legalname>
<legalshortname>University of Oxford</legalshortname>
</rel>
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="sysimport:actionset">
<to class="IsSupplementedBy" scheme="dnet:result_result_relations" type="publication">
doi_dedup___::015b27b0b7c55649236bf23a5c75f817
</to>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">10.6084/m9.figshare.15656924.v2
</pid>
<dateofacceptance>2021-01-01</dateofacceptance>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title">Implementation Details of the Reconstruction
Pipeline and Electrophysiological Inference Results from A completely automated pipeline
for 3D reconstruction of human heart from 2D cine magnetic resonance slices
</title>
<publisher>The Royal Society</publisher>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">10.6084/m9.figshare.15656924
</pid>
</rel>
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="sysimport:actionset">
<to class="isProducedBy" scheme="dnet:result_project_relations" type="project">
corda__h2020::27f89b49dee12d828cc0f90f51727204
</to>
<code>823712</code>
<funding>
<funder id="ec__________::EC" shortname="EC" name="European Commission"
jurisdiction="EU"/>
<funding_level_0 name="H2020">ec__________::EC::H2020</funding_level_0>
<funding_level_1 name="RIA">ec__________::EC::H2020::RIA</funding_level_1>
</funding>
<acronym>CompBioMed2</acronym>
<title>A Centre of Excellence in Computational Biomedicine</title>
</rel>
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="sysimport:actionset">
<to class="IsSupplementedBy" scheme="dnet:result_result_relations" type="dataset">
doi_dedup___::be1ef3b30a8d7aa7e4dfe1570d5febf7
</to>
<dateofacceptance>2021-01-01</dateofacceptance>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">10.6084/m9.figshare.15656927
</pid>
<publisher>The Royal Society</publisher>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">10.6084/m9.figshare.15656927.v1
</pid>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title">Montage Video of the Stepwise Performance of 3D
Reconstruction Pipeline on All 20 Patients from A completely automated pipeline for 3D
reconstruction of human heart from 2D cine magnetic resonance slices
</title>
</rel>
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="sysimport:actionset">
<to class="IsSupplementedBy" scheme="dnet:result_result_relations" type="dataset">
doi_________::9f9f2328e11d379b14cb888209e33088
</to>
<dateofacceptance>2021-01-01</dateofacceptance>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">10.6084/m9.figshare.15656924.v1
</pid>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title">Implementation Details of the Reconstruction
Pipeline and Electrophysiological Inference Results from A completely automated pipeline
for 3D reconstruction of human heart from 2D cine magnetic resonance slices
</title>
<publisher>The Royal Society</publisher>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
</rel>
</rels>
<children>
<result objidentifier="pmid________::9c855c9c4a8018df23f9d51f879b62b1">
<dateofacceptance>2021-10-01</dateofacceptance>
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">34689630
</pid>
<collectedfrom name="PubMed Central" id="opendoar____::eda80a3d5b344bc40f3bc04f65b7a357"/>
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">34689630
</pid>
<publisher>The Royal Society</publisher>
<collectedfrom name="Europe PubMed Central"
id="opendoar____::8b6dd7db9af49e67306feb59a8bdc52c"/>
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">PMC8543046
</pid>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">A completely automated
pipeline for 3D reconstruction of human heart from 2D cine magnetic resonance slices
</title>
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">PMC8543046
</pid>
</result>
<result objidentifier="od______1064::83eb0f76b60445d72bb7428a1b68ef1a">
<dateofacceptance>2023-01-05</dateofacceptance>
<publisher>Royal Society</publisher>
<collectedfrom name="Oxford University Research Archive"
id="opendoar____::2290a7385ed77cc5592dc2153229f082"/>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">A completely automated
pipeline for 3D reconstruction of human heart from 2D cine magnetic resonance slices
</title>
</result>
<result objidentifier="doi_________::e225555a08a082ad8f53f179bc59c5d0">
<dateofacceptance>2021-10-25</dateofacceptance>
<publisher>The Royal Society</publisher>
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2"/>
<title classid="alternative title" classname="alternative title"
schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">A completely
automated pipeline for 3D reconstruction of human heart from 2D cine magnetic resonance
slices.
</title>
<collectedfrom name="UnpayWall" id="openaire____::8ac8380272269217cb09a928c8caa993"/>
<collectedfrom name="ORCID" id="openaire____::806360c771262b4d6770e7cdf04b5c5a"/>
<collectedfrom name="Microsoft Academic Graph"
id="openaire____::5f532a3fc4f1ea403f37070f59a7a53a"/>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types">10.1098/rsta.2020.0257
</pid>
</result>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<collectedfrom name="Oxford University Research Archive"
id="opendoar____::2290a7385ed77cc5592dc2153229f082"/>
<hostedby name="Oxford University Research Archive"
id="opendoar____::2290a7385ed77cc5592dc2153229f082"/>
<dateofacceptance>2023-01-05</dateofacceptance>
<instancetype classid="0038" classname="Other literature type"
schemeid="dnet:publication_resource" schemename="dnet:publication_resource"/>
<alternateidentifier classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">34689630
</alternateidentifier>
<alternateidentifier classid="doi" classname="Digital Object Identifier"
schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">
10.1098/rsta.2020.0257
</alternateidentifier>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels"
schemename="dnet:review_levels"/>
<license>http://creativecommons.org/licenses/by/4.0/</license>
<webresource>
<url>https://ora.ox.ac.uk/objects/uuid:9fc4563a-07e1-41d1-8b99-31ce2f8ac027</url>
</webresource>
</instance>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<collectedfrom name="UnpayWall" id="openaire____::8ac8380272269217cb09a928c8caa993"/>
<hostedby
name="Philosophical Transactions of the Royal Society A Mathematical Physical and Engineering Sciences"
id="issn___print::c72e72d9860ed6d9ec013e20c9421ba0"/>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource"
schemename="dnet:publication_resource"/>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types">10.1098/rsta.2020.0257
</pid>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels"
schemename="dnet:review_levels"/>
<webresource>
<url>https://www.ncbi.nlm.nih.gov/pmc/articles/PMC8543046</url>
</webresource>
</instance>
<instance>
<accessright classid="CLOSED" classname="Closed Access" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2"/>
<hostedby
name="Philosophical Transactions of the Royal Society A Mathematical Physical and Engineering Sciences"
id="issn___print::c72e72d9860ed6d9ec013e20c9421ba0"/>
<dateofacceptance>2021-10-25</dateofacceptance>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource"
schemename="dnet:publication_resource"/>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types">10.1098/rsta.2020.0257
</pid>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels"
schemename="dnet:review_levels"/>
<license>https://royalsociety.org/journals/ethics-policies/data-sharing-mining/</license>
<webresource>
<url>https://doi.org/10.1098/rsta.2020.0257</url>
</webresource>
</instance>
<instance>
<accessright classid="UNKNOWN" classname="not available" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<collectedfrom name="Europe PubMed Central"
id="opendoar____::8b6dd7db9af49e67306feb59a8bdc52c"/>
<hostedby name="Unknown Repository" id="openaire____::55045bd2a65019fd8e6741a755395c8c"/>
<dateofacceptance>2021-10-25</dateofacceptance>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource"
schemename="dnet:publication_resource"/>
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">34689630
</pid>
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
trust="0.9">PMC8543046
</pid>
<alternateidentifier classid="doi" classname="Digital Object Identifier"
schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:actionset" trust="0.9">
10.1098/rsta.2020.0257
</alternateidentifier>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels"
schemename="dnet:review_levels"/>
<webresource>
<url>https://pubmed.ncbi.nlm.nih.gov/34689630</url>
</webresource>
</instance>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<collectedfrom name="PubMed Central" id="opendoar____::eda80a3d5b344bc40f3bc04f65b7a357"/>
<hostedby name="Europe PubMed Central" id="opendoar____::8b6dd7db9af49e67306feb59a8bdc52c"/>
<dateofacceptance>2021-10-01</dateofacceptance>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource"
schemename="dnet:publication_resource"/>
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">34689630
</pid>
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">PMC8543046
</pid>
<alternateidentifier classid="doi" classname="Digital Object Identifier"
schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:repository" trust="0.9">
10.1098/rsta.2020.0257
</alternateidentifier>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels"
schemename="dnet:review_levels"/>
<webresource>
<url>http://europepmc.org/articles/PMC8543046</url>
</webresource>
</instance>
</children>
</oaf:result>
</oaf:entity>
</metadata>
</result>
</record>

View File

@ -1,238 +0,0 @@
<record>
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<header>
<dri:objIdentifier>doi_dedup___::88a9861b26cdda1c3dd162d11f0bedbe</dri:objIdentifier>
<dri:dateOfCollection>2023-03-13T00:12:27+0000</dri:dateOfCollection>
<dri:dateOfTransformation>2023-03-13T04:39:52.807Z</dri:dateOfTransformation>
</header>
<metadata>
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
<oaf:result>
<collectedfrom name="NARCIS" id="openaire____::fdb035c8b3e0540a8d9a561a6c44f4de" />
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2" />
<collectedfrom name="UnpayWall" id="openaire____::8ac8380272269217cb09a928c8caa993" />
<collectedfrom name="ORCID" id="openaire____::806360c771262b4d6770e7cdf04b5c5a" />
<collectedfrom name="Microsoft Academic Graph" id="openaire____::5f532a3fc4f1ea403f37070f59a7a53a" />
<collectedfrom name="Europe PubMed Central" id="opendoar____::8b6dd7db9af49e67306feb59a8bdc52c" />
<collectedfrom name="NARCIS" id="eurocrisdris::fe4903425d9040f680d8610d9079ea14" />
<originalId>oai:pure.eur.nl:publications/70bf9bd0-5ea6-45fd-bb62-8d79b48cd69f</originalId>
<originalId>50|narcis______::3df5b8b060b819af0d439dd6751c8a77</originalId>
<originalId>10.2196/33081</originalId>
<originalId>50|doiboost____::88a9861b26cdda1c3dd162d11f0bedbe</originalId>
<originalId>3212148341</originalId>
<originalId>od_______267::276eb3ebee07cf1f3e8bfc43926fd0c2</originalId>
<originalId>35099399</originalId>
<originalId>PMC8844982</originalId>
<originalId>oai:services.nod.dans.knaw.nl:Publications/eur:oai:pure.eur.nl:publications/70bf9bd0-5ea6-45fd-bb62-8d79b48cd69f</originalId>
<originalId>50|dris___00893::107e97e645cbb06fb7b454ce2569d6c2</originalId>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types">10.2196/33081</pid>
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">35099399</pid>
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">PMC8844982</pid>
<measure id="influence" score="4.9560573E-9" class="C" />
<measure id="popularity" score="1.6705286E-8" class="C" />
<measure id="influence_alt" score="6" class="C" />
<measure id="popularity_alt" score="2.16" class="C" />
<measure id="impulse" score="6" class="C" />
<title classid="alternative title" classname="alternative title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Mapping the Ethical Issues of Digital Twins for Personalised Healthcare Service (Preprint)</title>
<title classid="subtitle" classname="subtitle" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title" inferred="false" provenanceaction="sysimport:crosswalk:datasetarchive" trust="0.9">Preliminary Mapping Study</title>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title" inferred="false" provenanceaction="sysimport:crosswalk:repository" trust="0.9">Ethical Issues of Digital Twins for Personalized Health Care Service: Preliminary Mapping Study</title>
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<creator rank="2" URL="https://academic.microsoft.com/#/detail/3211925536" orcid="0000-0001-9882-0389">Ki-hun Kim</creator>
<creator rank="1" URL="https://academic.microsoft.com/#/detail/3214669734" orcid="0000-0001-5334-8615">Pei-hua Huang</creator>
<creator rank="3" URL="https://academic.microsoft.com/#/detail/3214611946" orcid="0000-0003-4283-9659">Maartje Schermer</creator>
<contributor>Public Health</contributor>
<country classid="NL" classname="Netherlands" schemeid="dnet:countries" schemename="dnet:countries" />
<dateofacceptance>2022-01-01</dateofacceptance>
<description>Background: The concept of digital twins has great potential for transforming the existing health care system by making it more personalized. As a convergence of health care, artificial intelligence, and information and communication technologies, personalized health care services that are developed under the concept of digital twins raise a myriad of ethical issues. Although some of the ethical issues are known to researchers working on digital health and personalized medicine, currently, there is no comprehensive review that maps the major ethical risks of digital twins for personalized health care services. Objective This study aims to fill the research gap by identifying the major ethical risks of digital twins for personalized health care services. We first propose a working definition for digital twins for personalized health care services to facilitate future discussions on the ethical issues related to these emerging digital health services. We then develop a process-oriented ethical map to identify the major ethical risks in each of the different data processing phases. MethodsWe resorted to the literature on eHealth, personalized medicine, precision medicine, and information engineering to identify potential issues and developed a process-oriented ethical map to structure the inquiry in a more systematic way. The ethical map allows us to see how each of the major ethical concerns emerges during the process of transforming raw data into valuable information. Developers of a digital twin for personalized health care service may use this map to identify ethical risks during the development stage in a more systematic way and can proactively address them. ResultsThis paper provides a working definition of digital twins for personalized health care services by identifying 3 features that distinguish the new application from other eHealth services. On the basis of the working definition, this paper further layouts 10 major operational problems and the corresponding ethical risks. ConclusionsIt is challenging to address all the major ethical risks that a digital twin for a personalized health care service might encounter proactively without a conceptual map at hand. The process-oriented ethical map we propose here can assist the developers of digital twins for personalized health care services in analyzing ethical risks in a more systematic manner. </description>
<subject classid="mesh" classname="Medical Subject Headings" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="true" inferenceprovenance="iis::document_classes" provenanceaction="iis" trust="0.891">education</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">Health Informatics</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">Ethical issues</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">Healthcare service</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">Psychology</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">Internet privacy</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.4395309">business.industry</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.4395309">business</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">Preprint</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">Artificial Intelligence</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">Delivery of Health Care</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">Health Services</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">Humans</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">Precision Medicine</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">Telemedicine</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03 medical and health sciences</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0302 clinical medicine
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030212 General &amp; Internal Medicine
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03021201 Health care/Health care quality - datum/health care
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03021201 Health care/Health care quality - datum/electronic health
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0301 basic medicine
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030104 Developmental Biology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0303 health sciences
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030304 Developmental Biology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030304 Developmental Biology - datum/datum management/ethical
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">06 humanities and the arts</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0603 philosophy, ethics and religion
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">060301 Applied Ethics
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">06030101 Bioethics/Coordinates on Wikidata - intelligence/artificial intelligence/ethical
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">06030101 Bioethics/Coordinates on Wikidata - datum/ethical
</subject>
<language classid="eng" classname="English" schemeid="dnet:languages" schemename="dnet:languages" />
<relevantdate classid="created" classname="created" schemeid="dnet:dataCite_date" schemename="dnet:dataCite_date">2021-11-17</relevantdate>
<relevantdate classid="published-online" classname="published-online" schemeid="dnet:dataCite_date" schemename="dnet:dataCite_date">2022-01-31</relevantdate>
<relevantdate classid="issued" classname="issued" schemeid="dnet:dataCite_date" schemename="dnet:dataCite_date" inferred="false" provenanceaction="sysimport:crosswalk:datasetarchive" trust="0.9">2022-01-01</relevantdate>
<source>Crossref</source>
<source />
<source>Journal of Medical Internet Research, 24(1):e33081. Journal of medical Internet Research</source>
<source>urn:issn:1439-4456</source>
<source>VOLUME=24;ISSUE=1;ISSN=1439-4456;TITLE=Journal of Medical Internet Research</source>
<format>application/pdf</format>
<resulttype classid="publication" classname="publication" schemeid="dnet:result_typologies" schemename="dnet:result_typologies" />
<resourcetype classid="0001" classname="0001" schemeid="dnet:dataCite_resource" schemename="dnet:dataCite_resource" />
<context id="knowmad" label="Knowmad Institut" type="community" />
<context id="dth" label="Digital Twins in Health" type="community"/>
<datainfo>
<inferred>true</inferred>
<deletedbyinference>false</deletedbyinference>
<trust>0.8</trust>
<inferenceprovenance>dedup-result-decisiontree-v3</inferenceprovenance>
<provenanceaction classid="sysimport:dedup" classname="Inferred by OpenAIRE" schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions" />
</datainfo>
<rels />
<children>
<result objidentifier="narcis______::3df5b8b060b819af0d439dd6751c8a77">
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title" inferred="false" provenanceaction="sysimport:crosswalk:repository" trust="0.9">Ethical Issues of Digital Twins for Personalized Health Care Service: Preliminary Mapping Study</title>
<collectedfrom name="NARCIS" id="openaire____::fdb035c8b3e0540a8d9a561a6c44f4de" />
<dateofacceptance>2022-01-01</dateofacceptance>
</result>
<result objidentifier="pmid________::70f4213809e44f787bf9f6ffa50ff98f">
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">PMC8844982</pid>
<collectedfrom name="Europe PubMed Central" id="opendoar____::8b6dd7db9af49e67306feb59a8bdc52c" />
<dateofacceptance>2021-08-23</dateofacceptance>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">Ethical Issues of Digital Twins for Personalized Health Care Service: Preliminary Mapping Study.</title>
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">35099399</pid>
</result>
<result objidentifier="dris___00893::107e97e645cbb06fb7b454ce2569d6c2">
<title classid="subtitle" classname="subtitle" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title" inferred="false" provenanceaction="sysimport:crosswalk:datasetarchive" trust="0.9">Preliminary Mapping Study</title>
<dateofacceptance>2022-01-01</dateofacceptance>
<collectedfrom name="NARCIS" id="eurocrisdris::fe4903425d9040f680d8610d9079ea14" />
</result>
<result objidentifier="doi_________::88a9861b26cdda1c3dd162d11f0bedbe">
<dateofacceptance>2022-01-31</dateofacceptance>
<title classid="alternative title" classname="alternative title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Mapping the Ethical Issues of Digital Twins for Personalised Healthcare Service (Preprint)</title>
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2" />
<collectedfrom name="UnpayWall" id="openaire____::8ac8380272269217cb09a928c8caa993" />
<collectedfrom name="ORCID" id="openaire____::806360c771262b4d6770e7cdf04b5c5a" />
<collectedfrom name="Microsoft Academic Graph" id="openaire____::5f532a3fc4f1ea403f37070f59a7a53a" />
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types">10.2196/33081</pid>
<publisher>JMIR Publications Inc.</publisher>
</result>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<collectedfrom name="Europe PubMed Central" id="opendoar____::8b6dd7db9af49e67306feb59a8bdc52c" />
<hostedby name="Journal of Medical Internet Research" id="doajarticles::972bc6f95b43b5ad42f75695b1d8e948" />
<dateofacceptance>2021-08-23</dateofacceptance>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource" schemename="dnet:publication_resource" />
<pid classid="pmid" classname="PubMed ID" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">35099399</pid>
<pid classid="pmc" classname="PubMed Central ID" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">PMC8844982</pid>
<alternateidentifier classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.2196/33081</alternateidentifier>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels" schemename="dnet:review_levels" />
<webresource>
<url>https://pubmed.ncbi.nlm.nih.gov/35099399</url>
</webresource>
</instance>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2" />
<collectedfrom name="NARCIS" id="eurocrisdris::fe4903425d9040f680d8610d9079ea14" />
<hostedby name="Journal of Medical Internet Research" id="doajarticles::972bc6f95b43b5ad42f75695b1d8e948" />
<dateofacceptance>2022-01-01</dateofacceptance>
<dateofacceptance>2022-01-31</dateofacceptance>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource" schemename="dnet:publication_resource" />
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types">10.2196/33081</pid>
<alternateidentifier classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:crosswalk:datasetarchive" trust="0.9">10.2196/33081</alternateidentifier>
<alternateidentifier classid="urn" classname="urn" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:crosswalk:datasetarchive" trust="0.9">urn:nbn:nl:ui:15-70bf9bd0-5ea6-45fd-bb62-8d79b48cd69f</alternateidentifier>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels" schemename="dnet:review_levels" />
<webresource>
<url>https://doi.org/10.2196/33081</url>
</webresource>
</instance>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<collectedfrom name="NARCIS" id="openaire____::fdb035c8b3e0540a8d9a561a6c44f4de" />
<hostedby name="NARCIS" id="openaire____::fdb035c8b3e0540a8d9a561a6c44f4de" />
<dateofacceptance>2022-01-01</dateofacceptance>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource" schemename="dnet:publication_resource" />
<alternateidentifier classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:crosswalk:repository" trust="0.9">10.2196/33081</alternateidentifier>
<alternateidentifier classid="urn" classname="urn" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:crosswalk:repository" trust="0.9">urn:nbn:nl:ui:15-70bf9bd0-5ea6-45fd-bb62-8d79b48cd69f</alternateidentifier>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels" schemename="dnet:review_levels" />
<webresource>
<url>https://pure.eur.nl/en/publications/70bf9bd0-5ea6-45fd-bb62-8d79b48cd69f</url>
</webresource>
</instance>
</children>
</oaf:result>
</oaf:entity>
</metadata>
</result>
</record>

View File

@ -1,223 +0,0 @@
<record>
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<header>
<dri:objIdentifier>doi_________::c166d06aeaed817937a79a400906a4b9</dri:objIdentifier>
<dri:dateOfCollection>2023-03-09T00:12:02.045Z</dri:dateOfCollection>
<dri:dateOfTransformation>2023-03-09T00:24:00.8Z</dri:dateOfTransformation>
</header>
<metadata>
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
<oaf:result>
<collectedfrom name="OpenAPC Global Initiative"
id="apc_________::e2b1600b229fc30663c8a1f662debddf"/>
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2"/>
<collectedfrom name="UnpayWall" id="openaire____::8ac8380272269217cb09a928c8caa993"/>
<collectedfrom name="ORCID" id="openaire____::806360c771262b4d6770e7cdf04b5c5a"/>
<collectedfrom name="Microsoft Academic Graph" id="openaire____::5f532a3fc4f1ea403f37070f59a7a53a"/>
<originalId>50|openapc_____::c166d06aeaed817937a79a400906a4b9</originalId>
<originalId>10.3390/pr9111967</originalId>
<originalId>pr9111967</originalId>
<originalId>50|doiboost____::c166d06aeaed817937a79a400906a4b9</originalId>
<originalId>3209532762</originalId>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types" inferred="false"
provenanceaction="sysimport:crosswalk:datasetarchive" trust="0.9">10.3390/pr9111967
</pid>
<measure id="influence" score="5.187591E-9" class="C"/>
<measure id="popularity" score="2.4479982E-8" class="C"/>
<measure id="influence_alt" score="10" class="C"/>
<measure id="popularity_alt" score="3.6" class="C"/>
<measure id="impulse" score="10" class="C"/>
<title classid="alternative title" classname="alternative title" schemeid="dnet:dataCite_title"
schemename="dnet:dataCite_title">Digital Twins for Continuous mRNA Production
</title>
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<dateofacceptance>2021-11-04</dateofacceptance>
<description>The global coronavirus pandemic continues to restrict public life worldwide. An
effective means of limiting the pandemic is vaccination. Messenger ribonucleic acid (mRNA)
vaccines currently available on the market have proven to be a well-tolerated and effective
class of vaccine against coronavirus type 2 (CoV2). Accordingly, demand is presently
outstripping mRNA vaccine production. One way to increase productivity is to switch from the
currently performed batch to continuous in vitro transcription, which has proven to be a crucial
material-consuming step. In this article, a physico-chemical model of in vitro mRNA
transcription in a tubular reactor is presented and compared to classical batch and continuous
in vitro transcription in a stirred tank. The three models are validated based on a distinct and
quantitative validation workflow. Statistically significant parameters are identified as part of
the parameter determination concept. Monte Carlo simulations showed that the model is precise,
with a deviation of less than 1%. The advantages of continuous production are pointed out
compared to batchwise in vitro transcription by optimization of the spacetime yield.
Improvements of a factor of 56 (0.011 µM/min) in the case of the continuously stirred tank
reactor (CSTR) and 68 (0.013 µM/min) in the case of the plug flow reactor (PFR) were found.
</description>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Process Chemistry and Technology
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Chemical Engineering (miscellaneous)
</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Bioengineering
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Coronavirus
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.4019353">medicine.disease_cause
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="false"
provenanceaction="sysimport:actionset" trust="0.4019353">medicine
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Continuous production
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Messenger RNA
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Continuous stirred-tank reactor
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Plug flow reactor model
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Mathematics
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">In vitro transcription
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Biological system
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Yield (chemistry)
</subject>
<subject classid="MAG" classname="Microsoft Academic Graph classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies">Public life
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">03 medical and health sciences
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0301 basic medicine
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030104 Developmental Biology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0303 health sciences
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">030304 Developmental Biology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">02 engineering and technology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0210 nano-technology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">021001 Nanoscience &amp;
Nanotechnology
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">01 natural sciences
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">0104 chemical sciences
</subject>
<subject classid="FOS" classname="Fields of Science and Technology classification"
schemeid="dnet:subject_classification_typologies"
schemename="dnet:subject_classification_typologies" inferred="true"
inferenceprovenance="update" provenanceaction="subject:fos">010405 Organic Chemistry
</subject>
<language classid="UNKNOWN" classname="UNKNOWN" schemeid="dnet:languages"
schemename="dnet:languages"/>
<relevantdate classid="created" classname="created" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date">2021-11-05
</relevantdate>
<relevantdate classid="published-online" classname="published-online" schemeid="dnet:dataCite_date"
schemename="dnet:dataCite_date">2021-11-04
</relevantdate>
<source>Crossref</source>
<source/>
<resulttype classid="publication" classname="publication" schemeid="dnet:result_typologies"
schemename="dnet:result_typologies"/>
<resourcetype classid="0001" classname="Article" schemeid="dnet:publication_resource"
schemename="dnet:publication_resource"/>
<processingchargeamount>2146.08</processingchargeamount>
<processingchargecurrency>EUR</processingchargecurrency>
<journal issn="2227-9717">Processes</journal>
<context id="covid-19" label="COVID-19" type="community"/>
<context id="dth" label="Digital Twins in Health" type="community"/>
<datainfo>
<inferred>false</inferred>
<deletedbyinference>false</deletedbyinference>
<trust>0.9</trust>
<inferenceprovenance>null</inferenceprovenance>
<provenanceaction classid="sysimport:actionset" classname="Harvested"
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
</datainfo>
<rels/>
<children>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
schemename="dnet:access_modes"/>
<collectedfrom name="Crossref" id="openaire____::081b82f96300b6a6e3d282bad31cb6e2"/>
<hostedby name="Processes" id="doajarticles::9c4841602dcae937ac9c86cfb03c5ee1"/>
<dateofacceptance>2021-11-04</dateofacceptance>
<instancetype classid="0001" classname="Article" schemeid="dnet:publication_resource"
schemename="dnet:publication_resource"/>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
schemename="dnet:pid_types">10.3390/pr9111967
</pid>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels"
schemename="dnet:review_levels"/>
<license>https://creativecommons.org/licenses/by/4.0/</license>
<webresource>
<url>https://doi.org/10.3390/pr9111967</url>
</webresource>
</instance>
</children>
</oaf:result>
</oaf:entity>
</metadata>
</result>
</record>

View File

@ -1,138 +0,0 @@
<record>
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<header>
<dri:objIdentifier>doi_dedup___::10a910f4a66b7f4bce8407d7a486a80a</dri:objIdentifier>
<dri:dateOfCollection>2023-04-05T00:36:27+0000</dri:dateOfCollection>
<dri:dateOfTransformation>2023-04-05T07:33:52.185Z</dri:dateOfTransformation>
</header>
<metadata>
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
<oaf:result>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<originalId>50|datacite____::10a910f4a66b7f4bce8407d7a486a80a</originalId>
<originalId>10.5281/zenodo.6967373</originalId>
<originalId>50|datacite____::172969c66c312a9656fc745f0ec62ce5</originalId>
<originalId>10.5281/zenodo.6969999</originalId>
<originalId>50|datacite____::4fa8f1c89ff11e8e99f9ded870ade80d</originalId>
<originalId>10.5281/zenodo.6967372</originalId>
<originalId>50|datacite____::a466b6173773d742b7a5881682748a8c</originalId>
<originalId>10.5281/zenodo.6970067</originalId>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6967373</pid>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6969999</pid>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6967372</pid>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6970067</pid>
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Sentinel-3 NDVI ARD and Long Term Statistics (1999-2019) from the Copernicus Global Land Service over Lombardia</title>
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<creator rank="1">Marasco Pier Lorenzo</creator>
<dateofacceptance>2022-08-05</dateofacceptance>
<description>Sentinel-3 NDVI Analysis Ready Data (ARD) (C_GLS_NDVI_20220101_20220701_Lombardia_S3_2.nc) product provided by the Copernicus Global Land Service [3]. The file C_GLS_NDVI_20220101_20220701_Lombardia_S3_2_masked.nc is derived from C_GLS_NDVI_20220101_20220701_Lombardia_S3_2.nc but values have been scaled (raw_value * ( 1/250) - 0.08) and values lower then -0.08 and greater than 0.92 have been removed (set to missing values). The original dataset can also be discovered through the OpenEO API[5] from the CGLS distributor VITO [4]. Access is free of charge but an EGI registration is needed. The file called Italy.geojson has been created using the Global Administrative Unit Layers GAUL G2015_2014 provided by FAO-UN (see Documentation). It only contains information related to Italy. Further info about drought indexes can be found in the Integrated Drought Management Programme [5] [1] Application of vegetation index and brightness temperature for drought detection [2] NDVI [3] Copernicus Global Land Service [4] Vito [5] OpenEO [5] Integrated Drought Management</description>
<description>These datasets are used for training purposes. See https://pangeo-data.github.io/foss4g-2022/intro.html</description>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">NDVI</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">vegetaion</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">Copernicus Global Land Service</subject>
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">pangeo</subject>
<language classid="eng" classname="English" schemeid="dnet:languages" schemename="dnet:languages" />
<relevantdate classid="issued" classname="issued" schemeid="dnet:dataCite_date" schemename="dnet:dataCite_date">2022-08-05</relevantdate>
<publisher>Zenodo</publisher>
<resulttype classid="dataset" classname="dataset" schemeid="dnet:result_typologies" schemename="dnet:result_typologies" />
<resourcetype classid="UNKNOWN" classname="Unknown" schemeid="dnet:dataCite_resource" schemename="dnet:dataCite_resource" />
<eoscifguidelines code="EOSC::Jupyter Notebook" label="EOSC::Jupyter Notebook" semanticrelation="compliesWith" />
<datainfo>
<inferred>true</inferred>
<deletedbyinference>false</deletedbyinference>
<trust>0.8</trust>
<inferenceprovenance>dedup-result-decisiontree-v3</inferenceprovenance>
<provenanceaction classid="sysimport:dedup" classname="Inferred by OpenAIRE" schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions" />
</datainfo>
<rels></rels>
<children>
<result objidentifier="doi_________::4fa8f1c89ff11e8e99f9ded870ade80d">
<publisher>Zenodo</publisher>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6967372</pid>
<dateofacceptance>2022-08-05</dateofacceptance>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Sentinel-3 NDVI ARD and Long Term Statistics (1999-2019) from the Copernicus Global Land Service over Lombardia</title>
</result>
<result objidentifier="doi_________::a466b6173773d742b7a5881682748a8c">
<publisher>Zenodo</publisher>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6970067</pid>
<dateofacceptance>2022-08-05</dateofacceptance>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Sentinel-3 NDVI ARD and Long Term Statistics (1999-2019) from the Copernicus Global Land Service over Lombardia</title>
</result>
<result objidentifier="doi_________::172969c66c312a9656fc745f0ec62ce5">
<publisher>Zenodo</publisher>
<dateofacceptance>2022-08-05</dateofacceptance>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6969999</pid>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Sentinel-3 NDVI ARD and Long Term Statistics (1999-2019) from the Copernicus Global Land Service over Lombardia</title>
</result>
<result objidentifier="doi_________::10a910f4a66b7f4bce8407d7a486a80a">
<publisher>Zenodo</publisher>
<dateofacceptance>2022-08-05</dateofacceptance>
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Sentinel-3 NDVI ARD and Long Term Statistics (1999-2019) from the Copernicus Global Land Service over Lombardia</title>
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6967373</pid>
</result>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<hostedby name="ZENODO" id="opendoar____::358aee4cc897452c00244351e4d91f69" />
<dateofacceptance>2022-08-05</dateofacceptance>
<instancetype classid="0021" classname="Dataset" schemeid="dnet:publication_resource" schemename="dnet:publication_resource" />
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6967373</pid>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels" schemename="dnet:review_levels" />
<license>https://creativecommons.org/licenses/by/4.0/legalcode</license>
<webresource>
<url>https://doi.org/10.5281/zenodo.6967373</url>
</webresource>
</instance>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<hostedby name="ZENODO" id="opendoar____::358aee4cc897452c00244351e4d91f69" />
<dateofacceptance>2022-08-05</dateofacceptance>
<instancetype classid="0021" classname="Dataset" schemeid="dnet:publication_resource" schemename="dnet:publication_resource" />
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6970067</pid>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels" schemename="dnet:review_levels" />
<license>https://creativecommons.org/licenses/by/4.0/legalcode</license>
<webresource>
<url>https://doi.org/10.5281/zenodo.6970067</url>
</webresource>
</instance>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<hostedby name="ZENODO" id="opendoar____::358aee4cc897452c00244351e4d91f69" />
<dateofacceptance>2022-08-05</dateofacceptance>
<instancetype classid="0021" classname="Dataset" schemeid="dnet:publication_resource" schemename="dnet:publication_resource" />
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6969999</pid>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels" schemename="dnet:review_levels" />
<license>https://creativecommons.org/licenses/by/4.0/legalcode</license>
<webresource>
<url>https://doi.org/10.5281/zenodo.6969999</url>
</webresource>
</instance>
<instance>
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254" />
<hostedby name="ZENODO" id="opendoar____::358aee4cc897452c00244351e4d91f69" />
<dateofacceptance>2022-08-05</dateofacceptance>
<instancetype classid="0021" classname="Dataset" schemeid="dnet:publication_resource" schemename="dnet:publication_resource" />
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.6967372</pid>
<refereed classid="0000" classname="UNKNOWN" schemeid="dnet:review_levels" schemename="dnet:review_levels" />
<license>https://creativecommons.org/licenses/by/4.0/legalcode</license>
<webresource>
<url>https://doi.org/10.5281/zenodo.6967372</url>
</webresource>
</instance>
</children>
</oaf:result>
</oaf:entity>
</metadata>
</result>
</record>

View File

@ -1,10 +1,10 @@
{
"eoscifguidelines": [
{
"code": "EOSC::Jupyter Notebook",
"label": "EOSC::Jupyter Notebook",
"url": "",
"semanticRelation": "compliesWith"
"code" : "EOSC::Jupyter Notebook",
"label" : "EOSC::Jupyter Notebook",
"url" : "",
"semanticRelation" : "compliesWith"
}
],
"measures": [
@ -431,26 +431,7 @@
"value": "VIRTA"
}
],
"context": [
{
"id": "eosc",
"dataInfo": [
{
"invisible": false,
"inferred": false,
"deletedbyinference": false,
"trust": "0.9",
"inferenceprovenance": "",
"provenanceaction": {
"classid": "sysimport:crosswalk",
"classname": "sysimport:crosswalk",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
}
}
]
}
],
"context": [],
"contributor": [],
"country": [],
"coverage": [],

View File

@ -0,0 +1,110 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-workflows</artifactId>
<version>1.2.5-SNAPSHOT</version>
</parent>
<artifactId>dhp-swh</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-common</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
<groupId>net.sf.saxon</groupId>
<artifactId>Saxon-HE</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>dom4j</groupId>
<artifactId>dom4j</artifactId>
</dependency>
<dependency>
<groupId>xml-apis</groupId>
<artifactId>xml-apis</artifactId>
</dependency>
<dependency>
<groupId>jaxen</groupId>
<artifactId>jaxen</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-distcp</artifactId>
</dependency>
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-actionmanager-api</artifactId>
</dependency>
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-actionmanager-common</artifactId>
<exclusions>
<exclusion>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-openaireplus-mapping-utils</artifactId>
</exclusion>
<exclusion>
<groupId>saxonica</groupId>
<artifactId>saxon</artifactId>
</exclusion>
<exclusion>
<groupId>saxonica</groupId>
<artifactId>saxon-dom</artifactId>
</exclusion>
<exclusion>
<groupId>jgrapht</groupId>
<artifactId>jgrapht</artifactId>
</exclusion>
<exclusion>
<groupId>net.sf.ehcache</groupId>
<artifactId>ehcache</artifactId>
</exclusion>
<exclusion>
<groupId>org.springframework</groupId>
<artifactId>spring-test</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.*</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>apache</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
<version>4.5.13</version>
</dependency>
<dependency>
<groupId>org.datanucleus</groupId>
<artifactId>datanucleus-core</artifactId>
<version>3.2.10</version>
<scope>compile</scope>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,176 @@
package eu.dnetlib.dhp.swh;
import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration;
import java.io.IOException;
import java.net.URL;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions;
import eu.dnetlib.dhp.swh.models.LastVisitData;
import eu.dnetlib.dhp.swh.utils.SWHConnection;
import eu.dnetlib.dhp.swh.utils.SWHConstants;
import eu.dnetlib.dhp.swh.utils.SWHUtils;
/**
* Sends archive requests to the SWH API for those software repository URLs that are missing from them
*
* @author Serafeim Chatzopoulos
*/
public class ArchiveRepositoryURLs {
private static final Logger log = LoggerFactory.getLogger(ArchiveRepositoryURLs.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SWHConnection swhConnection = null;
public static void main(final String[] args) throws IOException, ParseException {
final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser(
IOUtils
.toString(
CollectLastVisitRepositoryData.class
.getResourceAsStream(
"/eu/dnetlib/dhp/swh/input_archive_repository_urls.json")));
argumentParser.parseArgument(args);
final String hdfsuri = argumentParser.get("namenode");
log.info("hdfsURI: {}", hdfsuri);
final String inputPath = argumentParser.get("lastVisitsPath");
log.info("inputPath: {}", inputPath);
final String outputPath = argumentParser.get("archiveRequestsPath");
log.info("outputPath: {}", outputPath);
final Integer archiveThresholdInDays = Integer.parseInt(argumentParser.get("archiveThresholdInDays"));
log.info("archiveThresholdInDays: {}", archiveThresholdInDays);
final String apiAccessToken = argumentParser.get("apiAccessToken");
log.info("apiAccessToken: {}", apiAccessToken);
final HttpClientParams clientParams = SWHUtils.getClientParams(argumentParser);
swhConnection = new SWHConnection(clientParams, apiAccessToken);
final FileSystem fs = FileSystem.get(getHadoopConfiguration(hdfsuri));
archive(fs, inputPath, outputPath, archiveThresholdInDays);
}
private static void archive(FileSystem fs, String inputPath, String outputPath, Integer archiveThresholdInDays)
throws IOException {
SequenceFile.Reader fr = SWHUtils.getSequenceFileReader(fs, inputPath);
SequenceFile.Writer fw = SWHUtils.getSequenceFileWriter(fs, outputPath);
// Create key and value objects to hold data
Text repoUrl = new Text();
Text lastVisitData = new Text();
// Read key-value pairs from the SequenceFile and handle appropriately
while (fr.next(repoUrl, lastVisitData)) {
String response = null;
try {
response = handleRecord(repoUrl.toString(), lastVisitData.toString(), archiveThresholdInDays);
} catch (java.text.ParseException e) {
log.error("Could not handle record with repo Url: {}", repoUrl.toString());
throw new RuntimeException(e);
}
// response is equal to null when no need for request
if (response != null) {
SWHUtils.appendToSequenceFile(fw, repoUrl.toString(), response);
}
}
// Close readers
fw.close();
fr.close();
}
public static String handleRecord(String repoUrl, String lastVisitData, Integer archiveThresholdInDays)
throws IOException, java.text.ParseException {
log.info("{ Key: {}, Value: {} }", repoUrl, lastVisitData);
LastVisitData lastVisit = OBJECT_MAPPER.readValue(lastVisitData, LastVisitData.class);
// a previous attempt for archival has been made, and repository URL was not found
// avoid performing the same archive request again
if (lastVisit.getStatus() != null &&
lastVisit.getStatus().equals(SWHConstants.VISIT_STATUS_NOT_FOUND)) {
log.info("Avoid request -- previous archive request returned NOT_FOUND");
return null;
}
// if we have last visit data
if (lastVisit.getSnapshot() != null) {
String cleanDate = GraphCleaningFunctions.cleanDate(lastVisit.getDate());
// and the last visit date can be parsed
if (cleanDate != null) {
SimpleDateFormat formatter = new SimpleDateFormat(ModelSupport.DATE_FORMAT);
Date lastVisitDate = formatter.parse(cleanDate);
// OR last visit time < (now() - archiveThresholdInDays)
long diffInMillies = Math.abs((new Date()).getTime() - lastVisitDate.getTime());
long diffInDays = TimeUnit.DAYS.convert(diffInMillies, TimeUnit.MILLISECONDS);
log.info("Date diff from now (in days): {}", diffInDays);
// do not perform a request, if the last visit date is no older than $archiveThresholdInDays
if (archiveThresholdInDays >= diffInDays) {
log.info("Avoid request -- no older than {} days", archiveThresholdInDays);
return null;
}
}
}
// ELSE perform an archive request
log.info("Perform archive request for: {}", repoUrl);
// if last visit data are available, re-use version control type,
// else use the default one (i.e., git)
String visitType = Optional
.ofNullable(lastVisit.getType())
.orElse(SWHConstants.DEFAULT_VISIT_TYPE);
URL url = new URL(String.format(SWHConstants.SWH_ARCHIVE_URL, visitType, repoUrl.trim()));
log.info("Sending archive request: {}", url);
String response;
try {
response = swhConnection.call(url.toString());
} catch (CollectorException e) {
log.error("Error in request: {}", url);
response = "{}";
}
return response;
}
}

View File

@ -0,0 +1,119 @@
package eu.dnetlib.dhp.swh;
import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration;
import java.io.BufferedReader;
import java.io.IOException;
import java.net.URL;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.dhp.swh.utils.SWHConnection;
import eu.dnetlib.dhp.swh.utils.SWHConstants;
import eu.dnetlib.dhp.swh.utils.SWHUtils;
/**
* Given a file with software repository URLs, this class
* collects last visit data from the Software Heritage API.
*
* @author Serafeim Chatzopoulos
*/
public class CollectLastVisitRepositoryData {
private static final Logger log = LoggerFactory.getLogger(CollectLastVisitRepositoryData.class);
private static SWHConnection swhConnection = null;
public static void main(final String[] args)
throws IOException, ParseException {
final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser(
IOUtils
.toString(
CollectLastVisitRepositoryData.class
.getResourceAsStream(
"/eu/dnetlib/dhp/swh/input_collect_last_visit_repository_data.json")));
argumentParser.parseArgument(args);
log.info("Java Xmx: {}m", Runtime.getRuntime().maxMemory() / (1024 * 1024));
final String hdfsuri = argumentParser.get("namenode");
log.info("hdfsURI: {}", hdfsuri);
final String inputPath = argumentParser.get("softwareCodeRepositoryURLs");
log.info("inputPath: {}", inputPath);
final String outputPath = argumentParser.get("lastVisitsPath");
log.info("outputPath: {}", outputPath);
final String apiAccessToken = argumentParser.get("apiAccessToken");
log.info("apiAccessToken: {}", apiAccessToken);
final HttpClientParams clientParams = SWHUtils.getClientParams(argumentParser);
swhConnection = new SWHConnection(clientParams, apiAccessToken);
final FileSystem fs = FileSystem.get(getHadoopConfiguration(hdfsuri));
collect(fs, inputPath, outputPath);
fs.close();
}
private static void collect(FileSystem fs, String inputPath, String outputPath)
throws IOException {
SequenceFile.Writer fw = SWHUtils.getSequenceFileWriter(fs, outputPath);
// Specify the HDFS directory path you want to read
Path directoryPath = new Path(inputPath);
// List all files in the directory
FileStatus[] partStatuses = fs.listStatus(directoryPath);
for (FileStatus partStatus : partStatuses) {
// Check if it's a file (not a directory)
if (partStatus.isFile()) {
handleFile(fs, partStatus.getPath(), fw);
}
}
fw.close();
}
private static void handleFile(FileSystem fs, Path partInputPath, SequenceFile.Writer fw)
throws IOException {
BufferedReader br = SWHUtils.getFileReader(fs, partInputPath);
String repoUrl;
while ((repoUrl = br.readLine()) != null) {
URL url = new URL(String.format(SWHConstants.SWH_LATEST_VISIT_URL, repoUrl.trim()));
String response;
try {
response = swhConnection.call(url.toString());
} catch (CollectorException e) {
log.error("Error in request: {}", url);
response = "{}";
}
SWHUtils.appendToSequenceFile(fw, repoUrl, response);
}
br.close();
}
}

View File

@ -0,0 +1,93 @@
package eu.dnetlib.dhp.swh;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Result;
/**
* Collects unique software repository URLs in the Graph using Hive
*
* @author Serafeim Chatzopoulos
*/
public class CollectSoftwareRepositoryURLs {
private static final Logger log = LoggerFactory.getLogger(CollectSoftwareRepositoryURLs.class);
public static <I extends Result> void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
CollectSoftwareRepositoryURLs.class
.getResourceAsStream("/eu/dnetlib/dhp/swh/input_collect_software_repository_urls.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String hiveDbName = parser.get("hiveDbName");
log.info("hiveDbName: {}", hiveDbName);
final String outputPath = parser.get("softwareCodeRepositoryURLs");
log.info("softwareCodeRepositoryURLs: {}", outputPath);
final String hiveMetastoreUris = parser.get("hiveMetastoreUris");
log.info("hiveMetastoreUris: {}", hiveMetastoreUris);
final Integer softwareLimit = Integer.parseInt(parser.get("softwareLimit"));
log.info("softwareLimit: {}", softwareLimit);
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", hiveMetastoreUris);
runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> {
doRun(spark, hiveDbName, softwareLimit, outputPath);
});
}
private static <I extends Result> void doRun(SparkSession spark, String hiveDbName, Integer limit,
String outputPath) {
String queryTemplate = "SELECT distinct coderepositoryurl.value " +
"FROM %s.software " +
"WHERE coderepositoryurl.value IS NOT NULL " +
"AND datainfo.deletedbyinference = FALSE " +
"AND datainfo.invisible = FALSE ";
if (limit != null) {
queryTemplate += String.format("LIMIT %s", limit);
}
String query = String.format(queryTemplate, hiveDbName);
log.info("Hive query to fetch software code URLs: {}", query);
Dataset<Row> df = spark.sql(query);
// write distinct repository URLs
df
.write()
.mode(SaveMode.Overwrite)
.csv(outputPath);
}
}

View File

@ -0,0 +1,185 @@
package eu.dnetlib.dhp.swh;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static org.apache.spark.sql.functions.col;
import java.io.Serializable;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
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.SparkConf;
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.*;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
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.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import eu.dnetlib.dhp.swh.models.LastVisitData;
import eu.dnetlib.dhp.swh.utils.SWHConstants;
import scala.Tuple2;
/**
* Creates action sets for Software Heritage data
*
* @author Serafeim Chatzopoulos
*/
public class PrepareSWHActionsets {
private static final Logger log = LoggerFactory.getLogger(PrepareSWHActionsets.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static <I extends Result> void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
PrepareSWHActionsets.class
.getResourceAsStream(
"/eu/dnetlib/dhp/swh/input_prepare_swh_actionsets.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("lastVisitsPath");
log.info("inputPath: {}", inputPath);
final String softwareInputPath = parser.get("softwareInputPath");
log.info("softwareInputPath: {}", softwareInputPath);
final String outputPath = parser.get("actionsetsPath");
log.info("outputPath: {}", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
JavaPairRDD<Text, Text> softwareRDD = prepareActionsets(spark, inputPath, softwareInputPath);
softwareRDD
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
});
}
private static Dataset<Row> loadSWHData(SparkSession spark, String inputPath) {
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
// read from file and transform to <origin, snapshotId> tuples
// Note: snapshot id is the SWH id for us
JavaRDD<Row> swhRDD = sc
.sequenceFile(inputPath, Text.class, Text.class)
.map(t -> t._2().toString())
.map(t -> OBJECT_MAPPER.readValue(t, LastVisitData.class))
.filter(t -> t.getOrigin() != null && t.getSnapshot() != null) // response from SWH API is empty if repo URL
// was not found
.map(item -> RowFactory.create(item.getOrigin(), item.getSnapshot()));
// convert RDD to 2-column DF
List<StructField> fields = Arrays
.asList(
DataTypes.createStructField("repoUrl", DataTypes.StringType, true),
DataTypes.createStructField("swhId", DataTypes.StringType, true));
StructType schema = DataTypes.createStructType(fields);
return spark.createDataFrame(swhRDD, schema);
}
private static Dataset<Row> loadGraphSoftwareData(SparkSession spark, String softwareInputPath) {
return spark
.read()
.textFile(softwareInputPath)
.map(
(MapFunction<String, Software>) t -> OBJECT_MAPPER.readValue(t, Software.class),
Encoders.bean(Software.class))
.filter(t -> t.getCodeRepositoryUrl() != null)
.select(col("id"), col("codeRepositoryUrl.value").as("repoUrl"));
}
private static <I extends Software> JavaPairRDD<Text, Text> prepareActionsets(SparkSession spark, String inputPath,
String softwareInputPath) {
Dataset<Row> swhDF = loadSWHData(spark, inputPath);
// swhDF.show(false);
Dataset<Row> graphSoftwareDF = loadGraphSoftwareData(spark, softwareInputPath);
// graphSoftwareDF.show(5);
Dataset<Row> joinedDF = graphSoftwareDF.join(swhDF, "repoUrl").select("id", "swhid");
// joinedDF.show(false);
return joinedDF.map((MapFunction<Row, Software>) row -> {
Software s = new Software();
// set openaire id
s.setId(row.getString(row.fieldIndex("id")));
// set swh id
Qualifier qualifier = OafMapperUtils
.qualifier(
SWHConstants.SWHID,
SWHConstants.SWHID_CLASSNAME,
ModelConstants.DNET_PID_TYPES,
ModelConstants.DNET_PID_TYPES);
DataInfo dataInfo = OafMapperUtils
.dataInfo(
false,
null,
false,
false,
ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER,
"");
s
.setPid(
Arrays
.asList(
OafMapperUtils
.structuredProperty(
String.format("swh:1:snp:%s", row.getString(row.fieldIndex("swhid"))),
qualifier,
dataInfo)));
// add SWH in the `collectedFrom` field
KeyValue kv = new KeyValue();
kv.setKey(SWHConstants.SWH_ID);
kv.setValue(SWHConstants.SWH_NAME);
s.setCollectedfrom(Arrays.asList(kv));
return s;
}, Encoders.bean(Software.class))
.toJavaRDD()
.map(p -> new AtomicAction(Software.class, p))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
}
}

View File

@ -0,0 +1,71 @@
package eu.dnetlib.dhp.swh.models;
import java.io.Serializable;
import com.cloudera.com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
@JsonIgnoreProperties(ignoreUnknown = true)
public class LastVisitData implements Serializable {
private String origin;
private String type;
private String date;
@JsonProperty("snapshot")
private String snapshotId;
private String status;
public String getType() {
return type;
}
public void setType(String type) {
this.type = type;
}
public String getDate() {
return date;
}
public void setDate(String date) {
this.date = date;
}
public String getSnapshot() {
return snapshotId;
}
public void setSnapshot(String snapshotId) {
this.snapshotId = snapshotId;
}
public String getStatus() {
return status;
}
public void setStatus(String status) {
this.status = status;
}
public String getOrigin() {
return origin;
}
public void setOrigin(String origin) {
this.origin = origin;
}
@Override
public String toString() {
return "LastVisitData{" +
"origin='" + origin + '\'' +
", type='" + type + '\'' +
", date='" + date + '\'' +
", snapshotId='" + snapshotId + '\'' +
", status='" + status + '\'' +
'}';
}
}

View File

@ -0,0 +1,40 @@
package eu.dnetlib.dhp.swh.utils;
import java.util.HashMap;
import java.util.Map;
import org.apache.http.HttpHeaders;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.dhp.common.collection.HttpConnector2;
public class SWHConnection {
HttpConnector2 conn;
public SWHConnection(HttpClientParams clientParams, String accessToken) {
// set custom headers
Map<String, String> headers = new HashMap<String, String>() {
{
put(HttpHeaders.ACCEPT, "application/json");
if (accessToken != null) {
put(HttpHeaders.AUTHORIZATION, String.format("Bearer %s", accessToken));
}
}
};
clientParams.setHeaders(headers);
// create http connector
conn = new HttpConnector2(clientParams);
}
public String call(String url) throws CollectorException {
return conn.getInputSource(url);
}
}

View File

@ -0,0 +1,21 @@
package eu.dnetlib.dhp.swh.utils;
public class SWHConstants {
public static final String SWH_LATEST_VISIT_URL = "https://archive.softwareheritage.org/api/1/origin/%s/visit/latest/";
public static final String SWH_ARCHIVE_URL = "https://archive.softwareheritage.org/api/1/origin/save/%s/url/%s/";
public static final String DEFAULT_VISIT_TYPE = "git";
public static final String VISIT_STATUS_NOT_FOUND = "not_found";
public static final String SWHID = "swhid";
public static final String SWHID_CLASSNAME = "Software Heritage Identifier";
public static final String SWH_ID = "10|openaire____::dbfd07503aaa1ed31beed7dec942f3f4";
public static final String SWH_NAME = "Software Heritage";
}

View File

@ -0,0 +1,95 @@
package eu.dnetlib.dhp.swh.utils;
import static eu.dnetlib.dhp.common.Constants.*;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.util.Optional;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
public class SWHUtils {
private static final Logger log = LoggerFactory.getLogger(SWHUtils.class);
public static HttpClientParams getClientParams(ArgumentApplicationParser argumentParser) {
final HttpClientParams clientParams = new HttpClientParams();
clientParams
.setMaxNumberOfRetry(
Optional
.ofNullable(argumentParser.get(MAX_NUMBER_OF_RETRY))
.map(Integer::parseInt)
.orElse(HttpClientParams._maxNumberOfRetry));
log.info("maxNumberOfRetry is {}", clientParams.getMaxNumberOfRetry());
clientParams
.setRequestDelay(
Optional
.ofNullable(argumentParser.get(REQUEST_DELAY))
.map(Integer::parseInt)
.orElse(HttpClientParams._requestDelay));
log.info("requestDelay is {}", clientParams.getRequestDelay());
clientParams
.setRetryDelay(
Optional
.ofNullable(argumentParser.get(RETRY_DELAY))
.map(Integer::parseInt)
.orElse(HttpClientParams._retryDelay));
log.info("retryDelay is {}", clientParams.getRetryDelay());
clientParams
.setRequestMethod(
Optional
.ofNullable(argumentParser.get(REQUEST_METHOD))
.orElse(HttpClientParams._requestMethod));
log.info("requestMethod is {}", clientParams.getRequestMethod());
return clientParams;
}
public static BufferedReader getFileReader(FileSystem fs, Path inputPath) throws IOException {
FSDataInputStream inputStream = fs.open(inputPath);
return new BufferedReader(
new InputStreamReader(inputStream, StandardCharsets.UTF_8));
}
public static SequenceFile.Writer getSequenceFileWriter(FileSystem fs, String outputPath) throws IOException {
return SequenceFile
.createWriter(
fs.getConf(),
SequenceFile.Writer.file(new Path(outputPath)),
SequenceFile.Writer.keyClass(Text.class),
SequenceFile.Writer.valueClass(Text.class));
}
public static SequenceFile.Reader getSequenceFileReader(FileSystem fs, String inputPath) throws IOException {
Path filePath = new Path(inputPath);
SequenceFile.Reader.Option fileOption = SequenceFile.Reader.file(filePath);
return new SequenceFile.Reader(fs.getConf(), fileOption);
}
public static void appendToSequenceFile(SequenceFile.Writer fw, String keyStr, String valueStr) throws IOException {
Text key = new Text();
key.set(keyStr);
Text value = new Text();
value.set(valueStr);
fw.append(key, value);
}
}

View File

@ -0,0 +1,56 @@
[
{
"paramName": "n",
"paramLongName": "namenode",
"paramDescription": "the Name Node URI",
"paramRequired": true
},
{
"paramName": "lv",
"paramLongName": "lastVisitsPath",
"paramDescription": "the URL where to store last visits data",
"paramRequired": true
},
{
"paramName": "arp",
"paramLongName": "archiveRequestsPath",
"paramDescription": "the URL where to store the responses of the archive requests",
"paramRequired": true
},
{
"paramName": "mnr",
"paramLongName": "maxNumberOfRetry",
"paramDescription": "the maximum number of admitted connection retries",
"paramRequired": false
},
{
"paramName": "rqd",
"paramLongName": "requestDelay",
"paramDescription": "the delay (ms) between requests",
"paramRequired": false
},
{
"paramName": "rtd",
"paramLongName": "retryDelay",
"paramDescription": "the delay (ms) between retries",
"paramRequired": false
},
{
"paramName": "rm",
"paramLongName": "requestMethod",
"paramDescription": "the method of the requests to perform",
"paramRequired": false
},
{
"paramName": "atid",
"paramLongName": "archiveThresholdInDays",
"paramDescription": "the thershold (in days) required to issue an archive request",
"paramRequired": false
},
{
"paramName": "aat",
"paramLongName": "apiAccessToken",
"paramDescription": "the API access token of the SWH API",
"paramRequired": false
}
]

View File

@ -0,0 +1,50 @@
[
{
"paramName": "n",
"paramLongName": "namenode",
"paramDescription": "the Name Node URI",
"paramRequired": true
},
{
"paramName": "scr",
"paramLongName": "softwareCodeRepositoryURLs",
"paramDescription": "the URL from where to read software repository URLs",
"paramRequired": true
},
{
"paramName": "lv",
"paramLongName": "lastVisitsPath",
"paramDescription": "the URL where to store last visits data",
"paramRequired": true
},
{
"paramName": "mnr",
"paramLongName": "maxNumberOfRetry",
"paramDescription": "the maximum number of admitted connection retries",
"paramRequired": false
},
{
"paramName": "rqd",
"paramLongName": "requestDelay",
"paramDescription": "the delay (ms) between requests",
"paramRequired": false
},
{
"paramName": "rtd",
"paramLongName": "retryDelay",
"paramDescription": "the delay (ms) between retries",
"paramRequired": false
},
{
"paramName": "rm",
"paramLongName": "requestMethod",
"paramDescription": "the method of the requests to perform",
"paramRequired": false
},
{
"paramName": "aat",
"paramLongName": "apiAccessToken",
"paramDescription": "the API access token of the SWH API",
"paramRequired": false
}
]

View File

@ -0,0 +1,32 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "scr",
"paramLongName": "softwareCodeRepositoryURLs",
"paramDescription": "the URL where to store software repository URLs",
"paramRequired": true
},
{
"paramName": "db",
"paramLongName": "hiveDbName",
"paramDescription": "the target hive database name",
"paramRequired": true
},
{
"paramName": "hmu",
"paramLongName": "hiveMetastoreUris",
"paramDescription": "the hive metastore uris",
"paramRequired": true
},
{
"paramName": "slim",
"paramLongName": "softwareLimit",
"paramDescription": "limit on the number of software repo URL to fetch",
"paramRequired": false
}
]

View File

@ -0,0 +1,26 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "lv",
"paramLongName": "lastVisitsPath",
"paramDescription": "the URL where to store last visits data",
"paramRequired": true
},
{
"paramName": "ap",
"paramLongName": "actionsetsPath",
"paramDescription": "the URL path where to store actionsets",
"paramRequired": true
},
{
"paramName": "sip",
"paramLongName": "softwareInputPath",
"paramDescription": "the URL path of the software in the graph",
"paramRequired": true
}
]

View File

@ -0,0 +1,19 @@
# hive
hiveDbName=openaire_prod_20230914
# input/output files
softwareCodeRepositoryURLs=${workingDir}/1_code_repo_urls.csv
lastVisitsPath=${workingDir}/2_last_visits.seq
archiveRequestsPath=${workingDir}/3_archive_requests.seq
actionsetsPath=${workingDir}/4_actionsets
graphPath=/tmp/prod_provision/graph/18_graph_blacklisted
apiAccessToken=eyJhbGciOiJIUzI1NiIsInR5cCIgOiAiSldUIiwia2lkIiA6ICJhMTMxYTQ1My1hM2IyLTQwMTUtODQ2Ny05MzAyZjk3MTFkOGEifQ.eyJpYXQiOjE2OTQ2MzYwMjAsImp0aSI6IjkwZjdkNTNjLTQ5YTktNGFiMy1hY2E0LTcwMTViMjEyZTNjNiIsImlzcyI6Imh0dHBzOi8vYXV0aC5zb2Z0d2FyZWhlcml0YWdlLm9yZy9hdXRoL3JlYWxtcy9Tb2Z0d2FyZUhlcml0YWdlIiwiYXVkIjoiaHR0cHM6Ly9hdXRoLnNvZnR3YXJlaGVyaXRhZ2Uub3JnL2F1dGgvcmVhbG1zL1NvZnR3YXJlSGVyaXRhZ2UiLCJzdWIiOiIzMTY5OWZkNC0xNmE0LTQxOWItYTdhMi00NjI5MDY4ZjI3OWEiLCJ0eXAiOiJPZmZsaW5lIiwiYXpwIjoic3doLXdlYiIsInNlc3Npb25fc3RhdGUiOiIzMjYzMzEwMS00ZDRkLTQwMjItODU2NC1iMzNlMTJiNTE3ZDkiLCJzY29wZSI6Im9wZW5pZCBvZmZsaW5lX2FjY2VzcyBwcm9maWxlIGVtYWlsIn0.XHj1VIZu1dZ4Ej32-oU84mFmaox9cLNjXosNxwZM0Xs
maxNumberOfRetry=2
retryDelay=1
requestDelay=100
softwareLimit=500
resume=collect-software-repository-urls

View File

@ -0,0 +1,54 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
</property>
<property>
<name>spark2EventLogDir</name>
<value>/user/spark/spark2ApplicationHistory</value>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>"com.cloudera.spark.lineage.NavigatorAppListener"</value>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>"com.cloudera.spark.lineage.NavigatorQueryListener"</value>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<value>spark2</value>
</property>
<property>
<name>resourceManager</name>
<value>http://iis-cdh5-test-m2.ocean.icm.edu.pl:8088/cluster</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
<property>
<name>sparkSqlWarehouseDir</name>
<value>/user/hive/warehouse</value>
</property>
</configuration>

View File

@ -0,0 +1,183 @@
<workflow-app name="Software-Heritage-Integration-Workflow" xmlns="uri:oozie:workflow:0.5">
<!-- Custom parameters -->
<parameters>
<property>
<name>hiveDbName</name>
<description>The name of the Hive DB to be used</description>
</property>
<property>
<name>softwareCodeRepositoryURLs</name>
<description>The path in the HDFS to save the software repository URLs</description>
</property>
<property>
<name>lastVisitsPath</name>
<description>The path in the HDFS to save the responses of the last visit requests</description>
</property>
<property>
<name>archiveRequestsPath</name>
<description>The path in the HDFS to save the responses of the archive requests</description>
</property>
<property>
<name>actionsetsPath</name>
<description>The path in the HDFS to save the action sets</description>
</property>
<property>
<name>graphPath</name>
<description>The path in the HDFS to the base folder of the graph</description>
</property>
<property>
<name>maxNumberOfRetry</name>
<description>Max number of retries for failed API calls</description>
</property>
<property>
<name>retryDelay</name>
<description>Retry delay for failed requests (in sec)</description>
</property>
<property>
<name>requestDelay</name>
<description>Delay between API requests (in ms)</description>
</property>
<property>
<name>apiAccessToken</name>
<description>The API Key of the SWH API</description>
</property>
<property>
<name>softwareLimit</name>
<description>Limit on the number of repo URLs to use (Optional); for debug purposes</description>
</property>
<property>
<name>resumeFrom</name>
<description>Variable that indicates the step to start from</description>
</property>
</parameters>
<!-- Global variables -->
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
<property>
<name>actionsetsPath</name>
<value>${actionsetsPath}</value>
</property>
<property>
<name>apiAccessToken</name>
<value>${apiAccessToken}</value>
</property>
</configuration>
</global>
<start to="startFrom"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<decision name="startFrom">
<switch>
<case to="collect-software-repository-urls">${wf:conf('resumeFrom') eq 'collect-software-repository-urls'}</case>
<case to="create-swh-actionsets">${wf:conf('resumeFrom') eq 'create-swh-actionsets'}</case>
<default to="collect-software-repository-urls"/>
</switch>
</decision>
<action name="collect-software-repository-urls">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Collect software repository URLs</name>
<class>eu.dnetlib.dhp.swh.CollectSoftwareRepositoryURLs</class>
<jar>dhp-swh-${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>--softwareCodeRepositoryURLs</arg><arg>${softwareCodeRepositoryURLs}</arg>
<arg>--hiveDbName</arg><arg>${hiveDbName}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
<arg>--softwareLimit</arg><arg>${softwareLimit}</arg>
</spark>
<ok to="collect-repository-last-visit-data"/>
<error to="Kill"/>
</action>
<action name="collect-repository-last-visit-data">
<java>
<main-class>eu.dnetlib.dhp.swh.CollectLastVisitRepositoryData</main-class>
<arg>--namenode</arg><arg>${nameNode}</arg>
<arg>--softwareCodeRepositoryURLs</arg><arg>${softwareCodeRepositoryURLs}</arg>
<arg>--lastVisitsPath</arg><arg>${lastVisitsPath}</arg>
<arg>--maxNumberOfRetry</arg><arg>${maxNumberOfRetry}</arg>
<arg>--requestDelay</arg><arg>${requestDelay}</arg>
<arg>--retryDelay</arg><arg>${retryDelay}</arg>
<arg>--requestMethod</arg><arg>GET</arg>
<arg>--apiAccessToken</arg><arg>${apiAccessToken}</arg>
</java>
<ok to="archive-repository-urls"/>
<error to="Kill"/>
</action>
<action name="archive-repository-urls">
<java>
<main-class>eu.dnetlib.dhp.swh.ArchiveRepositoryURLs</main-class>
<arg>--namenode</arg><arg>${nameNode}</arg>
<arg>--lastVisitsPath</arg><arg>${lastVisitsPath}</arg>
<arg>--archiveRequestsPath</arg><arg>${archiveRequestsPath}</arg>
<arg>--archiveThresholdInDays</arg><arg>365</arg>
<arg>--maxNumberOfRetry</arg><arg>${maxNumberOfRetry}</arg>
<arg>--requestDelay</arg><arg>${requestDelay}</arg>
<arg>--retryDelay</arg><arg>${retryDelay}</arg>
<arg>--requestMethod</arg><arg>POST</arg>
<arg>--apiAccessToken</arg><arg>${apiAccessToken}</arg>
</java>
<ok to="create-swh-actionsets"/>
<error to="Kill"/>
</action>
<action name="create-swh-actionsets">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Create actionsets for SWH data</name>
<class>eu.dnetlib.dhp.swh.PrepareSWHActionsets</class>
<jar>dhp-swh-${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>--lastVisitsPath</arg><arg>${lastVisitsPath}</arg>
<arg>--actionsetsPath</arg><arg>${actionsetsPath}</arg>
<arg>--softwareInputPath</arg><arg>${graphPath}/software</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,38 @@
package eu.dnetlib.dhp.swh;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.text.ParseException;
import java.util.Arrays;
import org.apache.hadoop.fs.FileSystem;
import org.junit.jupiter.api.Test;
import eu.dnetlib.dhp.swh.utils.SWHUtils;
public class ArchiveRepositoryURLsTest {
@Test
void testArchive() throws IOException, ParseException {
String inputPath = getClass()
.getResource("/eu/dnetlib/dhp/swh/lastVisitDataToArchive.csv")
.getPath();
File file = new File(inputPath);
FileReader fr = new FileReader(file);
BufferedReader br = new BufferedReader(fr); // creates a buffering character input stream
String line;
while ((line = br.readLine()) != null) {
String[] tokens = line.split("\t");
String response = ArchiveRepositoryURLs.handleRecord(tokens[0], tokens[1], 365);
System.out.println(tokens[0] + "\t" + response);
System.out.println();
}
fr.close();
}
}

View File

@ -0,0 +1,97 @@
package eu.dnetlib.dhp.swh;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.io.Text;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
public class PrepareSWHActionsetsTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
private static final Logger log = LoggerFactory
.getLogger(PrepareSWHActionsetsTest.class);
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(PrepareSWHActionsetsTest.class.getSimpleName());
log.info("Using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(PrepareSWHActionsetsTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
spark = SparkSession
.builder()
.appName(PrepareSWHActionsetsTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void testRun() throws Exception {
String lastVisitsPath = getClass()
.getResource("/eu/dnetlib/dhp/swh/last_visits_data.seq")
.getPath();
String outputPath = workingDir.toString() + "/actionSet";
String softwareInputPath = getClass()
.getResource("/eu/dnetlib/dhp/swh/software.json.gz")
.getPath();
PrepareSWHActionsets
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-lastVisitsPath", lastVisitsPath,
"-softwareInputPath", softwareInputPath,
"-actionsetsPath", outputPath
});
}
}

View File

@ -0,0 +1,58 @@
package eu.dnetlib.dhp.swh;
import java.io.IOException;
import java.net.MalformedURLException;
import java.net.URL;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.dhp.swh.utils.SWHConnection;
import eu.dnetlib.dhp.swh.utils.SWHConstants;
//import org.apache.hadoop.hdfs.MiniDFSCluster;
public class SWHConnectionTest {
private static final Logger log = LoggerFactory.getLogger(SWHConnectionTest.class);
@Test
void testGetCall() throws IOException {
HttpClientParams clientParams = new HttpClientParams();
clientParams.setRequestMethod("GET");
SWHConnection swhConnection = new SWHConnection(clientParams, null);
String repoUrl = "https://github.com/stanford-futuredata/FAST";
URL url = new URL(String.format(SWHConstants.SWH_LATEST_VISIT_URL, repoUrl));
String response = null;
try {
response = swhConnection.call(url.toString());
} catch (CollectorException e) {
System.out.println("Error in request: " + url);
}
System.out.println(response);
}
@Test
void testPostCall() throws MalformedURLException {
HttpClientParams clientParams = new HttpClientParams();
clientParams.setRequestMethod("POST");
SWHConnection swhConnection = new SWHConnection(clientParams, null);
String repoUrl = "https://github.com/stanford-futuredata/FAST";
URL url = new URL(String.format(SWHConstants.SWH_ARCHIVE_URL, SWHConstants.DEFAULT_VISIT_TYPE, repoUrl));
String response = null;
try {
response = swhConnection.call(url.toString());
} catch (CollectorException e) {
System.out.println("Error in request: " + url);
}
System.out.println(response);
}
}

View File

@ -0,0 +1,7 @@
https://bitbucket.org/samskillman/yt-stokes {"origin":"https://bitbucket.org/samskillman/yt-stokes","visit":43,"date":"2021-09-13T21:59:27.125171+00:00","status":"failed","snapshot":null,"type":"hg","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://bitbucket.org/samskillman/yt-stokes/get/","snapshot_url":null}
https://github.com/bioinsilico/BIPSPI {"origin":"https://github.com/bioinsilico/BIPSPI","visit":1,"date":"2020-03-18T14:50:21.541822+00:00","status":"full","snapshot":"c6c69d2cd73ce89811448da5f031611df6f63bdb","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/bioinsilico/BIPSPI/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/c6c69d2cd73ce89811448da5f031611df6f63bdb/"}
https://github.com/mloop/kdiff-type1-error-rate/blob/master/analysis/simulation.R {}
https://github.com/schwanbeck/YSMR {"origin":"https://github.com/schwanbeck/YSMR","visit":6,"date":"2023-08-02T15:25:02.650676+00:00","status":"full","snapshot":"a9d1c5f0bca2def198b89f65bc9f7da3be8439ed","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/schwanbeck/YSMR/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/a9d1c5f0bca2def198b89f65bc9f7da3be8439ed/"}
https://github.com/lvclark/TASSELGBS_combine {"origin":"https://github.com/lvclark/TASSELGBS_combine","visit":1,"date":"2020-04-12T20:44:09.405589+00:00","status":"full","snapshot":"ffa6fefd3f5becefbea9fe0e6d5d93859c95c071","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/lvclark/TASSELGBS_combine/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/ffa6fefd3f5becefbea9fe0e6d5d93859c95c071/"}
https://github.com/PRIDE-Toolsuite/inspector-example-files {"origin":"https://github.com/PRIDE-Toolsuite/inspector-example-files","visit":12,"date":"2021-01-25T08:54:13.394674+00:00","status":"full","snapshot":"0b56eb0ad07cf778df6dabefc4b73636e0ae8b37","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/PRIDE-Toolsuite/inspector-example-files/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/0b56eb0ad07cf778df6dabefc4b73636e0ae8b37/"}
https://bitbucket.org/matwey/chelyabinsk {"origin":"https://bitbucket.org/matwey/chelyabinsk","visit":6,"date":"2021-09-24T19:32:43.322909+00:00","status":"full","snapshot":"215913858c3ee0e61e1aaea18241c5ee006da1b0","type":"hg","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://bitbucket.org/matwey/chelyabinsk/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/215913858c3ee0e61e1aaea18241c5ee006da1b0/"}
Can't render this file because it contains an unexpected character in line 1 and column 46.

View File

@ -0,0 +1,111 @@
General notes
====================
Oozie-installer is a utility allowing building, uploading and running oozie workflows. In practice, it creates a `*.tar.gz` package that contains resouces that define a workflow and some helper scripts.
This module is automatically executed when running:
`mvn package -Poozie-package -Dworkflow.source.dir=classpath/to/parent/directory/of/oozie_app`
on module having set:
<parent>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-workflows</artifactId>
</parent>
in `pom.xml` file. `oozie-package` profile initializes oozie workflow packaging, `workflow.source.dir` property points to a workflow (notice: this is not a relative path but a classpath to directory usually holding `oozie_app` subdirectory).
The outcome of this packaging is `oozie-package.tar.gz` file containing inside all the resources required to run Oozie workflow:
- jar packages
- workflow definitions
- job properties
- maintenance scripts
Required properties
====================
In order to include proper workflow within package, `workflow.source.dir` property has to be set. It could be provided by setting `-Dworkflow.source.dir=some/job/dir` maven parameter.
In oder to define full set of cluster environment properties one should create `~/.dhp/application.properties` file with the following properties:
- `dhp.hadoop.frontend.user.name` - your user name on hadoop cluster and frontend machine
- `dhp.hadoop.frontend.host.name` - frontend host name
- `dhp.hadoop.frontend.temp.dir` - frontend directory for temporary files
- `dhp.hadoop.frontend.port.ssh` - frontend machine ssh port
- `oozieServiceLoc` - oozie service location required by run_workflow.sh script executing oozie job
- `nameNode` - name node address
- `jobTracker` - job tracker address
- `oozie.execution.log.file.location` - location of file that will be created when executing oozie job, it contains output produced by `run_workflow.sh` script (needed to obtain oozie job id)
- `maven.executable` - mvn command location, requires parameterization due to a different setup of CI cluster
- `sparkDriverMemory` - amount of memory assigned to spark jobs driver
- `sparkExecutorMemory` - amount of memory assigned to spark jobs executors
- `sparkExecutorCores` - number of cores assigned to spark jobs executors
All values will be overriden with the ones from `job.properties` and eventually `job-override.properties` stored in module's main folder.
When overriding properties from `job.properties`, `job-override.properties` file can be created in main module directory (the one containing `pom.xml` file) and define all new properties which will override existing properties. One can provide those properties one by one as command line -D arguments.
Properties overriding order is the following:
1. `pom.xml` defined properties (located in the project root dir)
2. `~/.dhp/application.properties` defined properties
3. `${workflow.source.dir}/job.properties`
4. `job-override.properties` (located in the project root dir)
5. `maven -Dparam=value`
where the maven `-Dparam` property is overriding all the other ones.
Workflow definition requirements
====================
`workflow.source.dir` property should point to the following directory structure:
[${workflow.source.dir}]
|
|-job.properties (optional)
|
\-[oozie_app]
|
\-workflow.xml
This property can be set using maven `-D` switch.
`[oozie_app]` is the default directory name however it can be set to any value as soon as `oozieAppDir` property is provided with directory name as value.
Subworkflows are supported as well and subworkflow directories should be nested within `[oozie_app]` directory.
Creating oozie installer step-by-step
=====================================
Automated oozie-installer steps are the following:
1. creating jar packages: `*.jar` and `*tests.jar` along with copying all dependancies in `target/dependencies`
2. reading properties from maven, `~/.dhp/application.properties`, `job.properties`, `job-override.properties`
3. invoking priming mechanism linking resources from import.txt file (currently resolving subworkflow resources)
4. assembling shell scripts for preparing Hadoop filesystem, uploading Oozie application and starting workflow
5. copying whole `${workflow.source.dir}` content to `target/${oozie.package.file.name}`
6. generating updated `job.properties` file in `target/${oozie.package.file.name}` based on maven, `~/.dhp/application.properties`, `job.properties` and `job-override.properties`
7. creating `lib` directory (or multiple directories for subworkflows for each nested directory) and copying jar packages created at step (1) to each one of them
8. bundling whole `${oozie.package.file.name}` directory into single tar.gz package
Uploading oozie package and running workflow on cluster
=======================================================
In order to simplify deployment and execution process two dedicated profiles were introduced:
- `deploy`
- `run`
to be used along with `oozie-package` profile e.g. by providing `-Poozie-package,deploy,run` maven parameters.
`deploy` profile supplements packaging process with:
1) uploading oozie-package via scp to `/home/${user.name}/oozie-packages` directory on `${dhp.hadoop.frontend.host.name}` machine
2) extracting uploaded package
3) uploading oozie content to hadoop cluster HDFS location defined in `oozie.wf.application.path` property (generated dynamically by maven build process, based on `${dhp.hadoop.frontend.user.name}` and `workflow.source.dir` properties)
`run` profile introduces:
1) executing oozie application uploaded to HDFS cluster using `deploy` command. Triggers `run_workflow.sh` script providing runtime properties defined in `job.properties` file.
Notice: ssh access to frontend machine has to be configured on system level and it is preferable to set key-based authentication in order to simplify remote operations.

View File

@ -25,6 +25,7 @@
<modules>
<module>dhp-workflow-profiles</module>
<module>dhp-aggregation</module>
<module>dhp-distcp</module>
<module>dhp-actionmanager</module>
<module>dhp-graph-mapper</module>
<module>dhp-dedup-openaire</module>
@ -38,6 +39,7 @@
<module>dhp-broker-events</module>
<module>dhp-doiboost</module>
<module>dhp-impact-indicators</module>
<module>dhp-swh</module>
</modules>
<pluginRepositories>

10
pom.xml
View File

@ -112,6 +112,16 @@
<url>https://maven.d4science.org/nexus/content/repositories/dnet-deps</url>
<layout>default</layout>
</repository>
<repository>
<id>maven-restlet</id>
<name>Restlet repository</name>
<url>https://maven.restlet.talend.com</url>
</repository>
<repository>
<id>conjars</id>
<name>conjars</name>
<url>https://conjars.wensel.net/repo/</url>
</repository>
</repositories>
<dependencies>