forked from D-Net/dnet-hadoop
[broker] integrating PR#147, notification record creation phase separated from indexing on ES
This commit is contained in:
commit
172363e7f1
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -2,28 +2,22 @@
|
|||
package eu.dnetlib.dhp.broker.oa;
|
||||
|
||||
import java.io.IOException;
|
||||
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.HttpDelete;
|
||||
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.JavaRDD;
|
||||
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.elasticsearch.spark.rdd.api.java.JavaEsSpark;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -31,14 +25,8 @@ import com.fasterxml.jackson.core.JsonProcessingException;
|
|||
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 IndexNotificationsJob {
|
||||
|
||||
|
@ -48,15 +36,14 @@ public class IndexNotificationsJob {
|
|||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
IndexNotificationsJob.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/index_notifications.json")));
|
||||
.toString(IndexNotificationsJob.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/index_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 index = parser.get("index");
|
||||
log.info("index: {}", index);
|
||||
|
@ -83,131 +70,38 @@ public class IndexNotificationsJob {
|
|||
|
||||
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) {
|
||||
final Map<String, Map<String, List<ConditionParams>>> conditionsMap = prepareConditionsMap(subscriptions);
|
||||
esCfg.put("es.index.auto.create", "false");
|
||||
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);
|
||||
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);
|
||||
log.info("*** Deleting old notifications");
|
||||
final String message = deleteOldNotifications(brokerApiBaseUrl, date - 1000);
|
||||
log.info("*** Deleted notifications: {}", message);
|
||||
|
||||
notifications
|
||||
.map((MapFunction<Notification, String>) n -> prepareForIndexing(n, total), Encoders.STRING())
|
||||
.javaRDD()
|
||||
.saveAsTextFile("/tmp/IndexNotificationsJob_test_6504");
|
||||
}
|
||||
}
|
||||
log.info("*** sendNotifications (emails, ...)");
|
||||
sendNotifications(brokerApiBaseUrl, date - 1000);
|
||||
log.info("*** ALL done.");
|
||||
|
||||
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 {
|
||||
|
|
|
@ -115,6 +115,11 @@
|
|||
<name>spark2EventLogDir</name>
|
||||
<description>spark 2.* event log dir location</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkMaxExecutorsForIndexing</name>
|
||||
<value>8</value>
|
||||
<description>Max number of workers for ElasticSearch indexing</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<global>
|
||||
|
@ -498,7 +503,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.dynamicAllocation.maxExecutors="8"
|
||||
--conf spark.dynamicAllocation.maxExecutors=${sparkMaxExecutorsForIndexing}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -542,6 +547,30 @@
|
|||
<arg>--dbPassword</arg><arg>${brokerDbPassword}</arg>
|
||||
<arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg>
|
||||
</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"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
@ -556,7 +585,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.dynamicAllocation.maxExecutors="8"
|
||||
--conf spark.dynamicAllocation.maxExecutors=${sparkMaxExecutorsForIndexing}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
]
|
|
@ -98,6 +98,11 @@
|
|||
<name>spark2EventLogDir</name>
|
||||
<description>spark 2.* event log dir location</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkMaxExecutorsForIndexing</name>
|
||||
<value>8</value>
|
||||
<description>Max number of workers for ElasticSearch indexing</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<global>
|
||||
|
@ -119,12 +124,36 @@
|
|||
</configuration>
|
||||
</global>
|
||||
|
||||
<start to="index_notifications"/>
|
||||
<start to="generate_notifications"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</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">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
|
@ -135,6 +164,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.dynamicAllocation.maxExecutors=${sparkMaxExecutorsForIndexing}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
|
|
@ -75,6 +75,11 @@
|
|||
<name>spark2EventLogDir</name>
|
||||
<description>spark 2.* event log dir location</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkMaxExecutorsForIndexing</name>
|
||||
<value>8</value>
|
||||
<description>Max number of workers for ElasticSearch indexing</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<global>
|
||||
|
@ -112,7 +117,7 @@
|
|||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.dynamicAllocation.maxExecutors="8"
|
||||
--conf spark.dynamicAllocation.maxExecutors=${sparkMaxExecutorsForIndexing}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
|
|
@ -0,0 +1,133 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
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.Subscription;
|
||||
import eu.dnetlib.dhp.broker.oa.util.NotificationGroup;
|
||||
|
||||
class IndexNotificationsJobTest {
|
||||
|
||||
private List<Subscription> subscriptions;
|
||||
|
||||
private Map<String, Map<String, List<ConditionParams>>> conditionsMap;
|
||||
|
||||
private static final int N_TIMES = 1_000_000;
|
||||
|
||||
@BeforeEach
|
||||
void setUp() throws Exception {
|
||||
final Subscription s = new Subscription();
|
||||
s.setTopic("ENRICH/MISSING/PID");
|
||||
s
|
||||
.setConditions(
|
||||
"[{\"field\":\"targetDatasourceName\",\"fieldType\":\"STRING\",\"operator\":\"EXACT\",\"listParams\":[{\"value\":\"reposiTUm\"}]},{\"field\":\"trust\",\"fieldType\":\"FLOAT\",\"operator\":\"RANGE\",\"listParams\":[{\"value\":\"0\",\"otherValue\":\"1\"}]}]");
|
||||
subscriptions = Arrays.asList(s);
|
||||
conditionsMap = IndexNotificationsJob.prepareConditionsMap(subscriptions);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGenerateNotifications_invalid_topic() {
|
||||
final Event event = new Event();
|
||||
event.setTopic("ENRICH/MISSING/PROJECT");
|
||||
|
||||
final NotificationGroup res = IndexNotificationsJob
|
||||
.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
assertEquals(0, res.getData().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGenerateNotifications_topic_match() {
|
||||
final Event event = new Event();
|
||||
event.setTopic("ENRICH/MISSING/PID");
|
||||
event.setMap(new MappedFields());
|
||||
event.getMap().setTargetDatasourceName("reposiTUm");
|
||||
event.getMap().setTrust(0.8f);
|
||||
|
||||
final NotificationGroup res = IndexNotificationsJob
|
||||
.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
assertEquals(1, res.getData().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGenerateNotifications_topic_no_match() {
|
||||
final Event event = new Event();
|
||||
event.setTopic("ENRICH/MISSING/PID");
|
||||
event.setMap(new MappedFields());
|
||||
event.getMap().setTargetDatasourceName("Puma");
|
||||
event.getMap().setTrust(0.8f);
|
||||
|
||||
final NotificationGroup res = IndexNotificationsJob
|
||||
.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
assertEquals(0, res.getData().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGenerateNotifications_invalid_topic_repeated() {
|
||||
final Event event = new Event();
|
||||
event.setTopic("ENRICH/MISSING/PROJECT");
|
||||
|
||||
// warm up
|
||||
IndexNotificationsJob.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
|
||||
final long start = System.currentTimeMillis();
|
||||
for (int i = 0; i < N_TIMES; i++) {
|
||||
IndexNotificationsJob.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
}
|
||||
final long end = System.currentTimeMillis();
|
||||
System.out
|
||||
.println(String.format("no topic - repeated %s times - execution time: %s ms ", N_TIMES, end - start));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGenerateNotifications_topic_match_repeated() {
|
||||
final Event event = new Event();
|
||||
event.setTopic("ENRICH/MISSING/PID");
|
||||
event.setMap(new MappedFields());
|
||||
event.getMap().setTargetDatasourceName("reposiTUm");
|
||||
event.getMap().setTrust(0.8f);
|
||||
|
||||
// warm up
|
||||
IndexNotificationsJob.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
|
||||
final long start = System.currentTimeMillis();
|
||||
for (int i = 0; i < N_TIMES; i++) {
|
||||
IndexNotificationsJob.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
}
|
||||
final long end = System.currentTimeMillis();
|
||||
System.out
|
||||
.println(String.format("topic match - repeated %s times - execution time: %s ms ", N_TIMES, end - start));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGenerateNotifications_topic_no_match_repeated() {
|
||||
final Event event = new Event();
|
||||
event.setTopic("ENRICH/MISSING/PID");
|
||||
event.setMap(new MappedFields());
|
||||
event.getMap().setTargetDatasourceName("Puma");
|
||||
event.getMap().setTrust(0.8f);
|
||||
|
||||
// warm up
|
||||
IndexNotificationsJob.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
|
||||
final long start = System.currentTimeMillis();
|
||||
for (int i = 0; i < N_TIMES; i++) {
|
||||
IndexNotificationsJob.generateNotifications(event, subscriptions, conditionsMap, 0);
|
||||
}
|
||||
final long end = System.currentTimeMillis();
|
||||
System.out
|
||||
.println(
|
||||
String.format("topic no match - repeated %s times - execution time: %s ms ", N_TIMES, end - start));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,132 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa.samples;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonParseException;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.broker.model.ConditionParams;
|
||||
import eu.dnetlib.dhp.broker.model.MapCondition;
|
||||
import eu.dnetlib.dhp.broker.oa.util.SubscriptionUtils;
|
||||
|
||||
@Disabled
|
||||
public class SimpleVariableJobTest {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(SimpleVariableJobTest.class);
|
||||
|
||||
private static Path workingDir;
|
||||
|
||||
private static SparkSession spark;
|
||||
|
||||
private final static List<String> inputList = new ArrayList<>();
|
||||
|
||||
private static final Map<String, Map<String, List<ConditionParams>>> staticMap = new HashMap<>();
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
|
||||
workingDir = Files.createTempDirectory(SimpleVariableJobTest.class.getSimpleName());
|
||||
log.info("using work dir {}", workingDir);
|
||||
|
||||
final SparkConf conf = new SparkConf();
|
||||
conf.setAppName(SimpleVariableJobTest.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(SimpleVariableJobTest.class.getSimpleName())
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
|
||||
for (int i = 0; i < 1_000_000; i++) {
|
||||
inputList.add("record " + i);
|
||||
}
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void afterAll() throws IOException {
|
||||
FileUtils.deleteDirectory(workingDir.toFile());
|
||||
spark.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleVariableJob() throws Exception {
|
||||
final Map<String, Map<String, List<ConditionParams>>> map = fillMap();
|
||||
|
||||
final long n = spark
|
||||
.createDataset(inputList, Encoders.STRING())
|
||||
.filter(s -> filter(map.get(s)))
|
||||
.map((MapFunction<String, String>) s -> s.toLowerCase(), Encoders.STRING())
|
||||
.count();
|
||||
|
||||
System.out.println(n);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleVariableJob_static() throws Exception {
|
||||
|
||||
staticMap.putAll(fillMap());
|
||||
|
||||
final long n = spark
|
||||
.createDataset(inputList, Encoders.STRING())
|
||||
.filter(s -> filter(staticMap.get(s)))
|
||||
.map((MapFunction<String, String>) s -> s.toLowerCase(), Encoders.STRING())
|
||||
.count();
|
||||
|
||||
System.out.println(n);
|
||||
}
|
||||
|
||||
private static Map<String, Map<String, List<ConditionParams>>> fillMap()
|
||||
throws JsonParseException, JsonMappingException, IOException {
|
||||
final String s = "[{\"field\":\"targetDatasourceName\",\"fieldType\":\"STRING\",\"operator\":\"EXACT\",\"listParams\":[{\"value\":\"reposiTUm\"}]},{\"field\":\"trust\",\"fieldType\":\"FLOAT\",\"operator\":\"RANGE\",\"listParams\":[{\"value\":\"0\",\"otherValue\":\"1\"}]}]";
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final List<MapCondition> list = mapper
|
||||
.readValue(s, mapper.getTypeFactory().constructCollectionType(List.class, MapCondition.class));
|
||||
final Map<String, List<ConditionParams>> conditions = list
|
||||
.stream()
|
||||
.filter(mc -> !mc.getListParams().isEmpty())
|
||||
.collect(Collectors.toMap(MapCondition::getField, MapCondition::getListParams));
|
||||
|
||||
final Map<String, Map<String, List<ConditionParams>>> map = new HashMap<>();
|
||||
inputList.forEach(i -> map.put(i, conditions));
|
||||
return map;
|
||||
}
|
||||
|
||||
private static boolean filter(final Map<String, List<ConditionParams>> conditions) {
|
||||
if (conditions.containsKey("targetDatasourceName")
|
||||
&& !SubscriptionUtils
|
||||
.verifyExact("reposiTUm", conditions.get("targetDatasourceName").get(0).getValue())) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue