merge branch with master
This commit is contained in:
commit
0d83f47166
|
@ -3,11 +3,16 @@ package eu.dnetlib.dhp.broker.oa;
|
|||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
||||
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.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.TypedColumn;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -29,7 +34,7 @@ public class GenerateStatsJob {
|
|||
IOUtils
|
||||
.toString(
|
||||
GenerateStatsJob.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/stats_params.json")));
|
||||
parser.parseArgument(args);
|
||||
|
||||
final Boolean isSparkSessionManaged = Optional
|
||||
|
@ -43,21 +48,50 @@ public class GenerateStatsJob {
|
|||
final String eventsPath = parser.get("workingPath") + "/events";
|
||||
log.info("eventsPath: {}", eventsPath);
|
||||
|
||||
final String statsPath = parser.get("workingPath") + "/stats";
|
||||
log.info("stats: {}", statsPath);
|
||||
final String dbUrl = parser.get("dbUrl");
|
||||
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 Properties connectionProperties = new Properties();
|
||||
connectionProperties.put("user", dbUser);
|
||||
connectionProperties.put("password", dbPassword);
|
||||
|
||||
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
||||
|
||||
final Dataset<DatasourceStats> stats = ClusterUtils
|
||||
ClusterUtils
|
||||
.readPath(spark, eventsPath, Event.class)
|
||||
.groupByKey(e -> e.getMap().getTargetDatasourceId(), Encoders.STRING())
|
||||
.groupByKey(e -> e.getTopic() + "@@@" + e.getMap().getTargetDatasourceId(), Encoders.STRING())
|
||||
.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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -47,8 +47,9 @@ 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();
|
||||
|
@ -116,7 +117,8 @@ public class IndexNotificationsJob {
|
|||
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 -> StringUtils.isBlank(s.getTopic()) || s.getTopic().equals("*") || s.getTopic().equals(e.getTopic()))
|
||||
.filter(s -> verifyConditions(e.getMap(), s.conditionsAsMap()))
|
||||
.map(s -> generateNotification(s, e, date))
|
||||
.collect(Collectors.toList());
|
||||
|
@ -147,15 +149,18 @@ public class IndexNotificationsJob {
|
|||
|
||||
if (conditions.containsKey("trust")
|
||||
&& !SubscriptionUtils
|
||||
.verifyFloatRange(map.getTrust(), conditions.get("trust").get(0).getValue(), conditions.get("trust").get(0).getOtherValue())) {
|
||||
.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()))) {
|
||||
.anyMatch(
|
||||
c -> SubscriptionUtils
|
||||
.verifyDateRange(map.getTargetDateofacceptance(), c.getValue(), c.getOtherValue()))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,113 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.slf4j.Logger;
|
||||
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.oa.util.ClusterUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class PartitionEventsByDsIdJob {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(PartitionEventsByDsIdJob.class);
|
||||
private static final String OPENDOAR_NSPREFIX = "opendoar____::";
|
||||
|
||||
public static void main(final String[] args) throws Exception {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
PartitionEventsByDsIdJob.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
|
||||
parser.parseArgument(args);
|
||||
|
||||
final Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final SparkConf conf = new SparkConf();
|
||||
|
||||
final String eventsPath = parser.get("workingPath") + "/events";
|
||||
log.info("eventsPath: {}", eventsPath);
|
||||
|
||||
final String partitionPath = parser.get("workingPath") + "/eventsByOpendoarId";
|
||||
log.info("partitionPath: {}", partitionPath);
|
||||
|
||||
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
||||
|
||||
ClusterUtils
|
||||
.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)))
|
||||
.write()
|
||||
.partitionBy("_1")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.json(partitionPath);
|
||||
|
||||
});
|
||||
renameSubDirs(partitionPath);
|
||||
|
||||
}
|
||||
|
||||
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);
|
||||
for (final FileStatus fileStatus : fs.listStatus(new Path(path))) {
|
||||
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));
|
||||
log.info(" * " + oldPath.getName() + " -> " + newPath.getName());
|
||||
fs.rename(oldPath, newPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static ShortEventMessage messageFromNotification(final Event e) {
|
||||
final Gson gson = new Gson();
|
||||
|
||||
final OaBrokerEventPayload payload = gson.fromJson(e.getPayload(), OaBrokerEventPayload.class);
|
||||
|
||||
final ShortEventMessage res = new ShortEventMessage();
|
||||
|
||||
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());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
|
@ -2,8 +2,6 @@
|
|||
package eu.dnetlib.dhp.broker.oa.util.aggregators.stats;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class DatasourceStats implements Serializable {
|
||||
|
||||
|
@ -15,7 +13,8 @@ public class DatasourceStats implements Serializable {
|
|||
private String id;
|
||||
private String name;
|
||||
private String type;
|
||||
private Map<String, Long> topics = new HashMap<>();
|
||||
private String topic;
|
||||
private long size = 0l;
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
|
@ -41,21 +40,24 @@ public class DatasourceStats implements Serializable {
|
|||
this.type = type;
|
||||
}
|
||||
|
||||
public Map<String, Long> getTopics() {
|
||||
return topics;
|
||||
public String getTopic() {
|
||||
return topic;
|
||||
}
|
||||
|
||||
public void setTopics(final Map<String, Long> topics) {
|
||||
this.topics = topics;
|
||||
public void setTopic(final String topic) {
|
||||
this.topic = topic;
|
||||
}
|
||||
|
||||
public void incrementTopic(final String topic, final long inc) {
|
||||
if (topics.containsKey(topic)) {
|
||||
topics.put(topic, topics.get(topic) + inc);
|
||||
} else {
|
||||
topics.put(topic, inc);
|
||||
}
|
||||
public long getSize() {
|
||||
return size;
|
||||
}
|
||||
|
||||
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.setName(e.getMap().getTargetDatasourceName());
|
||||
stats.setType(e.getMap().getTargetDatasourceType());
|
||||
stats.incrementTopic(e.getTopic(), 1l);
|
||||
stats.setTopic(e.getTopic());
|
||||
stats.incrementSize(1l);
|
||||
return stats;
|
||||
}
|
||||
|
||||
|
@ -35,8 +36,9 @@ public class StatsAggregator extends Aggregator<Event, DatasourceStats, Datasour
|
|||
stats0.setId(stats1.getId());
|
||||
stats0.setName(stats1.getName());
|
||||
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;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,18 @@
|
|||
<configuration>
|
||||
<property>
|
||||
<name>jobTracker</name>
|
||||
<value>yarnRM</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>nameNode</name>
|
||||
<value>hdfs://nameservice1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.use.system.libpath</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>spark2</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,137 @@
|
|||
<workflow-app name="create broker events - partial" xmlns="uri:oozie:workflow:0.5">
|
||||
|
||||
<parameters>
|
||||
<property>
|
||||
<name>graphInputPath</name>
|
||||
<description>the path where the graph is stored</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>workingPath</name>
|
||||
<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 for events</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>esNotificationsIndexName</name>
|
||||
<description>the elasticsearch index name for notifications</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>
|
||||
<name>brokerApiBaseUrl</name>
|
||||
<description>the url of the broker service api</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozieActionShareLibForSpark2</name>
|
||||
<description>oozie action sharelib for spark 2.*</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2ExtraListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
|
||||
<description>spark 2.* extra listeners classname</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2SqlQueryExecutionListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
|
||||
<description>spark 2.* sql query execution listeners classname</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2YarnHistoryServerAddress</name>
|
||||
<description>spark 2.* yarn history server address</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2EventLogDir</name>
|
||||
<description>spark 2.* event log dir location</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<global>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>mapreduce.job.queuename</name>
|
||||
<value>${queueName}</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapred.job.queue.name</name>
|
||||
<value>${oozieLauncherQueueName}</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>${oozieActionShareLibForSpark2}</value>
|
||||
</property>
|
||||
</configuration>
|
||||
</global>
|
||||
|
||||
<start to="index_notifications"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="index_notifications">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>IndexNotificationsOnESJob</name>
|
||||
<class>eu.dnetlib.dhp.broker.oa.IndexNotificationsJob</class>
|
||||
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.dynamicAllocation.maxExecutors="8"
|
||||
--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>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--index</arg><arg>${esNotificationsIndexName}</arg>
|
||||
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
||||
<arg>--brokerApiBaseUrl</arg><arg>${brokerApiBaseUrl}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<end name="End"/>
|
||||
|
||||
</workflow-app>
|
|
@ -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>
|
||||
|
@ -99,23 +111,23 @@
|
|||
</configuration>
|
||||
</global>
|
||||
|
||||
<start to="index_notifications"/>
|
||||
<start to="stats"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="index_notifications">
|
||||
<action name="stats">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>IndexNotificationsOnESJob</name>
|
||||
<class>eu.dnetlib.dhp.broker.oa.IndexNotificationsJob</class>
|
||||
<name>GenerateStatsJob</name>
|
||||
<class>eu.dnetlib.dhp.broker.oa.GenerateStatsJob</class>
|
||||
<jar>dhp-broker-events-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.dynamicAllocation.maxExecutors="8"
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -123,14 +135,14 @@
|
|||
--conf spark.sql.shuffle.partitions=3840
|
||||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--index</arg><arg>${esNotificationsIndexName}</arg>
|
||||
<arg>--esHost</arg><arg>${esIndexHost}</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"/>
|
||||
</action>
|
||||
|
||||
|
||||
<end name="End"/>
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
]
|
2
pom.xml
2
pom.xml
|
@ -663,7 +663,7 @@
|
|||
<mockito-core.version>3.3.3</mockito-core.version>
|
||||
<mongodb.driver.version>3.4.2</mongodb.driver.version>
|
||||
<vtd.version>[2.12,3.0)</vtd.version>
|
||||
<dnet.openaire.broker.common>3.1.0</dnet.openaire.broker.common>
|
||||
<dnet.openaire.broker.common>3.1.1</dnet.openaire.broker.common>
|
||||
<solr.version>7.5.0</solr.version>
|
||||
<okhttp.version>4.7.2</okhttp.version>
|
||||
<common.compress.version>1.1</common.compress.version>
|
||||
|
|
Loading…
Reference in New Issue