Changes in maven poms to build and test the project using Spark 3.4.x and scala 2.12 #327

Open
giambattista.bloisi wants to merge 17 commits from spark34-integration into beta
50 changed files with 2248 additions and 1099 deletions

1
.gitignore vendored
View File

@ -27,3 +27,4 @@ spark-warehouse
/**/.factorypath
/**/.scalafmt.conf
/.java-version
/dhp-shade-package/dependency-reduced-pom.xml

View File

@ -80,7 +80,15 @@ class WritePredefinedProjectPropertiesTest {
mojo.outputFile = testFolder;
// execute
Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute());
try {
mojo.execute();
Assertions.assertTrue(false); // not reached
} catch (Exception e) {
Assertions
.assertTrue(
MojoExecutionException.class.isAssignableFrom(e.getClass()) ||
IllegalArgumentException.class.isAssignableFrom(e.getClass()));
}
}
@Test

View File

@ -131,6 +131,12 @@
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>cnr-rmi-api</artifactId>
<exclusions>
<exclusion>
<artifactId>log4j</artifactId>
<groupId>log4j</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
@ -163,7 +169,7 @@
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>${dhp-schemas.artifact}</artifactId>
<artifactId>dhp-schemas</artifactId>
</dependency>
<dependency>

View File

@ -38,7 +38,7 @@ public class PacePerson {
PacePerson.class
.getResourceAsStream(
"/eu/dnetlib/dhp/common/name_particles.txt")));
} catch (IOException e) {
} catch (Exception e) {
throw new RuntimeException(e);
}
}

View File

@ -217,8 +217,6 @@ public class ZenodoAPIClient implements Serializable {
* part of the url for the DOI Zenodo suggests to use to cite all versions: DOI: 10.xxx/zenodo.656930
* concept_rec_id = 656930
* @return response code
* @throws IOException
* @throws MissingConceptDoiException
*/
public int newVersion(String concept_rec_id) throws IOException, MissingConceptDoiException {
setDepositionId(concept_rec_id, 1);

View File

@ -12,9 +12,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.math.NumberUtils;
import org.apache.commons.lang3.time.DateUtils;
import org.apache.http.HttpHeaders;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

View File

@ -1,19 +1,20 @@
package eu.dnetlib.pace.common;
import com.google.common.base.Splitter;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.ibm.icu.text.Transliterator;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import java.nio.charset.StandardCharsets;
import java.text.Normalizer;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import com.google.common.base.Splitter;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.ibm.icu.text.Transliterator;
/**
* Set of common functions for the framework
*

View File

@ -1,20 +1,21 @@
package eu.dnetlib.pace.model;
import java.nio.charset.Charset;
import java.text.Normalizer;
import java.util.List;
import java.util.Set;
import com.google.common.base.Joiner;
import com.google.common.base.Splitter;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.hash.Hashing;
import eu.dnetlib.pace.common.PaceCommonUtils;
import eu.dnetlib.pace.util.Capitalise;
import eu.dnetlib.pace.util.DotAbbreviations;
import java.nio.charset.Charset;
import java.text.Normalizer;
import java.util.List;
import java.util.Set;
public class Person {
private static final String UTF8 = "UTF-8";

View File

@ -1,9 +1,10 @@
package eu.dnetlib.pace.util;
import com.google.common.base.Function;
import org.apache.commons.lang3.text.WordUtils;
import com.google.common.base.Function;
public class Capitalise implements Function<String, String> {
private final char[] DELIM = {

View File

@ -154,5 +154,13 @@
"unknown":{
"original":"Unknown",
"inverse":"Unknown"
},
"isamongtopnsimilardocuments": {
"original": "IsAmongTopNSimilarDocuments",
"inverse": "HasAmongTopNSimilarDocuments"
},
"hasamongtopnsimilardocuments": {
"original": "HasAmongTopNSimilarDocuments",
"inverse": "IsAmongTopNSimilarDocuments"
}
}

View File

@ -65,12 +65,13 @@ abstract class AbstractScalaApplication(
val conf: SparkConf = new SparkConf()
val master = parser.get("master")
log.info(s"Creating Spark session: Master: $master")
SparkSession
val b = SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
.master(master)
.getOrCreate()
if (master != null)
b.master(master)
b.getOrCreate()
}
def reportTotalSize(targetPath: String, outputBasePath: String): Unit = {

View File

@ -65,7 +65,11 @@ object ScholixUtils extends Serializable {
}
def generateScholixResourceFromResult(r: Result): ScholixResource = {
generateScholixResourceFromSummary(ScholixUtils.resultToSummary(r))
val sum = ScholixUtils.resultToSummary(r)
if (sum != null)
generateScholixResourceFromSummary(ScholixUtils.resultToSummary(r))
else
null
}
val statsAggregator: Aggregator[(String, String, Long), RelatedEntities, RelatedEntities] =
@ -153,6 +157,14 @@ object ScholixUtils extends Serializable {
}
def invRel(rel: String): String = {
val semanticRelation = relations.getOrElse(rel.toLowerCase, null)
if (semanticRelation != null)
semanticRelation.inverse
else
null
}
def extractCollectedFrom(summary: ScholixResource): List[ScholixEntityId] = {
if (summary.getCollectedFrom != null && !summary.getCollectedFrom.isEmpty) {
val l: List[ScholixEntityId] = summary.getCollectedFrom.asScala.map { d =>
@ -377,10 +389,7 @@ object ScholixUtils extends Serializable {
if (persistentIdentifiers.isEmpty)
return null
s.setLocalIdentifier(persistentIdentifiers.asJava)
if (r.isInstanceOf[Publication])
s.setTypology(Typology.publication)
else
s.setTypology(Typology.dataset)
s.setTypology(r.getResulttype.getClassid)
s.setSubType(r.getInstance().get(0).getInstancetype.getClassname)

View File

@ -24,7 +24,7 @@
<executions>
<execution>
<id>scala-compile-first</id>
<phase>initialize</phase>
<phase>process-resources</phase>
<goals>
<goal>add-source</goal>
<goal>compile</goal>
@ -59,14 +59,6 @@
<groupId>edu.cmu</groupId>
<artifactId>secondstring</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>com.google.code.gson</groupId>
<artifactId>gson</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
@ -91,10 +83,6 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-math3</artifactId>
</dependency>
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
@ -113,4 +101,90 @@
</dependency>
</dependencies>
<profiles>
<profile>
<id>spark-24</id>
<activation>
<activeByDefault>true</activeByDefault>
</activation>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>3.4.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/spark-2</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>spark-34</id>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>3.4.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/spark-2</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>spark-35</id>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>3.4.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/spark-35</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -1,12 +1,6 @@
package eu.dnetlib.pace.common;
import com.google.common.base.Joiner;
import com.google.common.collect.Sets;
import com.ibm.icu.text.Transliterator;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import java.io.IOException;
import java.io.StringWriter;
import java.nio.charset.StandardCharsets;
@ -15,6 +9,13 @@ import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import com.google.common.base.Joiner;
import com.google.common.collect.Sets;
import com.ibm.icu.text.Transliterator;
/**
* Set of common functions for the framework
*

View File

@ -3,7 +3,7 @@ package eu.dnetlib.pace.model
import com.jayway.jsonpath.{Configuration, JsonPath}
import eu.dnetlib.pace.common.AbstractPaceFunctions
import eu.dnetlib.pace.config.{DedupConfig, Type}
import eu.dnetlib.pace.util.MapDocumentUtil
import eu.dnetlib.pace.util.{MapDocumentUtil, SparkCompatUtils}
import org.apache.commons.lang3.StringUtils
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
@ -52,7 +52,7 @@ case class SparkModel(conf: DedupConfig) {
val orderingFieldPosition: Int = schema.fieldIndex(orderingFieldName)
val parseJsonDataset: (Dataset[String] => Dataset[Row]) = df => {
df.map(r => rowFromJson(r))(RowEncoder(schema))
df.map(r => rowFromJson(r))(SparkCompatUtils.encoderFor(schema))
}
def rowFromJson(json: String): Row = {

View File

@ -0,0 +1,12 @@
package eu.dnetlib.pace.util
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder}
import org.apache.spark.sql.types.StructType
object SparkCompatUtils {
def encoderFor(schema: StructType): ExpressionEncoder[Row] = {
RowEncoder(schema)
}
}

View File

@ -0,0 +1,12 @@
package eu.dnetlib.pace.util
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.types.StructType
object SparkCompatUtils {
def encoderFor(schema: StructType): ExpressionEncoder[Row] = {
ExpressionEncoder(schema)
}
}

View File

@ -11,6 +11,7 @@ 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 {

169
dhp-shade-package/pom.xml Normal file
View File

@ -0,0 +1,169 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp</artifactId>
<version>1.2.5-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<artifactId>dhp-shade-package</artifactId>
<packaging>jar</packaging>
<distributionManagement>
<site>
<id>DHPSite</id>
<url>${dhp.site.stage.path}/dhp-common</url>
</site>
</distributionManagement>
<description>This module create a jar of all module dependencies</description>
<dependencies>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-actionmanager</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-aggregation</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-blacklist</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-broker-events</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-dedup-openaire</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-enrichment</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-graph-mapper</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-graph-provision</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-impact-indicators</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-stats-actionsets</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-stats-hist-snaps</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-stats-monitor-irish</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-stats-promote</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-stats-update</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-swh</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-usage-raw-data-update</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-usage-stats-build</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<transformers>
<transformer
implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
<mainClass>eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels</mainClass>
</transformer>
<!-- This is needed if you have dependencies that use Service Loader. Most Google Cloud client libraries do. -->
<transformer
implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
<transformer
implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>META-INF/cxf/bus-extensions.txt</resource>
</transformer>
</transformers>
<filters>
<filter>
<artifact>*:*</artifact>
<excludes>
<exclude>META-INF/maven/**</exclude>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
</excludes>
</filter>
</filters>
<relocations>
<relocation>
<pattern>com</pattern>
<shadedPattern>repackaged.com.google.common</shadedPattern>
<includes>
<include>com.google.common.**</include>
</includes>
</relocation>
</relocations>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View File

@ -9,6 +9,7 @@ import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.BZip2Codec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
@ -106,7 +107,7 @@ public class PrepareAffiliationRelations implements Serializable {
.union(openAPCRelations)
.union(dataciteRelations)
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class);
});
}

View File

@ -10,6 +10,7 @@ import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.BZip2Codec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
@ -83,7 +84,7 @@ public class SparkAtomicActionScoreJob implements Serializable {
resultsRDD
.union(projectsRDD)
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class);
});
}

View File

@ -1025,6 +1025,7 @@ case object Crossref2Oaf {
tp._1 match {
case "electronic" => journal.setIssnOnline(tp._2)
case "print" => journal.setIssnPrinted(tp._2)
case _ =>
}
})
}

View File

@ -2,12 +2,9 @@ package eu.dnetlib.dhp.sx.bio.ebi
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.collection.CollectionUtils
import eu.dnetlib.dhp.common.Constants.{MDSTORE_DATA_PATH, MDSTORE_SIZE_PATH}
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion
import eu.dnetlib.dhp.schema.oaf.{Oaf, Result}
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.sx.bio.pubmed._
import eu.dnetlib.dhp.utils.DHPUtils.{MAPPER, writeHdfsFile}
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
@ -17,13 +14,13 @@ import org.apache.http.client.methods.HttpGet
import org.apache.http.impl.client.HttpClientBuilder
import org.apache.spark.SparkConf
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql._
import org.apache.spark.sql.expressions.Aggregator
import org.slf4j.{Logger, LoggerFactory}
import java.io.InputStream
import scala.io.Source
import scala.xml.pull.XMLEventReader
import java.io.{ByteArrayInputStream, InputStream}
import java.nio.charset.Charset
import javax.xml.stream.XMLInputFactory
object SparkCreateBaselineDataFrame {
@ -86,7 +83,7 @@ object SparkCreateBaselineDataFrame {
if (response.getStatusLine.getStatusCode > 400) {
tries -= 1
} else
return IOUtils.toString(response.getEntity.getContent)
return IOUtils.toString(response.getEntity.getContent, Charset.defaultCharset())
} catch {
case e: Throwable =>
println(s"Error on requesting ${r.getURI}")
@ -158,7 +155,8 @@ object SparkCreateBaselineDataFrame {
IOUtils.toString(
SparkEBILinksToOaf.getClass.getResourceAsStream(
"/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json"
)
),
Charset.defaultCharset()
)
)
parser.parseArgument(args)
@ -167,15 +165,11 @@ object SparkCreateBaselineDataFrame {
val workingPath = parser.get("workingPath")
log.info("workingPath: {}", workingPath)
val mdstoreOutputVersion = parser.get("mdstoreOutputVersion")
log.info("mdstoreOutputVersion: {}", mdstoreOutputVersion)
val cleanedMdStoreVersion = MAPPER.readValue(mdstoreOutputVersion, classOf[MDStoreVersion])
val outputBasePath = cleanedMdStoreVersion.getHdfsPath
log.info("outputBasePath: {}", outputBasePath)
val targetPath = parser.get("targetPath")
log.info("targetPath: {}", targetPath)
val hdfsServerUri = parser.get("hdfsServerUri")
log.info("hdfsServerUri: {}", hdfsServerUri)
log.info("hdfsServerUri: {}", targetPath)
val skipUpdate = parser.get("skipUpdate")
log.info("skipUpdate: {}", skipUpdate)
@ -201,10 +195,11 @@ object SparkCreateBaselineDataFrame {
if (!"true".equalsIgnoreCase(skipUpdate)) {
downloadBaseLineUpdate(s"$workingPath/baseline", hdfsServerUri)
val k: RDD[(String, String)] = sc.wholeTextFiles(s"$workingPath/baseline", 2000)
val inputFactory = XMLInputFactory.newInstance
val ds: Dataset[PMArticle] = spark.createDataset(
k.filter(i => i._1.endsWith(".gz"))
.flatMap(i => {
val xml = new XMLEventReader(Source.fromBytes(i._2.getBytes()))
val xml = inputFactory.createXMLEventReader(new ByteArrayInputStream(i._2.getBytes()))
new PMParser(xml)
})
)
@ -223,11 +218,8 @@ object SparkCreateBaselineDataFrame {
.map(a => PubMedToOaf.convert(a, vocabularies))
.as[Oaf]
.filter(p => p != null),
s"$outputBasePath/$MDSTORE_DATA_PATH"
targetPath
)
val df = spark.read.text(s"$outputBasePath/$MDSTORE_DATA_PATH")
val mdStoreSize = df.count
writeHdfsFile(spark.sparkContext.hadoopConfiguration, s"$mdStoreSize", s"$outputBasePath/$MDSTORE_SIZE_PATH")
}
}

View File

@ -1,7 +1,8 @@
package eu.dnetlib.dhp.sx.bio.pubmed
import scala.xml.MetaData
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader}
import javax.xml.stream.XMLEventReader
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText}
/** @param xml
*/

View File

@ -15,10 +15,7 @@ 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.junit.jupiter.api.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

View File

@ -119,7 +119,9 @@ public class ReadCOCITest {
workingDir.toString() + "/COCI",
"-outputPath",
workingDir.toString() + "/COCI_json/",
"-inputFile", "input1;input2;input3;input4;input5"
"-inputFile", "input1;input2;input3;input4;input5",
"-format",
"COCI"
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());

View File

@ -789,10 +789,6 @@
"value": "2227-9717",
"type": "electronic"
},
{
"value": "VALUE",
"type": "PIPPO"
},
{
"value": "1063-4584",
"type": "pu"

View File

@ -2,7 +2,9 @@ package eu.dnetlib.dhp.collection.crossref
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
import org.junit.jupiter.api.BeforeEach
import eu.dnetlib.dhp.collection.crossref.Crossref2Oaf.TransformationType
import org.apache.commons.io.IOUtils
import org.junit.jupiter.api.{BeforeEach, Test}
import org.junit.jupiter.api.extension.ExtendWith
import org.mockito.junit.jupiter.MockitoExtension
import org.slf4j.{Logger, LoggerFactory}
@ -18,4 +20,13 @@ class CrossrefMappingTest extends AbstractVocabularyTest {
super.setUpVocabulary()
}
@Test
def mappingRecord(): Unit = {
val input =
IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/issn_pub.json"), "utf-8")
println(Crossref2Oaf.convert(input, vocabularies, TransformationType.All))
}
}

View File

@ -16,6 +16,7 @@ import org.mockito.junit.jupiter.MockitoExtension
import java.io.{BufferedReader, InputStream, InputStreamReader}
import java.util.zip.GZIPInputStream
import javax.xml.stream.XMLInputFactory
import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import scala.io.Source
@ -49,10 +50,8 @@ class BioScholixTest extends AbstractVocabularyTest {
@Test
def testEBIData() = {
val inputXML = Source
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed.xml"))
.mkString
val xml = new XMLEventReader(Source.fromBytes(inputXML.getBytes()))
val inputFactory = XMLInputFactory.newInstance
val xml = inputFactory.createXMLEventReader(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed.xml"))
new PMParser(xml).foreach(s => println(mapper.writeValueAsString(s)))
}
@ -91,9 +90,10 @@ class BioScholixTest extends AbstractVocabularyTest {
@Test
def testParsingPubmedXML(): Unit = {
val xml = new XMLEventReader(
Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed.xml"))
)
val inputFactory = XMLInputFactory.newInstance
val xml = inputFactory.createXMLEventReader(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed.xml"))
val parser = new PMParser(xml)
parser.foreach(checkPMArticle)
}
@ -156,9 +156,9 @@ class BioScholixTest extends AbstractVocabularyTest {
@Test
def testPubmedMapping(): Unit = {
val xml = new XMLEventReader(
Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed.xml"))
)
val inputFactory = XMLInputFactory.newInstance
val xml = inputFactory.createXMLEventReader(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed.xml"))
val parser = new PMParser(xml)
val results = ListBuffer[Oaf]()
parser.foreach(x => results += PubMedToOaf.convert(x, vocabularies))

View File

@ -53,24 +53,10 @@
<artifactId>dhp-pace-core</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>org.scala-lang.modules</groupId>
<artifactId>scala-java8-compat_${scala.binary.version}</artifactId>
<version>1.0.2</version>
</dependency>
<dependency>
<groupId>org.scala-lang.modules</groupId>
<artifactId>scala-collection-compat_${scala.binary.version}</artifactId>
<version>2.11.0</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
@ -79,16 +65,10 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-graphx_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>com.arakelian</groupId>
<artifactId>java-jq</artifactId>
</dependency>
<dependency>
<groupId>dom4j</groupId>
<artifactId>dom4j</artifactId>
@ -101,10 +81,6 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>

View File

@ -42,6 +42,7 @@ import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.util.SparkCompatUtils;
import scala.Tuple3;
import scala.collection.JavaConversions;
@ -148,8 +149,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
Dataset<Row> pivotHistory = spark
.createDataset(
Collections.emptyList(),
RowEncoder
.apply(StructType.fromDDL("id STRING, lastUsage STRING")));
SparkCompatUtils.encoderFor(StructType.fromDDL("id STRING, lastUsage STRING")));
if (StringUtils.isNotBlank(pivotHistoryDatabase)) {
pivotHistory = spark

View File

@ -8,7 +8,6 @@ import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.ReduceFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
import org.apache.spark.sql.types.StructType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -23,6 +22,7 @@ import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.util.SparkCompatUtils;
import scala.Tuple2;
import scala.Tuple3;
@ -145,7 +145,7 @@ public class SparkPropagateRelation extends AbstractSparkAction {
StructType idsSchema = StructType
.fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>");
Dataset<Row> allIds = spark.emptyDataset(RowEncoder.apply(idsSchema));
Dataset<Row> allIds = spark.emptyDataset(SparkCompatUtils.encoderFor(idsSchema));
for (EntityType entityType : ModelSupport.entityTypes.keySet()) {
String entityPath = graphBasePath + '/' + entityType.name();

View File

@ -0,0 +1,5 @@
[
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": false},
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source Path", "paramRequired": true},
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the scholix dump", "paramRequired": true}
]

View File

@ -0,0 +1,166 @@
{
"cites":{
"original":"Cites",
"inverse":"IsCitedBy"
},
"compiles":{
"original":"Compiles",
"inverse":"IsCompiledBy"
},
"continues":{
"original":"Continues",
"inverse":"IsContinuedBy"
},
"derives":{
"original":"IsSourceOf",
"inverse":"IsDerivedFrom"
},
"describes":{
"original":"Describes",
"inverse":"IsDescribedBy"
},
"documents":{
"original":"Documents",
"inverse":"IsDocumentedBy"
},
"hasmetadata":{
"original":"HasMetadata",
"inverse":"IsMetadataOf"
},
"hasassociationwith":{
"original":"HasAssociationWith",
"inverse":"HasAssociationWith"
},
"haspart":{
"original":"HasPart",
"inverse":"IsPartOf"
},
"hasversion":{
"original":"HasVersion",
"inverse":"IsVersionOf"
},
"iscitedby":{
"original":"IsCitedBy",
"inverse":"Cites"
},
"iscompiledby":{
"original":"IsCompiledBy",
"inverse":"Compiles"
},
"iscontinuedby":{
"original":"IsContinuedBy",
"inverse":"Continues"
},
"isderivedfrom":{
"original":"IsDerivedFrom",
"inverse":"IsSourceOf"
},
"isdescribedby":{
"original":"IsDescribedBy",
"inverse":"Describes"
},
"isdocumentedby":{
"original":"IsDocumentedBy",
"inverse":"Documents"
},
"isidenticalto":{
"original":"IsIdenticalTo",
"inverse":"IsIdenticalTo"
},
"ismetadatafor":{
"original":"IsMetadataFor",
"inverse":"IsMetadataOf"
},
"ismetadataof":{
"original":"IsMetadataOf",
"inverse":"IsMetadataFor"
},
"isnewversionof":{
"original":"IsNewVersionOf",
"inverse":"IsPreviousVersionOf"
},
"isobsoletedby":{
"original":"IsObsoletedBy",
"inverse":"Obsoletes"
},
"isoriginalformof":{
"original":"IsOriginalFormOf",
"inverse":"IsVariantFormOf"
},
"ispartof":{
"original":"IsPartOf",
"inverse":"HasPart"
},
"ispreviousversionof":{
"original":"IsPreviousVersionOf",
"inverse":"IsNewVersionOf"
},
"isreferencedby":{
"original":"IsReferencedBy",
"inverse":"References"
},
"isrelatedto":{
"original":"IsRelatedTo",
"inverse":"IsRelatedTo"
},
"isrequiredby":{
"original":"IsRequiredBy",
"inverse":"Requires"
},
"isreviewedby":{
"original":"IsReviewedBy",
"inverse":"Reviews"
},
"issourceof":{
"original":"IsSourceOf",
"inverse":"IsDerivedFrom"
},
"issupplementedby":{
"original":"IsSupplementedBy",
"inverse":"IsSupplementTo"
},
"issupplementto":{
"original":"IsSupplementTo",
"inverse":"IsSupplementedBy"
},
"isvariantformof":{
"original":"IsVariantFormOf",
"inverse":"IsOriginalFormOf"
},
"isversionof":{
"original":"IsVersionOf",
"inverse":"HasVersion"
},
"obsoletes":{
"original":"Obsoletes",
"inverse":"IsObsoletedBy"
},
"references":{
"original":"References",
"inverse":"IsReferencedBy"
},
"requires":{
"original":"Requires",
"inverse":"IsRequiredBy"
},
"related":{
"original":"IsRelatedTo",
"inverse":"IsRelatedTo"
},
"reviews":{
"original":"Reviews",
"inverse":"IsReviewedBy"
},
"unknown":{
"original":"Unknown",
"inverse":"Unknown"
},
"isamongtopnsimilardocuments": {
"original": "IsAmongTopNSimilarDocuments",
"inverse": "HasAmongTopNSimilarDocuments"
},
"hasamongtopnsimilardocuments": {
"original": "HasAmongTopNSimilarDocuments",
"inverse": "IsAmongTopNSimilarDocuments"
}
}

View File

@ -0,0 +1,256 @@
package eu.dnetlib.dhp.sx.graph
import eu.dnetlib.dhp.schema.oaf.{KeyValue, Result, StructuredProperty}
import eu.dnetlib.dhp.schema.sx.scholix.{
Scholix,
ScholixCollectedFrom,
ScholixEntityId,
ScholixIdentifier,
ScholixRelationship,
ScholixResource
}
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import scala.collection.JavaConverters._
import scala.io.Source
case class RelationInfo(
source: String,
target: String,
relclass: String,
id: String,
collectedfrom: Seq[RelKeyValue]
) {}
case class RelKeyValue(key: String, value: String) {}
object ScholexplorerUtils {
val OPENAIRE_IDENTIFIER_SCHEMA: String = "OpenAIRE Identifier"
case class RelationVocabulary(original: String, inverse: String) {}
val relations: Map[String, RelationVocabulary] = {
val input = Source
.fromInputStream(
getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/relation/relations.json")
)
.mkString
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(input)
json.extract[Map[String, RelationVocabulary]]
}
def invRel(rel: String): String = {
val semanticRelation = relations.getOrElse(rel.toLowerCase, null)
if (semanticRelation != null)
semanticRelation.inverse
else
null
}
def generateDatasourceOpenAIREURLS(id: String): String = {
if (id != null && id.length > 12)
s"https://explore.openaire.eu/search/dataprovider?datasourceId=${id.substring(3)}"
else
null
}
def findURLForPID(
pidValue: List[StructuredProperty],
urls: List[String]
): List[(StructuredProperty, String)] = {
pidValue.map { p =>
val pv = p.getValue
val r = urls.find(u => u.toLowerCase.contains(pv.toLowerCase))
(p, r.orNull)
}
}
def extractTypedIdentifierFromInstance(r: Result): List[ScholixIdentifier] = {
if (r.getInstance() == null || r.getInstance().isEmpty)
return List()
r.getInstance()
.asScala
.filter(i => i.getUrl != null && !i.getUrl.isEmpty)
.filter(i => i.getPid != null && i.getUrl != null)
.flatMap(i => findURLForPID(i.getPid.asScala.toList, i.getUrl.asScala.toList))
.map(i => new ScholixIdentifier(i._1.getValue, i._1.getQualifier.getClassid, i._2))
.distinct
.toList
}
def generateScholixResourceFromResult(result: Result): ScholixResource = {
if (result.getInstance() == null || result.getInstance().size() == 0)
return null
if (result.getPid == null || result.getPid.isEmpty)
return null
val r = new ScholixResource
r.setDnetIdentifier(result.getId)
val persistentIdentifiers: List[ScholixIdentifier] = extractTypedIdentifierFromInstance(result)
if (persistentIdentifiers.isEmpty)
return null
r.setIdentifier(persistentIdentifiers.asJava)
r.setObjectType(result.getResulttype.getClassid)
r.setObjectSubType(
result
.getInstance()
.asScala
.filter(i => i != null && i.getInstancetype != null)
.map(i => i.getInstancetype.getClassname)
.distinct
.head
)
if (result.getTitle != null && result.getTitle.asScala.nonEmpty) {
val titles: List[String] = result.getTitle.asScala.map(t => t.getValue).toList
if (titles.nonEmpty)
r.setTitle(titles.head)
else
return null
}
if (result.getAuthor != null && !result.getAuthor.isEmpty) {
val authors: List[ScholixEntityId] =
result.getAuthor.asScala
.map(a => {
val entity = new ScholixEntityId()
entity.setName(a.getFullname)
if (a.getPid != null && a.getPid.size() > 0)
entity.setIdentifiers(
a.getPid.asScala
.map(sp => {
val id = new ScholixIdentifier()
id.setIdentifier(sp.getValue)
id.setSchema(sp.getQualifier.getClassid)
id
})
.take(3)
.toList
.asJava
)
entity
})
.toList
if (authors.nonEmpty)
r.setCreator(authors.asJava)
}
val dt: List[String] = result
.getInstance()
.asScala
.filter(i => i.getDateofacceptance != null)
.map(i => i.getDateofacceptance.getValue)
.toList
if (dt.nonEmpty)
r.setPublicationDate(dt.distinct.head)
r.setPublisher(
result
.getInstance()
.asScala
.map(i => i.getHostedby)
.filter(h => !"unknown".equalsIgnoreCase(h.getValue))
.map(h => {
val eid = new ScholixEntityId()
eid.setName(h.getValue)
val id = new ScholixIdentifier()
id.setIdentifier(h.getKey)
id.setSchema(OPENAIRE_IDENTIFIER_SCHEMA)
id.setUrl(generateDatasourceOpenAIREURLS(h.getKey))
eid.setIdentifiers(List(id).asJava)
eid
})
.distinct
.asJava
)
r.setCollectedFrom(
result.getCollectedfrom.asScala
.map(cf => {
val scf = new ScholixCollectedFrom()
scf.setProvisionMode("collected")
scf.setCompletionStatus("complete")
val eid = new ScholixEntityId()
eid.setName(cf.getValue)
val id = new ScholixIdentifier()
id.setIdentifier(cf.getKey)
id.setSchema(OPENAIRE_IDENTIFIER_SCHEMA)
id.setUrl(generateDatasourceOpenAIREURLS(cf.getKey))
eid.setIdentifiers(List(id).asJava)
scf.setProvider(eid)
scf
})
.asJava
)
r
}
def generateScholix(relation: RelationInfo, source: ScholixResource): Scholix = {
val s: Scholix = new Scholix
s.setSource(source)
if (relation.collectedfrom != null && relation.collectedfrom.nonEmpty)
s.setLinkprovider(
relation.collectedfrom
.map(cf => {
val eid = new ScholixEntityId()
eid.setName(cf.value)
val id = new ScholixIdentifier()
id.setIdentifier(cf.key)
id.setSchema(OPENAIRE_IDENTIFIER_SCHEMA)
id.setUrl(generateDatasourceOpenAIREURLS(cf.key))
eid.setIdentifiers(List(id).asJava)
eid
})
.toList
.asJava
)
else {
val eid = new ScholixEntityId()
eid.setName("OpenAIRE")
val id = new ScholixIdentifier()
id.setIdentifier("10|infrastruct_::f66f1bd369679b5b077dcdf006089556")
id.setSchema(OPENAIRE_IDENTIFIER_SCHEMA)
id.setUrl(generateDatasourceOpenAIREURLS(id.getIdentifier))
eid.setIdentifiers(List(id).asJava)
s.setLinkprovider(List(eid).asJava)
}
s.setIdentifier(relation.id)
val semanticRelation = relations.getOrElse(relation.relclass.toLowerCase, null)
if (semanticRelation == null)
return null
s.setRelationship(
new ScholixRelationship(semanticRelation.original, "datacite", semanticRelation.inverse)
)
s.setPublicationDate(source.getPublicationDate)
s.setPublisher(source.getPublisher)
val mockTarget = new ScholixResource
mockTarget.setDnetIdentifier(relation.target)
s.setTarget(mockTarget)
s
}
def updateTarget(s: Scholix, t: ScholixResource): Scholix = {
s.setTarget(t)
val spublishers: Seq[ScholixEntityId] =
if (s.getPublisher != null && !s.getPublisher.isEmpty) s.getPublisher.asScala else List()
val tpublishers: Seq[ScholixEntityId] =
if (t.getPublisher != null && !t.getPublisher.isEmpty) t.getPublisher.asScala else List()
val mergedPublishers = spublishers.union(tpublishers).distinct.take(10).toList
s.setPublisher(mergedPublishers.asJava)
s
}
}

View File

@ -0,0 +1,134 @@
package eu.dnetlib.dhp.sx.graph
import eu.dnetlib.dhp.application.AbstractScalaApplication
import eu.dnetlib.dhp.schema.oaf.{
KeyValue,
OtherResearchProduct,
Publication,
Relation,
Result,
Software,
Dataset => OafDataset
}
import eu.dnetlib.dhp.schema.sx.scholix.{Scholix, ScholixResource}
import org.apache.spark.sql.functions.{col, concat, expr, md5}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
class SparkCreateScholexplorerDump(propertyPath: String, args: Array[String], log: Logger)
extends AbstractScalaApplication(propertyPath, args, log: Logger) {
/** Here all the spark applications runs this method
* where the whole logic of the spark node is defined
*/
override def run(): Unit = {
val sourcePath = parser.get("sourcePath")
log.info("sourcePath: {}", sourcePath)
val targetPath = parser.get("targetPath")
log.info("targetPath: {}", targetPath)
generateBidirectionalRelations(sourcePath, targetPath, spark)
generateScholixResource(sourcePath, targetPath, spark)
generateScholix(targetPath, spark)
}
def generateScholixResource(inputPath: String, outputPath: String, spark: SparkSession): Unit = {
val entityMap: Map[String, StructType] = Map(
"publication" -> Encoders.bean(classOf[Publication]).schema,
"dataset" -> Encoders.bean(classOf[OafDataset]).schema,
"software" -> Encoders.bean(classOf[Software]).schema,
"otherresearchproduct" -> Encoders.bean(classOf[OtherResearchProduct]).schema
)
implicit val scholixResourceEncoder: Encoder[ScholixResource] = Encoders.bean(classOf[ScholixResource])
implicit val resultEncoder: Encoder[Result] = Encoders.bean(classOf[Result])
val resDs = spark.emptyDataset[ScholixResource]
val scholixResourceDS = entityMap.foldLeft[Dataset[ScholixResource]](resDs)((res, item) => {
println(s"adding ${item._1}")
res.union(
spark.read
.schema(item._2)
.json(s"$inputPath/${item._1}")
.as[Result]
.map(r => ScholexplorerUtils.generateScholixResourceFromResult(r))
.filter(s => s != null)
)
})
scholixResourceDS.write.mode(SaveMode.Overwrite).save(s"$outputPath/resource")
}
def generateBidirectionalRelations(inputPath: String, otuputPath: String, spark: SparkSession): Unit = {
val relSchema = Encoders.bean(classOf[Relation]).schema
val relDF = spark.read
.schema(relSchema)
.json(s"$inputPath/relation")
.where(
"datainfo.deletedbyinference is false and source like '50%' and target like '50%' " +
"and relClass <> 'merges' and relClass <> 'isMergedIn'"
)
.select("source", "target", "collectedfrom", "relClass")
def invRel: String => String = { s =>
ScholexplorerUtils.invRel(s)
}
import org.apache.spark.sql.functions.udf
val inverseRelationUDF = udf(invRel)
val inverseRelation = relDF.select(
col("target").alias("source"),
col("source").alias("target"),
col("collectedfrom"),
inverseRelationUDF(col("relClass")).alias("relClass")
)
val bidRel = inverseRelation
.union(relDF)
.withColumn("id", md5(concat(col("source"), col("relClass"), col("target"))))
.withColumn("cf", expr("transform(collectedfrom, x -> struct(x.key, x.value))"))
.drop("collectedfrom")
.withColumnRenamed("cf", "collectedfrom")
.distinct()
bidRel.write.mode(SaveMode.Overwrite).save(s"$otuputPath/relation")
}
def generateScholix(outputPath: String, spark: SparkSession): Unit = {
implicit val scholixResourceEncoder: Encoder[ScholixResource] = Encoders.bean(classOf[ScholixResource])
implicit val scholixEncoder: Encoder[Scholix] = Encoders.bean(classOf[Scholix])
import spark.implicits._
val relations = spark.read.load(s"$outputPath/relation").as[RelationInfo]
val resource = spark.read.load(s"$outputPath/resource").as[ScholixResource]
val scholix_one_verse = relations
.joinWith(resource, relations("source") === resource("dnetIdentifier"), "inner")
.map(res => ScholexplorerUtils.generateScholix(res._1, res._2))
val resourceTarget = relations
.joinWith(resource, relations("target") === resource("dnetIdentifier"), "inner")
.map(res => (res._1.id, res._2))(Encoders.tuple(Encoders.STRING, Encoders.kryo(classOf[ScholixResource])))
scholix_one_verse
.joinWith(resourceTarget, scholix_one_verse("identifier") === resourceTarget("_1"), "inner")
.map(k => ScholexplorerUtils.updateTarget(k._1, k._2._2))
.write
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(s"$outputPath/scholix")
}
}
object SparkCreateScholexplorerDump {
val logger: Logger = LoggerFactory.getLogger(SparkCreateScholexplorerDump.getClass)
def main(args: Array[String]): Unit = {
new SparkCreateScholexplorerDump(
log = logger,
args = args,
propertyPath = "/eu/dnetlib/dhp/sx/create_scholix_dump_params.json"
).initialize().run()
}
}

View File

@ -0,0 +1,17 @@
package eu.dnetlib.dhp.sx.graph.scholix
import eu.dnetlib.dhp.sx.graph.SparkCreateScholexplorerDump
import org.apache.spark.sql.SparkSession
import org.junit.jupiter.api.Test
class ScholixGenerationTest {
@Test
def generateScholix(): Unit = {
val spark: SparkSession = SparkSession.builder().master("local[*]").getOrCreate()
val app = new SparkCreateScholexplorerDump(null, null, null)
// app.generateScholixResource("/home/sandro/Downloads/scholix_sample/", "/home/sandro/Downloads/scholix/", spark)
// app.generateBidirectionalRelations("/home/sandro/Downloads/scholix_sample/", "/home/sandro/Downloads/scholix/", spark)
app.generateScholix("/home/sandro/Downloads/scholix/", spark)
}
}

View File

@ -18,7 +18,7 @@
<executions>
<execution>
<id>scala-compile-first</id>
<phase>initialize</phase>
<phase>process-resources</phase>
<goals>
<goal>add-source</goal>
<goal>compile</goal>
@ -59,12 +59,6 @@
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>dom4j</groupId>
@ -160,6 +154,26 @@
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</exclusion>
<exclusion>
<artifactId>ant</artifactId>
<groupId>org.apache.ant</groupId>
</exclusion>
<exclusion>
<artifactId>antlr4-runtime</artifactId>
<groupId>org.antlr</groupId>
</exclusion>
<exclusion>
<artifactId>woodstox-core</artifactId>
<groupId>com.fasterxml.woodstox</groupId>
</exclusion>
<exclusion>
<artifactId>log4j</artifactId>
<groupId>*</groupId>
</exclusion>
<exclusion>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
@ -206,5 +220,90 @@
</dependencies>
<profiles>
<profile>
<id>spark-24</id>
<activation>
<activeByDefault>true</activeByDefault>
</activation>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>3.4.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/sparksolr-3</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>spark-34</id>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>3.4.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/sparksolr-4</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>spark-35</id>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<version>3.4.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/sparksolr-4</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -25,6 +25,7 @@ import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory;
import eu.dnetlib.dhp.sparksolr.DHPSolrSupport;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
@ -129,7 +130,7 @@ public class XmlIndexingJob extends AbstractSolrRecordTransformJob {
.javaRDD()
.map(
t -> new StreamingInputDocumentFactory().parseDocument(t.getXml(), t.getJson()));
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
DHPSolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
}
}

View File

@ -5,14 +5,11 @@ import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.removePrefix;
import static eu.dnetlib.dhp.oa.provision.utils.XmlSerializationUtils.escapeXml;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import javax.swing.text.html.Option;
import org.apache.commons.lang3.StringUtils;
import org.stringtemplate.v4.ST;

View File

@ -0,0 +1,12 @@
package eu.dnetlib.dhp.sparksolr;
import com.lucidworks.spark.util.SolrSupport;
import org.apache.solr.common.SolrInputDocument;
import org.apache.spark.rdd.RDD;
public class DHPSolrSupport {
static public void indexDocs(String zkhost, String collection, int batchSize, RDD<SolrInputDocument> docs) {
SolrSupport.indexDocs(zkhost, collection, batchSize, docs);
}
}

View File

@ -0,0 +1,12 @@
package eu.dnetlib.dhp.sparksolr;
import com.lucidworks.spark.util.SolrSupport;
import org.apache.solr.common.SolrInputDocument;
import org.apache.spark.rdd.RDD;
public class DHPSolrSupport {
static public void indexDocs(String zkhost, String collection, int batchSize, RDD<SolrInputDocument> docs) {
SolrSupport.indexDocs(zkhost, collection, batchSize, com.lucidworks.spark.BatchSizeType.NUM_DOCS, docs);
}
}

View File

@ -16,11 +16,11 @@
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
</dependency>
<dependency>

View File

@ -10,11 +10,11 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
</dependency>
</dependencies>
<build>

View File

@ -10,11 +10,11 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
</dependency>
</dependencies>
<build>

View File

@ -10,11 +10,11 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
</dependency>
</dependencies>
<build>

View File

@ -17,6 +17,7 @@ import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
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.*;
import org.apache.spark.sql.Dataset;
@ -117,7 +118,7 @@ public class PrepareSWHActionsets {
.map(
(MapFunction<String, Software>) t -> OBJECT_MAPPER.readValue(t, Software.class),
Encoders.bean(Software.class))
.filter(t -> t.getCodeRepositoryUrl() != null)
.filter((FilterFunction<Software>) t -> t.getCodeRepositoryUrl() != null)
.select(col("id"), col("codeRepositoryUrl.value").as("repoUrl"));
}

View File

@ -39,8 +39,8 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
<cdh.hive.version>0.13.1-cdh5.2.1</cdh.hive.version>
<cdh.hadoop.version>2.5.0-cdh5.2.1</cdh.hadoop.version>
<cdh.hive.version>1.1.0-cdh5.16.2</cdh.hive.version>
<cdh.hadoop.version>2.6.0-cdh5.16.2</cdh.hadoop.version>
</properties>
<dependencies>
@ -72,7 +72,13 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<version>${cdh.hadoop.version}</version>
</dependency>
<exclusions>
<exclusion>
<groupId>jdk.tools</groupId>
<artifactId>jdk.tools</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-common</artifactId>

View File

@ -39,8 +39,8 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
<cdh.hive.version>0.13.1-cdh5.2.1</cdh.hive.version>
<cdh.hadoop.version>2.5.0-cdh5.2.1</cdh.hadoop.version>
<cdh.hive.version>1.1.0-cdh5.16.2</cdh.hive.version>
<cdh.hadoop.version>2.6.0-cdh5.16.2</cdh.hadoop.version>
</properties>
<dependencies>
@ -67,11 +67,23 @@
<groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${cdh.hive.version}</version>
</dependency>
<exclusions>
<exclusion>
<groupId>jdk.tools</groupId>
<artifactId>jdk.tools</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<version>${cdh.hadoop.version}</version>
<exclusions>
<exclusion>
<groupId>jdk.tools</groupId>
<artifactId>jdk.tools</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>

2054
pom.xml

File diff suppressed because it is too large Load Diff