generateNotificationsJob and indexNotificationsJob

This commit is contained in:
Michele Artini 2021-10-05 13:57:46 +02:00
parent 69008e20c2
commit 210d6c0e6d
5 changed files with 279 additions and 138 deletions

View File

@ -0,0 +1,184 @@
package eu.dnetlib.dhp.broker.oa;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.model.ConditionParams;
import eu.dnetlib.dhp.broker.model.Event;
import eu.dnetlib.dhp.broker.model.MappedFields;
import eu.dnetlib.dhp.broker.model.Notification;
import eu.dnetlib.dhp.broker.model.Subscription;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.NotificationGroup;
import eu.dnetlib.dhp.broker.oa.util.SubscriptionUtils;
public class GenerateNotificationsJob {
private static final Logger log = LoggerFactory.getLogger(GenerateNotificationsJob.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(GenerateNotificationsJob.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/generate_notifications.json")));
parser.parseArgument(args);
final SparkConf conf = new SparkConf();
final String eventsPath = parser.get("outputDir") + "/events";
log.info("eventsPath: {}", eventsPath);
final String notificationsPath = parser.get("outputDir") + "/notifications";
log.info("notificationsPath: {}", notificationsPath);
final String brokerApiBaseUrl = parser.get("brokerApiBaseUrl");
log.info("brokerApiBaseUrl: {}", brokerApiBaseUrl);
final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
final LongAccumulator total = spark.sparkContext().longAccumulator("total_notifications");
final long startTime = new Date().getTime();
final List<Subscription> subscriptions = listSubscriptions(brokerApiBaseUrl);
log.info("Number of subscriptions: " + subscriptions.size());
if (subscriptions.size() > 0) {
final Map<String, Map<String, List<ConditionParams>>> conditionsMap = prepareConditionsMap(subscriptions);
log.info("ConditionsMap: " + new ObjectMapper().writeValueAsString(conditionsMap));
final Encoder<NotificationGroup> ngEncoder = Encoders.bean(NotificationGroup.class);
final Encoder<Notification> nEncoder = Encoders.bean(Notification.class);
final Dataset<Notification> notifications = ClusterUtils
.readPath(spark, eventsPath, Event.class)
.map((MapFunction<Event, NotificationGroup>) e -> generateNotifications(e, subscriptions, conditionsMap, startTime), ngEncoder)
.flatMap((FlatMapFunction<NotificationGroup, Notification>) g -> g.getData().iterator(), nEncoder);
ClusterUtils.save(notifications, notificationsPath, Notification.class, total);
}
}
protected static Map<String, Map<String, List<ConditionParams>>> prepareConditionsMap(
final List<Subscription> subscriptions) {
final Map<String, Map<String, List<ConditionParams>>> map = new HashMap<>();
subscriptions.forEach(s -> map.put(s.getSubscriptionId(), s.conditionsAsMap()));
return map;
}
protected static NotificationGroup generateNotifications(final Event e,
final List<Subscription> subscriptions,
final Map<String, Map<String, List<ConditionParams>>> conditionsMap,
final long date) {
final List<Notification> list = subscriptions
.stream()
.filter(s -> StringUtils.isBlank(s.getTopic()) || s.getTopic().equals("*") || s.getTopic().equals(e.getTopic()))
.filter(s -> verifyConditions(e.getMap(), conditionsMap.get(s.getSubscriptionId())))
.map(s -> generateNotification(s, e, date))
.collect(Collectors.toList());
return new NotificationGroup(list);
}
private static Notification generateNotification(final Subscription s, final Event e, final long date) {
final Notification n = new Notification();
n.setNotificationId("ntf-" + DigestUtils.md5Hex(s.getSubscriptionId() + "@@@" + e.getEventId()));
n.setSubscriptionId(s.getSubscriptionId());
n.setEventId(e.getEventId());
n.setProducerId(e.getProducerId());
n.setTopic(e.getTopic());
n.setPayload(e.getPayload());
n.setMap(e.getMap());
n.setDate(date);
return n;
}
private static boolean verifyConditions(final MappedFields map,
final Map<String, List<ConditionParams>> conditions) {
if (conditions.containsKey("targetDatasourceName")
&& !SubscriptionUtils
.verifyExact(map.getTargetDatasourceName(), conditions.get("targetDatasourceName").get(0).getValue())) {
return false;
}
if (conditions.containsKey("trust")
&& !SubscriptionUtils
.verifyFloatRange(map.getTrust(), conditions.get("trust").get(0).getValue(), conditions.get("trust").get(0).getOtherValue())) {
return false;
}
if (conditions.containsKey("targetDateofacceptance") && !conditions
.get("targetDateofacceptance")
.stream()
.anyMatch(c -> SubscriptionUtils
.verifyDateRange(map.getTargetDateofacceptance(), c.getValue(), c.getOtherValue()))) {
return false;
}
if (conditions.containsKey("targetResultTitle")
&& !conditions
.get("targetResultTitle")
.stream()
.anyMatch(c -> SubscriptionUtils.verifySimilar(map.getTargetResultTitle(), c.getValue()))) {
return false;
}
if (conditions.containsKey("targetAuthors")
&& !conditions
.get("targetAuthors")
.stream()
.allMatch(c -> SubscriptionUtils.verifyListSimilar(map.getTargetAuthors(), c.getValue()))) {
return false;
}
return !conditions.containsKey("targetSubjects")
|| conditions
.get("targetSubjects")
.stream()
.allMatch(c -> SubscriptionUtils.verifyListExact(map.getTargetSubjects(), c.getValue()));
}
private static List<Subscription> listSubscriptions(final String brokerApiBaseUrl) throws Exception {
final String url = brokerApiBaseUrl + "/api/subscriptions";
final HttpGet req = new HttpGet(url);
final ObjectMapper mapper = new ObjectMapper();
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {
final String s = IOUtils.toString(response.getEntity().getContent());
return mapper
.readValue(s, mapper.getTypeFactory().constructCollectionType(List.class, Subscription.class));
}
}
}
}

View File

@ -2,28 +2,22 @@
package eu.dnetlib.dhp.broker.oa; package eu.dnetlib.dhp.broker.oa;
import java.io.IOException; import java.io.IOException;
import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpDelete; import org.apache.http.client.methods.HttpDelete;
import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpGet;
import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients; import org.apache.http.impl.client.HttpClients;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.SparkSession;
import org.apache.spark.util.LongAccumulator; import org.apache.spark.util.LongAccumulator;
import org.elasticsearch.spark.rdd.api.java.JavaEsSpark;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -31,14 +25,8 @@ import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.model.ConditionParams;
import eu.dnetlib.dhp.broker.model.Event;
import eu.dnetlib.dhp.broker.model.MappedFields;
import eu.dnetlib.dhp.broker.model.Notification; import eu.dnetlib.dhp.broker.model.Notification;
import eu.dnetlib.dhp.broker.model.Subscription;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.NotificationGroup;
import eu.dnetlib.dhp.broker.oa.util.SubscriptionUtils;
public class IndexNotificationsJob { public class IndexNotificationsJob {
@ -48,15 +36,14 @@ public class IndexNotificationsJob {
final ArgumentApplicationParser parser = new ArgumentApplicationParser( final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils IOUtils
.toString( .toString(IndexNotificationsJob.class
IndexNotificationsJob.class .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/index_notifications.json")));
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/index_notifications.json")));
parser.parseArgument(args); parser.parseArgument(args);
final SparkConf conf = new SparkConf(); final SparkConf conf = new SparkConf();
final String eventsPath = parser.get("outputDir") + "/events"; final String notificationsPath = parser.get("outputDir") + "/notifications";
log.info("eventsPath: {}", eventsPath); log.info("notificationsPath: {}", notificationsPath);
final String index = parser.get("index"); final String index = parser.get("index");
log.info("index: {}", index); log.info("index: {}", index);
@ -83,131 +70,38 @@ public class IndexNotificationsJob {
final LongAccumulator total = spark.sparkContext().longAccumulator("total_indexed"); final LongAccumulator total = spark.sparkContext().longAccumulator("total_indexed");
final long startTime = new Date().getTime(); final Long date = ClusterUtils
.readPath(spark, notificationsPath, Notification.class)
.first()
.getDate();
final List<Subscription> subscriptions = listSubscriptions(brokerApiBaseUrl); final JavaRDD<String> toIndexRdd = ClusterUtils
.readPath(spark, notificationsPath, Notification.class)
.map((MapFunction<Notification, String>) n -> prepareForIndexing(n, total), Encoders.STRING())
.javaRDD();
log.info("Number of subscriptions: " + subscriptions.size()); final Map<String, String> esCfg = new HashMap<>();
if (subscriptions.size() > 0) { esCfg.put("es.index.auto.create", "false");
final Map<String, Map<String, List<ConditionParams>>> conditionsMap = prepareConditionsMap(subscriptions); esCfg.put("es.nodes", indexHost);
esCfg.put("es.mapping.id", "notificationId"); // THE PRIMARY KEY
esCfg.put("es.batch.write.retry.count", esBatchWriteRetryCount);
esCfg.put("es.batch.write.retry.wait", esBatchWriteRetryWait);
esCfg.put("es.batch.size.entries", esBatchSizeEntries);
esCfg.put("es.nodes.wan.only", esNodesWanOnly);
log.info("ConditionsMap: " + new ObjectMapper().writeValueAsString(conditionsMap)); log.info("*** Start indexing");
JavaEsSpark.saveJsonToEs(toIndexRdd, index, esCfg);
log.info("*** End indexing");
final Encoder<NotificationGroup> ngEncoder = Encoders.bean(NotificationGroup.class); log.info("*** Deleting old notifications");
final Encoder<Notification> nEncoder = Encoders.bean(Notification.class); final String message = deleteOldNotifications(brokerApiBaseUrl, date - 1000);
final Dataset<Notification> notifications = ClusterUtils log.info("*** Deleted notifications: {}", message);
.readPath(spark, eventsPath, Event.class)
.map(
(MapFunction<Event, NotificationGroup>) e -> generateNotifications(
e, subscriptions, conditionsMap, startTime),
ngEncoder)
.flatMap((FlatMapFunction<NotificationGroup, Notification>) g -> g.getData().iterator(), nEncoder);
notifications log.info("*** sendNotifications (emails, ...)");
.map((MapFunction<Notification, String>) n -> prepareForIndexing(n, total), Encoders.STRING()) sendNotifications(brokerApiBaseUrl, date - 1000);
.javaRDD() log.info("*** ALL done.");
.saveAsTextFile("/tmp/IndexNotificationsJob_test_6504");
}
}
protected static Map<String, Map<String, List<ConditionParams>>> prepareConditionsMap(
final List<Subscription> subscriptions) {
final Map<String, Map<String, List<ConditionParams>>> map = new HashMap<>();
subscriptions.forEach(s -> map.put(s.getSubscriptionId(), s.conditionsAsMap()));
return map;
}
protected static NotificationGroup generateNotifications(final Event e,
final List<Subscription> subscriptions,
final Map<String, Map<String, List<ConditionParams>>> conditionsMap,
final long date) {
final List<Notification> list = subscriptions
.stream()
.filter(
s -> StringUtils.isBlank(s.getTopic()) || s.getTopic().equals("*") || s.getTopic().equals(e.getTopic()))
.filter(s -> verifyConditions(e.getMap(), conditionsMap.get(s.getSubscriptionId())))
.map(s -> generateNotification(s, e, date))
.collect(Collectors.toList());
return new NotificationGroup(list);
}
private static Notification generateNotification(final Subscription s, final Event e, final long date) {
final Notification n = new Notification();
n.setNotificationId("ntf-" + DigestUtils.md5Hex(s.getSubscriptionId() + "@@@" + e.getEventId()));
n.setSubscriptionId(s.getSubscriptionId());
n.setEventId(e.getEventId());
n.setProducerId(e.getProducerId());
n.setTopic(e.getTopic());
n.setPayload(e.getPayload());
n.setMap(e.getMap());
n.setDate(date);
return n;
}
private static boolean verifyConditions(final MappedFields map,
final Map<String, List<ConditionParams>> conditions) {
if (conditions.containsKey("targetDatasourceName")
&& !SubscriptionUtils
.verifyExact(map.getTargetDatasourceName(), conditions.get("targetDatasourceName").get(0).getValue())) {
return false;
}
if (conditions.containsKey("trust")
&& !SubscriptionUtils
.verifyFloatRange(
map.getTrust(), conditions.get("trust").get(0).getValue(),
conditions.get("trust").get(0).getOtherValue())) {
return false;
}
if (conditions.containsKey("targetDateofacceptance") && !conditions
.get("targetDateofacceptance")
.stream()
.anyMatch(
c -> SubscriptionUtils
.verifyDateRange(map.getTargetDateofacceptance(), c.getValue(), c.getOtherValue()))) {
return false;
}
if (conditions.containsKey("targetResultTitle")
&& !conditions
.get("targetResultTitle")
.stream()
.anyMatch(c -> SubscriptionUtils.verifySimilar(map.getTargetResultTitle(), c.getValue()))) {
return false;
}
if (conditions.containsKey("targetAuthors")
&& !conditions
.get("targetAuthors")
.stream()
.allMatch(c -> SubscriptionUtils.verifyListSimilar(map.getTargetAuthors(), c.getValue()))) {
return false;
}
return !conditions.containsKey("targetSubjects")
|| conditions
.get("targetSubjects")
.stream()
.allMatch(c -> SubscriptionUtils.verifyListExact(map.getTargetSubjects(), c.getValue()));
}
private static List<Subscription> listSubscriptions(final String brokerApiBaseUrl) throws Exception {
final String url = brokerApiBaseUrl + "/api/subscriptions";
final HttpGet req = new HttpGet(url);
final ObjectMapper mapper = new ObjectMapper();
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {
final String s = IOUtils.toString(response.getEntity().getContent());
return mapper
.readValue(s, mapper.getTypeFactory().constructCollectionType(List.class, Subscription.class));
}
}
} }
private static String deleteOldNotifications(final String brokerApiBaseUrl, final long l) throws Exception { private static String deleteOldNotifications(final String brokerApiBaseUrl, final long l) throws Exception {

View File

@ -542,6 +542,30 @@
<arg>--dbPassword</arg><arg>${brokerDbPassword}</arg> <arg>--dbPassword</arg><arg>${brokerDbPassword}</arg>
<arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg> <arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg>
</spark> </spark>
<ok to="generate_notifications"/>
<error to="Kill"/>
</action>
<action name="generate_notifications">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>GenerateNotificationsJob</name>
<class>eu.dnetlib.dhp.broker.oa.GenerateNotificationsJob</class>
<jar>dhp-broker-events-${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=3840
</spark-opts>
<arg>--outputDir</arg><arg>${outputDir}</arg>
<arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg>
</spark>
<ok to="index_notifications"/> <ok to="index_notifications"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>

View File

@ -0,0 +1,14 @@
[
{
"paramName": "o",
"paramLongName": "outputDir",
"paramDescription": "the dir that contains the events folder",
"paramRequired": true
},
{
"paramName": "broker",
"paramLongName": "brokerApiBaseUrl",
"paramDescription": "the url of the broker service api",
"paramRequired": true
}
]

View File

@ -119,12 +119,36 @@
</configuration> </configuration>
</global> </global>
<start to="index_notifications"/> <start to="generate_notifications"/>
<kill name="Kill"> <kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message> <message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill> </kill>
<action name="generate_notifications">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>GenerateNotificationsJob</name>
<class>eu.dnetlib.dhp.broker.oa.GenerateNotificationsJob</class>
<jar>dhp-broker-events-${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=3840
</spark-opts>
<arg>--outputDir</arg><arg>${outputDir}</arg>
<arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg>
</spark>
<ok to="index_notifications"/>
<error to="Kill"/>
</action>
<action name="index_notifications"> <action name="index_notifications">
<spark xmlns="uri:oozie:spark-action:0.2"> <spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master> <master>yarn</master>
@ -135,6 +159,7 @@
<spark-opts> <spark-opts>
--executor-memory=${sparkExecutorMemory} --executor-memory=${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory} --driver-memory=${sparkDriverMemory}
--conf spark.dynamicAllocation.maxExecutors="8"
--conf spark.extraListeners=${spark2ExtraListeners} --conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}