merged again from beta
This commit is contained in:
commit
0d628cd62b
|
@ -10,6 +10,7 @@ import org.apache.commons.lang3.StringUtils;
|
|||
import com.wcohen.ss.JaroWinkler;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||
import eu.dnetlib.pace.model.Person;
|
||||
import scala.Tuple2;
|
||||
|
@ -146,10 +147,20 @@ public class AuthorMerger {
|
|||
}
|
||||
|
||||
public static String pidToComparableString(StructuredProperty pid) {
|
||||
final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase()
|
||||
: "";
|
||||
return (pid.getQualifier() != null ? classid : "")
|
||||
+ (pid.getValue() != null ? pid.getValue().toLowerCase() : "");
|
||||
final String classId = Optional
|
||||
.ofNullable(pid)
|
||||
.map(
|
||||
p -> Optional
|
||||
.ofNullable(p.getQualifier())
|
||||
.map(Qualifier::getClassid)
|
||||
.map(String::toLowerCase)
|
||||
.orElse(""))
|
||||
.orElse("");
|
||||
return Optional
|
||||
.ofNullable(pid)
|
||||
.map(StructuredProperty::getValue)
|
||||
.map(v -> String.join("|", v, classId))
|
||||
.orElse("");
|
||||
}
|
||||
|
||||
public static int countAuthorsPids(List<Author> authors) {
|
||||
|
|
|
@ -270,8 +270,9 @@ public class MergeUtils {
|
|||
}
|
||||
|
||||
HashMap<String, KeyValue> values = new HashMap<>();
|
||||
left.forEach(kv -> values.put(kv.getKey(), kv));
|
||||
right.forEach(kv -> values.putIfAbsent(kv.getKey(), kv));
|
||||
|
||||
Optional.ofNullable(left).ifPresent(l -> l.forEach(kv -> values.put(kv.getKey(), kv)));
|
||||
Optional.ofNullable(right).ifPresent(r -> r.forEach(kv -> values.putIfAbsent(kv.getKey(), kv)));
|
||||
|
||||
return new ArrayList<>(values.values());
|
||||
}
|
||||
|
|
|
@ -7,9 +7,11 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import eu.dnetlib.pace.model.Person;
|
||||
import jdk.nashorn.internal.ir.annotations.Ignore;
|
||||
|
||||
public class UtilTest {
|
||||
|
||||
|
@ -20,7 +22,8 @@ public class UtilTest {
|
|||
params = new HashMap<>();
|
||||
}
|
||||
|
||||
// @Test
|
||||
@Test
|
||||
@Disabled
|
||||
public void paceResolverTest() {
|
||||
PaceResolver paceResolver = new PaceResolver();
|
||||
paceResolver.getComparator("keywordMatch", params);
|
||||
|
|
|
@ -93,7 +93,7 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
.filter((FilterFunction<Relation>) Objects::nonNull)
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(p.getClass(), p));
|
||||
//TODO relations in stand-by waiting to know if we need to create them or not In case we need just make a union before saving the sequence file
|
||||
|
||||
spark
|
||||
.read()
|
||||
.textFile(inputPath)
|
||||
|
@ -108,6 +108,7 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
.filter((FilterFunction<Result>) r -> r != null)
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(p.getClass(), p))
|
||||
.union(relations)
|
||||
.mapToPair(
|
||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||
|
|
|
@ -0,0 +1,232 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.webcrawl;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
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.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 18/04/24
|
||||
*/
|
||||
public class CreateActionSetFromWebEntries implements Serializable {
|
||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetFromWebEntries.class);
|
||||
private static final String DOI_PREFIX = "50|doi_________::";
|
||||
|
||||
private static final String ROR_PREFIX = "20|ror_________::";
|
||||
|
||||
private static final String PMID_PREFIX = "50|pmid________::";
|
||||
|
||||
private static final String PMCID_PREFIX = "50|pmc_________::";
|
||||
private static final String WEB_CRAWL_ID = "10|openaire____::fb98a192f6a055ba495ef414c330834b";
|
||||
private static final String WEB_CRAWL_NAME = "Web Crawl";
|
||||
public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
CreateActionSetFromWebEntries.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/actionmanager/webcrawl/as_parameters.json"));
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String inputPath = parser.get("sourcePath");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
|
||||
createActionSet(spark, inputPath, outputPath);
|
||||
|
||||
});
|
||||
}
|
||||
|
||||
public static void createActionSet(SparkSession spark, String inputPath,
|
||||
String outputPath) {
|
||||
|
||||
final Dataset<Row> dataset = readWebCrawl(spark, inputPath)
|
||||
.filter("publication_year <= 2020 or country_code=='IE'")
|
||||
.drop("publication_year");
|
||||
|
||||
dataset.flatMap((FlatMapFunction<Row, Relation>) row -> {
|
||||
List<Relation> ret = new ArrayList<>();
|
||||
final String ror = ROR_PREFIX
|
||||
+ IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAs("ror")));
|
||||
ret.addAll(createAffiliationRelationPairDOI(row.getAs("doi"), ror));
|
||||
ret.addAll(createAffiliationRelationPairPMID(row.getAs("pmid"), ror));
|
||||
ret.addAll(createAffiliationRelationPairPMCID(row.getAs("pmcid"), ror));
|
||||
|
||||
return ret
|
||||
.iterator();
|
||||
}, Encoders.bean(Relation.class))
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(p.getClass(), p))
|
||||
.mapToPair(
|
||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
|
||||
}
|
||||
|
||||
private static Dataset<Row> readWebCrawl(SparkSession spark, String inputPath) {
|
||||
StructType webInfo = StructType
|
||||
.fromDDL(
|
||||
"`id` STRING , `doi` STRING, `ids` STRUCT<`pmid` :STRING, `pmcid`: STRING >, `publication_year` STRING, "
|
||||
+
|
||||
"`authorships` ARRAY<STRUCT <`institutions`: ARRAY <STRUCT <`ror`: STRING, `country_code` :STRING>>>>");
|
||||
|
||||
return spark
|
||||
.read()
|
||||
.schema(webInfo)
|
||||
.json(inputPath)
|
||||
.withColumn(
|
||||
"authors", functions
|
||||
.explode(
|
||||
functions.col("authorships")))
|
||||
.selectExpr("id", "doi", "ids", "publication_year", "authors.institutions as institutions")
|
||||
.withColumn(
|
||||
"institution", functions
|
||||
.explode(
|
||||
functions.col("institutions")))
|
||||
.selectExpr(
|
||||
"id", "doi", "ids.pmcid as pmcid", "ids.pmid as pmid", "institution.ror as ror",
|
||||
"institution.country_code as country_code", "publication_year")
|
||||
.distinct();
|
||||
|
||||
}
|
||||
|
||||
private static List<Relation> createAffiliationRelationPairPMCID(String pmcid, String ror) {
|
||||
if (pmcid == null)
|
||||
return new ArrayList<>();
|
||||
|
||||
return createAffiliatioRelationPair(
|
||||
PMCID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), removeResolver("PMC", pmcid))),
|
||||
ror);
|
||||
}
|
||||
|
||||
private static List<Relation> createAffiliationRelationPairPMID(String pmid, String ror) {
|
||||
if (pmid == null)
|
||||
return new ArrayList<>();
|
||||
|
||||
return createAffiliatioRelationPair(
|
||||
PMID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), removeResolver("PMID", pmid))),
|
||||
ror);
|
||||
}
|
||||
|
||||
private static String removeResolver(String pidType, String pid) {
|
||||
switch (pidType) {
|
||||
case "PMID":
|
||||
return pid.substring(33);
|
||||
case "PMC":
|
||||
return "PMC" + pid.substring(43);
|
||||
case "DOI":
|
||||
return pid.substring(16);
|
||||
}
|
||||
|
||||
throw new RuntimeException();
|
||||
|
||||
}
|
||||
|
||||
private static List<Relation> createAffiliationRelationPairDOI(String doi, String ror) {
|
||||
if (doi == null)
|
||||
return new ArrayList<>();
|
||||
|
||||
return createAffiliatioRelationPair(
|
||||
DOI_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), removeResolver("DOI", doi))),
|
||||
ror);
|
||||
|
||||
}
|
||||
|
||||
private static List<Relation> createAffiliatioRelationPair(String resultId, String orgId) {
|
||||
ArrayList<Relation> newRelations = new ArrayList();
|
||||
|
||||
newRelations
|
||||
.add(
|
||||
OafMapperUtils
|
||||
.getRelation(
|
||||
orgId, resultId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION,
|
||||
ModelConstants.IS_AUTHOR_INSTITUTION_OF,
|
||||
Arrays
|
||||
.asList(
|
||||
OafMapperUtils.keyValue(WEB_CRAWL_ID, WEB_CRAWL_NAME)),
|
||||
OafMapperUtils
|
||||
.dataInfo(
|
||||
false, null, false, false,
|
||||
OafMapperUtils
|
||||
.qualifier(
|
||||
"sysimport:crasswalk:webcrawl", "Imported from Webcrawl",
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
|
||||
"0.9"),
|
||||
null));
|
||||
|
||||
newRelations
|
||||
.add(
|
||||
OafMapperUtils
|
||||
.getRelation(
|
||||
resultId, orgId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION,
|
||||
ModelConstants.HAS_AUTHOR_INSTITUTION,
|
||||
Arrays
|
||||
.asList(
|
||||
OafMapperUtils.keyValue(WEB_CRAWL_ID, WEB_CRAWL_NAME)),
|
||||
OafMapperUtils
|
||||
.dataInfo(
|
||||
false, null, false, false,
|
||||
OafMapperUtils
|
||||
.qualifier(
|
||||
"sysimport:crasswalk:webcrawl", "Imported from Webcrawl",
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
|
||||
"0.9"),
|
||||
null));
|
||||
|
||||
return newRelations;
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,20 @@
|
|||
[
|
||||
{
|
||||
"paramName": "sp",
|
||||
"paramLongName": "sourcePath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -0,0 +1,2 @@
|
|||
sourcePath=/user/miriam.baglioni/openalex-snapshot/data/works/
|
||||
outputPath=/tmp/miriam/webcrawlComplete/
|
|
@ -0,0 +1,58 @@
|
|||
<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>
|
||||
<property>
|
||||
<name>hive_metastore_uris</name>
|
||||
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2YarnHistoryServerAddress</name>
|
||||
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2ExtraListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2SqlQueryExecutionListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorNumber</name>
|
||||
<value>4</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2EventLogDir</name>
|
||||
<value>/user/spark/spark2ApplicationHistory</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<value>15G</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<value>6G</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<value>1</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,53 @@
|
|||
<workflow-app name="WebCrawl Integration" xmlns="uri:oozie:workflow:0.5">
|
||||
|
||||
<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="create_actionset"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="create_actionset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Produces the AS for WC</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.webcrawl.CreateActionSetFromWebEntries</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--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.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -271,12 +271,6 @@
|
|||
"name": "An Roinn Sl\u00e1inte",
|
||||
"synonym": []
|
||||
},
|
||||
{
|
||||
"id": "100018998",
|
||||
"uri": "http://dx.doi.org/10.13039/100018998",
|
||||
"name": "Irish Research eLibrary",
|
||||
"synonym": []
|
||||
},
|
||||
{
|
||||
"id": "100019428",
|
||||
"uri": "http://dx.doi.org/10.13039/100019428",
|
||||
|
|
|
@ -0,0 +1,287 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.webcrawl;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 22/04/24
|
||||
*/
|
||||
public class CreateASTest {
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
private static SparkSession spark;
|
||||
|
||||
private static Path workingDir;
|
||||
private static final Logger log = LoggerFactory
|
||||
.getLogger(CreateASTest.class);
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
workingDir = Files
|
||||
.createTempDirectory(CreateASTest.class.getSimpleName());
|
||||
log.info("using work dir {}", workingDir);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.setAppName(CreateASTest.class.getSimpleName());
|
||||
|
||||
conf.setMaster("local[*]");
|
||||
conf.set("spark.driver.host", "localhost");
|
||||
conf.set("hive.metastore.local", "true");
|
||||
conf.set("spark.ui.enabled", "false");
|
||||
conf.set("spark.sql.warehouse.dir", workingDir.toString());
|
||||
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
|
||||
|
||||
spark = SparkSession
|
||||
.builder()
|
||||
.appName(CreateASTest.class.getSimpleName())
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void afterAll() throws IOException {
|
||||
FileUtils.deleteDirectory(workingDir.toFile());
|
||||
spark.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNumberofRelations() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/webcrawl/")
|
||||
.getPath();
|
||||
|
||||
CreateActionSetFromWebEntries
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-sourcePath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet1"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
Assertions.assertEquals(64, tmp.count());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRelations() throws Exception {
|
||||
|
||||
// , "doi":"https://doi.org/10.1126/science.1188021", "pmid":"https://pubmed.ncbi.nlm.nih.gov/20448178", https://www.ncbi.nlm.nih.gov/pmc/articles/5100745
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/webcrawl/")
|
||||
.getPath();
|
||||
|
||||
CreateActionSetFromWebEntries
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-sourcePath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet1"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r)));
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getSource()
|
||||
.equals(
|
||||
"50|doi_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue(PidType.doi.toString(), "10.1098/rstl.1684.0023"))))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getTarget()
|
||||
.equals(
|
||||
"50|doi_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue(PidType.doi.toString(), "10.1098/rstl.1684.0023"))))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getSource()
|
||||
.equals(
|
||||
"20|ror_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue("ROR", "https://ror.org/03argrj65"))))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getTarget()
|
||||
.equals(
|
||||
"20|ror_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue("ROR", "https://ror.org/03argrj65"))))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
5, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getSource()
|
||||
.equals(
|
||||
"20|ror_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue("ROR", "https://ror.org/03265fv13"))))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
5, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getTarget()
|
||||
.equals(
|
||||
"20|ror_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue("ROR", "https://ror.org/03265fv13"))))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
2, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getTarget()
|
||||
.equals(
|
||||
"20|ror_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue(PidType.doi.toString(), "https://ror.org/03265fv13")))
|
||||
&& r.getSource().startsWith("50|doi"))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
2, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getTarget()
|
||||
.equals(
|
||||
"20|ror_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue(PidType.doi.toString(), "https://ror.org/03265fv13")))
|
||||
&& r.getSource().startsWith("50|pmid"))
|
||||
.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1, tmp
|
||||
.filter(
|
||||
r -> r
|
||||
.getTarget()
|
||||
.equals(
|
||||
"20|ror_________::" + IdentifierFactory
|
||||
.md5(
|
||||
PidCleaner
|
||||
.normalizePidValue(PidType.doi.toString(), "https://ror.org/03265fv13")))
|
||||
&& r.getSource().startsWith("50|pmc"))
|
||||
.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRelationsCollectedFrom() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/webcrawl")
|
||||
.getPath();
|
||||
|
||||
CreateActionSetFromWebEntries
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-sourcePath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet1"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
tmp.foreach(r -> {
|
||||
assertEquals("Web Crawl", r.getCollectedfrom().get(0).getValue());
|
||||
assertEquals("10|openaire____::fb98a192f6a055ba495ef414c330834b", r.getCollectedfrom().get(0).getKey());
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
}
|
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
|
@ -47,7 +47,7 @@ public class OrcidClientTest {
|
|||
private static Path testPath;
|
||||
|
||||
@BeforeAll
|
||||
private static void setUp() throws IOException {
|
||||
public static void setUp() throws IOException {
|
||||
testPath = Files.createTempDirectory(OrcidClientTest.class.getName());
|
||||
System.out.println("using test path: " + testPath);
|
||||
}
|
||||
|
@ -150,8 +150,9 @@ public class OrcidClientTest {
|
|||
System.out.println(valueDt.toString());
|
||||
}
|
||||
|
||||
// @Test
|
||||
private void testModifiedDate() throws ParseException {
|
||||
@Test
|
||||
@Disabled
|
||||
public void testModifiedDate() throws ParseException {
|
||||
testDate(toRetrieveDate);
|
||||
testDate(toNotRetrieveDate);
|
||||
testDate(shortDate);
|
||||
|
@ -224,7 +225,7 @@ public class OrcidClientTest {
|
|||
|
||||
@Test
|
||||
@Disabled
|
||||
private void slowedDownDownloadTest() throws Exception {
|
||||
public void slowedDownDownloadTest() throws Exception {
|
||||
String orcid = "0000-0001-5496-1243";
|
||||
String record = slowedDownDownload(orcid);
|
||||
String filename = "/tmp/downloaded_".concat(orcid).concat(".xml");
|
||||
|
@ -330,6 +331,7 @@ public class OrcidClientTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
void testUpdatedRecord() throws Exception {
|
||||
final String base64CompressedRecord = IOUtils
|
||||
.toString(getClass().getResourceAsStream("0000-0001-7281-6306.compressed.base64"));
|
||||
|
@ -338,6 +340,7 @@ public class OrcidClientTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
void testUpdatedWork() throws Exception {
|
||||
final String base64CompressedWork = "H4sIAAAAAAAAAM1XS2/jNhC+51cQOuxJsiXZSR03Vmq0G6Bo013E6R56oyXaZiOJWpKy4y783zvUg5Ksh5uiCJogisX5Zjj85sHx3f1rFKI94YKyeGE4I9tAJPZZQOPtwvj9+cGaGUhIHAc4ZDFZGEcijHvv6u7A+MtcPVCSSgsUQObYzuzaccBEguVuYYxt+LHgbwKP6a11M3WnY6UzrpB7KuiahlQeF0aSrkPqGwhcisWcxpLwGIcLYydlMh+PD4fDiHGfBvDcjmMxLhGlBglSH8vsIH0qGlLqBFRIGvvDWjWQ1iMJJ2CKBANqGlNqMbkj3IpxRPq1KkypFZFoDRHa0aRfq8JoNjhnfIAJJS6xPouiIQJyeYmGQzE+cO5cXqITcItBlKyASExD0a93jiwtvJDjYXDDAqBPHoH2wMmVWGNf8xyyaEBiSTeUDHHWBpd2Nmmc10yfbgHQrHCyIRxKjQwRUoFKPRwEnIgBnQJQVdGeQgJaCRN0OMnPkaUFVbD9WkpaIndQJowf+8EFoIpTErJjBFQOBavElFpfUxwC9ZcqvQErdQXhe+oPFF8BaObupYzVsYEOARzSoZBWmKqaBMHcV0Wf8oG0beIqD+Gdkz0lhyE3NajUW6fhQFSV9Nw/MCBYyofYa0EN7wrBz13eP+Y+J6obWgE8Pdd2JpYD94P77Ezmjj13b0bu5PqPu3EXumEnxEJaEVxSUIHammsra+53z44zt2/m1/bItaeVtQ6dhs3c4XytvW75IYUchMKvEHVUyqmnWBFAS0VJrqSvQde6vp251ux2NtFuKcVOi+oK9YY0M0Cn6o4J6WkvtEK2XJ1vfPGAZxSoK8lb+SxJBbLQx1CohOLndjJUywQWUFmqEi3G6Zaqf/7buOyYJd5IYpfmf0XipfP18pDR9cQCeEuJQI/Lx36bFbVnpBeL2UwmqQw7ApAvf4GeGGQdEbENgolui/wdpjHaYCmPCIPPAmGBIsxfoLUhyRCB0SeCakEBJRKBtfJ+UBbI15TG4PaGBAhWthx8DmFYtHZQujv1CWbLLdzmmUKmHEOWCe1/zdu78bn/+YH+hCOqOzcXfFwuP6OVT/P710crwqGXFrpNaM2GT3MXarw01i15TIi3pmtJXgtbTVGf3h6HKfF+wBAnPyTfdCChudlm5gZaoG//F9pPZsGQcqqbyZN5hBau5OoIJ3PPwjTKDuG4s5MZp2rMzF5PZoK34IT6PIFOPrk+mTiVO5aJH2C+JJRjE/06eoRfpJxa4VgyYaLlaJUv/EhCfATMU/76gEOfmehL/qbJNNHjaFna+CQYB8wvo9PpPFJ5MOrJ1Ix7USBZqBl7KRNOx1d3jex7SG6zuijqCMWRusBsncjZSrM2u82UJmqzpGhvUJN2t6caIM9QQgO9c0t40UROnWsJd2Rbs+nsxpna9u30ttNkjechmzHjEST+X5CkkuNY0GzQkzyFseAf7lSZuLwdh1xSXKvvQJ4g4abTYgPV7uMt3rskohlJmMa82kQkshtyBEIYqQ+YB8X3oRHg7iFKi/bZP+Ao+T6BJhIT/vNPi8ffZs+flk+r2v0WNroZiyWn6xRmadHqTJXsjLJczElAZX6TnJdoWTM1SI2gfutv3rjeBt5t06rVvNuWup29246tlvluO+u2/G92bK9DXheL6uFd/Q3EaRDZqBIAAA==";
|
||||
final String work = ArgumentApplicationParser.decompressValue(base64CompressedWork);
|
||||
|
@ -409,6 +412,7 @@ public class OrcidClientTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
void testDownloadedAuthor() throws Exception {
|
||||
final String base64CompressedWork = "H4sIAAAAAAAAAI2Yy26jMBSG932KiD0hIe1MiwiVZjGLkWbX2XRHsFOsgs3YJmnefszFFy4+mUhtVPz9P/gcH/vQ9PWrrjYXzAVh9Bjst7tgg2nBEKEfx+DP28/wOdgImVOUV4ziY3DDInjNHlKOC8ZRMnxtmlyWxyDaqU+ofg7h/uX7IYwfn+Ngo25ARUKoxJzm1TEopWySKLper1vGC4LU74+IikgTWoFRW+SyfyyfxCBag4iQhBawyoGMDjdqJrnECJAZRquYLDEPaV5jv8oyWlXj+qTiXZLGr7KMiQbnjAOR6IY1W7C6hgIwjGt6SKGfHsY13ajHYipLIcIyJ5Xw6+akdvjEtyt4wxEwM6+VGph5N2zYr2ENhQRhKsmZYChmS1j7nFs6VIBPOwImKhyfMVeFg6GAWEjrcoQ4FoBmBGwVXYhagGHDBIEX+ZzUDiqyn35VN6rJUpUJ4zc/PAI2T03FbrUKJZQszWjV3zavVOjvVfoE01qB+YUUQPGNwHTt3luxJjdqh1AxJFBKLWOrSeCcF13RtxxYtlPOPqH6m+MLwVfoMQ2kdae2ArLajc6fTxkI1nIoegs0yB426pMO+0fSw07xDKMu0XKSde5C2VvrlVMijRzFwqY7XTJI1QMLWcmEzMxtDdxfHiYSgTNJnYJ1K9y5k0tUrMgrnGGaRiuXxxuClulYUbr0nBvpkYLjvgTCGsuSoex3f1CEvRPHKI184NJKtKeaiO7cD5E61bJ4F+9DFd7d01u8Tw6H5BBvvz8f3q3nXLGIeJULGdaqeVBBRK7rS7h/fNvvk/gpedxt4923dxP7Fc3KtKuc1BhlkrfYmeN4dcmrhmbw60+HmWw2CKgbTuqc32CXKTTmeTWT6bDBjPsQ0DTpnchdaYO0ayQ2FyLIiVREqs25aU8VKYLRbK0BsyZuqvr1MU2Sm/rDdhe/2CRN6FU/b+oBVyj1zqRtC5F8kAumfTclsl+s7EoNQu64nfOaVLeezX60Z3XCULLi6GI2IZGTEeey7fec9lBAuXawIHKcpifE7GABHWfoxLVfpUNPBXoMbZWrHFsR3bPAk9J9i2sw9nW6AQT1mpk++7JhW+v44Hmt8PomJqfD13jRnvFOSxCKtu6qHoyBbQ7cMFo750UEfGaXm6bEeplXIXj2hvL6mA7tzvIwmM9pbJFBG834POZdLGi2gH2u9u0K9HMwn5PTioFWLufzmrS4oNuU9Pkt2rf/2jMs7fMdm2rQTTM+j+49AzToAVuXYA1mD2k0+XdE9vAP+JYR5NcQAAA=";
|
||||
final String work = ArgumentApplicationParser.decompressValue(base64CompressedWork);
|
||||
|
@ -416,6 +420,7 @@ public class OrcidClientTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
void testDownloadedWork() throws Exception {
|
||||
final String base64CompressedWork = "H4sIAAAAAAAAANVa63LiOBb+z1Oo+LVbhbkGAlTCLE1Id9IhTQV6unr/CVvB2tiWR5Khmal5rX2BfbE9ki3b3Jzt6Y13h6pQSPrOXTo6knL10zffQxvCBWXBdbVVb1YRCWzm0GB9Xf28vLX6VSQkDhzssYBcV3dEVH8aVa62jL8M1RcKI2kBAYwNLnrtXrMPFCGW7nW10YSPBX8dq3XRb1swNGgomkaG3FBBV9SjcnddDaOVR+0qApUCMaSBJDzA3nXVlTIcNhrb7bbOuE0d+F43AtEwCENBnMjGUhtyjiSFGBqHCkkDu5gqB0rpSMgJsCJOAVmKMVRMuoRbAfbJeaoMY6h84q8gQi4Nz1NlmNQbnDNe4Ak1bLA28/0iB8TjBg1GMV5gdzxu0CGoxSBKlkMkpp44T3eINBxeyG5bKDABpJb7QF1guRpOsd/iOWRRhwSSPlNS5LNjsOHzHAXxmjlHmwBSr3DyTDgsNVLkkAxk6LDjcCIKaBJAtoo2FCagFTJBiyf5IdJwUAv2PJUaNUgXlgnju/PgBJDFKfTYzgdXFgXLYAzVLxH2wPWvrfQ9mKEVhG+oXbD4EsD+3H1txqaxgQwBPqRFIc0w2WoSBHNbLfqIF0zbfVymIbQ52VCyLVIzBRm6VeQVRFWNHuoHDASLeJH3jqDVUQXB5yrOH0ObE5UNLQe+R+1mu2U1u1Z7sGy2hq3esN2tt5oXf79qnELv8fGwkJYPmxSswD1uA6vVXrY7w+5g2G3WuxedjNsJmj2escJx33G/ZXsU5iAs/AyRR0WcjpRXBLglc0lM1BjP59bX1qw9Hn/+dH87/dy9vBikeinKkyzVHjoqJNWIk7QuE3KU6pES6O7MwsarJh44QW1KowcWOCxAC9tlzEPsGX3YrYGQICgS0JKzENach2bEoTYNyKEQzaJyQnzSqesKSaV3IhRx92L8tLAm7GerjbZUujSwlFnIobqKkTuth+Q4ED4Vqqypp5JyfK8ah5Ji0f8AZVSGT2TZVGXfBLw/liOyqdRpJqfyXr8ldyEZrehKkm8Jr/2hc3Qb7EVk9DfMJbU98pu3k+6aETXXBebCZpt23tBaBUfSZRxdo98eYmgNfRxrh3zAnldDM/37FvZ+IiWtoQfddgiaEGBIDGCG7btA7jgBP9svAK2h90l4yYqIGop5jgMHXA4J0NB9ksR+YTX0qFtfqACO01jGjDHFPx552AW2W0P3uvGROk4NLfTvCeNS8X9MaDg1rL9Qz6PYh7En3f4ZNmKS6nUfQYFmE6PYe05IYBqPFGaq5wHlYpaoDbYqxokVK+JBerz51z+BIzc+SfSdTHVrTiSYtZzGFNOdGrr5ohsLF2+NUguqppkDoua6/S6yXwAYu44pM+/HiZ1BwEDWMqYbC5fjZ+MEBwMjb4PRLdTFYWrUwiUhJH/H+G3pMl/7fjqJhTGwSwU5lnfLsVDmxIPvmRetbJeCOsvfaxWXbXWxLVziqNky51BLW1OP2JKzgNoASSa7Gk1WAfrLI9mirzBBIUD1r/W/AgrMla7CjEMOzYBJolo30/mnxd0SzadPt5+eZtMb9O7rEN1wNINgEA8Ha+IxNMdrHLCQRR4TFRCudnmB7m6GqD0YDCqW+lQqlfnndw93iw/TJ/RwN5k+TqZDNJkAQyUvUlWvktjrdgbQEeI1EapN8Grd7MOeYJlfajSxWVOMfcIhVQXgfcFsqhcceobVA/U3GjsbDCYrjVSKSz0wHo8Xym6dArRvvjsbAfUGouFr8s5lG9o72DVVSy1saDqMqlarWW+12r2GiIXXMzuAU6AQcLLqWf3mZRf6iOlsNQdda9BudhQnvNNdPWN8XA7BgU5G2k3pLADA75XD3BSnn3y+3M90SbZWGczkxiRVmfSaJrd0V8u0yG3CeYRyht7O07Ste45weuqNmhcpLO44woEPRq1eilLN/f3ntEqGPFfzi2PmudHTO3EOEKf60LdTyUeDr7KIIzKfTfqtdr896JxklQtbES/IQD7UyL+SZIJSXYhLHkHZ9oqEjPR1MRzWu550cDYdCeI9n+S4hzouUU76+UeCQJ0fjkKn0+v3m703i0Eh/z97BCDH/XAAziTIt4rH94j7s4dHbSY/HJ90e3qriBQL+MMxCGETs9j/QxiSQ5PaS63/QsZqdS8vOxdvtj7Oc//fL4dTI2LvDAfVA6erSDKe3+cPxw70j4c5HHZlfLT9iAEZYKjZkxOYKZxymJy659l/t+QZllC5bvVJrzShD5GN0/NkiaZyqNcJh0NrdngtTfp7wviaHB+SS1Ng7O+Sk3h5HodT4S8RyY78pUmGM6eEg1l8tVCa1KnvY/SgrzDKsxRLF46j+uahNKH3BE6lsIb1lUxpUhdS3WUE+u6nPP/qiyAsklumMhMz9SBNqeus0oQ+QXqwIa7m3qy87IhXnBLPI8kVXXlZMaASm5vAEqWuKYkvHMtbPdiPiIdm6dVmeVMZjX+lfnKDWmaRAT7ev6ctTfhEF3RoWnJeXlKfSXcHcsf69rk0wTd4Qx30RV9yl5et2Ipwqe/SS5MJXiU8vbIv2b/qZaC8PZ65AUwj9QJR3vx1mQ9b7VPy1FFebnSpWq7xi0qJuwA+fLYpL7rwJdLXobcSa97kM4Cl35f3YXmofp0+8R9gBc/XeXL9Vn38pH7mLTs27z9T8ky1n7ynlZ0I4le78rYzl6t/woG5krwQlpcRcLDD2UPkH5F73C9G5tFKfY0q/wa1TIHI0CgAAA==";
|
||||
final String work = ArgumentApplicationParser.decompressValue(base64CompressedWork);
|
||||
|
|
|
@ -33,7 +33,7 @@ SELECT
|
|||
dc.officialname AS collectedfromname,
|
||||
p.contracttype || '@@@' || p.contracttypescheme AS contracttype,
|
||||
p.provenanceactionclass || '@@@' || p.provenanceactionscheme AS provenanceaction,
|
||||
array_remove(array_agg(DISTINCT i.pid || '###' || i.issuertype || '@@@' || i.issuertype), NULL) AS pid,,
|
||||
array_remove(array_agg(DISTINCT i.pid || '###' || i.issuertype || '@@@' || i.issuertype), NULL) AS pid,
|
||||
array_agg(DISTINCT s.name || '###' || s.semanticclass || '@@@' || s.semanticscheme) AS subjects,
|
||||
array_agg(DISTINCT fp.path) AS fundingtree
|
||||
|
||||
|
|
|
@ -185,6 +185,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -212,6 +213,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -239,6 +241,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -266,6 +269,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -293,6 +297,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -320,6 +325,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -347,6 +353,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -386,6 +393,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -414,6 +422,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -442,6 +451,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -470,6 +480,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -498,6 +509,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -526,6 +538,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
@ -554,6 +567,7 @@
|
|||
--executor-cores=${sparkExecutorCoresForJoining}
|
||||
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||
--driver-memory=${sparkDriverMemoryForJoining}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryForJoining}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
|
2
pom.xml
2
pom.xml
|
@ -960,7 +960,7 @@
|
|||
<commons.logging.version>1.1.3</commons.logging.version>
|
||||
<commons-validator.version>1.7</commons-validator.version>
|
||||
<dateparser.version>1.0.7</dateparser.version>
|
||||
<dhp-schemas.version>[6.1.0]</dhp-schemas.version>
|
||||
<dhp-schemas.version>[6.1.1]</dhp-schemas.version>
|
||||
<dhp.cdh.version>cdh5.9.2</dhp.cdh.version>
|
||||
<dhp.commons.lang.version>3.5</dhp.commons.lang.version>
|
||||
<dhp.guava.version>11.0.2</dhp.guava.version>
|
||||
|
|
Loading…
Reference in New Issue