dataset based provision WIP
This commit is contained in:
parent
daa26acc9d
commit
24b2c9012e
|
@ -23,6 +23,8 @@ import java.io.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
|
|
||||||
|
import static org.apache.spark.sql.functions.*;
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.asRelatedEntity;
|
import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.asRelatedEntity;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -93,8 +95,9 @@ public class GraphJoiner_v2 implements Serializable {
|
||||||
.union(otherresearchproduct)
|
.union(otherresearchproduct)
|
||||||
.union(software)
|
.union(software)
|
||||||
.union(publication)
|
.union(publication)
|
||||||
.repartition(20000)
|
.repartition(7000)
|
||||||
.write()
|
.write()
|
||||||
|
.partitionBy("id")
|
||||||
.parquet(getOutPath() + "/entities");
|
.parquet(getOutPath() + "/entities");
|
||||||
|
|
||||||
Dataset<Tuple2<String, TypedRow>> entities = getSpark()
|
Dataset<Tuple2<String, TypedRow>> entities = getSpark()
|
||||||
|
@ -108,29 +111,51 @@ public class GraphJoiner_v2 implements Serializable {
|
||||||
t.setOaf(r.getAs("oaf"));
|
t.setOaf(r.getAs("oaf"));
|
||||||
|
|
||||||
return new Tuple2<>(t.getId(), t);
|
return new Tuple2<>(t.getId(), t);
|
||||||
}, Encoders.tuple(Encoders.STRING(), Encoders.kryo(TypedRow.class)));
|
}, Encoders.tuple(Encoders.STRING(), Encoders.kryo(TypedRow.class)))
|
||||||
|
.cache();
|
||||||
|
|
||||||
System.out.println("Entities, number of partitions: " + entities.rdd().getNumPartitions());
|
System.out.println("Entities, number of partitions: " + entities.rdd().getNumPartitions());
|
||||||
System.out.println("Entities schema:");
|
System.out.println("Entities schema:");
|
||||||
entities.printSchema();
|
entities.printSchema();
|
||||||
|
System.out.println("Entities count:" + entities.count());
|
||||||
|
|
||||||
/*
|
|
||||||
// reads the relationships
|
// reads the relationships
|
||||||
Dataset<Relation> rels = readPathRelation(jsc, getInputPath())
|
readPathRelation(jsc, getInputPath())
|
||||||
.groupByKey((MapFunction<Relation, SortableRelationKey>) t -> SortableRelationKey.from(t), Encoders.kryo(SortableRelationKey.class))
|
.groupByKey((MapFunction<Relation, SortableRelationKey>) t -> SortableRelationKey.from(t), Encoders.kryo(SortableRelationKey.class))
|
||||||
.flatMapGroups((FlatMapGroupsFunction<SortableRelationKey, Relation, Relation>) (key, values) -> Iterators.limit(values, MAX_RELS), Encoders.bean(Relation.class))
|
.flatMapGroups((FlatMapGroupsFunction<SortableRelationKey, Relation, Relation>) (key, values) -> Iterators.limit(values, MAX_RELS), Encoders.kryo(Relation.class))
|
||||||
|
.repartition(3000)
|
||||||
|
.write()
|
||||||
|
.partitionBy("source", "target")
|
||||||
|
.parquet(getOutPath() + "/relations");
|
||||||
|
|
||||||
|
Dataset<Relation> rels = getSpark()
|
||||||
|
.read()
|
||||||
|
.load(getOutPath() + "/relations")
|
||||||
|
.map((MapFunction<Row, Relation>) r -> {
|
||||||
|
Relation rel = new Relation();
|
||||||
|
rel.setSource(r.getAs("source"));
|
||||||
|
rel.setTarget(r.getAs("target"));
|
||||||
|
rel.setRelType(r.getAs("relType"));
|
||||||
|
rel.setSubRelType(r.getAs("subRelType"));
|
||||||
|
rel.setRelClass(r.getAs("relClass"));
|
||||||
|
rel.setDataInfo(r.getAs("dataInfo"));
|
||||||
|
rel.setCollectedFrom(r.getList(r.fieldIndex("collectedFrom")));
|
||||||
|
return rel;
|
||||||
|
}, Encoders.kryo(Relation.class))
|
||||||
.cache();
|
.cache();
|
||||||
|
|
||||||
System.out.println("Relation schema:");
|
System.out.println("Relation schema:");
|
||||||
rels.printSchema();
|
System.out.println("Relation, number of partitions: " + rels.rdd().getNumPartitions());
|
||||||
|
System.out.println("Relation schema:");
|
||||||
|
entities.printSchema();
|
||||||
|
System.out.println("Relation count:" + rels.count());
|
||||||
|
|
||||||
|
/*
|
||||||
Dataset<Tuple2<String, Relation>> relsByTarget = rels
|
Dataset<Tuple2<String, Relation>> relsByTarget = rels
|
||||||
.map((MapFunction<Relation, Tuple2<String, Relation>>) r -> new Tuple2<>(r.getTarget(), r), Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class)));
|
.map((MapFunction<Relation, Tuple2<String, Relation>>) r -> new Tuple2<>(r.getTarget(), r), Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class)));
|
||||||
|
|
||||||
System.out.println("Relation by target schema:");
|
|
||||||
relsByTarget.printSchema();
|
|
||||||
|
|
||||||
Dataset<Tuple2<String, EntityRelEntity>> bySource = relsByTarget
|
relsByTarget
|
||||||
.joinWith(entities, relsByTarget.col("_1").equalTo(entities.col("_1")), "inner")
|
.joinWith(entities, relsByTarget.col("_1").equalTo(entities.col("_1")), "inner")
|
||||||
.filter((FilterFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, TypedRow>>>) value -> value._2()._2().getDeleted() == false)
|
.filter((FilterFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, TypedRow>>>) value -> value._2()._2().getDeleted() == false)
|
||||||
.map((MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, TypedRow>>, EntityRelEntity>) t -> {
|
.map((MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, TypedRow>>, EntityRelEntity>) t -> {
|
||||||
|
@ -139,12 +164,28 @@ public class GraphJoiner_v2 implements Serializable {
|
||||||
e.setTarget(asRelatedEntity(t._2()._2()));
|
e.setTarget(asRelatedEntity(t._2()._2()));
|
||||||
return e;
|
return e;
|
||||||
}, Encoders.kryo(EntityRelEntity.class))
|
}, Encoders.kryo(EntityRelEntity.class))
|
||||||
|
.repartition(20000)
|
||||||
|
.write()
|
||||||
|
.parquet(getOutPath() + "/bySource");
|
||||||
|
|
||||||
|
Dataset<Tuple2<String, EntityRelEntity>> bySource = getSpark()
|
||||||
|
.read()
|
||||||
|
.load(getOutPath() + "/bySource")
|
||||||
|
.map(new MapFunction<Row, EntityRelEntity>() {
|
||||||
|
@Override
|
||||||
|
public EntityRelEntity call(Row value) throws Exception {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}, Encoders.kryo(EntityRelEntity.class))
|
||||||
.map((MapFunction<EntityRelEntity, Tuple2<String, EntityRelEntity>>) e -> new Tuple2<>(e.getRelation().getSource(), e),
|
.map((MapFunction<EntityRelEntity, Tuple2<String, EntityRelEntity>>) e -> new Tuple2<>(e.getRelation().getSource(), e),
|
||||||
Encoders.tuple(Encoders.STRING(), Encoders.kryo(EntityRelEntity.class)));
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(EntityRelEntity.class)))
|
||||||
|
|
||||||
System.out.println("bySource schema");
|
System.out.println("bySource schema");
|
||||||
bySource.printSchema();
|
bySource.printSchema();
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
Dataset<EntityRelEntity> joined = entities
|
Dataset<EntityRelEntity> joined = entities
|
||||||
.joinWith(bySource, entities.col("_1").equalTo(bySource.col("_1")), "left")
|
.joinWith(bySource, entities.col("_1").equalTo(bySource.col("_1")), "left")
|
||||||
.map((MapFunction<Tuple2<Tuple2<String, TypedRow>, Tuple2<String, EntityRelEntity>>, EntityRelEntity>) value -> {
|
.map((MapFunction<Tuple2<Tuple2<String, TypedRow>, Tuple2<String, EntityRelEntity>>, EntityRelEntity>) value -> {
|
||||||
|
|
|
@ -27,4 +27,16 @@
|
||||||
<name>spark2EventLogDir</name>
|
<name>spark2EventLogDir</name>
|
||||||
<value>/user/spark/spark2ApplicationHistory</value>
|
<value>/user/spark/spark2ApplicationHistory</value>
|
||||||
</property>
|
</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>
|
||||||
|
<property>
|
||||||
|
<name>oozieActionShareLibForSpark2</name>
|
||||||
|
<value>spark2</value>
|
||||||
|
</property>
|
||||||
</configuration>
|
</configuration>
|
|
@ -25,6 +25,20 @@
|
||||||
<name>sparkExecutorCoresForIndexing</name>
|
<name>sparkExecutorCoresForIndexing</name>
|
||||||
<description>number of cores used by single executor</description>
|
<description>number of cores used by single executor</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>oozieActionShareLibForSpark2</name>
|
||||||
|
<description>oozie action sharelib for spark 2.*</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2ExtraListeners</name>
|
||||||
|
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
|
||||||
|
<description>spark 2.* extra listeners classname</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2SqlQueryExecutionListeners</name>
|
||||||
|
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
|
||||||
|
<description>spark 2.* sql query execution listeners classname</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>spark2YarnHistoryServerAddress</name>
|
<name>spark2YarnHistoryServerAddress</name>
|
||||||
<description>spark 2.* yarn history server address</description>
|
<description>spark 2.* yarn history server address</description>
|
||||||
|
@ -40,12 +54,8 @@
|
||||||
<name-node>${nameNode}</name-node>
|
<name-node>${nameNode}</name-node>
|
||||||
<configuration>
|
<configuration>
|
||||||
<property>
|
<property>
|
||||||
<name>mapreduce.job.queuename</name>
|
<name>oozie.action.sharelib.for.spark</name>
|
||||||
<value>${queueName}</value>
|
<value>${oozieActionShareLibForSpark2}</value>
|
||||||
</property>
|
|
||||||
<property>
|
|
||||||
<name>oozie.launcher.mapred.job.queue.name</name>
|
|
||||||
<value>${oozieLauncherQueueName}</value>
|
|
||||||
</property>
|
</property>
|
||||||
</configuration>
|
</configuration>
|
||||||
</global>
|
</global>
|
||||||
|
@ -76,11 +86,11 @@
|
||||||
<class>eu.dnetlib.dhp.oa.provision.SparkXmlRecordBuilderJob_v2</class>
|
<class>eu.dnetlib.dhp.oa.provision.SparkXmlRecordBuilderJob_v2</class>
|
||||||
<jar>dhp-graph-provision-${projectVersion}.jar</jar>
|
<jar>dhp-graph-provision-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-cores ${sparkExecutorCoresForJoining}
|
--executor-cores=${sparkExecutorCoresForJoining}
|
||||||
--executor-memory ${sparkExecutorMemoryForJoining}
|
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||||
--driver-memory=${sparkDriverMemoryForJoining}
|
--driver-memory=${sparkDriverMemoryForJoining}
|
||||||
--conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
|
@ -107,8 +117,8 @@
|
||||||
--executor-memory ${sparkExecutorMemoryForIndexing}
|
--executor-memory ${sparkExecutorMemoryForIndexing}
|
||||||
--driver-memory=${sparkDriverMemoryForIndexing}
|
--driver-memory=${sparkDriverMemoryForIndexing}
|
||||||
--conf spark.dynamicAllocation.maxExecutors=${sparkExecutorCoresForIndexing}
|
--conf spark.dynamicAllocation.maxExecutors=${sparkExecutorCoresForIndexing}
|
||||||
--conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
|
|
Loading…
Reference in New Issue