forked from D-Net/dnet-hadoop
stats to sql file
This commit is contained in:
parent
19eb8f9dcc
commit
9e681609fd
|
@ -3,11 +3,16 @@ package eu.dnetlib.dhp.broker.oa;
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
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.SparkConf;
|
||||||
import org.apache.spark.sql.Dataset;
|
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.TypedColumn;
|
import org.apache.spark.sql.TypedColumn;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -29,7 +34,7 @@ public class GenerateStatsJob {
|
||||||
IOUtils
|
IOUtils
|
||||||
.toString(
|
.toString(
|
||||||
GenerateStatsJob.class
|
GenerateStatsJob.class
|
||||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
|
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/stats_params.json")));
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
final Boolean isSparkSessionManaged = Optional
|
final Boolean isSparkSessionManaged = Optional
|
||||||
|
@ -43,21 +48,50 @@ public class GenerateStatsJob {
|
||||||
final String eventsPath = parser.get("workingPath") + "/events";
|
final String eventsPath = parser.get("workingPath") + "/events";
|
||||||
log.info("eventsPath: {}", eventsPath);
|
log.info("eventsPath: {}", eventsPath);
|
||||||
|
|
||||||
final String statsPath = parser.get("workingPath") + "/stats";
|
final String dbUrl = parser.get("dbUrl");
|
||||||
log.info("stats: {}", statsPath);
|
log.info("dbUrl: {}", dbUrl);
|
||||||
|
|
||||||
|
final String dbUser = parser.get("dbUser");
|
||||||
|
log.info("dbUser: {}", dbUser);
|
||||||
|
|
||||||
|
final String dbPassword = parser.get("dbPassword");
|
||||||
|
log.info("dbPassword: {}", "***");
|
||||||
|
|
||||||
|
final String brokerApiBaseUrl = parser.get("brokerApiBaseUrl");
|
||||||
|
log.info("brokerApiBaseUrl: {}", brokerApiBaseUrl);
|
||||||
|
|
||||||
final TypedColumn<Event, DatasourceStats> aggr = new StatsAggregator().toColumn();
|
final TypedColumn<Event, DatasourceStats> aggr = new StatsAggregator().toColumn();
|
||||||
|
|
||||||
|
final Properties connectionProperties = new Properties();
|
||||||
|
connectionProperties.put("user", dbUser);
|
||||||
|
connectionProperties.put("password", dbPassword);
|
||||||
|
|
||||||
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
||||||
|
|
||||||
final Dataset<DatasourceStats> stats = ClusterUtils
|
ClusterUtils
|
||||||
.readPath(spark, eventsPath, Event.class)
|
.readPath(spark, eventsPath, Event.class)
|
||||||
.groupByKey(e -> e.getMap().getTargetDatasourceId(), Encoders.STRING())
|
.groupByKey(e -> e.getTopic() + "@@@" + e.getMap().getTargetDatasourceId(), Encoders.STRING())
|
||||||
.agg(aggr)
|
.agg(aggr)
|
||||||
.map(t -> t._2, Encoders.bean(DatasourceStats.class));
|
.map(t -> t._2, Encoders.bean(DatasourceStats.class))
|
||||||
|
.write()
|
||||||
|
.jdbc(dbUrl, "oa_datasource_stats_temp", connectionProperties);
|
||||||
|
|
||||||
|
log.info("*** updateStats");
|
||||||
|
updateStats(brokerApiBaseUrl);
|
||||||
|
log.info("*** ALL done.");
|
||||||
|
|
||||||
ClusterUtils.save(stats, statsPath, DatasourceStats.class, null);
|
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static String updateStats(final String brokerApiBaseUrl) throws IOException {
|
||||||
|
final String url = brokerApiBaseUrl + "/api/openaireBroker/stats/update";
|
||||||
|
final HttpGet req = new HttpGet(url);
|
||||||
|
|
||||||
|
try (final CloseableHttpClient client = HttpClients.createDefault()) {
|
||||||
|
try (final CloseableHttpResponse response = client.execute(req)) {
|
||||||
|
return IOUtils.toString(response.getEntity().getContent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -2,8 +2,6 @@
|
||||||
package eu.dnetlib.dhp.broker.oa.util.aggregators.stats;
|
package eu.dnetlib.dhp.broker.oa.util.aggregators.stats;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
public class DatasourceStats implements Serializable {
|
public class DatasourceStats implements Serializable {
|
||||||
|
|
||||||
|
@ -15,7 +13,8 @@ public class DatasourceStats implements Serializable {
|
||||||
private String id;
|
private String id;
|
||||||
private String name;
|
private String name;
|
||||||
private String type;
|
private String type;
|
||||||
private Map<String, Long> topics = new HashMap<>();
|
private String topic;
|
||||||
|
private long size = 0l;
|
||||||
|
|
||||||
public String getId() {
|
public String getId() {
|
||||||
return id;
|
return id;
|
||||||
|
@ -41,21 +40,24 @@ public class DatasourceStats implements Serializable {
|
||||||
this.type = type;
|
this.type = type;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<String, Long> getTopics() {
|
public String getTopic() {
|
||||||
return topics;
|
return topic;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setTopics(final Map<String, Long> topics) {
|
public void setTopic(final String topic) {
|
||||||
this.topics = topics;
|
this.topic = topic;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void incrementTopic(final String topic, final long inc) {
|
public long getSize() {
|
||||||
if (topics.containsKey(topic)) {
|
return size;
|
||||||
topics.put(topic, topics.get(topic) + inc);
|
|
||||||
} else {
|
|
||||||
topics.put(topic, inc);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void setSize(final long size) {
|
||||||
|
this.size = size;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void incrementSize(final long inc) {
|
||||||
|
this.size = this.size + inc;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,8 @@ public class StatsAggregator extends Aggregator<Event, DatasourceStats, Datasour
|
||||||
stats.setId(e.getMap().getTargetDatasourceId());
|
stats.setId(e.getMap().getTargetDatasourceId());
|
||||||
stats.setName(e.getMap().getTargetDatasourceName());
|
stats.setName(e.getMap().getTargetDatasourceName());
|
||||||
stats.setType(e.getMap().getTargetDatasourceType());
|
stats.setType(e.getMap().getTargetDatasourceType());
|
||||||
stats.incrementTopic(e.getTopic(), 1l);
|
stats.setTopic(e.getTopic());
|
||||||
|
stats.incrementSize(1l);
|
||||||
return stats;
|
return stats;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -35,8 +36,9 @@ public class StatsAggregator extends Aggregator<Event, DatasourceStats, Datasour
|
||||||
stats0.setId(stats1.getId());
|
stats0.setId(stats1.getId());
|
||||||
stats0.setName(stats1.getName());
|
stats0.setName(stats1.getName());
|
||||||
stats0.setType(stats1.getType());
|
stats0.setType(stats1.getType());
|
||||||
|
stats0.setTopic(stats1.getTopic());
|
||||||
}
|
}
|
||||||
stats1.getTopics().entrySet().forEach(e -> stats0.incrementTopic(e.getKey(), e.getValue()));
|
stats0.incrementSize(stats1.getSize());
|
||||||
return stats0;
|
return stats0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -44,6 +44,18 @@
|
||||||
<name>brokerApiBaseUrl</name>
|
<name>brokerApiBaseUrl</name>
|
||||||
<description>the url of the broker service api</description>
|
<description>the url of the broker service api</description>
|
||||||
</property>
|
</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>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
<description>memory for driver process</description>
|
<description>memory for driver process</description>
|
||||||
|
@ -99,18 +111,18 @@
|
||||||
</configuration>
|
</configuration>
|
||||||
</global>
|
</global>
|
||||||
|
|
||||||
<start to="partitionByDsId"/>
|
<start to="stats"/>
|
||||||
|
|
||||||
<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="partitionByDsId">
|
<action name="stats">
|
||||||
<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>PartitionEventsByDsIdJob</name>
|
<name>GenerateStatsJob</name>
|
||||||
<class>eu.dnetlib.dhp.broker.oa.PartitionEventsByDsIdJob</class>
|
<class>eu.dnetlib.dhp.broker.oa.GenerateStatsJob</class>
|
||||||
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-cores=${sparkExecutorCores}
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
@ -122,8 +134,11 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
|
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</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>
|
</spark>
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
|
|
@ -0,0 +1,32 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "wp",
|
||||||
|
"paramLongName": "workingPath",
|
||||||
|
"paramDescription": "the working path",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dburl",
|
||||||
|
"paramLongName": "dbUrl",
|
||||||
|
"paramDescription": "the broker database url",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "u",
|
||||||
|
"paramLongName": "dbUser",
|
||||||
|
"paramDescription": "the broker database user",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "p",
|
||||||
|
"paramLongName": "dbPassword",
|
||||||
|
"paramDescription": "the broker database password",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "broker",
|
||||||
|
"paramLongName": "brokerApiBaseUrl",
|
||||||
|
"paramDescription": "the url of the broker service api",
|
||||||
|
"paramRequired": true
|
||||||
|
}
|
||||||
|
]
|
Loading…
Reference in New Issue