forked from antonis.lempesis/dnet-hadoop
Merge branch 'master' of https://code-repo.d4science.org/miriam.baglioni/dnet-hadoop
This commit is contained in:
commit
182225becb
|
@ -86,15 +86,4 @@ public class Author implements Serializable {
|
||||||
return Objects.hash(fullname, name, surname, rank, pid, affiliation);
|
return Objects.hash(fullname, name, surname, rank, pid, affiliation);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addPid(StructuredProperty pid) {
|
|
||||||
|
|
||||||
if (pid == null)
|
|
||||||
return;
|
|
||||||
|
|
||||||
if (this.pid == null) {
|
|
||||||
this.pid = Arrays.asList(pid);
|
|
||||||
} else {
|
|
||||||
this.pid.add(pid);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -4,10 +4,8 @@ package eu.dnetlib.dhp.countrypropagation;
|
||||||
import static eu.dnetlib.dhp.PropagationConstant.*;
|
import static eu.dnetlib.dhp.PropagationConstant.*;
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.*;
|
||||||
import java.util.HashSet;
|
import java.util.function.Function;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
@ -17,6 +15,7 @@ import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.api.java.function.MapGroupsFunction;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SaveMode;
|
import org.apache.spark.sql.SaveMode;
|
||||||
|
@ -27,6 +26,7 @@ import org.slf4j.LoggerFactory;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Country;
|
import eu.dnetlib.dhp.schema.oaf.Country;
|
||||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||||
|
@ -71,6 +71,7 @@ public class SparkCountryPropagationJob3 {
|
||||||
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
|
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||||
|
|
||||||
runWithSparkSession(
|
runWithSparkSession(
|
||||||
conf,
|
conf,
|
||||||
|
@ -96,50 +97,51 @@ public class SparkCountryPropagationJob3 {
|
||||||
if (saveGraph) {
|
if (saveGraph) {
|
||||||
// updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath);
|
// updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath);
|
||||||
log.info("Reading Graph table from: {}", inputPath);
|
log.info("Reading Graph table from: {}", inputPath);
|
||||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
|
||||||
JavaPairRDD<String, R> results = sc
|
|
||||||
.textFile(inputPath)
|
|
||||||
.map(r -> OBJECT_MAPPER.readValue(r, resultClazz))
|
|
||||||
.mapToPair(r -> new Tuple2<>(r.getId(), r));
|
|
||||||
|
|
||||||
JavaPairRDD<String, R> tmp = results.reduceByKey((r1, r2) -> {
|
spark
|
||||||
if (r1 == null) {
|
.read()
|
||||||
return r2;
|
.json(inputPath)
|
||||||
}
|
.as(Encoders.kryo(resultClazz))
|
||||||
if (r2 == null) {
|
.groupByKey((MapFunction<R, String>) result1 -> result1.getId(), Encoders.STRING())
|
||||||
return r1;
|
.mapGroups(getCountryMergeFn(resultClazz), Encoders.bean(resultClazz))
|
||||||
}
|
.write()
|
||||||
if (Optional.ofNullable(r1.getCollectedfrom()).isPresent()) {
|
.option("compression", "gzip")
|
||||||
r1.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry()));
|
.mode(SaveMode.Overwrite)
|
||||||
return r1;
|
.json(outputPath);
|
||||||
}
|
|
||||||
if (Optional.ofNullable(r2.getCollectedfrom()).isPresent()) {
|
|
||||||
r2.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry()));
|
|
||||||
return r2;
|
|
||||||
}
|
|
||||||
r1.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry()));
|
|
||||||
return r1;
|
|
||||||
});
|
|
||||||
|
|
||||||
tmp
|
|
||||||
.map(c -> c._2())
|
|
||||||
.map(r -> OBJECT_MAPPER.writeValueAsString(r))
|
|
||||||
.saveAsTextFile(outputPath, GzipCodec.class);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static List<Country> getUnionCountries(List<Country> country, List<Country> country1) {
|
private static <R extends Result> MapGroupsFunction<String, R, R> getCountryMergeFn(Class<R> resultClazz) {
|
||||||
HashSet<String> countries = country
|
return (MapGroupsFunction<String, R, R>) (key, values) -> {
|
||||||
.stream()
|
R res = resultClazz.newInstance();
|
||||||
.map(c -> c.getClassid())
|
List<Country> countries = new ArrayList<>();
|
||||||
.collect(Collectors.toCollection(HashSet::new));
|
values.forEachRemaining(r -> {
|
||||||
country
|
res.mergeFrom(r);
|
||||||
.addAll(
|
countries.addAll(r.getCountry());
|
||||||
country1
|
});
|
||||||
.stream()
|
res
|
||||||
.filter(c -> !(countries.contains(c.getClassid())))
|
.setCountry(
|
||||||
.collect(Collectors.toList()));
|
countries
|
||||||
return country;
|
.stream()
|
||||||
|
.collect(
|
||||||
|
Collectors
|
||||||
|
.toMap(
|
||||||
|
Country::getClassid,
|
||||||
|
Function.identity(),
|
||||||
|
(c1, c2) -> {
|
||||||
|
if (Optional
|
||||||
|
.ofNullable(
|
||||||
|
c1.getDataInfo().getInferenceprovenance())
|
||||||
|
.isPresent()) {
|
||||||
|
return c2;
|
||||||
|
}
|
||||||
|
return c1;
|
||||||
|
}))
|
||||||
|
.values()
|
||||||
|
.stream()
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
return res;
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,6 +6,7 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
@ -18,6 +19,7 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||||
|
@ -181,7 +183,14 @@ public class SparkOrcidToResultFromSemRelJob3 {
|
||||||
PROPAGATION_DATA_INFO_TYPE,
|
PROPAGATION_DATA_INFO_TYPE,
|
||||||
PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID,
|
PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID,
|
||||||
PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME));
|
PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME));
|
||||||
author.addPid(p);
|
|
||||||
|
Optional<List<StructuredProperty>> authorPid = Optional.ofNullable(author.getPid());
|
||||||
|
if (authorPid.isPresent()) {
|
||||||
|
authorPid.get().add(p);
|
||||||
|
} else {
|
||||||
|
author.setPid(Lists.newArrayList(p));
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
return toaddpid;
|
return toaddpid;
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@
|
||||||
<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="reset-outputpath">
|
<action name="reset-outputpath">
|
||||||
<fs>
|
<fs>
|
||||||
<delete path='${workingDir}/preparedInfo'/>
|
<delete path='${workingDir}/preparedInfo'/>
|
||||||
|
@ -158,6 +159,7 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
--conf spark.sql.shuffle.partitions=300
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||||
<arg>--whitelist</arg><arg>${whitelist}</arg>
|
<arg>--whitelist</arg><arg>${whitelist}</arg>
|
||||||
|
@ -191,11 +193,10 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/publication</arg>
|
<arg>--sourcePath</arg><arg>${workingDir}/publication</arg>
|
||||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||||
|
@ -221,11 +222,10 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/dataset</arg>
|
<arg>--sourcePath</arg><arg>${workingDir}/dataset</arg>
|
||||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||||
|
@ -251,11 +251,10 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/otherresearchproduct</arg>
|
<arg>--sourcePath</arg><arg>${workingDir}/otherresearchproduct</arg>
|
||||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||||
|
@ -281,11 +280,10 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/software</arg>
|
<arg>--sourcePath</arg><arg>${workingDir}/software</arg>
|
||||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||||
|
@ -313,15 +311,13 @@
|
||||||
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3</class>
|
<class>eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3</class>
|
||||||
<jar>dhp-propagation-${projectVersion}.jar</jar>
|
<jar>dhp-propagation-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
<!-- --executor-cores=${sparkExecutorCores}-->
|
--executor-cores=${sparkExecutorCores}
|
||||||
<!-- --executor-memory=${sparkExecutorMemory}-->
|
--executor-memory=${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
<!-- --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}-->
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
@ -351,11 +347,10 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/dataset</arg>
|
<arg>--sourcePath</arg><arg>${workingDir}/dataset</arg>
|
||||||
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
||||||
|
@ -381,11 +376,10 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/otherresearchproduct</arg>
|
<arg>--sourcePath</arg><arg>${workingDir}/otherresearchproduct</arg>
|
||||||
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
||||||
|
@ -411,11 +405,10 @@
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.dynamicAllocation.enabled=true
|
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
|
|
||||||
--conf spark.speculation=false
|
--conf spark.speculation=false
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/software</arg>
|
<arg>--sourcePath</arg><arg>${workingDir}/software</arg>
|
||||||
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
<arg>--saveGraph</arg><arg>${saveGraph}</arg>
|
||||||
|
@ -425,6 +418,9 @@
|
||||||
<ok to="wait"/>
|
<ok to="wait"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
<join name="wait" to="End"/>
|
|
||||||
|
<join name="wait" to="End"/>
|
||||||
|
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
|
|
||||||
</workflow-app>
|
</workflow-app>
|
Loading…
Reference in New Issue