WIP: materialize graph as Hive DB, configured spark actions to include hive support]

This commit is contained in:
Claudio Atzori 2020-08-06 21:48:29 +02:00
parent fde8738ed2
commit cce21eafc2
17 changed files with 361 additions and 131 deletions

View File

@ -1,6 +1,8 @@
package eu.dnetlib.dhp.oa.graph.clean;
import static eu.dnetlib.dhp.common.GraphSupport.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Objects;
@ -13,7 +15,6 @@ import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -50,50 +51,64 @@ public class CleanGraphSparkJob {
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String inputPath = parser.get("inputPath");
log.info("inputPath: {}", inputPath);
String inputGraph = parser.get("inputGraph");
log.info("inputGraph: {}", inputGraph);
String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
String outputGraph = parser.get("outputGraph");
log.info("outputGraph: {}", outputGraph);
String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
GraphFormat inputGraphFormat = Optional
.ofNullable(parser.get("inputGraphFormat"))
.map(GraphFormat::valueOf)
.orElse(GraphFormat.DEFAULT);
log.info("inputGraphFormat: {}", inputGraphFormat);
GraphFormat outputGraphFormat = Optional
.ofNullable(parser.get("outputGraphFormat"))
.map(GraphFormat::valueOf)
.orElse(GraphFormat.DEFAULT);
log.info("outputGraphFormat: {}", outputGraphFormat);
String isLookUpUrl = parser.get("isLookUpUrl");
log.info("isLookUpUrl: {}", isLookUpUrl);
String graphTableClassName = parser.get("graphTableClassName");
log.info("graphTableClassName: {}", graphTableClassName);
Class<? extends OafEntity> entityClazz = (Class<? extends OafEntity>) Class.forName(graphTableClassName);
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupService);
String hiveMetastoreUris = parser.get("hiveMetastoreUris");
log.info("hiveMetastoreUris: {}", hiveMetastoreUris);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf.set("hive.metastore.uris", hiveMetastoreUris);
Class<? extends Oaf> clazz = (Class<? extends Oaf>) Class.forName(graphTableClassName);
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookUpUrl);
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupService);
runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> {
removeOutputDir(spark, outputPath);
fixGraphTable(spark, vocs, inputPath, entityClazz, outputPath);
});
spark -> cleanGraphTable(spark, vocs, inputGraph, inputGraphFormat, outputGraph, outputGraphFormat, clazz));
}
private static <T extends Oaf> void fixGraphTable(
private static <T extends Oaf> void cleanGraphTable(
SparkSession spark,
VocabularyGroup vocs,
String inputPath,
Class<T> clazz,
String outputPath) {
String inputGraph,
GraphFormat inputGraphFormat,
String outputGraph,
GraphFormat outputGraphFormat,
Class<T> clazz) {
final CleaningRuleMap mapping = CleaningRuleMap.create(vocs);
readTableFromPath(spark, inputPath, clazz)
Dataset<T> cleaned = readGraph(spark, inputGraph, clazz, inputGraphFormat)
.map((MapFunction<T, T>) value -> fixVocabularyNames(value), Encoders.bean(clazz))
.map((MapFunction<T, T>) value -> OafCleaner.apply(value, mapping), Encoders.bean(clazz))
.map((MapFunction<T, T>) value -> fixDefaults(value), Encoders.bean(clazz))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
.map((MapFunction<T, T>) value -> fixDefaults(value), Encoders.bean(clazz));
saveGraphTable(cleaned, clazz, outputGraph, outputGraphFormat);
}
protected static <T extends Oaf> T fixVocabularyNames(T value) {

View File

@ -3,13 +3,12 @@ package eu.dnetlib.dhp.oa.graph.merge;
import static eu.dnetlib.dhp.common.GraphSupport.deleteGraphTable;
import static eu.dnetlib.dhp.common.GraphSupport.saveGraphTable;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Objects;
import java.util.Optional;
import eu.dnetlib.dhp.common.GraphFormat;
import eu.dnetlib.dhp.common.GraphSupport;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
@ -24,6 +23,8 @@ import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.GraphFormat;
import eu.dnetlib.dhp.common.GraphSupport;
import eu.dnetlib.dhp.oa.graph.clean.CleanGraphSparkJob;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
@ -71,15 +72,15 @@ public class MergeGraphSparkJob {
log.info("outputGraph: {}", outputGraph);
GraphFormat inputGraphFormat = Optional
.ofNullable(parser.get("inputGraphFormat"))
.map(GraphFormat::valueOf)
.orElse(GraphFormat.DEFAULT);
.ofNullable(parser.get("inputGraphFormat"))
.map(GraphFormat::valueOf)
.orElse(GraphFormat.DEFAULT);
log.info("inputGraphFormat: {}", inputGraphFormat);
GraphFormat outputGraphFormat = Optional
.ofNullable(parser.get("outputGraphFormat"))
.map(GraphFormat::valueOf)
.orElse(GraphFormat.DEFAULT);
.ofNullable(parser.get("outputGraphFormat"))
.map(GraphFormat::valueOf)
.orElse(GraphFormat.DEFAULT);
log.info("outputGraphFormat: {}", outputGraphFormat);
String graphTableClassName = parser.get("graphTableClassName");
@ -87,45 +88,48 @@ public class MergeGraphSparkJob {
Class<? extends Oaf> clazz = (Class<? extends Oaf>) Class.forName(graphTableClassName);
String hiveMetastoreUris = parser.get("hiveMetastoreUris");
log.info("hiveMetastoreUris: {}", hiveMetastoreUris);
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", hiveMetastoreUris);
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
runWithSparkSession(
runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> {
deleteGraphTable(spark, clazz, outputGraph, outputGraphFormat);
mergeGraphTable(spark, priority, betaInputGraph, prodInputGraph, clazz, clazz, outputGraph, inputGraphFormat, outputGraphFormat);
});
spark -> mergeGraphTable(
spark, priority, betaInputGraph, clazz, prodInputGraph, clazz, outputGraph, inputGraphFormat,
outputGraphFormat));
}
private static <P extends Oaf, B extends Oaf> void mergeGraphTable(
SparkSession spark,
String priority,
String betaInputGraph,
String prodInputGraph,
Class<P> p_clazz,
Class<B> b_clazz,
String outputGraph, GraphFormat inputGraphFormat, GraphFormat outputGraphFormat) {
SparkSession spark,
String priority,
String betaInputGraph,
Class<B> b_clazz,
String prodInputGraph,
Class<P> p_clazz,
String outputGraph, GraphFormat inputGraphFormat, GraphFormat outputGraphFormat) {
Dataset<Tuple2<String, B>> beta = readGraph(spark, betaInputGraph, b_clazz, inputGraphFormat);
Dataset<Tuple2<String, P>> prod = readGraph(spark, prodInputGraph, p_clazz, inputGraphFormat);
Dataset<P> merged = prod
.joinWith(beta, prod.col("_1").equalTo(beta.col("_1")), "full_outer")
.map((MapFunction<Tuple2<Tuple2<String, P>, Tuple2<String, B>>, P>) value -> {
Optional<P> p = Optional.ofNullable(value._1()).map(Tuple2::_2);
Optional<B> b = Optional.ofNullable(value._2()).map(Tuple2::_2);
switch (priority) {
default:
case "BETA":
return mergeWithPriorityToBETA(p, b);
case "PROD":
return mergeWithPriorityToPROD(p, b);
}
}, Encoders.bean(p_clazz))
.filter((FilterFunction<P>) Objects::nonNull);
.joinWith(beta, prod.col("_1").equalTo(beta.col("_1")), "full_outer")
.map((MapFunction<Tuple2<Tuple2<String, P>, Tuple2<String, B>>, P>) value -> {
Optional<P> p = Optional.ofNullable(value._1()).map(Tuple2::_2);
Optional<B> b = Optional.ofNullable(value._2()).map(Tuple2::_2);
switch (priority) {
default:
case "BETA":
return mergeWithPriorityToBETA(p, b);
case "PROD":
return mergeWithPriorityToPROD(p, b);
}
}, Encoders.bean(p_clazz))
.filter((FilterFunction<P>) Objects::nonNull);
saveGraphTable(merged, p_clazz, outputGraph, outputGraphFormat);
}
@ -151,14 +155,15 @@ public class MergeGraphSparkJob {
}
private static <T extends Oaf> Dataset<Tuple2<String, T>> readGraph(
SparkSession spark, String inputGraph, Class<T> clazz, GraphFormat inputGraphFormat) {
SparkSession spark, String inputGraph, Class<T> clazz, GraphFormat inputGraphFormat) {
log.info("Reading Graph table from: {}", inputGraph);
return GraphSupport.readGraph(spark, inputGraph, clazz, inputGraphFormat)
.map((MapFunction<T, Tuple2<String, T>>) t -> {
final String id = ModelSupport.idFn().apply(t);
return new Tuple2<>(id, t);
}, Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz)));
return GraphSupport
.readGraph(spark, inputGraph, clazz, inputGraphFormat)
.map((MapFunction<T, Tuple2<String, T>>) t -> {
final String id = ModelSupport.idFn().apply(t);
return new Tuple2<>(id, t);
}, Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz)));
}
}

View File

@ -70,10 +70,10 @@ public class GenerateEntitiesApplication {
final String targetPath = parser.get("targetPath");
log.info("targetPath: {}", targetPath);
final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
final String isLookUpUrl = parser.get("isLookUpUrl");
log.info("isLookUpUrl: {}", isLookUpUrl);
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookUpUrl);
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupService);
final SparkConf conf = new SparkConf();

View File

@ -62,22 +62,18 @@ public class MergeClaimsApplication {
Class<? extends Oaf> clazz = (Class<? extends Oaf>) Class.forName(graphTableClassName);
String hiveMetastoreUris = parser.get("hiveMetastoreUris");
log.info("hiveMetastoreUris: {}", hiveMetastoreUris);
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", hiveMetastoreUris);
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> {
String type = clazz.getSimpleName().toLowerCase();
String rawPath = rawGraphPath + "/" + type;
String claimPath = claimsGraphPath + "/" + type;
deleteGraphTable(spark, clazz, outputGraph, format);
mergeByType(spark, rawPath, claimPath, outputGraph, format, clazz);
});
spark -> mergeByType(spark, rawGraphPath, claimsGraphPath, outputGraph, format, clazz));
}
private static <T extends Oaf> void mergeByType(

View File

@ -15,4 +15,8 @@
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
</configuration>

View File

@ -2,18 +2,32 @@
<parameters>
<property>
<name>graphInputPath</name>
<description>the input path to read graph content</description>
<name>inputGraph</name>
<description>the input graph name (or path)</description>
</property>
<property>
<name>graphOutputPath</name>
<description>the target path to store cleaned graph</description>
<name>outputGraph</name>
<description>the output graph name (or path)</description>
</property>
<property>
<name>isLookupUrl</name>
<name>isLookUpUrl</name>
<description>the address of the lookUp service</description>
</property>
<property>
<name>inputGraphFormat</name>
<value>HIVE</value>
<description>the input graph data format</description>
</property>
<property>
<name>outputGraphFormat</name>
<value>HIVE</value>
<description>the output graph data format</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
@ -48,14 +62,70 @@
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
<property>
<name>sparkSqlWarehouseDir</name>
<value>spark 2.* db directory location</value>
</property>
</parameters>
<start to="fork_clean_graph"/>
<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="should_drop_db"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<decision name="should_drop_db">
<switch>
<case to="fork_clean_graph">${wf:conf('outputGraphFormat') eq 'JSON'}</case>
<case to="reset_DB">${wf:conf('outputGraphFormat') eq 'HIVE'}</case>
<default to="reset_DB"/>
</switch>
</decision>
<action name="reset_DB">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>reset_DB</name>
<class>eu.dnetlib.dhp.common.ResetHiveDbApplication</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}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--dbName</arg><arg>${outputGraph}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="fork_clean_graph"/>
<error to="Kill"/>
</action>
<fork name="fork_clean_graph">
<path start="clean_publication"/>
<path start="clean_dataset"/>
@ -82,12 +152,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/publication</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/publication</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -108,12 +182,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/dataset</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/dataset</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -134,12 +212,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/otherresearchproduct</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/otherresearchproduct</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -160,12 +242,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/software</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/software</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -186,12 +272,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/datasource</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/datasource</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -212,12 +302,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/organization</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/organization</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -238,12 +332,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/project</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/project</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -264,12 +362,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphInputPath}/relation</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/relation</arg>
<arg>--inputGraph</arg><arg>${inputGraph}</arg>
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--inputGraphFormat</arg><arg>${inputGraphFormat}</arg>
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>

View File

@ -19,7 +19,7 @@
},
{
"paramName": "isu",
"paramLongName": "isLookupUrl",
"paramLongName": "isLookUpUrl",
"paramDescription": "the url of the ISLookupService",
"paramRequired": true
}

View File

@ -11,6 +11,10 @@
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
@ -19,8 +23,4 @@
<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>
</configuration>

View File

@ -51,6 +51,10 @@
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
<property>
<name>sparkSqlWarehouseDir</name>
<value>spark 2.* db directory location</value>
</property>
</parameters>
<global>

View File

@ -6,20 +6,32 @@
"paramRequired": false
},
{
"paramName": "in",
"paramLongName": "inputPath",
"paramDescription": "the path to the graph data dump to read",
"paramName": "ig",
"paramLongName": "inputGraph",
"paramDescription": "the input graph name (or path)",
"paramRequired": true
},
{
"paramName": "out",
"paramLongName": "outputPath",
"paramDescription": "the path to store the output graph",
"paramName": "og",
"paramLongName": "outputGraph",
"paramDescription": "the output graph name (or path)",
"paramRequired": true
},
{
"paramName": "igf",
"paramLongName": "inputGraphFormat",
"paramDescription": "the input graph data format",
"paramRequired": true
},
{
"paramName": "ogf",
"paramLongName": "outputGraphFormat",
"paramDescription": "the output graph data format",
"paramRequired": true
},
{
"paramName": "isu",
"paramLongName": "isLookupUrl",
"paramLongName": "isLookUpUrl",
"paramDescription": "url to the ISLookup Service",
"paramRequired": true
},
@ -28,5 +40,11 @@
"paramLongName": "graphTableClassName",
"paramDescription": "class name moelling the graph table",
"paramRequired": true
},
{
"paramName": "hmu",
"paramLongName": "hiveMetastoreUris",
"paramDescription": "the hive metastore uris",
"paramRequired": true
}
]

View File

@ -28,6 +28,10 @@
<description>the output graph data format</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
@ -62,6 +66,10 @@
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
<property>
<name>sparkSqlWarehouseDir</name>
<value>spark 2.* db directory location</value>
</property>
</parameters>
<start to="should_drop_db"/>
@ -93,14 +101,16 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--dbName</arg><arg>${outputGraph}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="fork_merge_graph"/>
<error to="Kill"/>
</action>
<fork name="fork_merge_graph">
<path start="merge_publication"/>
<path start="merge_dataset"/>
@ -127,6 +137,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -136,6 +147,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -156,6 +168,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -165,6 +178,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -185,6 +199,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -194,6 +209,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -214,6 +230,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -223,6 +240,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -243,6 +261,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -252,6 +271,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -272,6 +292,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -281,6 +302,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -301,6 +323,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -310,6 +333,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -330,6 +354,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--betaInputGraph</arg><arg>${betaInputGraph}</arg>
@ -339,6 +364,7 @@
<arg>--outputGraphFormat</arg><arg>${outputGraphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>
<arg>--priority</arg><arg>${priority}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>

View File

@ -34,5 +34,11 @@
"paramLongName": "graphTableClassName",
"paramDescription": "class name associated to the input entity path",
"paramRequired": true
},
{
"paramName": "hmu",
"paramLongName": "hiveMetastoreUris",
"paramDescription": "the hive metastore uris",
"paramRequired": true
}
]

View File

@ -46,5 +46,11 @@
"paramLongName": "priority",
"paramDescription": "decides from which infrastructure the content must win in case of ID clash",
"paramRequired": false
},
{
"paramName": "hmu",
"paramLongName": "hiveMetastoreUris",
"paramDescription": "the hive metastore uris",
"paramRequired": true
}
]

View File

@ -45,7 +45,7 @@
<description>mongo database</description>
</property>
<property>
<name>isLookupUrl</name>
<name>isLookUpUrl</name>
<description>the address of the lookUp service</description>
</property>
<property>
@ -53,6 +53,11 @@
<value></value>
<description>a blacklist of nsprefixes (comma separeted)</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
@ -87,6 +92,10 @@
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
<property>
<name>sparkSqlWarehouseDir</name>
<value>spark 2.* db directory location</value>
</property>
</parameters>
<global>
@ -137,7 +146,7 @@
<arg>--postgresUrl</arg><arg>${postgresURL}</arg>
<arg>--postgresUser</arg><arg>${postgresUser}</arg>
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--action</arg><arg>claims</arg>
<arg>--dbschema</arg><arg>${dbSchema}</arg>
<arg>--nsPrefixBlacklist</arg><arg>${nsPrefixBlacklist}</arg>
@ -190,7 +199,7 @@
<arg>--postgresUrl</arg><arg>${postgresURL}</arg>
<arg>--postgresUser</arg><arg>${postgresUser}</arg>
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--dbschema</arg><arg>${dbSchema}</arg>
<arg>--nsPrefixBlacklist</arg><arg>${nsPrefixBlacklist}</arg>
</java>
@ -274,7 +283,7 @@
</spark-opts>
<arg>--sourcePaths</arg><arg>${contentPath}/db_claims,${contentPath}/oaf_claims,${contentPath}/odf_claims,${contentPath}/oaf_records_invisible</arg>
<arg>--targetPath</arg><arg>${workingDir}/entities_claim</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
</spark>
<ok to="GenerateGraph_claims"/>
<error to="Kill"/>
@ -321,7 +330,7 @@
</spark-opts>
<arg>--sourcePaths</arg><arg>${contentPath}/db_records,${contentPath}/oaf_records,${contentPath}/odf_records</arg>
<arg>--targetPath</arg><arg>${workingDir}/entities</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
</spark>
<ok to="GenerateGraph"/>
<error to="Kill"/>
@ -376,9 +385,11 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--dbName</arg><arg>${outputGraph}</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="fork_merge_claims"/>
<error to="Kill"/>
@ -410,6 +421,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -417,6 +429,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -437,6 +450,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -444,6 +458,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -464,6 +479,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -471,6 +487,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -491,6 +508,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=1920
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -498,6 +516,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -518,6 +537,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=1920
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -525,6 +545,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -545,6 +566,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=200
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -552,6 +574,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -572,6 +595,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=200
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -579,6 +603,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>
@ -599,6 +624,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=200
</spark-opts>
<arg>--rawGraphPath</arg><arg>${workingDir}/graph_raw</arg>
@ -606,6 +632,7 @@
<arg>--outputGraph</arg><arg>${outputGraph}</arg>
<arg>--graphFormat</arg><arg>${graphFormat}</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
</spark>
<ok to="wait_merge"/>
<error to="Kill"/>

View File

@ -30,7 +30,7 @@ public class GraphSupport {
break;
case HIVE:
String table = ModelSupport.tableIdentifier(outputGraph, clazz);
String sql = String.format("DROP TABLE IF EXISTS %s PURGE;", table);
String sql = String.format("DROP TABLE IF EXISTS %s PURGE", table);
log.info("running SQL: '{}'", sql);
spark.sql(sql);
break;
@ -46,14 +46,14 @@ public class GraphSupport {
String type = clazz.getSimpleName().toLowerCase();
String outPath = outputGraph + "/" + type;
log.info("saving graph in {} mode to {}", outputGraph, graphFormat.toString());
log.info("saving graph to path {},", outPath);
writer.option("compression", "gzip").json(outPath);
break;
case HIVE:
final String db_table = ModelSupport.tableIdentifier(outputGraph, clazz);
log.info("saving graph in {} mode to {}", outputGraph, graphFormat.toString());
log.info("saving graph to '{}'", db_table);
writer.saveAsTable(db_table);
break;
@ -63,17 +63,22 @@ public class GraphSupport {
public static <T extends Oaf> Dataset<T> readGraph(
SparkSession spark, String graph, Class<T> clazz, GraphFormat format) {
log.info("reading graph {}, format {}, class {}", graph, format, clazz);
switch (format) {
case JSON:
String path = graph + "/" + clazz.getSimpleName().toLowerCase();
log.info("reading path {}", path);
return spark
.read()
.textFile(graph)
.textFile(path)
.map(
(MapFunction<String, T>) value -> OBJECT_MAPPER.readValue(value, clazz),
Encoders.bean(clazz))
.filter((FilterFunction<T>) value -> Objects.nonNull(ModelSupport.idFn().apply(value)));
case HIVE:
String table = ModelSupport.tableIdentifier(graph, clazz);
log.info("reading table {}", table);
return spark.read().table(table).as(Encoders.bean(clazz));
default:
throw new IllegalStateException(String.format("format not managed: '%s'", format));

View File

@ -7,6 +7,7 @@ import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -36,15 +37,24 @@ public class ResetHiveDbApplication {
.orElseThrow(() -> new IllegalArgumentException("missing DB name"));
log.info("dbName: {}", dbName);
String hiveMetastoreUris = parser.get("hiveMetastoreUris");
log.info("hiveMetastoreUris: {}", hiveMetastoreUris);
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", hiveMetastoreUris);
runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> {
spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", dbName));
spark.sql(String.format("CREATE DATABASE %s", dbName));
runSQL(spark, String.format("DROP DATABASE IF EXISTS %s CASCADE", dbName));
runSQL(spark, String.format("CREATE DATABASE %s", dbName));
});
}
protected static void runSQL(SparkSession spark, String sql) {
log.info("running SQL '{}'", sql);
spark.sqlContext().sql(sql);
}
}

View File

@ -5,6 +5,12 @@
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "hmu",
"paramLongName": "hiveMetastoreUris",
"paramDescription": "the hive metastore uris",
"paramRequired": true
},
{
"paramName": "db",
"paramLongName": "dbName",