1
0
Fork 0

Merge pull request 'adding context information to projects and datasources' (#407) from taggingProjects into beta

Reviewed-on: D-Net/dnet-hadoop#407
This commit is contained in:
Claudio Atzori 2024-03-26 14:53:38 +01:00
commit 9e700a8b0d
15 changed files with 469 additions and 44 deletions

View File

@ -95,7 +95,7 @@ public class SparkAtomicActionScoreJob implements Serializable {
return projectScores.map((MapFunction<BipProjectModel, Project>) bipProjectScores -> {
Project project = new Project();
// project.setId(bipProjectScores.getProjectId());
project.setId(bipProjectScores.getProjectId());
project.setMeasures(bipProjectScores.toMeasures());
return project;
}, Encoders.bean(Project.class))

View File

@ -6,6 +6,7 @@ import java.io.IOException;
import java.io.InputStreamReader;
import java.net.HttpURLConnection;
import java.net.URL;
import java.util.List;
import org.jetbrains.annotations.NotNull;

View File

@ -3,14 +3,10 @@ package eu.dnetlib.dhp.api;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.*;
import java.util.stream.Collectors;
import javax.management.Query;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -24,7 +20,10 @@ import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration;
import eu.dnetlib.dhp.bulktag.community.Provider;
import eu.dnetlib.dhp.bulktag.criteria.VerbResolver;
import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory;
import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Project;
/**
* @author miriam.baglioni
@ -58,7 +57,7 @@ public class Utils implements Serializable {
if (d.getEnabled() == null || Boolean.FALSE.equals(d.getEnabled()))
return null;
Provider p = new Provider();
p.setOpenaireId("10|" + d.getOpenaireId());
p.setOpenaireId(ModelSupport.getIdPrefix(Datasource.class) + "|" + d.getOpenaireId());
p.setSelectionConstraints(d.getSelectioncriteria());
if (p.getSelectionConstraints() != null)
p.getSelectionConstraints().setSelection(resolver);
@ -113,6 +112,7 @@ public class Utils implements Serializable {
*/
public static CommunityEntityMap getCommunityOrganization(String baseURL) throws IOException {
CommunityEntityMap organizationMap = new CommunityEntityMap();
String entityPrefix = ModelSupport.getIdPrefix(Organization.class);
getValidCommunities(baseURL)
.forEach(community -> {
String id = community.getId();
@ -124,9 +124,9 @@ public class Utils implements Serializable {
if (!organizationMap
.keySet()
.contains(
"20|" + o))
organizationMap.put("20|" + o, new ArrayList<>());
organizationMap.get("20|" + o).add(community.getId());
entityPrefix + "|" + o))
organizationMap.put(entityPrefix + "|" + o, new ArrayList<>());
organizationMap.get(entityPrefix + "|" + o).add(community.getId());
});
} catch (IOException e) {
throw new RuntimeException(e);
@ -138,7 +138,7 @@ public class Utils implements Serializable {
public static CommunityEntityMap getCommunityProjects(String baseURL) throws IOException {
CommunityEntityMap projectMap = new CommunityEntityMap();
String entityPrefix = ModelSupport.getIdPrefix(Project.class);
getValidCommunities(baseURL)
.forEach(community -> {
int page = -1;
@ -155,9 +155,9 @@ public class Utils implements Serializable {
ContentModel.class);
if (cm.getContent().size() > 0) {
cm.getContent().forEach(p -> {
if (!projectMap.keySet().contains("40|" + p.getOpenaireId()))
projectMap.put("40|" + p.getOpenaireId(), new ArrayList<>());
projectMap.get("40|" + p.getOpenaireId()).add(community.getId());
if (!projectMap.keySet().contains(entityPrefix + "|" + p.getOpenaireId()))
projectMap.put(entityPrefix + "|" + p.getOpenaireId(), new ArrayList<>());
projectMap.get(entityPrefix + "|" + p.getOpenaireId()).add(community.getId());
});
}
} catch (IOException e) {
@ -174,4 +174,41 @@ public class Utils implements Serializable {
.map(community -> community.getId())
.collect(Collectors.toList());
}
public static List<EntityCommunities> getDatasourceCommunities(String baseURL) throws IOException {
List<CommunityModel> validCommunities = getValidCommunities(baseURL);
HashMap<String, Set<String>> map = new HashMap<>();
String entityPrefix = ModelSupport.getIdPrefix(Datasource.class) + "|";
validCommunities.forEach(c -> {
try {
new ObjectMapper()
.readValue(QueryCommunityAPI.communityDatasource(c.getId(), baseURL), DatasourceList.class)
.forEach(d -> {
if (!map.keySet().contains(d.getOpenaireId()))
map.put(d.getOpenaireId(), new HashSet<>());
map.get(d.getOpenaireId()).add(c.getId());
});
} catch (IOException e) {
throw new RuntimeException(e);
}
});
List<EntityCommunities> temp = map
.keySet()
.stream()
.map(k -> EntityCommunities.newInstance(entityPrefix + k, getCollect(k, map)))
.collect(Collectors.toList());
return temp;
}
@NotNull
private static List<String> getCollect(String k, HashMap<String, Set<String>> map) {
List<String> temp = map.get(k).stream().collect(Collectors.toList());
return temp;
}
}

View File

@ -0,0 +1,40 @@
package eu.dnetlib.dhp.api.model;
import java.io.Serializable;
import java.util.List;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Datasource;
/**
* @author miriam.baglioni
* @Date 13/02/24
*/
public class EntityCommunities implements Serializable {
private String entityId;
private List<String> communitiesId;
public String getEntityId() {
return entityId;
}
public void setEntityId(String entityId) {
this.entityId = entityId;
}
public List<String> getCommunitiesId() {
return communitiesId;
}
public void setCommunitiesId(List<String> communitiesId) {
this.communitiesId = communitiesId;
}
public static EntityCommunities newInstance(String dsid, List<String> csid) {
EntityCommunities dsc = new EntityCommunities();
dsc.entityId = dsid;
dsc.communitiesId = csid;
return dsc;
}
}

View File

@ -4,9 +4,23 @@ package eu.dnetlib.dhp.bulktag;
import static eu.dnetlib.dhp.PropagationConstant.removeOutputDir;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.BufferedOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.*;
import java.util.stream.Collectors;
import java.util.zip.GZIPOutputStream;
import org.apache.avro.TestAnnotation;
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
@ -17,17 +31,24 @@ import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.gson.Gson;
import com.sun.media.sound.ModelInstrumentComparator;
import eu.dnetlib.dhp.api.Utils;
import eu.dnetlib.dhp.api.model.CommunityEntityMap;
import eu.dnetlib.dhp.api.model.EntityCommunities;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.bulktag.community.*;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Context;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import scala.Tuple2;
public class SparkBulkTagJob {
@ -47,6 +68,7 @@ public class SparkBulkTagJob {
.getResourceAsStream(
"/eu/dnetlib/dhp/wf/subworkflows/bulktag/input_bulkTag_parameters.json"));
log.info(args.toString());
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
@ -65,8 +87,20 @@ public class SparkBulkTagJob {
final String baseURL = parser.get("baseURL");
log.info("baseURL: {}", baseURL);
ProtoMap protoMappingParams = new Gson().fromJson(parser.get("pathMap"), ProtoMap.class);
log.info("pathMap: {}", new Gson().toJson(protoMappingParams));
log.info("pathMap: {}", parser.get("pathMap"));
String protoMappingPath = parser.get("pathMap");
final String hdfsNameNode = parser.get("nameNode");
log.info("nameNode: {}", hdfsNameNode);
Configuration configuration = new Configuration();
configuration.set("fs.defaultFS", hdfsNameNode);
FileSystem fs = FileSystem.get(configuration);
String temp = IOUtils.toString(fs.open(new Path(protoMappingPath)), StandardCharsets.UTF_8);
log.info("protoMap: {}", temp);
ProtoMap protoMap = new Gson().fromJson(temp, ProtoMap.class);
log.info("pathMap: {}", new Gson().toJson(protoMap));
SparkConf conf = new SparkConf();
CommunityConfiguration cc;
@ -88,10 +122,130 @@ public class SparkBulkTagJob {
isSparkSessionManaged,
spark -> {
extendCommunityConfigurationForEOSC(spark, inputPath, cc);
execBulkTag(spark, inputPath, outputPath, protoMappingParams, cc);
execBulkTag(
spark, inputPath, outputPath, protoMap, cc);
execDatasourceTag(spark, inputPath, outputPath, Utils.getDatasourceCommunities(baseURL));
execProjectTag(spark, inputPath, outputPath, Utils.getCommunityProjects(baseURL));
});
}
private static void execProjectTag(SparkSession spark, String inputPath, String outputPath,
CommunityEntityMap communityProjects) {
Dataset<Project> projects = readPath(spark, inputPath + "project", Project.class);
Dataset<EntityCommunities> pc = spark
.createDataset(
communityProjects
.keySet()
.stream()
.map(k -> EntityCommunities.newInstance(k, communityProjects.get(k)))
.collect(Collectors.toList()),
Encoders.bean(EntityCommunities.class));
projects
.joinWith(pc, projects.col("id").equalTo(pc.col("entityId")), "left")
.map((MapFunction<Tuple2<Project, EntityCommunities>, Project>) t2 -> {
Project ds = t2._1();
if (t2._2() != null) {
List<String> context = Optional
.ofNullable(ds.getContext())
.map(v -> v.stream().map(c -> c.getId()).collect(Collectors.toList()))
.orElse(new ArrayList<>());
if (!Optional.ofNullable(ds.getContext()).isPresent())
ds.setContext(new ArrayList<>());
t2._2().getCommunitiesId().forEach(c -> {
if (!context.contains(c)) {
Context con = new Context();
con.setId(c);
con
.setDataInfo(
Arrays
.asList(
OafMapperUtils
.dataInfo(
false, TaggingConstants.BULKTAG_DATA_INFO_TYPE, true, false,
OafMapperUtils
.qualifier(
TaggingConstants.CLASS_ID_DATASOURCE,
TaggingConstants.CLASS_NAME_BULKTAG_DATASOURCE,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"1")));
ds.getContext().add(con);
}
});
}
return ds;
}, Encoders.bean(Project.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath + "project");
readPath(spark, outputPath + "project", Datasource.class)
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(inputPath + "project");
}
private static void execDatasourceTag(SparkSession spark, String inputPath, String outputPath,
List<EntityCommunities> datasourceCommunities) {
Dataset<Datasource> datasource = readPath(spark, inputPath + "datasource", Datasource.class);
Dataset<EntityCommunities> dc = spark
.createDataset(datasourceCommunities, Encoders.bean(EntityCommunities.class));
datasource
.joinWith(dc, datasource.col("id").equalTo(dc.col("entityId")), "left")
.map((MapFunction<Tuple2<Datasource, EntityCommunities>, Datasource>) t2 -> {
Datasource ds = t2._1();
if (t2._2() != null) {
List<String> context = Optional
.ofNullable(ds.getContext())
.map(v -> v.stream().map(c -> c.getId()).collect(Collectors.toList()))
.orElse(new ArrayList<>());
if (!Optional.ofNullable(ds.getContext()).isPresent())
ds.setContext(new ArrayList<>());
t2._2().getCommunitiesId().forEach(c -> {
if (!context.contains(c)) {
Context con = new Context();
con.setId(c);
con
.setDataInfo(
Arrays
.asList(
OafMapperUtils
.dataInfo(
false, TaggingConstants.BULKTAG_DATA_INFO_TYPE, true, false,
OafMapperUtils
.qualifier(
TaggingConstants.CLASS_ID_DATASOURCE,
TaggingConstants.CLASS_NAME_BULKTAG_DATASOURCE,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"1")));
ds.getContext().add(con);
}
});
}
return ds;
}, Encoders.bean(Datasource.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath + "datasource");
readPath(spark, outputPath + "datasource", Datasource.class)
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(inputPath + "datasource");
}
private static void extendCommunityConfigurationForEOSC(SparkSession spark, String inputPath,
CommunityConfiguration cc) {
@ -129,6 +283,11 @@ public class SparkBulkTagJob {
ProtoMap protoMappingParams,
CommunityConfiguration communityConfiguration) {
try {
System.out.println(new ObjectMapper().writeValueAsString(protoMappingParams));
} catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
ModelSupport.entityTypes
.keySet()
.parallelStream()

View File

@ -38,7 +38,6 @@ public class ExecSubstringAction implements Serializable {
}
public String execSubstring() {
return this.value.substring(Integer.valueOf(this.from), Integer.valueOf(this.to));
}

View File

@ -1,4 +1,4 @@
sourcePath=/tmp/beta_provision/graph/10_graph_orcid_enriched
sourcePath=/tmp/beta_provision/graph/09_graph_orcid_enriched
resumeFrom=ResultProject
allowedsemrelsorcidprop=isSupplementedBy;isSupplementTo
allowedsemrelsresultproject=isSupplementedBy;isSupplementTo
@ -7,20 +7,23 @@ datasourceWhitelistForCountryPropagation=10|opendoar____::16e6a3326dd7d868cbc926
#allowedtypes=pubsrepository::institutional
allowedtypes=Institutional
outputPath=/tmp/miriam/graph/11_graph_orcid
pathMap ={"author":"$['author'][*]['fullname']", \
"title":"$['title'][*]['value']",\
"orcid":"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid')]['value']" ,\
"orcid_pending":"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid_pending')]['value']" ,\
"contributor" : "$['contributor'][*]['value']",\
"description" : "$['description'][*]['value']",\
"subject" :"$['subject'][*]['value']" , \
"fos" : "$['subject'][?(@['qualifier']['classid']=='FOS')].value" ,\
"sdg" : "$['subject'][?(@['qualifier']['classid']=='SDG')].value",\
"journal":"$['journal'].name",\
"hostedby":"$['instance'][*]['hostedby']['key']",\
"collectedfrom":"$['instance'][*]['collectedfrom']['key']",\
"publisher":"$['publisher'].value",\
"publicationyear":"$['dateofacceptance'].value"}
pathMap ={"author":{"path":"$['author'][*]['fullname']"}, \
"title":{"path":"$['title'][*]['value']"},\
"orcid":{"path":"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid')]['value']"} ,\
"orcid_pending":{"path":"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid_pending')]['value']"} ,\
"contributor" : {"path":"$['contributor'][*]['value']"},\
"description" : {"path":"$['description'][*]['value']"},\
"subject" :{"path":"$['subject'][*]['value']"}, \
"fos" : {"path":"$['subject'][?(@['qualifier']['classid']=='FOS')].value"} ,\
"sdg" : {"path":"$['subject'][?(@['qualifier']['classid']=='SDG')].value"},\
"journal":{"path":"$['journal'].name"},\
"hostedby":{"path":"$['instance'][*]['hostedby']['key']"},\
"collectedfrom":{"path":"$['instance'][*]['collectedfrom']['key']"},\
"publisher":{"path":"$['publisher'].value"},\
"publicationyear":{"path":"$['dateofacceptance'].value", "action":{"class":"eu.dnetlib.dhp.bulktag.actions.ExecSubstringAction",\
"method":"execSubstring",\
"params":[{"param_name":"From","param_value":0},\
{"param_name":"To","param_value":4}]}}}
blacklist=empty
allowedpids=orcid;orcid_pending
baseURL = https://services.openaire.eu/openaire/community/

View File

@ -33,6 +33,11 @@
"paramName": "bu",
"paramLongName": "baseURL",
"paramDescription": "this parameter is to specify the api to be queried (beta or production)",
"paramRequired": false
}
"paramRequired": true
},{
"paramName": "nn",
"paramLongName": "nameNode",
"paramDescription": "this parameter is to specify the api to be queried (beta or production)",
"paramRequired": true
}
]

View File

@ -53,10 +53,10 @@
</property>
<property>
<name>memoryOverhead</name>
<value>3G</value>
<value>4G</value>
</property>
<property>
<name>partitions</name>
<value>3284</value>
<value>15000</value>
</property>
</configuration>

View File

@ -76,6 +76,7 @@
<arg>--outputPath</arg><arg>${workingDir}/bulktag/</arg>
<arg>--pathMap</arg><arg>${pathMap}</arg>
<arg>--baseURL</arg><arg>${baseURL}</arg>
<arg>--nameNode</arg><arg>${nameNode}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -6,14 +6,19 @@ import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.ZENODO_COMMUNITY
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
@ -25,14 +30,16 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.gson.Gson;
import eu.dnetlib.dhp.bulktag.community.ProtoMap;
import eu.dnetlib.dhp.schema.oaf.*;
public class BulkTagJobTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static final String pathMap = "{\"author\":{\"path\":\"$['author'][*]['fullname']\"}," +
public static final String pathMap = "{\"protoMap\":{\"author\":{\"path\":\"$['author'][*]['fullname']\"}," +
" \"title\":{\"path\":\"$['title'][*]['value']\"}, " +
" \"orcid\":{\"path\":\"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid')]['value']\"} , " +
" \"orcid_pending\":{\"path\":\"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid_pending')]['value']\"} ,"
@ -51,7 +58,7 @@ public class BulkTagJobTest {
"\"method\":\"execSubstring\"," +
"\"params\":[" +
"{\"paramName\":\"From\", \"paramValue\":0}, " +
"{\"paramName\":\"To\",\"paramValue\":4}]}}}";
"{\"paramName\":\"To\",\"paramValue\":4}]}}}}";
private static SparkSession spark;
@ -231,6 +238,14 @@ public class BulkTagJobTest {
@Test
void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception {
LocalFileSystem fs = FileSystem.getLocal(new Configuration());
fs
.copyFromLocalFile(
false, new org.apache.hadoop.fs.Path(getClass()
.getResource("/eu/dnetlib/dhp/bulktag/pathMap/")
.getPath()),
new org.apache.hadoop.fs.Path(workingDir.toString() + "/data/bulktagging/protoMap"));
final String sourcePath = getClass()
.getResource(
"/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/")
@ -246,7 +261,8 @@ public class BulkTagJobTest {
"-outputPath", workingDir.toString() + "/",
"-pathMap", pathMap
"-pathMap", workingDir.toString() + "/data/bulktagging/protoMap",
"-nameNode", "local"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
@ -316,6 +332,7 @@ public class BulkTagJobTest {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/")
.getPath();
SparkBulkTagJob
.main(
new String[] {
@ -325,7 +342,7 @@ public class BulkTagJobTest {
"-taggingConf", taggingConf,
"-outputPath", workingDir.toString() + "/",
"-baseURL", "https://services.openaire.eu/openaire/community/",
"-pathMap", pathMap
});
@ -383,6 +400,71 @@ public class BulkTagJobTest {
.count());
}
@Test
void datasourceTag() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/")
.getPath();
LocalFileSystem fs = FileSystem.getLocal(new Configuration());
fs
.copyFromLocalFile(
false, new org.apache.hadoop.fs.Path(getClass()
.getResource("/eu/dnetlib/dhp/bulktag/pathMap/")
.getPath()),
new org.apache.hadoop.fs.Path(workingDir.toString() + "/data/bulktagging/protoMap"));
SparkBulkTagJob
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-sourcePath", sourcePath,
"-taggingConf", taggingConf,
"-outputPath", workingDir.toString() + "/",
"-baseURL", "https://services.openaire.eu/openaire/community/",
"-pathMap", workingDir.toString() + "/data/bulktagging/protoMap/pathMap",
"-nameNode", "local"
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Datasource> tmp = sc
.textFile(workingDir.toString() + "/datasource")
.map(item -> OBJECT_MAPPER.readValue(item, Datasource.class));
Assertions.assertEquals(3, tmp.count());
org.apache.spark.sql.Dataset<Datasource> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(Datasource.class));
verificationDataset.createOrReplaceTempView("datasource");
String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name "
+ "from datasource "
+ "lateral view explode(context) c as MyT "
+ "lateral view explode(MyT.datainfo) d as MyD "
+ "where MyD.inferenceprovenance = 'bulktagging'";
org.apache.spark.sql.Dataset<Row> idExplodeCommunity = spark.sql(query);
idExplodeCommunity.show(false);
Assertions.assertEquals(3, idExplodeCommunity.count());
Assertions
.assertEquals(
3, idExplodeCommunity.filter("provenance = 'community:datasource'").count());
Assertions
.assertEquals(
3,
idExplodeCommunity
.filter("name = 'Bulktagging for Community - Datasource'")
.count());
Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'dh-ch'").count());
Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'clarin'").count());
}
@Test
void bulktagByZenodoCommunityTest() throws Exception {
final String sourcePath = getClass()
@ -1699,4 +1781,40 @@ public class BulkTagJobTest {
}
@Test
public void prova() throws Exception {
LocalFileSystem fs = FileSystem.getLocal(new Configuration());
fs
.copyFromLocalFile(
false, new org.apache.hadoop.fs.Path(getClass()
.getResource("/eu/dnetlib/dhp/bulktag/pathMap/")
.getPath()),
new org.apache.hadoop.fs.Path(workingDir.toString() + "/data/bulktagging/protoMap"));
final String sourcePath = getClass()
.getResource(
"/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/")
.getPath();
ProtoMap prova = new Gson()
.fromJson(
"{\"author\":{\"path\":\"$['author'][]['fullname']\"},\"title\":{\"path\":\"$['title'][]['value']\"},\"orcid\":{\"path\":\"$['author'][]['pid'][][?(@['qualifier']['classid']=='orcid')]['value']\"},\"orcid_pending\":{\"path\":\"$['author'][]['pid'][][?(@['qualifier']['classid']=='orcid_pending')]['value']\"},\"contributor\":{\"path\":\"$['contributor'][]['value']\"},\"description\":{\"path\":\"$['description'][]['value']\"},\"subject\":{\"path\":\"$['subject'][]['value']\"},\"fos\":{\"path\":\"$['subject'][?(@['qualifier']['classid']=='FOS')].value\"},\"sdg\":{\"path\":\"$['subject'][?(@['qualifier']['classid']=='SDG')].value\"},\"journal\":{\"path\":\"$['journal'].name\"},\"hostedby\":{\"path\":\"$['instance'][]['hostedby']['key']\"},\"collectedfrom\":{\"path\":\"$['instance'][*]['collectedfrom']['key']\"},\"publisher\":{\"path\":\"$['publisher'].value\"},\"publicationyear\":{\"path\":\"$['dateofacceptance'].value\",\"action\":{\"clazz\":\"eu.dnetlib.dhp.bulktag.actions.ExecSubstringAction\",\"method\":\"execSubstring\",\"params\":[{\"paramName\":\"From\",\"paramValue\":0},{\"paramName\":\"To\",\"paramValue\":4}]}}}",
ProtoMap.class);
SparkBulkTagJob
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-sourcePath", sourcePath,
"-taggingConf", taggingConf,
"-outputPath", workingDir.toString() + "/",
"-pathMap", workingDir.toString() + "/data/bulktagging/protoMap/pathMap",
"-baseURL", "none",
"-nameNode", "local"
});
}
}

View File

@ -0,0 +1,58 @@
{
"author":{
"path":"$['author'][*]['fullname']"
},
"title":{
"path":"$['title'][*]['value']"
},
"orcid":{
"path":"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid')]['value']"
},
"orcid_pending":{
"path":"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid_pending')]['value']"
},
"contributor":{
"path":"$['contributor'][*]['value']"
},
"description":{
"path":"$['description'][*]['value']"
},
"subject":{
"path":"$['subject'][*]['value']"
},
"fos":{
"path":"$['subject'][?(@['qualifier']['classid']=='FOS')].value"
},
"sdg":{
"path":"$['subject'][?(@['qualifier']['classid']=='SDG')].value"
},
"journal":{
"path":"$['journal'].name"
},
"hostedby":{
"path":"$['instance'][*]['hostedby']['key']"
},
"collectedfrom":{
"path":"$['instance'][*]['collectedfrom']['key']"
},
"publisher":{
"path":"$['publisher'].value"
},
"publicationyear":{
"path":"$['dateofacceptance'].value",
"action":{
"clazz":"eu.dnetlib.dhp.bulktag.actions.ExecSubstringAction",
"method":"execSubstring",
"params":[
{
"paramName":"From",
"paramValue":0
},
{
"paramName":"To",
"paramValue":4
}
]
}
}
}

View File

@ -888,6 +888,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>
<dhp-schemas.version>[5.17.3]</dhp-schemas.version>
<dhp-schemas.version>[6.1.0]</dhp-schemas.version>
<dnet-actionmanager-api.version>[4.0.3]</dnet-actionmanager-api.version>
<dnet-actionmanager-common.version>[6.0.5]</dnet-actionmanager-common.version>