forked from antonis.lempesis/dnet-hadoop
new workflow and parameter for country propagation
This commit is contained in:
parent
5a476c7a13
commit
16193cf0ba
|
@ -1,4 +1,134 @@
|
|||
|
||||
package eu.dnetlib.dhp.countrypropagation;
|
||||
|
||||
import static eu.dnetlib.dhp.PropagationConstant.*;
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
|
||||
public class PrepareResultCountrySet {
|
||||
private static final Logger log = LoggerFactory.getLogger(PrepareResultCountrySet.class);
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
SparkCountryPropagationJob2.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json"));
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
String inputPath = parser.get("sourcePath");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
|
||||
final String datasourcecountrypath = parser.get("preparedInfoPath");
|
||||
log.info("preparedInfoPath: {}", datasourcecountrypath);
|
||||
|
||||
final String resultClassName = parser.get("resultTableName");
|
||||
log.info("resultTableName: {}", resultClassName);
|
||||
|
||||
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
|
||||
|
||||
runWithSparkHiveSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
getPotentialResultToUpdate(
|
||||
spark,
|
||||
inputPath,
|
||||
datasourcecountrypath,
|
||||
resultClazz);
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
private static <R extends Result> void getPotentialResultToUpdate(
|
||||
SparkSession spark,
|
||||
String inputPath,
|
||||
String datasourcecountrypath,
|
||||
Class<R> resultClazz) {
|
||||
|
||||
Dataset<R> result = readPathEntity(spark, inputPath, resultClazz);
|
||||
result.createOrReplaceTempView("result");
|
||||
// log.info("number of results: {}", result.count());
|
||||
createCfHbforresult(spark);
|
||||
Dataset<DatasourceCountry> datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath);
|
||||
countryPropagationAssoc(spark, datasourcecountryassoc)
|
||||
.map((MapFunction<ResultCountrySet, R>) value -> {
|
||||
R ret = resultClazz.newInstance();
|
||||
ret.setId(value.getResultId());
|
||||
ret
|
||||
.setCountry(
|
||||
value
|
||||
.getCountrySet()
|
||||
.stream()
|
||||
.map(c -> getCountry(c.getClassid(), c.getClassname()))
|
||||
.collect(Collectors.toList()));
|
||||
return ret;
|
||||
}, Encoders.bean(resultClazz))
|
||||
.write()
|
||||
.option("compression", "gzip")
|
||||
.mode(SaveMode.Append)
|
||||
.json(inputPath);
|
||||
}
|
||||
|
||||
private static Dataset<ResultCountrySet> countryPropagationAssoc(
|
||||
SparkSession spark,
|
||||
Dataset<DatasourceCountry> datasource_country) {
|
||||
|
||||
// Dataset<DatasourceCountry> datasource_country = broadcast_datasourcecountryassoc.value();
|
||||
datasource_country.createOrReplaceTempView("datasource_country");
|
||||
log.info("datasource_country number : {}", datasource_country.count());
|
||||
|
||||
String query = "SELECT id resultId, collect_set(country) countrySet "
|
||||
+ "FROM ( SELECT id, country "
|
||||
+ "FROM datasource_country "
|
||||
+ "JOIN cfhb "
|
||||
+ " ON cf = dataSourceId "
|
||||
+ "UNION ALL "
|
||||
+ "SELECT id , country "
|
||||
+ "FROM datasource_country "
|
||||
+ "JOIN cfhb "
|
||||
+ " ON hb = dataSourceId ) tmp "
|
||||
+ "GROUP BY id";
|
||||
Dataset<ResultCountrySet> potentialUpdates = spark
|
||||
.sql(query)
|
||||
.as(Encoders.bean(ResultCountrySet.class));
|
||||
// log.info("potential update number : {}", potentialUpdates.count());
|
||||
return potentialUpdates;
|
||||
}
|
||||
|
||||
private static Dataset<DatasourceCountry> readAssocDatasourceCountry(
|
||||
SparkSession spark, String relationPath) {
|
||||
return spark
|
||||
.read()
|
||||
.textFile(relationPath)
|
||||
.map(
|
||||
value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class),
|
||||
Encoders.bean(DatasourceCountry.class));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -9,7 +9,7 @@
|
|||
"paramName":"h",
|
||||
"paramLongName":"hive_metastore_uris",
|
||||
"paramDescription": "the hive metastore uris",
|
||||
"paramRequired": true
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName":"sg",
|
||||
|
@ -33,7 +33,7 @@
|
|||
"paramName": "p",
|
||||
"paramLongName": "preparedInfoPath",
|
||||
"paramDescription": "the path where prepared info have been stored",
|
||||
"paramRequired": true
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "ssm",
|
||||
|
|
|
@ -27,6 +27,10 @@
|
|||
<action name="reset-outputpath">
|
||||
<fs>
|
||||
<delete path='${workingDir}/preparedInfo'/>
|
||||
<delete path='${workingDir}/publication'/>
|
||||
<delete path='${workingDir}/dataset'/>
|
||||
<delete path='${workingDir}/otherresearchproduct'/>
|
||||
<delete path='${workingDir}/software'/>
|
||||
<delete path='${outputPath}/relation'/>
|
||||
<delete path='${outputPath}/dataset'/>
|
||||
<delete path='${outputPath}/software'/>
|
||||
|
@ -45,6 +49,10 @@
|
|||
<path start="copy_organization"/>
|
||||
<path start="copy_projects"/>
|
||||
<path start="copy_datasources"/>
|
||||
<path start="copy_publication"/>
|
||||
<path start="copy_dataset"/>
|
||||
<path start="copy_orp"/>
|
||||
<path start="copy_software"/>
|
||||
</fork>
|
||||
<action name="copy_relation">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
|
@ -89,6 +97,50 @@
|
|||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="copy_publication">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/${sourcePath}/publication</arg>
|
||||
<arg>${nameNode}/${workingDir}/publication</arg>
|
||||
</distcp>
|
||||
<ok to="copy_wait"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="copy_dataset">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/${sourcePath}/dataset</arg>
|
||||
<arg>${nameNode}/${workingDir}/dataset</arg>
|
||||
</distcp>
|
||||
<ok to="copy_wait"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="copy_orp">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/${sourcePath}/otherresearchproduct</arg>
|
||||
<arg>${nameNode}/${workingDir}/otherresearchproduct</arg>
|
||||
</distcp>
|
||||
<ok to="copy_wait"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="copy_software">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<arg>${nameNode}/${sourcePath}/software</arg>
|
||||
<arg>${nameNode}/${workingDir}/software</arg>
|
||||
</distcp>
|
||||
<ok to="copy_wait"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<join name="copy_wait" to="prepare_datasource_country_association"/>
|
||||
|
||||
<action name="prepare_datasource_country_association">
|
||||
|
@ -113,9 +165,139 @@
|
|||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="fork_join_apply_country_propagation"/>
|
||||
<ok to="fork_join_prepare_result_country"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<fork name="fork_join_prepare_result_country">
|
||||
<path start="join_prepareresult_publication"/>
|
||||
<path start="join_prepareresult_dataset"/>
|
||||
<path start="join_prepareresult_otherresearchproduct"/>
|
||||
<path start="join_prepareresult_software"/>
|
||||
</fork>
|
||||
|
||||
<action name="join_prepareresult_publication">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>prepareResultCountry-Publication</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet</class>
|
||||
<jar>dhp-propagation-${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.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/publication</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait_prepare"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="join_prepareresult_dataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>prepareResultCountry-Dataset</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet</class>
|
||||
<jar>dhp-propagation-${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.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/dataset</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait_prepare"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="join_prepareresult_otherresearchproduct">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>prepareResultCountry-ORP</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet</class>
|
||||
<jar>dhp-propagation-${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.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/otherresearchproduct</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait_prepare"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="join_prepareresult_software">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>prepareResultCountry-Software</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet</class>
|
||||
<jar>dhp-propagation-${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.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/software</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait_prepare"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<join name="wait_prepare" to="fork_join_apply_country_propagation"/>
|
||||
|
||||
<fork name="fork_join_apply_country_propagation">
|
||||
<path start="join_propagation_publication"/>
|
||||
<path start="join_propagation_dataset"/>
|
||||
|
@ -128,26 +310,27 @@
|
|||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>countryPropagationForPublications</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2</class>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3</class>
|
||||
<jar>dhp-propagation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
<!-- --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.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
<!-- --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}-->
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}/publication</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/publication</arg>
|
||||
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}/publication</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -158,7 +341,7 @@
|
|||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>countryPropagationForDataset</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2</class>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3</class>
|
||||
<jar>dhp-propagation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
|
@ -171,13 +354,13 @@
|
|||
--conf spark.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}/dataset</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/dataset</arg>
|
||||
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}/dataset</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -188,7 +371,7 @@
|
|||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>countryPropagationForORP</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2</class>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3</class>
|
||||
<jar>dhp-propagation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
|
@ -201,13 +384,13 @@
|
|||
--conf spark.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}/otherresearchproduct</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/otherresearchproduct</arg>
|
||||
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}/otherresearchproduct</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -218,7 +401,7 @@
|
|||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>countryPropagationForSoftware</name>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2</class>
|
||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3</class>
|
||||
<jar>dhp-propagation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
|
@ -231,13 +414,13 @@
|
|||
--conf spark.dynamicAllocation.enabled=true
|
||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
||||
--conf spark.speculation=false
|
||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}/software</arg>
|
||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/software</arg>
|
||||
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
||||
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}/software</arg>
|
||||
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
|
||||
</spark>
|
||||
<ok to="wait"/>
|
||||
<error to="Kill"/>
|
||||
|
|
Loading…
Reference in New Issue