hostedbymap

pull/136/head
Miriam Baglioni 3 years ago
parent 3d2bba3d5d
commit baad01cadc

@ -38,8 +38,7 @@ import scala.Tuple2;
/**
* Groups the graph content by entity identifier to ensure ID uniqueness
*/
public class
GroupEntitiesSparkJob {
public class GroupEntitiesSparkJob {
private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class);

@ -245,6 +245,8 @@ object DoiBoostMappingUtil {
if (item != null) {
hb.setValue(item.officialname)
hb.setKey(generateDSId(item.id))
//TODO replace with the one above as soon as the new HBM will be used
//hb.setKey(item.id)
if (item.openAccess) {
i.setAccessright(getOpenAccessQualifier())
i.getAccessright.setOpenAccessRoute(OpenAccessRoute.gold)

@ -1,54 +0,0 @@
package eu.dnetlib.dhp.oa.graph.hostebymap
import org.apache.spark.sql.{Dataset, Encoder, Encoders, TypedColumn}
import org.apache.spark.sql.expressions.Aggregator
case class HostedByItemType(id: String, officialname: String, issn: String, eissn: String, lissn: String, openAccess: Boolean) {}
case class HostedByInfo(id: String, officialname: String, journal_id: String, provenance : String, id_type: String) {}
object Aggregators {
def getId(s1:String, s2:String) : String = {
if (!s1.equals("")){
return s1}
s2
}
def createHostedByItemTypes(df: Dataset[HostedByItemType]): Dataset[HostedByItemType] = {
val transformedData : Dataset[HostedByItemType] = df
.groupByKey(_.id)(Encoders.STRING)
.agg(Aggregators.hostedByAggregator)
.map{
case (id:String , res:HostedByItemType) => res
}(Encoders.product[HostedByItemType])
transformedData
}
val hostedByAggregator: TypedColumn[HostedByItemType, HostedByItemType] = new Aggregator[HostedByItemType, HostedByItemType, HostedByItemType] {
override def zero: HostedByItemType = HostedByItemType("","","","","",false)
override def reduce(b: HostedByItemType, a:HostedByItemType): HostedByItemType = {
return merge(b, a)
}
override def merge(b1: HostedByItemType, b2: HostedByItemType): HostedByItemType = {
if (b1 == null){
return b2
}
if(b2 == null){
return b1
}
HostedByItemType(getId(b1.id, b2.id), getId(b1.officialname, b2.officialname), getId(b1.issn, b2.issn), getId(b1.eissn, b2.eissn), getId(b1.lissn, b2.lissn), b1.openAccess || b2.openAccess)
}
override def finish(reduction: HostedByItemType): HostedByItemType = reduction
override def bufferEncoder: Encoder[HostedByItemType] = Encoders.product[HostedByItemType]
override def outputEncoder: Encoder[HostedByItemType] = Encoders.product[HostedByItemType]
}.toColumn
}

@ -1,15 +0,0 @@
package eu.dnetlib.dhp.oa.graph.hostebymap;
public class Constants {
public static final String OPENAIRE = "openaire";
public static final String DOAJ = "doaj";
public static final String UNIBI = "unibi";
public static final String ISSN = "issn";
public static final String EISSN = "eissn";
public static final String ISSNL = "issnl";
}

@ -1,111 +0,0 @@
package eu.dnetlib.dhp.oa.graph.hostebymap;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.opencsv.bean.CsvToBeanBuilder;
import eu.dnetlib.dhp.oa.graph.hostebymap.model.UnibiGoldModel;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.*;
import java.net.URL;
import java.net.URLConnection;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Optional;
public class GetCSV {
private static final Log log = LogFactory.getLog(eu.dnetlib.dhp.oa.graph.hostebymap.GetCSV.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
GetCSV.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/hostedbymap/download_csv_parameters.json")));
parser.parseArgument(args);
final String fileURL = parser.get("fileURL");
final String hdfsPath = parser.get("hdfsPath");
final String hdfsNameNode = parser.get("hdfsNameNode");
final String classForName = parser.get("classForName");
final Boolean shouldReplace = Optional.ofNullable((parser.get("replace")))
.map(Boolean::valueOf)
.orElse(false);
URLConnection connection = new URL(fileURL).openConnection();
connection.setRequestProperty("User-Agent", "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.11 (KHTML, like Gecko) Chrome/23.0.1271.95 Safari/537.11");
connection.connect();
BufferedReader in = new BufferedReader(new InputStreamReader(connection.getInputStream(), Charset.forName("UTF-8")));
if(shouldReplace){
PrintWriter writer = new PrintWriter(new BufferedWriter(new FileWriter("/tmp/DOAJ.csv")));
String line = null;
while((line = in.readLine())!= null){
writer.println(line.replace("\\\"", "\""));
}
writer.close();
in.close();
in = new BufferedReader(new FileReader("/tmp/DOAJ.csv"));
}
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
Path hdfsWritePath = new Path(hdfsPath);
FSDataOutputStream fsDataOutputStream = null;
if (fileSystem.exists(hdfsWritePath)) {
fileSystem.delete(hdfsWritePath, false);
}
fsDataOutputStream = fileSystem.create(hdfsWritePath);
BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8));
Class<?> clazz = Class.forName(classForName);
ObjectMapper mapper = new ObjectMapper();
new CsvToBeanBuilder(in)
.withType(clazz)
.withMultilineLimit(1)
.build()
.parse()
.forEach(line -> {
try {
writer.write(mapper.writeValueAsString(line));
writer.newLine();
} catch (IOException e) {
throw new RuntimeException(e);
}
});
writer.close();
in.close();
if(shouldReplace){
File f = new File("/tmp/DOAJ.csv");
f.delete();
}
}
}

@ -1,53 +0,0 @@
package eu.dnetlib.dhp.oa.graph.hostebymap.model;
import java.io.Serializable;
import com.opencsv.bean.CsvBindByName;
public class DOAJModel implements Serializable {
@CsvBindByName(column = "Journal title")
private String journalTitle;
@CsvBindByName(column = "Journal ISSN (print version)")
private String issn ;
@CsvBindByName(column = "Journal EISSN (online version)")
private String eissn;
@CsvBindByName(column = "Review process")
private String reviewProcess;
public String getJournalTitle() {
return journalTitle;
}
public void setJournalTitle(String journalTitle) {
this.journalTitle = journalTitle;
}
public String getIssn() {
return issn;
}
public void setIssn(String issn) {
this.issn = issn;
}
public String getEissn() {
return eissn;
}
public void setEissn(String eissn) {
this.eissn = eissn;
}
public String getReviewProcess() {
return reviewProcess;
}
public void setReviewProcess(String reviewProcess) {
this.reviewProcess = reviewProcess;
}
}

@ -1,44 +0,0 @@
package eu.dnetlib.dhp.oa.graph.hostebymap.model;
import com.opencsv.bean.CsvBindByName;
import java.io.Serializable;
public class UnibiGoldModel implements Serializable {
@CsvBindByName(column = "ISSN")
private String issn;
@CsvBindByName(column = "ISSN_L")
private String issn_l;
@CsvBindByName(column = "TITLE")
private String title;
@CsvBindByName(column = "TITLE_SOURCE")
private String title_source;
public String getIssn() {
return issn;
}
public void setIssn(String issn) {
this.issn = issn;
}
public String getIssn_l() {
return issn_l;
}
public String getTitle() {
return title;
}
public void setTitle(String title) {
this.title = title;
}
public String getTitle_source() {
return title_source;
}
public void setTitle_source(String title_source) {
this.title_source = title_source;
}
}

@ -0,0 +1,97 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap
import org.apache.spark.sql.{Dataset, Encoder, Encoders, TypedColumn}
import org.apache.spark.sql.expressions.Aggregator
case class HostedByItemType(id: String, officialname: String, issn: String, eissn: String, lissn: String, openAccess: Boolean) {}
case class HostedByInfo(id: String, officialname: String, journal_id: String, provenance : String, id_type: String) {}
object Aggregators {
def getId(s1:String, s2:String) : String = {
if (s1.startsWith("10|")){
return s1}
s2
}
def getValue(s1:String, s2:String) : String = {
if(!s1.equals("")){
return s1
}
s2
}
def createHostedByItemTypes(df: Dataset[HostedByItemType]): Dataset[HostedByItemType] = {
val transformedData : Dataset[HostedByItemType] = df
.groupByKey(_.id)(Encoders.STRING)
.agg(Aggregators.hostedByAggregator)
.map{
case (id:String , res:HostedByItemType) => res
}(Encoders.product[HostedByItemType])
transformedData
}
val hostedByAggregator: TypedColumn[HostedByItemType, HostedByItemType] = new Aggregator[HostedByItemType, HostedByItemType, HostedByItemType] {
override def zero: HostedByItemType = HostedByItemType("","","","","",false)
override def reduce(b: HostedByItemType, a:HostedByItemType): HostedByItemType = {
return merge(b, a)
}
override def merge(b1: HostedByItemType, b2: HostedByItemType): HostedByItemType = {
if (b1 == null){
return b2
}
if(b2 == null){
return b1
}
HostedByItemType(getId(b1.id, b2.id), getId(b1.officialname, b2.officialname), getId(b1.issn, b2.issn), getId(b1.eissn, b2.eissn), getId(b1.lissn, b2.lissn), b1.openAccess || b2.openAccess)
}
override def finish(reduction: HostedByItemType): HostedByItemType = reduction
override def bufferEncoder: Encoder[HostedByItemType] = Encoders.product[HostedByItemType]
override def outputEncoder: Encoder[HostedByItemType] = Encoders.product[HostedByItemType]
}.toColumn
def explodeHostedByItemType(df: Dataset[(String, HostedByItemType)]): Dataset[(String, HostedByItemType)] = {
val transformedData : Dataset[(String, HostedByItemType)] = df
.groupByKey(_._1)(Encoders.STRING)
.agg(Aggregators.hostedByAggregator1)
.map{
case (id:String , res:(String, HostedByItemType)) => res
}(Encoders.tuple(Encoders.STRING, Encoders.product[HostedByItemType]))
transformedData
}
val hostedByAggregator1: TypedColumn[(String, HostedByItemType), (String, HostedByItemType)] = new Aggregator[(String, HostedByItemType), (String, HostedByItemType), (String, HostedByItemType)] {
override def zero: (String, HostedByItemType) = ("", HostedByItemType("","","","","",false))
override def reduce(b: (String, HostedByItemType), a:(String,HostedByItemType)): (String, HostedByItemType) = {
return merge(b, a)
}
override def merge(b1: (String, HostedByItemType), b2: (String, HostedByItemType)): (String, HostedByItemType) = {
if (b1 == null){
return b2
}
if(b2 == null){
return b1
}
if(b1._2.id.startsWith("10|")){
return (b1._1, HostedByItemType(b1._2.id, b1._2.officialname, b1._2.issn, b1._2.eissn, b1._2.lissn, b1._2.openAccess || b2._2.openAccess))
}
return (b2._1, HostedByItemType(b2._2.id, b2._2.officialname, b2._2.issn, b2._2.eissn, b2._2.lissn, b1._2.openAccess || b2._2.openAccess))
}
override def finish(reduction: (String,HostedByItemType)): (String, HostedByItemType) = reduction
override def bufferEncoder: Encoder[(String,HostedByItemType)] = Encoders.tuple(Encoders.STRING,Encoders.product[HostedByItemType])
override def outputEncoder: Encoder[(String,HostedByItemType)] = Encoders.tuple(Encoders.STRING,Encoders.product[HostedByItemType])
}.toColumn
}

@ -0,0 +1,13 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap;
public class Constants {
public static final String OPENAIRE = "openaire";
public static final String DOAJ = "doaj";
public static final String UNIBI = "unibi";
public static final String ISSN = "issn";
public static final String EISSN = "eissn";
public static final String ISSNL = "issnl";
}

@ -0,0 +1,107 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap;
import java.io.*;
import java.net.URL;
import java.net.URLConnection;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.opencsv.bean.CsvToBeanBuilder;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
public class GetCSV {
private static final Log log = LogFactory.getLog(eu.dnetlib.dhp.oa.graph.hostedbymap.GetCSV.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
GetCSV.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/hostedbymap/download_csv_parameters.json")));
parser.parseArgument(args);
final String fileURL = parser.get("fileURL");
final String hdfsPath = parser.get("workingPath");
final String hdfsNameNode = parser.get("hdfsNameNode");
final String classForName = parser.get("classForName");
final Boolean shouldReplace = Optional
.ofNullable((parser.get("replace")))
.map(Boolean::valueOf)
.orElse(false);
URLConnection connection = new URL(fileURL).openConnection();
connection
.setRequestProperty(
"User-Agent",
"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.11 (KHTML, like Gecko) Chrome/23.0.1271.95 Safari/537.11");
connection.connect();
BufferedReader in = new BufferedReader(
new InputStreamReader(connection.getInputStream(), Charset.forName("UTF-8")));
if (shouldReplace) {
PrintWriter writer = new PrintWriter(new BufferedWriter(new FileWriter("/tmp/DOAJ.csv")));
String line = null;
while ((line = in.readLine()) != null) {
writer.println(line.replace("\\\"", "\""));
}
writer.close();
in.close();
in = new BufferedReader(new FileReader("/tmp/DOAJ.csv"));
}
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
Path hdfsWritePath = new Path(hdfsPath);
FSDataOutputStream fsDataOutputStream = null;
if (fileSystem.exists(hdfsWritePath)) {
fileSystem.delete(hdfsWritePath, false);
}
fsDataOutputStream = fileSystem.create(hdfsWritePath);
BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8));
Class<?> clazz = Class.forName(classForName);
ObjectMapper mapper = new ObjectMapper();
new CsvToBeanBuilder(in)
.withType(clazz)
.withMultilineLimit(1)
.build()
.parse()
.forEach(line -> {
try {
writer.write(mapper.writeValueAsString(line));
writer.newLine();
} catch (IOException e) {
throw new RuntimeException(e);
}
});
writer.close();
in.close();
if (shouldReplace) {
File f = new File("/tmp/DOAJ.csv");
f.delete();
}
}
}

@ -1,17 +1,23 @@
package eu.dnetlib.dhp.oa.graph.hostebymap
package eu.dnetlib.dhp.oa.graph.hostedbymap
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.oa.graph.hostebymap.model.{DOAJModel, UnibiGoldModel}
import eu.dnetlib.dhp.schema.oaf.{Datasource}
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.{DOAJModel, UnibiGoldModel}
import eu.dnetlib.dhp.schema.oaf.Datasource
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.json4s.DefaultFormats
import org.slf4j.{Logger, LoggerFactory}
import com.fasterxml.jackson.databind.ObjectMapper
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.fs.Path
import java.io.PrintWriter
import org.apache.hadoop.io.compress.GzipCodec
object SparkPrepareHostedByMapData {
object SparkProduceHostedByMap {
implicit val tupleForJoinEncoder: Encoder[(String, HostedByItemType)] = Encoders.tuple(Encoders.STRING, Encoders.product[HostedByItemType])
@ -37,24 +43,32 @@ object SparkPrepareHostedByMapData {
}
}
// def toHostedByMap(input: HostedByItemType): ListBuffer[String] = {
// implicit val formats = DefaultFormats
// val serializedJSON:String = write(input)
//
// var hostedBy = new ListBuffer[String]()
// if(!input.issn.equals("")){
// hostedBy += "{\"" + input.issn + "\":" + serializedJSON + "}"
// }
// if(!input.eissn.equals("")){
// hostedBy += "{\"" + input.eissn + "\":" + serializedJSON + "}"
// }
// if(!input.lissn.equals("")){
// hostedBy += "{\"" + input.lissn + "\":" + serializedJSON + "}"
// }
//
// hostedBy
//
// }
def toHostedByMap(input: (String, HostedByItemType)): String = {
import org.json4s.jackson.Serialization
implicit val formats = org.json4s.DefaultFormats
val map: Map [String, HostedByItemType] = Map (input._1 -> input._2 )
Serialization.write(map)
}
/**
*
def toHostedByMap(input: Map[String, HostedByItemType]): String = {
import org.json4s.jackson.Serialization
implicit val formats = org.json4s.DefaultFormats
Serialization.write(input)
}
*/
def getHostedByItemType(id:String, officialname: String, issn:String, eissn:String, issnl:String, oa:Boolean): HostedByItemType = {
if(issn != null){
@ -166,11 +180,31 @@ object SparkPrepareHostedByMapData {
}
def writeToHDFS(input: Array[String], outputPath: String, hdfsNameNode : String):Unit = {
val conf = new Configuration()
conf.set("fs.defaultFS", hdfsNameNode)
val fs= FileSystem.get(conf)
val output = fs.create(new Path(outputPath))
val writer = new PrintWriter(output)
try {
input.foreach(hbi => writer.println(hbi))
}
finally {
writer.close()
}
}
def main(args: Array[String]): Unit = {
val logger: Logger = LoggerFactory.getLogger(getClass)
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/hostedby/prepare_hostedby_params.json")))
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/hostedbymap/hostedby_params.json")))
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
@ -179,11 +213,10 @@ object SparkPrepareHostedByMapData {
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
import spark.implicits._
val datasourcePath = parser.get("datasourcePath")
val workingDirPath = parser.get("workingPath")
val outputPath = parser.get("outputPath")
implicit val formats = DefaultFormats
@ -191,29 +224,15 @@ object SparkPrepareHostedByMapData {
logger.info("Getting the Datasources")
// val doajDataset: Dataset[DOAJModel] = spark.read.textFile(workingDirPath + "/doaj").as[DOAJModel]
val dats : Dataset[HostedByItemType] =
oaHostedByDataset(spark, datasourcePath)
Aggregators.explodeHostedByItemType(oaHostedByDataset(spark, datasourcePath)
.union(goldHostedByDataset(spark, workingDirPath + "/unibi_gold"))
.union(doajHostedByDataset(spark, workingDirPath + "/doaj"))
dats.flatMap(hbi => toList(hbi))
.groupByKey(_._1)
//
//
.flatMap(hbi => toList(hbi))).filter(hbi => hbi._2.id.startsWith("10|"))
.map(hbi => toHostedByMap(hbi))(Encoders.STRING)
.rdd.saveAsTextFile(outputPath + "/hostedByMap", classOf[GzipCodec])
//
//
// Aggregators.createHostedByItemTypes(oa.joinWith(doaj, oa.col("journal_id").equalTo(doaj.col("journal_id")), "left")
// .joinWith(gold, $"_1.col('journal_id')".equalTo(gold.col("journal_id")), "left").map(toHostedByItemType)
// .filter(i => i != null))
// .flatMap(toHostedByMap)
// .write.mode(SaveMode.Overwrite).save(s"$workingDirPath/HostedByMap")
//
//
}

@ -0,0 +1,52 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap.model;
import java.io.Serializable;
import com.opencsv.bean.CsvBindByName;
public class DOAJModel implements Serializable {
@CsvBindByName(column = "Journal title")
private String journalTitle;
@CsvBindByName(column = "Journal ISSN (print version)")
private String issn;
@CsvBindByName(column = "Journal EISSN (online version)")
private String eissn;
@CsvBindByName(column = "Review process")
private String reviewProcess;
public String getJournalTitle() {
return journalTitle;
}
public void setJournalTitle(String journalTitle) {
this.journalTitle = journalTitle;
}
public String getIssn() {
return issn;
}
public void setIssn(String issn) {
this.issn = issn;
}
public String getEissn() {
return eissn;
}
public void setEissn(String eissn) {
this.eissn = eissn;
}
public String getReviewProcess() {
return reviewProcess;
}
public void setReviewProcess(String reviewProcess) {
this.reviewProcess = reviewProcess;
}
}

@ -0,0 +1,45 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap.model;
import java.io.Serializable;
import com.opencsv.bean.CsvBindByName;
public class UnibiGoldModel implements Serializable {
@CsvBindByName(column = "ISSN")
private String issn;
@CsvBindByName(column = "ISSN_L")
private String issn_l;
@CsvBindByName(column = "TITLE")
private String title;
@CsvBindByName(column = "TITLE_SOURCE")
private String title_source;
public String getIssn() {
return issn;
}
public void setIssn(String issn) {
this.issn = issn;
}
public String getIssn_l() {
return issn_l;
}
public String getTitle() {
return title;
}
public void setTitle(String title) {
this.title = title;
}
public String getTitle_source() {
return title_source;
}
public void setTitle_source(String title_source) {
this.title_source = title_source;
}
}

@ -0,0 +1,37 @@
[
{
"paramName":"fu",
"paramLongName":"fileURL",
"paramDescription": "the url to download the csv file ",
"paramRequired": true
},
{
"paramName":"wp",
"paramLongName":"workingPath",
"paramDescription": "the path where to find the pre-processed data for unibi gold list and doj artciles",
"paramRequired": true
},
{
"paramName": "hnn",
"paramLongName": "hdfsNameNode",
"paramDescription": "the path used to store the HostedByMap",
"paramRequired": true
},
{
"paramName": "cfn",
"paramLongName": "classForName",
"paramDescription": "true if the spark session is managed, false otherwise",
"paramRequired": true
},
{
"paramName": "sr",
"paramLongName": "replace",
"paramDescription": "true if the input file has to be cleaned before parsing",
"paramRequired": false
}
]

@ -0,0 +1,38 @@
[
{
"paramName":"dsp",
"paramLongName":"datasourcePath",
"paramDescription": "the path to the datasource ",
"paramRequired": true
},
{
"paramName":"wp",
"paramLongName":"workingPath",
"paramDescription": "the path where to find the pre-processed data for unibi gold list and doj artciles",
"paramRequired": true
},
{
"paramName": "out",
"paramLongName": "outputPath",
"paramDescription": "the path used to store the HostedByMap",
"paramRequired": true
},
{
"paramName": "ssm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "true if the spark session is managed, false otherwise",
"paramRequired": false
},
{
"paramName": "m",
"paramLongName": "master",
"paramDescription": "true if the spark session is managed, false otherwise",
"paramRequired": true
}
]

@ -0,0 +1,30 @@
<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>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

@ -0,0 +1,148 @@
<workflow-app name="hosted_by_map" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>outputPath</name>
<description>the output path</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="reset_outputpath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="reset_outputpath">
<fs>
<delete path="${outputPath}"/>
<mkdir path="${outputPath}"/>
</fs>
<ok to="fork_downloads_csv"/>
<error to="Kill"/>
</action>
<fork name="fork_downloads_csv">
<path start="download_gold"/>
<path start="download_doaj"/>
</fork>
<action name="download_gold">
<java>
<main-class>eu.dnetlib.dhp.oa.graph.hostedbymap.GetCSV</main-class>
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
<arg>--fileURL</arg><arg>${unibiFileURL}</arg>
<arg>--workingPath</arg><arg>${workingDir}/unibi_gold</arg>
<arg>--classForName</arg><arg>eu.dnetlib.dhp.oa.graph.hostedbymap.model.UnibiGoldModel</arg>
</java>
<ok to="join_download"/>
<error to="Kill"/>
</action>
<action name="download_doaj">
<java>
<main-class>eu.dnetlib.dhp.oa.graph.hostedbymap.GetCSV</main-class>
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
<arg>--fileURL</arg><arg>${doajFileURL}</arg>
<arg>--workingPath</arg><arg>${workingDir}/doaj</arg>
<arg>--classForName</arg><arg>eu.dnetlib.dhp.oa.graph.hostedbymap.model.DOAJModel</arg>
<arg>--replace</arg><arg>true</arg>
</java>
<ok to="join_download"/>
<error to="Kill"/>
</action>
<join name="join_download" to="produceHBM"/>
<action name="produceHBM">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<name>Produce the new HostedByMap</name>
<class>eu.dnetlib.dhp.oa.graph.hostedbymap.SparkProduceHostedByMap</class>
<jar>dhp-graph-mapper-${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>--datasourcePath</arg><arg>${sourcePath}/datasource</arg>
<arg>--workingPath</arg><arg>${workingDir}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -1,19 +1,14 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap
import java.sql.Timestamp
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.oa.graph.hostebymap.{Constants, HostedByInfo, SparkPrepareHostedByMapData}
import eu.dnetlib.dhp.oa.graph.hostedbymap.{Aggregators, Constants, HostedByInfo, HostedByItemType, SparkProduceHostedByMap}
import eu.dnetlib.dhp.schema.oaf.Datasource
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.json4s.DefaultFormats
import org.junit.jupiter.api.Assertions.{assertNotNull, assertTrue}
import org.junit.jupiter.api.Test
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.mutable.ListBuffer
import scala.io.Source
import org.junit.jupiter.api.Assertions._
import org.json4s.jackson.Serialization.write
class TestPreprocess extends java.io.Serializable{
@ -21,19 +16,14 @@ class TestPreprocess extends java.io.Serializable{
implicit val schema = Encoders.product[HostedByInfo]
@Test
def readDatasource():Unit = {
import org.apache.spark.sql.Encoders
def toHBIString (hbi:HostedByItemType): String = {
implicit val formats = DefaultFormats
val logger: Logger = LoggerFactory.getLogger(getClass)
val mapper = new ObjectMapper()
write(hbi)
}
@Test
def readDatasource():Unit = {
val conf = new SparkConf()
conf.setMaster("local[*]")
conf.set("spark.driver.host", "localhost")
@ -45,25 +35,29 @@ class TestPreprocess extends java.io.Serializable{
.getOrCreate()
val path = getClass.getResource("datasource.json").getPath
val ds :Dataset[HostedByItemType]= SparkProduceHostedByMap.oaHostedByDataset(spark, path)
println(SparkPrepareHostedByMapData.oaHostedByDataset(spark, path).count)
assertEquals(9, ds.count)
assertEquals(8, ds.filter(hbi => !hbi.issn.equals("")).count)
assertEquals(5, ds.filter(hbi => !hbi.eissn.equals("")).count)
assertEquals(0, ds.filter(hbi => !hbi.lissn.equals("")).count)
assertEquals(0, ds.filter(hbi => hbi.issn.equals("") && hbi.eissn.equals("") && hbi.lissn.equals("")).count)
assertTrue(ds.filter(hbi => hbi.issn.equals("0212-8365")).count == 1)
assertTrue(ds.filter(hbi => hbi.eissn.equals("2253-900X")).count == 1)
assertTrue(ds.filter(hbi => hbi.issn.equals("0212-8365") && hbi.eissn.equals("2253-900X")).count == 1)
assertTrue(ds.filter(hbi => hbi.issn.equals("0212-8365") && hbi.officialname.equals("Thémata")).count == 1)
assertTrue(ds.filter(hbi => hbi.issn.equals("0212-8365") && hbi.id.equals("10|doajarticles::abbc9265bea9ff62776a1c39785af00c")).count == 1)
ds.foreach(hbi => assertTrue(hbi.id.startsWith("10|")))
ds.foreach(hbi => println(toHBIString(hbi)))
spark.close()
}
@Test
def readGold():Unit = {
implicit val formats = DefaultFormats
val logger: Logger = LoggerFactory.getLogger(getClass)
val mapper = new ObjectMapper()
val conf = new SparkConf()
conf.setMaster("local[*]")
conf.set("spark.driver.host", "localhost")
@ -76,22 +70,60 @@ class TestPreprocess extends java.io.Serializable{
val path = getClass.getResource("unibi_transformed.json").getPath
println(SparkPrepareHostedByMapData.goldHostedByDataset(spark, path).count)
val ds :Dataset[HostedByItemType]= SparkProduceHostedByMap.goldHostedByDataset(spark, path)
assertEquals(29, ds.count)
assertEquals(29, ds.filter(hbi => !hbi.issn.equals("")).count)
assertEquals(0, ds.filter(hbi => !hbi.eissn.equals("")).count)
assertEquals(29, ds.filter(hbi => !hbi.lissn.equals("")).count)
assertEquals(0, ds.filter(hbi => hbi.issn.equals("") && hbi.eissn.equals("") && hbi.lissn.equals("")).count)
assertTrue(ds.filter(hbi => hbi.issn.equals("2239-6101")).first().officialname.equals("European journal of sustainable development."))
assertTrue(ds.filter(hbi => hbi.issn.equals("2239-6101")).first().lissn.equals("2239-5938"))
assertTrue(ds.filter(hbi => hbi.issn.equals("2239-6101")).count == 1)
ds.foreach(hbi => assertTrue(hbi.id.equals(Constants.UNIBI)))
ds.foreach(hbi => println(toHBIString(hbi)))
spark.close()
}
@Test
def readDoaj():Unit = {
val conf = new SparkConf()
conf.setMaster("local[*]")
conf.set("spark.driver.host", "localhost")
val spark: SparkSession =
SparkSession
.builder()
.appName(getClass.getSimpleName)
.config(conf)
.getOrCreate()
val path = getClass.getResource("doaj_transformed.json").getPath
implicit val formats = DefaultFormats
val ds :Dataset[HostedByItemType]= SparkProduceHostedByMap.doajHostedByDataset(spark, path)
assertEquals(25, ds.count)
assertEquals(14, ds.filter(hbi => !hbi.issn.equals("")).count)
assertEquals(21, ds.filter(hbi => !hbi.eissn.equals("")).count)
assertEquals(0, ds.filter(hbi => !hbi.lissn.equals("")).count)
val logger: Logger = LoggerFactory.getLogger(getClass)
val mapper = new ObjectMapper()
assertEquals(0, ds.filter(hbi => hbi.issn.equals("") && hbi.eissn.equals("") && hbi.lissn.equals("")).count)
assertTrue(ds.filter(hbi => hbi.issn.equals("2077-3099")).first().officialname.equals("Journal of Space Technology"))
assertTrue(ds.filter(hbi => hbi.issn.equals("2077-3099")).first().eissn.equals("2411-5029"))
assertTrue(ds.filter(hbi => hbi.issn.equals("2077-3099")).count == 1)
assertTrue(ds.filter(hbi => hbi.eissn.equals("2077-2955")).first().issn.equals(""))
ds.foreach(hbi => assertTrue(hbi.id.equals(Constants.DOAJ)))
ds.foreach(hbi => println(toHBIString(hbi)))
spark.close()
}
@Test
def testAggregator() : Unit = {
val conf = new SparkConf()
conf.setMaster("local[*]")
@ -102,14 +134,35 @@ class TestPreprocess extends java.io.Serializable{
.appName(getClass.getSimpleName)
.config(conf)
.getOrCreate()
val path = getClass.getResource("doaj_transformed.json").getPath
println(SparkPrepareHostedByMapData.doajHostedByDataset(spark, path).count)
val tmp = SparkProduceHostedByMap.oaHostedByDataset(spark, getClass.getResource("datasource.json").getPath)
.union(SparkProduceHostedByMap.goldHostedByDataset(spark,getClass.getResource("unibi_transformed.json").getPath))
.union(SparkProduceHostedByMap.doajHostedByDataset(spark, getClass.getResource("doaj_transformed.json").getPath))
.flatMap(hbi => SparkProduceHostedByMap.toList(hbi))(Encoders.tuple(Encoders.STRING, Encoders.product[HostedByItemType]))
assertEquals(106, tmp.count)
assertEquals(82, tmp.map(i => i._1)(Encoders.STRING).distinct().count)
val ds :Dataset[(String, HostedByItemType)] = Aggregators.explodeHostedByItemType(SparkProduceHostedByMap.oaHostedByDataset(spark, getClass.getResource("datasource.json").getPath)
.union(SparkProduceHostedByMap.goldHostedByDataset(spark,getClass.getResource("unibi_transformed.json").getPath))
.union(SparkProduceHostedByMap.doajHostedByDataset(spark, getClass.getResource("doaj_transformed.json").getPath))
.flatMap(hbi => SparkProduceHostedByMap.toList(hbi))(Encoders.tuple(Encoders.STRING, Encoders.product[HostedByItemType])))
assertEquals(82, ds.count)
assertEquals(13, ds.filter(i => i._2.id.startsWith("10|")).count)
assertTrue(ds.filter(i => i._1.equals("2077-3757")).first()._2.id.startsWith("10|"))
assertTrue(ds.filter(i => i._1.equals("2077-3757")).first()._2.openAccess)
assertEquals(1, ds.filter(i => i._1.equals("2077-3757")).count)
val hbmap : Dataset[String] = ds.filter(hbi => hbi._2.id.startsWith("10|")).map(SparkProduceHostedByMap.toHostedByMap)(Encoders.STRING)
hbmap.foreach(entry => println(entry))
spark.close()
}

@ -1,111 +1,109 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.opencsv.bean.CsvToBeanBuilder;
import eu.dnetlib.dhp.oa.graph.hostebymap.GetCSV;
import eu.dnetlib.dhp.oa.graph.hostebymap.model.UnibiGoldModel;
import org.junit.jupiter.api.Test;
package eu.dnetlib.dhp.oa.graph.hostedbymap;
import java.io.*;
import java.net.MalformedURLException;
import java.net.URL;
import java.net.URLConnection;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.List;
public class TestReadCSV {
@Test
public void testCSVUnibi() throws FileNotFoundException {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/hostedbymap/unibiGold.csv")
.getPath();
List<UnibiGoldModel> beans = new CsvToBeanBuilder(new FileReader(sourcePath))
.withType(UnibiGoldModel.class)
.build()
.parse();
ObjectMapper mapper = new ObjectMapper();
beans.forEach(r -> {
try {
System.out.println(mapper.writeValueAsString(r));
} catch (JsonProcessingException e) {
e.printStackTrace();
}
});
}
@Test
public void testCSVUrlUnibi() throws IOException {
URL csv = new URL("https://pub.uni-bielefeld.de/download/2944717/2944718/issn_gold_oa_version_4.csv");
BufferedReader in = new BufferedReader(new InputStreamReader(csv.openStream()));
ObjectMapper mapper = new ObjectMapper();
new CsvToBeanBuilder(in)
.withType(eu.dnetlib.dhp.oa.graph.hostebymap.model.UnibiGoldModel.class)
.build()
.parse()
.forEach(line ->
{
try {
System.out.println(mapper.writeValueAsString(line));
} catch (JsonProcessingException e) {
e.printStackTrace();
}
}
);
}
@Test
public void testCSVUrlDOAJ() throws IOException {
URLConnection connection = new URL("https://doaj.org/csv").openConnection();
connection.setRequestProperty("User-Agent", "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.11 (KHTML, like Gecko) Chrome/23.0.1271.95 Safari/537.11");
connection.connect();
BufferedReader in = new BufferedReader(new InputStreamReader(connection.getInputStream(), Charset.forName("UTF-8")));
//BufferedReader in = new BufferedReader(new FileReader("/tmp/DOAJ.csv"));
PrintWriter writer = new PrintWriter(new BufferedWriter(new FileWriter("/tmp/DOAJ_1.csv")));
String line = null;
while((line = in.readLine())!= null){
writer.println(line.replace("\\\"", "\""));
}
writer.close();
in.close();
in = new BufferedReader(new FileReader("/tmp/DOAJ_1.csv"));
ObjectMapper mapper = new ObjectMapper();
new CsvToBeanBuilder(in)
.withType(eu.dnetlib.dhp.oa.graph.hostebymap.model.DOAJModel.class)
.withMultilineLimit(1)
.build()
.parse()
.forEach(lline ->
{
try {
System.out.println(mapper.writeValueAsString(lline));
} catch (JsonProcessingException e) {
e.printStackTrace();
}
}
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.opencsv.bean.CsvToBeanBuilder;
);
}
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.UnibiGoldModel;
public class TestReadCSV {
@Test
public void testCSVUnibi() throws FileNotFoundException {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/hostedbymap/unibiGold.csv")
.getPath();
List<UnibiGoldModel> beans = new CsvToBeanBuilder(new FileReader(sourcePath))
.withType(UnibiGoldModel.class)
.build()
.parse();
ObjectMapper mapper = new ObjectMapper();
beans.forEach(r -> {
try {
System.out.println(mapper.writeValueAsString(r));
} catch (JsonProcessingException e) {
e.printStackTrace();
}
});
}
@Test
public void testCSVUrlUnibi() throws IOException {
URL csv = new URL("https://pub.uni-bielefeld.de/download/2944717/2944718/issn_gold_oa_version_4.csv");
BufferedReader in = new BufferedReader(new InputStreamReader(csv.openStream()));
ObjectMapper mapper = new ObjectMapper();
new CsvToBeanBuilder(in)
.withType(eu.dnetlib.dhp.oa.graph.hostedbymap.model.UnibiGoldModel.class)
.build()
.parse()
.forEach(line ->
{
try {
System.out.println(mapper.writeValueAsString(line));
} catch (JsonProcessingException e) {
e.printStackTrace();
}
}
);
}
@Test
public void testCSVUrlDOAJ() throws IOException {
URLConnection connection = new URL("https://doaj.org/csv").openConnection();
connection
.setRequestProperty(
"User-Agent",
"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.11 (KHTML, like Gecko) Chrome/23.0.1271.95 Safari/537.11");
connection.connect();
BufferedReader in = new BufferedReader(
new InputStreamReader(connection.getInputStream(), Charset.forName("UTF-8")));
// BufferedReader in = new BufferedReader(new FileReader("/tmp/DOAJ.csv"));
PrintWriter writer = new PrintWriter(new BufferedWriter(new FileWriter("/tmp/DOAJ_1.csv")));
String line = null;
while ((line = in.readLine()) != null) {
writer.println(line.replace("\\\"", "\""));
}
writer.close();
in.close();
in = new BufferedReader(new FileReader("/tmp/DOAJ_1.csv"));
ObjectMapper mapper = new ObjectMapper();
new CsvToBeanBuilder(in)
.withType(eu.dnetlib.dhp.oa.graph.hostedbymap.model.DOAJModel.class)
.withMultilineLimit(1)
.build()
.parse()
.forEach(lline ->
{
try {
System.out.println(mapper.writeValueAsString(lline));
} catch (JsonProcessingException e) {
e.printStackTrace();
}
}
);
}
}

@ -1,13 +1,13 @@
package eu.dnetlib.dhp.oa.graph.raw;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.oa.graph.clean.GraphCleaningFunctionsTest;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import static org.junit.jupiter.api.Assertions.*;
import static org.mockito.Mockito.lenient;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.junit.jupiter.api.BeforeEach;
@ -16,12 +16,14 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import com.fasterxml.jackson.databind.ObjectMapper;
import static org.junit.jupiter.api.Assertions.*;
import static org.mockito.Mockito.lenient;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.oa.graph.clean.GraphCleaningFunctionsTest;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
@ExtendWith(MockitoExtension.class)
public class MappersTest {
@ -340,7 +342,7 @@ public class MappersTest {
assertEquals(2, p.getOriginalId().size());
assertTrue(p.getOriginalId().stream().anyMatch(oid -> oid.equals("oai:pub.uni-bielefeld.de:2949739")));
//assertEquals("oai:pub.uni-bielefeld.de:2949739", p.getOriginalId().get(0));
// assertEquals("oai:pub.uni-bielefeld.de:2949739", p.getOriginalId().get(0));
assertValidId(p.getCollectedfrom().get(0).getKey());
assertTrue(p.getAuthor().size() > 0);

@ -0,0 +1,9 @@
{"id":"10|doajarticles::0ab37b7620eb9a73ac95d3ca4320c97d","officialname":"Известия высших учебных заведений: Проблемы энергетики","issn":"1998-9903","eissn":"","lissn":"","openAccess":false}
{"id":"10|doajarticles::abbc9265bea9ff62776a1c39785af00c","officialname":"Thémata","issn":"0212-8365","eissn":"2253-900X","lissn":"","openAccess":false}
{"id":"10|issn___print::051e86306840dc8255d95c5671e97928","officialname":"Science Technology & Public Policy","issn":"2640-4613","eissn":"","lissn":"","openAccess":false}
{"id":"10|issn___print::4b2e7f05b6353940e5a7a592f2a87c94","officialname":"Cahiers détudes germaniques","issn":"0751-4239","eissn":"2605-8359","lissn":"","openAccess":false}
{"id":"10|issn___print::4c950a72660642d69e767d1c2daad4a2","officialname":"Regional Economics Theory and Practice","issn":"2073-1477","eissn":"2311-8733","lissn":"","openAccess":false}
{"id":"10|issn___print::9241f8ebd40dd55cbb179028b84ebb12","officialname":"Transplantation","issn":"0041-1337","eissn":"","lissn":"","openAccess":false}
{"id":"10|issn___print::982b4d2537d3f800b596fbec3dae0c7c","officialname":"International Journal of Operations Research and Information Systems","issn":"1947-9328","eissn":"1947-9336","lissn":"","openAccess":false}
{"id":"10|issn___print::b9faf9c36c47169d4328e586eb62247c","officialname":"Bulletin of the British Mycological Society","issn":"0007-1528","eissn":"","lissn":"","openAccess":false}
{"id":"10|issn__online::709e633c2ecf46396a4ed1b0096da1d0","officialname":"Journal of Technology and Innovation","issn":"","eissn":"2410-3993","lissn":"","openAccess":false}

@ -0,0 +1,25 @@
{"id":"doaj","officialname":"Lëd i Sneg","issn":"2076-6734","eissn":"2412-3765","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Компьютерные исследования и моделирование","issn":"2076-7633","eissn":"2077-6853","lissn":"","openAccess":true}
{"id":"doaj","officialname":" Историко-биологические исследования","issn":"2076-8176","eissn":"2500-1221","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Інформаційні технології і засоби навчання","issn":"2076-8184","eissn":"","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Revue Internationale de Pédagogie de lEnseignement Supérieur","issn":"","eissn":"2076-8427","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Проблемы развития территории","issn":"2076-8915","eissn":"2409-9007","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Rambam Maimonides Medical Journal","issn":"","eissn":"2076-9172","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Membranes","issn":"2077-0375","eissn":"","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Journal of Clinical Medicine","issn":"","eissn":"2077-0383","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Agriculture","issn":"","eissn":"2077-0472","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Standartnye Obrazcy","issn":"2077-1177","eissn":"","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Металл и литье Украины","issn":"2077-1304","eissn":"2706-5529","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Journal of Marine Science and Engineering","issn":"","eissn":"2077-1312","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Religions","issn":"","eissn":"2077-1444","lissn":"","openAccess":true}
{"id":"doaj","officialname":"GW-Unterricht","issn":"2077-1517","eissn":"2414-4169","lissn":"","openAccess":true}
{"id":"doaj","officialname":"UCV-Scientia","issn":"2077-172X","eissn":"","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Sovremennye Issledovaniâ Socialʹnyh Problem","issn":"2077-1770","eissn":"2218-7405","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Granì","issn":"2077-1800","eissn":"2413-8738","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Journal of Economics Finance and Administrative Science","issn":"2077-1886","eissn":"2218-0648","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Science Education International","issn":"","eissn":"2077-2327","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Edumecentro","issn":"","eissn":"2077-2874","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Monteverdia","issn":"","eissn":"2077-2890","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Transformación","issn":"","eissn":"2077-2955","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Journal of Space Technology","issn":"2077-3099","eissn":"2411-5029","lissn":"","openAccess":true}
{"id":"doaj","officialname":"Revue de Primatologie","issn":"","eissn":"2077-3757","lissn":"","openAccess":true}

@ -0,0 +1,29 @@
{"id":"unibi","officialname":"JIMKESMAS (Jurnal Ilmiah Mahasiswa Kesehatan Masyarakat)","issn":"2502-731X","eissn":"","lissn":"2502-731X","openAccess":true}
{"id":"unibi","officialname":"Jurnal ilmu informasi, perpustakaan, dan kearsipan","issn":"2502-7409","eissn":"","lissn":"1411-0253","openAccess":true}
{"id":"unibi","officialname":"At-Tadbir : jurnal ilmiah manajemen","issn":"2502-7433","eissn":"","lissn":"2502-7433","openAccess":true}
{"id":"unibi","officialname":"Jurnal Kesehatan Panrita Husada.","issn":"2502-745X","eissn":"","lissn":"2502-745X","openAccess":true}
{"id":"unibi","officialname":"ELang journal (An English Education journal)","issn":"2502-7549","eissn":"","lissn":"2502-7549","openAccess":true}
{"id":"unibi","officialname":"̒Ulūm-i darmāngāhī-i dāmpizishkī-i Īrān.","issn":"2423-3633","eissn":"","lissn":"2423-3625","openAccess":true}
{"id":"unibi","officialname":"Pizhūhishnāmah-i ̒ilm/sanjī.","issn":"2423-5563","eissn":"","lissn":"2423-3773","openAccess":true}
{"id":"unibi","officialname":"Iranian journal of animal biosystematics.","issn":"1735-434X","eissn":"","lissn":"1735-434X","openAccess":true}
{"id":"unibi","officialname":"Majallah-i jangal-i Īrān.","issn":"2423-4435","eissn":"","lissn":"2008-6113","openAccess":true}
{"id":"unibi","officialname":"Ābziyān-i zinatī.","issn":"2423-4575","eissn":"","lissn":"2423-4575","openAccess":true}
{"id":"unibi","officialname":"Pizhūhishnāmah-i ravābiṭ-i biyn/al- milal.","issn":"2423-4974","eissn":"","lissn":"2423-4974","openAccess":true}
{"id":"unibi","officialname":"AIHM journal club.","issn":"2380-0607","eissn":"","lissn":"2380-0607","openAccess":true}
{"id":"unibi","officialname":"Frontiers.","issn":"1085-4568","eissn":"","lissn":"1085-4568","openAccess":true}
{"id":"unibi","officialname":"˜The œjournal of contemporary archival studies.","issn":"2380-8845","eissn":"","lissn":"2380-8845","openAccess":true}
{"id":"unibi","officialname":"International journal of complementary & alternative medicine.","issn":"2381-1803","eissn":"","lissn":"2381-1803","openAccess":true}
{"id":"unibi","officialname":"Palapala.","issn":"2381-2478","eissn":"","lissn":"2381-2478","openAccess":true}
{"id":"unibi","officialname":"Asia pacific journal of environment ecology and sustainable development.","issn":"2382-5170","eissn":"","lissn":"2382-5170","openAccess":true}
{"id":"unibi","officialname":"Majallah-i salāmat va bihdāsht","issn":"2382-9737","eissn":"","lissn":"2382-9737","openAccess":true}
{"id":"unibi","officialname":"UCT journal of research in science ,engineering and technology","issn":"2382-977X","eissn":"","lissn":"2382-977X","openAccess":true}
{"id":"unibi","officialname":"Bih/nizhādī-i giyāhān-i zirā̒ī va bāghī.","issn":"2382-9974","eissn":"","lissn":"2382-9974","openAccess":true}
{"id":"unibi","officialname":"Problemi endokrinnoï patologìï.","issn":"2227-4782","eissn":"","lissn":"2227-4782","openAccess":true}
{"id":"unibi","officialname":"Jurnal Kebijakan Pembangunan Daerah : Jurnal Penelitian dan Pengembangan Kebijakan Pembangunan Daerah.","issn":"2685-0079","eissn":"","lissn":"2597-4971","openAccess":true}
{"id":"unibi","officialname":"Hypermedia magazine.","issn":"2574-0075","eissn":"","lissn":"2574-0075","openAccess":true}
{"id":"unibi","officialname":"˜The œmuseum review.","issn":"2574-0296","eissn":"","lissn":"2574-0296","openAccess":true}
{"id":"unibi","officialname":"Bioactive compounds in health and disease.","issn":"2574-0334","eissn":"","lissn":"2574-0334","openAccess":true}
{"id":"unibi","officialname":"Journal of computer science integration.","issn":"2574-108X","eissn":"","lissn":"2574-108X","openAccess":true}
{"id":"unibi","officialname":"Child and adolescent obesity.","issn":"2574-254X","eissn":"","lissn":"2574-254X","openAccess":true}
{"id":"unibi","officialname":"Journal of research on the college president.","issn":"2574-3325","eissn":"","lissn":"2574-3325","openAccess":true}
{"id":"unibi","officialname":"European journal of sustainable development.","issn":"2239-6101","eissn":"","lissn":"2239-5938","openAccess":true}
Loading…
Cancel
Save