Adding H2020 Classification, topic code and topic description to H2020 projects #46
|
@ -0,0 +1,25 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
import eu.dnetlib.broker.api.ShortEventMessage;
|
||||
|
||||
public class ShortEventMessageWithGroupId extends ShortEventMessage implements Serializable {
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private static final long serialVersionUID = 4704889388757626630L;
|
||||
|
||||
private String group;
|
||||
|
||||
public String getGroup() {
|
||||
return group;
|
||||
}
|
||||
|
||||
public void setGroup(final String group) {
|
||||
this.group = group;
|
||||
}
|
||||
|
||||
}
|
|
@ -14,6 +14,7 @@ import org.apache.http.impl.client.CloseableHttpClient;
|
|||
import org.apache.http.impl.client.HttpClients;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.TypedColumn;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -74,6 +75,7 @@ public class GenerateStatsJob {
|
|||
.agg(aggr)
|
||||
.map(t -> t._2, Encoders.bean(DatasourceStats.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.jdbc(dbUrl, "oa_datasource_stats_temp", connectionProperties);
|
||||
|
||||
log.info("*** updateStats");
|
||||
|
|
|
@ -20,12 +20,11 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import com.google.gson.Gson;
|
||||
|
||||
import eu.dnetlib.broker.api.ShortEventMessage;
|
||||
import eu.dnetlib.broker.objects.OaBrokerEventPayload;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.broker.model.Event;
|
||||
import eu.dnetlib.dhp.broker.model.ShortEventMessageWithGroupId;
|
||||
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class PartitionEventsByDsIdJob {
|
||||
|
||||
|
@ -61,13 +60,11 @@ public class PartitionEventsByDsIdJob {
|
|||
.readPath(spark, eventsPath, Event.class)
|
||||
.filter(e -> StringUtils.isNotBlank(e.getMap().getTargetDatasourceId()))
|
||||
.filter(e -> e.getMap().getTargetDatasourceId().contains(OPENDOAR_NSPREFIX))
|
||||
.map(
|
||||
e -> new Tuple2<>(
|
||||
StringUtils.substringAfter(e.getMap().getTargetDatasourceId(), OPENDOAR_NSPREFIX),
|
||||
messageFromNotification(e)),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.bean(ShortEventMessage.class)))
|
||||
.limit(10000)
|
||||
.map(e -> messageFromNotification(e), Encoders.bean(ShortEventMessageWithGroupId.class))
|
||||
.coalesce(1)
|
||||
.write()
|
||||
.partitionBy("_1")
|
||||
.partitionBy("group")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.json(partitionPath);
|
||||
|
||||
|
@ -77,7 +74,6 @@ public class PartitionEventsByDsIdJob {
|
|||
}
|
||||
|
||||
private static void renameSubDirs(final String path) throws IOException {
|
||||
final String prefix = "_1=";
|
||||
final FileSystem fs = FileSystem.get(new Configuration());
|
||||
|
||||
log.info("** Renaming subdirs of " + path);
|
||||
|
@ -85,8 +81,8 @@ public class PartitionEventsByDsIdJob {
|
|||
if (fileStatus.isDirectory()) {
|
||||
final Path oldPath = fileStatus.getPath();
|
||||
final String oldName = oldPath.getName();
|
||||
if (oldName.startsWith(prefix)) {
|
||||
final Path newPath = new Path(path + "/" + StringUtils.substringAfter(oldName, prefix));
|
||||
if (oldName.contains("=")) {
|
||||
final Path newPath = new Path(path + "/" + StringUtils.substringAfter(oldName, "="));
|
||||
log.info(" * " + oldPath.getName() + " -> " + newPath.getName());
|
||||
fs.rename(oldPath, newPath);
|
||||
}
|
||||
|
@ -94,18 +90,19 @@ public class PartitionEventsByDsIdJob {
|
|||
}
|
||||
}
|
||||
|
||||
private static ShortEventMessage messageFromNotification(final Event e) {
|
||||
private static ShortEventMessageWithGroupId messageFromNotification(final Event e) {
|
||||
final Gson gson = new Gson();
|
||||
|
||||
final OaBrokerEventPayload payload = gson.fromJson(e.getPayload(), OaBrokerEventPayload.class);
|
||||
|
||||
final ShortEventMessage res = new ShortEventMessage();
|
||||
final ShortEventMessageWithGroupId res = new ShortEventMessageWithGroupId();
|
||||
|
||||
res.setOriginalId(payload.getResult().getOriginalId());
|
||||
res.setTitle(payload.getResult().getTitles().stream().filter(StringUtils::isNotBlank).findFirst().orElse(null));
|
||||
res.setTopic(e.getTopic());
|
||||
res.setTrust(payload.getTrust());
|
||||
res.generateMessageFromObject(payload.getHighlight());
|
||||
res.setGroup(StringUtils.substringAfter(e.getMap().getTargetDatasourceId(), OPENDOAR_NSPREFIX));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
|
|
@ -44,6 +44,18 @@
|
|||
<name>brokerApiBaseUrl</name>
|
||||
<description>the url of the broker service api</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>brokerDbUrl</name>
|
||||
<description>the url of the broker database</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>brokerDbUser</name>
|
||||
<description>the user of the broker database</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>brokerDbPassword</name>
|
||||
<description>the password of the broker database</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
|
@ -509,8 +521,11 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
</spark-opts>
|
||||
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--dbUrl</arg><arg>${brokerDbUrl}</arg>
|
||||
<arg>--dbUser</arg><arg>${brokerDbUser}</arg>
|
||||
<arg>--dbPassword</arg><arg>${brokerDbPassword}</arg>
|
||||
<arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
|
|
|
@ -111,18 +111,18 @@
|
|||
</configuration>
|
||||
</global>
|
||||
|
||||
<start to="stats"/>
|
||||
<start to="partition"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="stats">
|
||||
<action name="partition">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>GenerateStatsJob</name>
|
||||
<class>eu.dnetlib.dhp.broker.oa.GenerateStatsJob</class>
|
||||
<name>PartitionEventsByDsIdJob</name>
|
||||
<class>eu.dnetlib.dhp.broker.oa.PartitionEventsByDsIdJob</class>
|
||||
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
|
@ -134,11 +134,8 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
</spark-opts>
|
||||
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--dbUrl</arg><arg>${brokerDbUrl}</arg>
|
||||
<arg>--dbUser</arg><arg>${brokerDbUser}</arg>
|
||||
<arg>--dbPassword</arg><arg>${brokerDbPassword}</arg>
|
||||
<arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
|
|
|
@ -577,7 +577,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
|
|||
final String lissn = StringUtils.isNotBlank(arr[2]) ? arr[2].trim() : null;
|
||||
|
||||
if (issn != null || eissn != null || lissn != null) {
|
||||
return journal(name, issn, eissn, eissn, null, null, null, null, null, null, null, info);
|
||||
return journal(name, issn, eissn, lissn, null, null, null, null, null, null, null, info);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -3,7 +3,7 @@ SELECT
|
|||
d.id || array_agg(distinct di.pid) AS identities,
|
||||
d.officialname AS officialname,
|
||||
d.englishname AS englishname,
|
||||
d.contactemail AS contactemail,
|
||||
d.contactemail AS contactemail,
|
||||
CASE
|
||||
WHEN (array_agg(DISTINCT COALESCE (a.compatibility_override, a.compatibility):: TEXT) @> ARRAY ['openaire-cris_1.1'])
|
||||
THEN
|
||||
|
@ -85,7 +85,7 @@ SELECT
|
|||
dc.officialname AS collectedfromname,
|
||||
d.typology||'@@@dnet:datasource_typologies' AS datasourcetype,
|
||||
'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction,
|
||||
d.issn || ' @@@ ' || d.eissn || ' @@@ ' || d.lissn AS journal
|
||||
concat_ws(' @@@ ', d.issn, d.eissn, d.lissn) AS journal
|
||||
|
||||
FROM dsm_datasources d
|
||||
|
||||
|
|
|
@ -73,12 +73,16 @@ public class MigrateDbEntitiesApplicationTest {
|
|||
final Datasource ds = (Datasource) list.get(0);
|
||||
assertValidId(ds.getId());
|
||||
assertValidId(ds.getCollectedfrom().get(0).getKey());
|
||||
assertEquals(ds.getOfficialname().getValue(), getValueAsString("officialname", fields));
|
||||
assertEquals(ds.getEnglishname().getValue(), getValueAsString("englishname", fields));
|
||||
assertEquals(ds.getContactemail().getValue(), getValueAsString("contactemail", fields));
|
||||
assertEquals(ds.getWebsiteurl().getValue(), getValueAsString("websiteurl", fields));
|
||||
assertEquals(ds.getNamespaceprefix().getValue(), getValueAsString("namespaceprefix", fields));
|
||||
assertEquals(ds.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields));
|
||||
assertEquals(getValueAsString("officialname", fields), ds.getOfficialname().getValue());
|
||||
assertEquals(getValueAsString("englishname", fields), ds.getEnglishname().getValue());
|
||||
assertEquals(getValueAsString("contactemail", fields), ds.getContactemail().getValue());
|
||||
assertEquals(getValueAsString("websiteurl", fields), ds.getWebsiteurl().getValue());
|
||||
assertEquals(getValueAsString("namespaceprefix", fields), ds.getNamespaceprefix().getValue());
|
||||
assertEquals(getValueAsString("collectedfromname", fields), ds.getCollectedfrom().get(0).getValue());
|
||||
assertEquals(getValueAsString("officialname", fields), ds.getJournal().getName());
|
||||
assertEquals("2579-5449", ds.getJournal().getIssnPrinted());
|
||||
assertEquals("2597-6540", ds.getJournal().getIssnOnline());
|
||||
assertEquals(null, ds.getJournal().getIssnLinking());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -92,9 +96,11 @@ public class MigrateDbEntitiesApplicationTest {
|
|||
final Project p = (Project) list.get(0);
|
||||
assertValidId(p.getId());
|
||||
assertValidId(p.getCollectedfrom().get(0).getKey());
|
||||
assertEquals(p.getAcronym().getValue(), getValueAsString("acronym", fields));
|
||||
assertEquals(p.getTitle().getValue(), getValueAsString("title", fields));
|
||||
assertEquals(p.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields));
|
||||
assertEquals(getValueAsString("acronym", fields), p.getAcronym().getValue());
|
||||
assertEquals(getValueAsString("title", fields), p.getTitle().getValue());
|
||||
assertEquals(getValueAsString("collectedfromname", fields), p.getCollectedfrom().get(0).getValue());
|
||||
assertEquals(getValueAsFloat("fundedamount", fields), p.getFundedamount());
|
||||
assertEquals(getValueAsFloat("totalcost", fields), p.getTotalcost());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -110,14 +116,14 @@ public class MigrateDbEntitiesApplicationTest {
|
|||
final Organization o = (Organization) list.get(0);
|
||||
assertValidId(o.getId());
|
||||
assertValidId(o.getCollectedfrom().get(0).getKey());
|
||||
assertEquals(o.getLegalshortname().getValue(), getValueAsString("legalshortname", fields));
|
||||
assertEquals(o.getLegalname().getValue(), getValueAsString("legalname", fields));
|
||||
assertEquals(o.getWebsiteurl().getValue(), getValueAsString("websiteurl", fields));
|
||||
assertEquals(o.getCountry().getClassid(), getValueAsString("country", fields).split("@@@")[0]);
|
||||
assertEquals(o.getCountry().getClassname(), getValueAsString("country", fields).split("@@@")[0]);
|
||||
assertEquals(o.getCountry().getSchemeid(), getValueAsString("country", fields).split("@@@")[1]);
|
||||
assertEquals(o.getCountry().getSchemename(), getValueAsString("country", fields).split("@@@")[1]);
|
||||
assertEquals(o.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields));
|
||||
assertEquals(getValueAsString("legalshortname", fields), o.getLegalshortname().getValue());
|
||||
assertEquals(getValueAsString("legalname", fields), o.getLegalname().getValue());
|
||||
assertEquals(getValueAsString("websiteurl", fields), o.getWebsiteurl().getValue());
|
||||
assertEquals(getValueAsString("country", fields).split("@@@")[0], o.getCountry().getClassid());
|
||||
assertEquals(getValueAsString("country", fields).split("@@@")[0], o.getCountry().getClassname());
|
||||
assertEquals(getValueAsString("country", fields).split("@@@")[1], o.getCountry().getSchemeid());
|
||||
assertEquals(getValueAsString("country", fields).split("@@@")[1], o.getCountry().getSchemename());
|
||||
assertEquals(getValueAsString("collectedfromname", fields), o.getCollectedfrom().get(0).getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -322,12 +328,20 @@ public class MigrateDbEntitiesApplicationTest {
|
|||
}
|
||||
|
||||
private String getValueAsString(final String name, final List<TypedField> fields) {
|
||||
return getValueAs(name, fields);
|
||||
}
|
||||
|
||||
private Float getValueAsFloat(final String name, final List<TypedField> fields) {
|
||||
return new Float(getValueAs(name, fields).toString());
|
||||
}
|
||||
|
||||
private <T> T getValueAs(final String name, final List<TypedField> fields) {
|
||||
return fields
|
||||
.stream()
|
||||
.filter(f -> f.getField().equals(name))
|
||||
.map(TypedField::getValue)
|
||||
.filter(Objects::nonNull)
|
||||
.map(o -> o.toString())
|
||||
.map(o -> (T) o)
|
||||
.findFirst()
|
||||
.get();
|
||||
}
|
||||
|
|
|
@ -142,12 +142,12 @@
|
|||
{
|
||||
"field": "totalcost",
|
||||
"type": "double",
|
||||
"value": null
|
||||
"value": 157846
|
||||
},
|
||||
{
|
||||
"field": "fundedamount",
|
||||
"type": "double",
|
||||
"value": null
|
||||
"value": 157846
|
||||
},
|
||||
{
|
||||
"field": "collectedfromid",
|
||||
|
|
Loading…
Reference in New Issue