forked from D-Net/dnet-hadoop
indexing only a subset of eventsa
This commit is contained in:
parent
55e24c2547
commit
6e60bf026a
|
@ -2,7 +2,6 @@
|
||||||
package eu.dnetlib.dhp.broker.model;
|
package eu.dnetlib.dhp.broker.model;
|
||||||
|
|
||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.util.Date;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ -19,16 +18,12 @@ public class EventFactory {
|
||||||
|
|
||||||
private final static String PRODUCER_ID = "OpenAIRE";
|
private final static String PRODUCER_ID = "OpenAIRE";
|
||||||
|
|
||||||
private static final int TTH_DAYS = 365;
|
|
||||||
|
|
||||||
private final static String[] DATE_PATTERNS = {
|
private final static String[] DATE_PATTERNS = {
|
||||||
"yyyy-MM-dd"
|
"yyyy-MM-dd"
|
||||||
};
|
};
|
||||||
|
|
||||||
public static Event newBrokerEvent(final UpdateInfo<?> updateInfo) {
|
public static Event newBrokerEvent(final UpdateInfo<?> updateInfo) {
|
||||||
|
|
||||||
final long now = new Date().getTime();
|
|
||||||
|
|
||||||
final Event res = new Event();
|
final Event res = new Event();
|
||||||
|
|
||||||
final MappedFields map = createMapFromResult(updateInfo);
|
final MappedFields map = createMapFromResult(updateInfo);
|
||||||
|
@ -44,8 +39,8 @@ public class EventFactory {
|
||||||
res.setPayload(updateInfo.asBrokerPayload().toJSON());
|
res.setPayload(updateInfo.asBrokerPayload().toJSON());
|
||||||
res.setMap(map);
|
res.setMap(map);
|
||||||
res.setTopic(updateInfo.getTopicPath());
|
res.setTopic(updateInfo.getTopicPath());
|
||||||
res.setCreationDate(now);
|
res.setCreationDate(0l);
|
||||||
res.setExpiryDate(calculateExpiryDate(now));
|
res.setExpiryDate(Long.MAX_VALUE);
|
||||||
res.setInstantMessage(false);
|
res.setInstantMessage(false);
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
|
@ -96,7 +91,9 @@ public class EventFactory {
|
||||||
return map;
|
return map;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static String calculateEventId(final String topic, final String dsId, final String publicationId,
|
private static String calculateEventId(final String topic,
|
||||||
|
final String dsId,
|
||||||
|
final String publicationId,
|
||||||
final String value) {
|
final String value) {
|
||||||
return "event-"
|
return "event-"
|
||||||
+ DigestUtils.md5Hex(topic).substring(0, 4) + "-"
|
+ DigestUtils.md5Hex(topic).substring(0, 4) + "-"
|
||||||
|
@ -105,10 +102,6 @@ public class EventFactory {
|
||||||
+ DigestUtils.md5Hex(value).substring(0, 5);
|
+ DigestUtils.md5Hex(value).substring(0, 5);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static long calculateExpiryDate(final long now) {
|
|
||||||
return now + TTH_DAYS * 24 * 60 * 60 * 1000;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static long parseDateTolong(final String date) {
|
private static long parseDateTolong(final String date) {
|
||||||
if (StringUtils.isBlank(date)) {
|
if (StringUtils.isBlank(date)) {
|
||||||
return -1;
|
return -1;
|
||||||
|
|
|
@ -28,7 +28,7 @@ public class GenerateStatsJob {
|
||||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
IOUtils
|
IOUtils
|
||||||
.toString(
|
.toString(
|
||||||
IndexOnESJob.class
|
GenerateStatsJob.class
|
||||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
|
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,100 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.broker.oa;
|
||||||
|
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.commons.lang3.math.NumberUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.apache.spark.sql.TypedColumn;
|
||||||
|
import org.apache.spark.util.LongAccumulator;
|
||||||
|
import org.elasticsearch.spark.rdd.api.java.JavaEsSpark;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.broker.model.Event;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.EventGroup;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.subset.EventSubsetAggregator;
|
||||||
|
|
||||||
|
public class IndexEventSubsetJob {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(IndexEventSubsetJob.class);
|
||||||
|
|
||||||
|
public static void main(final String[] args) throws Exception {
|
||||||
|
|
||||||
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
IndexEventSubsetJob.class
|
||||||
|
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/index_event_subset.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
final SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
|
final String eventsPath = parser.get("workingPath") + "/events";
|
||||||
|
log.info("eventsPath: {}", eventsPath);
|
||||||
|
|
||||||
|
final String index = parser.get("index");
|
||||||
|
log.info("index: {}", index);
|
||||||
|
|
||||||
|
final String indexHost = parser.get("esHost");
|
||||||
|
log.info("indexHost: {}", indexHost);
|
||||||
|
|
||||||
|
final int maxEventsForTopic = NumberUtils.toInt(parser.get("maxEventsForTopic"));
|
||||||
|
log.info("maxEventsForTopic: {}", maxEventsForTopic);
|
||||||
|
|
||||||
|
final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
|
||||||
|
|
||||||
|
final TypedColumn<Event, EventGroup> aggr = new EventSubsetAggregator(maxEventsForTopic).toColumn();
|
||||||
|
|
||||||
|
final LongAccumulator total = spark.sparkContext().longAccumulator("total_indexed");
|
||||||
|
|
||||||
|
final long now = new Date().getTime();
|
||||||
|
|
||||||
|
final Dataset<Event> subset = ClusterUtils
|
||||||
|
.readPath(spark, eventsPath, Event.class)
|
||||||
|
.groupByKey(e -> e.getTopic() + '@' + e.getMap().getTargetDatasourceId(), Encoders.STRING())
|
||||||
|
.agg(aggr)
|
||||||
|
.map(t -> t._2, Encoders.bean(EventGroup.class))
|
||||||
|
.flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class));
|
||||||
|
|
||||||
|
final JavaRDD<String> inputRdd = subset
|
||||||
|
.map(e -> prepareEventForIndexing(e, now, total), Encoders.STRING())
|
||||||
|
.javaRDD();
|
||||||
|
|
||||||
|
final Map<String, String> esCfg = new HashMap<>();
|
||||||
|
// esCfg.put("es.nodes", "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54");
|
||||||
|
|
||||||
|
esCfg.put("es.index.auto.create", "false");
|
||||||
|
esCfg.put("es.nodes", indexHost);
|
||||||
|
esCfg.put("es.mapping.id", "eventId"); // THE PRIMARY KEY
|
||||||
|
esCfg.put("es.batch.write.retry.count", "8");
|
||||||
|
esCfg.put("es.batch.write.retry.wait", "60s");
|
||||||
|
esCfg.put("es.batch.size.entries", "200");
|
||||||
|
esCfg.put("es.nodes.wan.only", "true");
|
||||||
|
|
||||||
|
JavaEsSpark.saveJsonToEs(inputRdd, index, esCfg);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String prepareEventForIndexing(final Event e, final long creationDate, final LongAccumulator acc)
|
||||||
|
throws JsonProcessingException {
|
||||||
|
acc.add(1);
|
||||||
|
|
||||||
|
e.setCreationDate(creationDate);
|
||||||
|
e.setExpiryDate(Long.MAX_VALUE);
|
||||||
|
|
||||||
|
return new ObjectMapper().writeValueAsString(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -20,6 +20,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.broker.model.Event;
|
import eu.dnetlib.dhp.broker.model.Event;
|
||||||
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
|
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
public class IndexOnESJob {
|
public class IndexOnESJob {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(IndexOnESJob.class);
|
private static final Logger log = LoggerFactory.getLogger(IndexOnESJob.class);
|
||||||
|
|
|
@ -0,0 +1,67 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.broker.oa.util.aggregators.subset;
|
||||||
|
|
||||||
|
import org.apache.spark.sql.Encoder;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.expressions.Aggregator;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.broker.model.Event;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.EventGroup;
|
||||||
|
|
||||||
|
public class EventSubsetAggregator extends Aggregator<Event, EventGroup, EventGroup> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
private static final long serialVersionUID = -678071078823059805L;
|
||||||
|
|
||||||
|
private final int maxEventsForTopic;
|
||||||
|
|
||||||
|
public EventSubsetAggregator(final int maxEventsForTopic) {
|
||||||
|
this.maxEventsForTopic = maxEventsForTopic;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public EventGroup zero() {
|
||||||
|
return new EventGroup();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public EventGroup reduce(final EventGroup g, final Event e) {
|
||||||
|
if (g.getData().size() < maxEventsForTopic) {
|
||||||
|
g.getData().add(e);
|
||||||
|
}
|
||||||
|
return g;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public EventGroup merge(final EventGroup g0, final EventGroup g1) {
|
||||||
|
final int missing = maxEventsForTopic - g0.getData().size();
|
||||||
|
|
||||||
|
if (missing > 0) {
|
||||||
|
if (g1.getData().size() < missing) {
|
||||||
|
g0.getData().addAll(g1.getData());
|
||||||
|
} else {
|
||||||
|
g0.getData().addAll(g1.getData().subList(0, missing));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return g0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public EventGroup finish(final EventGroup g) {
|
||||||
|
return g;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<EventGroup> outputEncoder() {
|
||||||
|
return Encoders.bean(EventGroup.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<EventGroup> bufferEncoder() {
|
||||||
|
return Encoders.bean(EventGroup.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -25,13 +25,17 @@
|
||||||
<description>a black list (comma separeted, - for empty list) of datasource ids</description>
|
<description>a black list (comma separeted, - for empty list) of datasource ids</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>esIndexName</name>
|
<name>esEventIndexName</name>
|
||||||
<description>the elasticsearch index name</description>
|
<description>the elasticsearch index name</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>esIndexHost</name>
|
<name>esIndexHost</name>
|
||||||
<description>the elasticsearch host</description>
|
<description>the elasticsearch host</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>maxIndexedEventsForDsAndTopic</name>
|
||||||
|
<description>the max number of events for each couple (ds/topic)</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
<description>memory for driver process</description>
|
<description>memory for driver process</description>
|
||||||
|
@ -423,16 +427,16 @@
|
||||||
<arg>--datasourceTypeWhitelist</arg><arg>${datasourceTypeWhitelist}</arg>
|
<arg>--datasourceTypeWhitelist</arg><arg>${datasourceTypeWhitelist}</arg>
|
||||||
<arg>--datasourceIdBlacklist</arg><arg>${datasourceIdBlacklist}</arg>
|
<arg>--datasourceIdBlacklist</arg><arg>${datasourceIdBlacklist}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="index_es"/>
|
<ok to="index_event_subset"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
<action name="index_es">
|
<action name="index_event_subset">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>IndexOnESJob</name>
|
<name>IndexEventSubsetOnESJob</name>
|
||||||
<class>eu.dnetlib.dhp.broker.oa.IndexOnESJob</class>
|
<class>eu.dnetlib.dhp.broker.oa.IndexEventSubsetJob</class>
|
||||||
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory=${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
@ -445,8 +449,9 @@
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--index</arg><arg>${esIndexName}</arg>
|
<arg>--index</arg><arg>${esEventIndexName}</arg>
|
||||||
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
||||||
|
<arg>--maxEventsForTopic</arg><arg>${maxIndexedEventsForDsAndTopic}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="stats"/>
|
<ok to="stats"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
|
|
@ -0,0 +1,26 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "o",
|
||||||
|
"paramLongName": "workingPath",
|
||||||
|
"paramDescription": "the workinh path",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "idx",
|
||||||
|
"paramLongName": "index",
|
||||||
|
"paramDescription": "the ES index",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "es",
|
||||||
|
"paramLongName": "esHost",
|
||||||
|
"paramDescription": "the ES host",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "n",
|
||||||
|
"paramLongName": "maxEventsForTopic",
|
||||||
|
"paramDescription": "the max number of events for each couple (ds/topic)",
|
||||||
|
"paramRequired": true
|
||||||
|
}
|
||||||
|
]
|
|
@ -8,6 +8,33 @@
|
||||||
<property>
|
<property>
|
||||||
<name>workingPath</name>
|
<name>workingPath</name>
|
||||||
<description>the path where the the generated data will be stored</description>
|
<description>the path where the the generated data will be stored</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>datasourceIdWhitelist</name>
|
||||||
|
<value>-</value>
|
||||||
|
<description>a white list (comma separeted, - for empty list) of datasource ids</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>datasourceTypeWhitelist</name>
|
||||||
|
<value>-</value>
|
||||||
|
<description>a white list (comma separeted, - for empty list) of datasource types</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>datasourceIdBlacklist</name>
|
||||||
|
<value>-</value>
|
||||||
|
<description>a black list (comma separeted, - for empty list) of datasource ids</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>esEventIndexName</name>
|
||||||
|
<description>the elasticsearch index name</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>esIndexHost</name>
|
||||||
|
<description>the elasticsearch host</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>maxIndexedEventsForDsAndTopic</name>
|
||||||
|
<description>the max number of events for each couple (ds/topic)</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
|
@ -64,18 +91,18 @@
|
||||||
</configuration>
|
</configuration>
|
||||||
</global>
|
</global>
|
||||||
|
|
||||||
<start to="index_es"/>
|
<start to="index_event_subset"/>
|
||||||
|
|
||||||
<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="index_es">
|
<action name="index_event_subset">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>IndexOnESJob</name>
|
<name>IndexEventSubsetOnESJob</name>
|
||||||
<class>eu.dnetlib.dhp.broker.oa.IndexOnESJob</class>
|
<class>eu.dnetlib.dhp.broker.oa.IndexEventSubsetJob</class>
|
||||||
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory=${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
@ -88,8 +115,9 @@
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--index</arg><arg>${esIndexName}</arg>
|
<arg>--index</arg><arg>${esEventIndexName}</arg>
|
||||||
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
||||||
|
<arg>--maxEventsForTopic</arg><arg>${maxIndexedEventsForDsAndTopic}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
|
Loading…
Reference in New Issue