forked from D-Net/dnet-hadoop
Updated all aggregation workflow to use mdstore manager
in this way the commit/rollback mechanism is implemented.
This commit is contained in:
parent
96cd96f3c2
commit
5f82c98494
|
@ -65,7 +65,7 @@
|
|||
<arg>--master</arg><arg>yarn</arg>
|
||||
</spark>
|
||||
<ok to="CommitVersion"/>
|
||||
<error to="Kill"/>
|
||||
<error to="RollBack"/>
|
||||
</action>
|
||||
|
||||
<action name="CommitVersion">
|
||||
|
|
|
@ -8,44 +8,121 @@
|
|||
<name>database</name>
|
||||
<description>the PDB Database Working Path</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>targetPath</name>
|
||||
<description>the Target Working dir path</description>
|
||||
<name>mdStoreOutputId</name>
|
||||
<description>the identifier of the cleaned MDStore</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>mdStoreManagerURI</name>
|
||||
<description>the path of the cleaned mdstore</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<start to="ConvertDB"/>
|
||||
|
||||
<start to="StartTransaction"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="ConvertDB">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Bio DB to OAF Dataset</name>
|
||||
<class>eu.dnetlib.dhp.sx.bio.SparkTransformBioDatabaseToOAF</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.shuffle.partitions=2000
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--master</arg><arg>yarn</arg>
|
||||
<arg>--dbPath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--database</arg><arg>${database}</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<end name="End"/>
|
||||
<action name="StartTransaction">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>NEW_VERSION</arg>
|
||||
<arg>--mdStoreID</arg><arg>${mdStoreOutputId}</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
<capture-output/>
|
||||
</java>
|
||||
<ok to="ConvertDB"/>
|
||||
<error to="EndReadRollBack"/>
|
||||
</action>
|
||||
|
||||
<action name="ConvertDB">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Bio DB to OAF Dataset</name>
|
||||
<class>eu.dnetlib.dhp.sx.bio.SparkTransformBioDatabaseToOAF</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.shuffle.partitions=2000
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--master</arg><arg>yarn</arg>
|
||||
<arg>--dbPath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--database</arg><arg>${database}</arg>
|
||||
<arg>--mdstoreOutputVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
|
||||
</spark>
|
||||
<ok to="CommitVersion"/>
|
||||
<error to="RollBack"/>
|
||||
</action>
|
||||
|
||||
<action name="CommitVersion">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>COMMIT</arg>
|
||||
<arg>--namenode</arg><arg>${nameNode}</arg>
|
||||
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
</java>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="EndReadRollBack">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>READ_UNLOCK</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
<arg>--readMDStoreId</arg><arg>${wf:actionData('BeginRead')['mdStoreReadLockVersion']}</arg>
|
||||
<capture-output/>
|
||||
</java>
|
||||
<ok to="RollBack"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="RollBack">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>ROLLBACK</arg>
|
||||
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
</java>
|
||||
<ok to="Kill"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<end name="End"/>
|
||||
|
||||
</workflow-app>
|
|
@ -2,5 +2,5 @@
|
|||
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
|
||||
{"paramName":"db", "paramLongName":"database", "paramDescription": "should be PDB or UNIPROT", "paramRequired": true},
|
||||
{"paramName":"p", "paramLongName":"dbPath", "paramDescription": "the path of the database to transform", "paramRequired": true},
|
||||
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the OAF target path ", "paramRequired": true}
|
||||
{"paramName":"mo", "paramLongName":"mdstoreOutputVersion", "paramDescription": "the mdstore Output Version", "paramRequired": true}
|
||||
]
|
|
@ -0,0 +1,68 @@
|
|||
<configuration>
|
||||
|
||||
<!-- OCEAN -->
|
||||
|
||||
<property>
|
||||
<name>jobTracker</name>
|
||||
<value>yarnRM</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>nameNode</name>
|
||||
<value>hdfs://nameservice1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hive_metastore_uris</name>
|
||||
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2YarnHistoryServerAddress</name>
|
||||
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
|
||||
</property>
|
||||
|
||||
|
||||
<!-- GARR -->
|
||||
|
||||
<!-- <property>-->
|
||||
<!-- <name>jobTracker</name>-->
|
||||
<!-- <value>yarn</value>-->
|
||||
<!-- </property>-->
|
||||
<!-- <property>-->
|
||||
<!-- <name>nameNode</name>-->
|
||||
<!-- <value>hdfs://hadoop-rm1.garr-pa1.d4science.org:8020</value>-->
|
||||
<!-- </property>-->
|
||||
<!-- <property>-->
|
||||
<!-- <name>hive_metastore_uris</name>-->
|
||||
<!-- <value>thrift://hadoop-edge3.garr-pa1.d4science.org:9083</value>-->
|
||||
<!-- </property>-->
|
||||
<!-- <property>-->
|
||||
<!-- <name>spark2YarnHistoryServerAddress</name>-->
|
||||
<!-- <value>http://hadoop-rm2.garr-pa1.d4science.org:19888</value>-->
|
||||
<!-- </property>-->
|
||||
|
||||
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>oozie.use.system.libpath</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>spark2</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2EventLogDir</name>
|
||||
<value>/user/spark/spark2ApplicationHistory</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2ExtraListeners</name>
|
||||
<value>"com.cloudera.spark.lineage.NavigatorAppListener"</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2SqlQueryExecutionListeners</name>
|
||||
<value>"com.cloudera.spark.lineage.NavigatorQueryListener"</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,118 @@
|
|||
<workflow-app name="Transform_EBILINKS_Workflow" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sourcePath</name>
|
||||
<description>the PDB Database Working Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>mdStoreOutputId</name>
|
||||
<description>the identifier of the cleaned MDStore</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>mdStoreManagerURI</name>
|
||||
<description>the path of the cleaned mdstore</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<start to="StartTransaction"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="StartTransaction">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>NEW_VERSION</arg>
|
||||
<arg>--mdStoreID</arg><arg>${mdStoreOutputId}</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
<capture-output/>
|
||||
</java>
|
||||
<ok to="CreateEBIDataSet"/>
|
||||
<error to="EndReadRollBack"/>
|
||||
</action>
|
||||
<action name="CreateEBIDataSet">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Create OAF DataSet</name>
|
||||
<class>eu.dnetlib.dhp.sx.bio.ebi.SparkTransformEBILinksToOaf</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=2000
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--mdstoreOutputVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
|
||||
<arg>--master</arg><arg>yarn</arg>
|
||||
</spark>
|
||||
<ok to="CommitVersion"/>
|
||||
<error to="RollBack"/>
|
||||
</action>
|
||||
|
||||
<action name="CommitVersion">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>COMMIT</arg>
|
||||
<arg>--namenode</arg><arg>${nameNode}</arg>
|
||||
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
</java>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="EndReadRollBack">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>READ_UNLOCK</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
<arg>--readMDStoreId</arg><arg>${wf:actionData('BeginRead')['mdStoreReadLockVersion']}</arg>
|
||||
<capture-output/>
|
||||
</java>
|
||||
<ok to="RollBack"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="RollBack">
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
|
||||
<arg>--action</arg><arg>ROLLBACK</arg>
|
||||
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
|
||||
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
|
||||
</java>
|
||||
<ok to="Kill"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<end name="End"/>
|
||||
|
||||
</workflow-app>
|
|
@ -0,0 +1,5 @@
|
|||
[
|
||||
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
|
||||
{"paramName":"s", "paramLongName":"sourcePath","paramDescription": "the source Path", "paramRequired": true},
|
||||
{"paramName":"mo", "paramLongName":"mdstoreOutputVersion", "paramDescription": "the mdstore Output Version", "paramRequired": true}
|
||||
]
|
|
@ -82,8 +82,8 @@ object BioDBToOAF {
|
|||
"Springer Nature"
|
||||
)
|
||||
val EBICollectedFrom: KeyValue = OafMapperUtils.keyValue(
|
||||
"10|opendoar____::83e60e09c222f206c725385f53d7e567c",
|
||||
"EMBL-EBIs Protein Data Bank in Europe (PDBe)"
|
||||
"10|fairsharing_::96c5c28becf18e71190460a9955aa4d8",
|
||||
"Protein Data Bank in Europe"
|
||||
)
|
||||
val pubmedCollectedFrom: KeyValue =
|
||||
OafMapperUtils.keyValue(ModelConstants.EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central")
|
||||
|
@ -120,7 +120,7 @@ object BioDBToOAF {
|
|||
|
||||
val relation_semantic = (json \ "RelationshipType" \ "Name").extract[String]
|
||||
|
||||
val date = GraphCleaningFunctions.cleanDate((json \ "LinkedPublicationDate").extract[String])
|
||||
val date = GraphCleaningFunctions.cleanDate((json \ "LinkPublicationDate").extract[String])
|
||||
|
||||
createRelation(
|
||||
target_pid,
|
||||
|
|
|
@ -1,42 +1,26 @@
|
|||
package eu.dnetlib.dhp.sx.bio
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import eu.dnetlib.dhp.application.AbstractScalaApplication
|
||||
import eu.dnetlib.dhp.collection.CollectionUtils
|
||||
import eu.dnetlib.dhp.common.Constants.MDSTORE_DATA_PATH
|
||||
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
||||
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.ScholixResolved
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
object SparkTransformBioDatabaseToOAF {
|
||||
class SparkTransformBioDatabaseToOAF(propertyPath: String, args: Array[String], log: Logger)
|
||||
extends AbstractScalaApplication(propertyPath, args, log: Logger) {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
val parser = new ArgumentApplicationParser(
|
||||
IOUtils.toString(
|
||||
getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/bio/ebi/bio_to_oaf_params.json")
|
||||
)
|
||||
)
|
||||
parser.parseArgument(args)
|
||||
val database: String = parser.get("database")
|
||||
log.info("database: {}", database)
|
||||
|
||||
val dbPath: String = parser.get("dbPath")
|
||||
log.info("dbPath: {}", database)
|
||||
val targetPath: String = parser.get("targetPath")
|
||||
log.info("targetPath: {}", database)
|
||||
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate()
|
||||
private def convertDatabase(
|
||||
spark: SparkSession,
|
||||
dbPath: String,
|
||||
database: String,
|
||||
targetPath: String,
|
||||
outputBasePath: String
|
||||
): Unit = {
|
||||
val sc = spark.sparkContext
|
||||
|
||||
implicit val resultEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
import spark.implicits._
|
||||
database.toUpperCase() match {
|
||||
|
@ -45,22 +29,60 @@ object SparkTransformBioDatabaseToOAF {
|
|||
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.uniprotToOAF(i))),
|
||||
targetPath
|
||||
)
|
||||
reportTotalSize(targetPath, outputBasePath)
|
||||
case "PDB" =>
|
||||
CollectionUtils.saveDataset(
|
||||
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.pdbTOOaf(i))),
|
||||
targetPath
|
||||
)
|
||||
reportTotalSize(targetPath, outputBasePath)
|
||||
case "SCHOLIX" =>
|
||||
CollectionUtils.saveDataset(
|
||||
spark.read.load(dbPath).as[ScholixResolved].map(i => BioDBToOAF.scholixResolvedToOAF(i)),
|
||||
targetPath
|
||||
)
|
||||
reportTotalSize(targetPath, outputBasePath)
|
||||
case "CROSSREF_LINKS" =>
|
||||
CollectionUtils.saveDataset(
|
||||
spark.createDataset(sc.textFile(dbPath).map(i => BioDBToOAF.crossrefLinksToOaf(i))),
|
||||
spark.read.load(dbPath).select("json").as[String].map(i => BioDBToOAF.crossrefLinksToOaf(i)),
|
||||
targetPath
|
||||
)
|
||||
reportTotalSize(targetPath, outputBasePath)
|
||||
}
|
||||
}
|
||||
|
||||
/** Here all the spark applications runs this method
|
||||
* where the whole logic of the spark node is defined
|
||||
*/
|
||||
override def run(): Unit = {
|
||||
val database: String = parser.get("database")
|
||||
log.info("database: {}", database)
|
||||
|
||||
val dbPath: String = parser.get("dbPath")
|
||||
log.info("dbPath: {}", database)
|
||||
|
||||
val mdstoreOutputVersion = parser.get("mdstoreOutputVersion")
|
||||
log.info(s"mdstoreOutputVersion is '$mdstoreOutputVersion'")
|
||||
|
||||
val mapper = new ObjectMapper()
|
||||
val cleanedMdStoreVersion = mapper.readValue(mdstoreOutputVersion, classOf[MDStoreVersion])
|
||||
val outputBasePath = cleanedMdStoreVersion.getHdfsPath
|
||||
log.info(s"outputBasePath is '$outputBasePath'")
|
||||
|
||||
val targetPath = s"$outputBasePath$MDSTORE_DATA_PATH"
|
||||
log.info(s"targetPath is '$targetPath'")
|
||||
convertDatabase(spark, dbPath, database, targetPath, outputBasePath)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
object SparkTransformBioDatabaseToOAF {
|
||||
val log: Logger = LoggerFactory.getLogger(SparkTransformBioDatabaseToOAF.getClass)
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
new SparkTransformBioDatabaseToOAF("/eu/dnetlib/dhp/sx/bio/ebi/bio_to_oaf_params.json", args, log)
|
||||
.initialize()
|
||||
.run()
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,66 @@
|
|||
package eu.dnetlib.dhp.sx.bio.ebi
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import eu.dnetlib.dhp.application.AbstractScalaApplication
|
||||
import eu.dnetlib.dhp.collection.CollectionUtils
|
||||
import eu.dnetlib.dhp.common.Constants.MDSTORE_DATA_PATH
|
||||
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
||||
import eu.dnetlib.dhp.sx.bio.BioDBToOAF
|
||||
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.EBILinkItem
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
class SparkTransformEBILinksToOaf(propertyPath: String, args: Array[String], log: Logger)
|
||||
extends AbstractScalaApplication(propertyPath, args, log: Logger) {
|
||||
|
||||
def transformLinks(spark: SparkSession, sourcePath: String, outputBasePath: String, targetPath: String) = {
|
||||
implicit val PMEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
import spark.implicits._
|
||||
val ebLinks: Dataset[EBILinkItem] = spark.read
|
||||
.load(sourcePath)
|
||||
.as[EBILinkItem]
|
||||
.filter(l => l.links != null && l.links.startsWith("{"))
|
||||
|
||||
CollectionUtils.saveDataset(
|
||||
ebLinks
|
||||
.flatMap(j => BioDBToOAF.parse_ebi_links(j.links))
|
||||
.filter(p => BioDBToOAF.EBITargetLinksFilter(p))
|
||||
.flatMap(p => BioDBToOAF.convertEBILinksToOaf(p)),
|
||||
targetPath
|
||||
)
|
||||
reportTotalSize(targetPath, outputBasePath)
|
||||
}
|
||||
|
||||
/** 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(s"sourcePath is '$sourcePath'")
|
||||
|
||||
val mdstoreOutputVersion = parser.get("mdstoreOutputVersion")
|
||||
log.info(s"mdstoreOutputVersion is '$mdstoreOutputVersion'")
|
||||
|
||||
val mapper = new ObjectMapper()
|
||||
val cleanedMdStoreVersion = mapper.readValue(mdstoreOutputVersion, classOf[MDStoreVersion])
|
||||
val outputBasePath = cleanedMdStoreVersion.getHdfsPath
|
||||
log.info(s"outputBasePath is '$outputBasePath'")
|
||||
|
||||
val targetPath = s"$outputBasePath$MDSTORE_DATA_PATH"
|
||||
log.info(s"targetPath is '$targetPath'")
|
||||
transformLinks(spark, sourcePath, outputBasePath, targetPath)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
object SparkTransformEBILinksToOaf {
|
||||
val log: Logger = LoggerFactory.getLogger(SparkTransformEBILinksToOaf.getClass)
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
new SparkTransformEBILinksToOaf("/eu/dnetlib/dhp/sx/bio/ebi/transform_ebi_to_df_params.json", args, log)
|
||||
.initialize()
|
||||
.run()
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue