From 6b5f9ca9cba9fc90a26e987e3f7a00024e76aae7 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Apr 2020 17:53:07 +0200 Subject: [PATCH 1/9] raw graph creation workflow moved under dhp-graph-mapper, claims integration is included --- dhp-workflows/dhp-aggregation/pom.xml | 12 - .../step3/DispatchEntitiesApplication.java | 71 --- .../regular_all_steps/oozie_app/workflow.xml | 197 ------- dhp-workflows/dhp-graph-mapper/pom.xml | 13 + .../raw}/AbstractMdRecordToOafMapper.java | 40 +- .../raw/DispatchEntitiesApplication.java | 94 +++ .../raw}/GenerateEntitiesApplication.java | 87 ++- .../oa/graph/raw/MergeClaimsApplication.java | 151 +++++ .../raw}/MigrateDbEntitiesApplication.java | 49 +- .../raw}/MigrateMongoMdstoresApplication.java | 19 +- .../dhp/oa/graph/raw}/OafToOafMapper.java | 24 +- .../dhp/oa/graph/raw}/OdfToOafMapper.java | 24 +- .../common}/AbstractMigrationApplication.java | 11 +- .../dhp/oa/graph/raw/common}/DbClient.java | 16 +- .../oa/graph/raw/common}/MdstoreClient.java | 23 +- .../oa/graph/raw/common}/OafMapperUtils.java | 19 +- .../dhp/oa/graph/raw/common}/PacePerson.java | 21 +- .../graph}/dispatch_entities_parameters.json | 12 +- .../graph}/generate_entities_parameters.json | 12 +- .../dhp/oa/graph/merge_claims_parameters.json | 32 ++ .../graph/migrate_actionsets_parameters.json | 10 + .../migrate_db_entities_parameters.json | 0 .../migrate_mongo_mstores_parameters.json | 0 .../dhp/oa/graph}/pace/name_particles.txt | 0 .../raw_all}/oozie_app/config-default.xml | 0 .../oa/graph/raw_all/oozie_app/workflow.xml | 538 ++++++++++++++++++ .../oa/graph/raw_claims}/config-default.xml | 0 .../dhp/oa/graph/raw_claims}/workflow.xml | 0 .../raw_db}/oozie_app/config-default.xml | 0 .../oa/graph/raw_db}/oozie_app/workflow.xml | 0 .../raw_step1}/oozie_app/config-default.xml | 0 .../graph/raw_step1}/oozie_app/workflow.xml | 0 .../raw_step2}/oozie_app/config-default.xml | 0 .../graph/raw_step2}/oozie_app/workflow.xml | 0 .../raw_step3}/oozie_app/config-default.xml | 0 .../graph/raw_step3}/oozie_app/workflow.xml | 0 .../dnetlib/dhp/oa/graph}/sql/queryClaims.sql | 0 .../sql/queryDatasourceOrganization.sql | 0 .../dhp/oa/graph}/sql/queryDatasources.sql | 0 .../dhp/oa/graph}/sql/queryOrganizations.sql | 0 .../sql/queryOrganizationsFromOpenOrgsDB.sql | 0 .../graph}/sql/queryProjectOrganization.sql | 0 .../dhp/oa/graph}/sql/queryProjects.sql | 0 .../graph}/sql/queryProjects_production.sql | 0 .../sql/querySimilarityFromOpenOrgsDB.sql | 0 .../transform_actionsets_parameters.json | 20 + .../oa/graph/GraphHiveImporterJobTest.java | 17 +- .../dhp/oa/graph/raw}/MappersTest.java | 29 +- .../MigrateDbEntitiesApplicationTest.java | 2 +- .../raw}/claimscontext_resultset_entry.json | 0 .../graph/raw}/claimsrel_resultset_entry.json | 0 ...atasourceorganization_resultset_entry.json | 0 .../raw}/datasources_resultset_entry.json | 0 .../dnetlib/dhp/oa/graph/raw}/oaf_record.xml | 0 .../dnetlib/dhp/oa/graph/raw}/odf_dataset.xml | 0 .../dhp/oa/graph/raw}/odf_software.xml | 0 .../raw}/organizations_resultset_entry.json | 0 .../projectorganization_resultset_entry.json | 0 .../graph/raw}/projects_resultset_entry.json | 0 pom.xml | 6 + 60 files changed, 1013 insertions(+), 536 deletions(-) delete mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step3/DispatchEntitiesApplication.java delete mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/workflow.xml rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw}/AbstractMdRecordToOafMapper.java (90%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw}/GenerateEntitiesApplication.java (70%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw}/MigrateDbEntitiesApplication.java (91%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw}/MigrateMongoMdstoresApplication.java (89%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw}/OafToOafMapper.java (91%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw}/OdfToOafMapper.java (93%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common}/AbstractMigrationApplication.java (98%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common}/DbClient.java (90%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common}/MdstoreClient.java (98%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common}/OafMapperUtils.java (92%) rename dhp-workflows/{dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils => dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common}/PacePerson.java (97%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/dispatch_entities_parameters.json (61%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/generate_entities_parameters.json (82%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_actionsets_parameters.json rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/migrate_db_entities_parameters.json (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/migrate_mongo_mstores_parameters.json (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/pace/name_particles.txt (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all}/oozie_app/config-default.xml (100%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims}/config-default.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims}/workflow.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1 => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db}/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1_onlydb => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db}/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1_onlydb => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step1}/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1 => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step1}/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2 => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step2}/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2 => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step2}/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3 => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step3}/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3 => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step3}/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryClaims.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryDatasourceOrganization.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryDatasources.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryOrganizations.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryOrganizationsFromOpenOrgsDB.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryProjectOrganization.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryProjects.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/queryProjects_production.sql (100%) rename dhp-workflows/{dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration => dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph}/sql/querySimilarityFromOpenOrgsDB.sql (100%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/transform_actionsets_parameters.json rename dhp-workflows/{dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw}/MappersTest.java (91%) rename dhp-workflows/{dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw}/MigrateDbEntitiesApplicationTest.java (99%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/claimscontext_resultset_entry.json (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/claimsrel_resultset_entry.json (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/datasourceorganization_resultset_entry.json (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/datasources_resultset_entry.json (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/oaf_record.xml (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/odf_dataset.xml (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/odf_software.xml (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/organizations_resultset_entry.json (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/projectorganization_resultset_entry.json (100%) rename dhp-workflows/{dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1 => dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw}/projects_resultset_entry.json (100%) diff --git a/dhp-workflows/dhp-aggregation/pom.xml b/dhp-workflows/dhp-aggregation/pom.xml index 8d4d880b3..3e7b1a375 100644 --- a/dhp-workflows/dhp-aggregation/pom.xml +++ b/dhp-workflows/dhp-aggregation/pom.xml @@ -99,23 +99,11 @@ jaxen jaxen - - - org.mongodb - mongo-java-driver - - org.apache.hadoop hadoop-distcp - - - org.postgresql - postgresql - 42.2.10 - diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step3/DispatchEntitiesApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step3/DispatchEntitiesApplication.java deleted file mode 100644 index 4ee24cba0..000000000 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step3/DispatchEntitiesApplication.java +++ /dev/null @@ -1,71 +0,0 @@ -package eu.dnetlib.dhp.migration.step3; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.compress.GzipCodec; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.Datasource; -import eu.dnetlib.dhp.schema.oaf.Organization; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Project; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Software; - -public class DispatchEntitiesApplication { - - private static final Log log = LogFactory.getLog(DispatchEntitiesApplication.class); - - public static void main(final String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils.toString(MigrateMongoMdstoresApplication.class - .getResourceAsStream("/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json"))); - parser.parseArgument(args); - - try (final SparkSession spark = newSparkSession(parser); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext())) { - - final String sourcePath = parser.get("sourcePath"); - final String targetPath = parser.get("graphRawPath"); - - processEntity(sc, Publication.class, sourcePath, targetPath); - processEntity(sc, Dataset.class, sourcePath, targetPath); - processEntity(sc, Software.class, sourcePath, targetPath); - processEntity(sc, OtherResearchProduct.class, sourcePath, targetPath); - processEntity(sc, Datasource.class, sourcePath, targetPath); - processEntity(sc, Organization.class, sourcePath, targetPath); - processEntity(sc, Project.class, sourcePath, targetPath); - processEntity(sc, Relation.class, sourcePath, targetPath); - } - } - - private static SparkSession newSparkSession(final ArgumentApplicationParser parser) { - return SparkSession - .builder() - .appName(DispatchEntitiesApplication.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); - } - - private static void processEntity(final JavaSparkContext sc, final Class clazz, final String sourcePath, final String targetPath) { - final String type = clazz.getSimpleName().toLowerCase(); - - log.info(String.format("Processing entities (%s) in file: %s", type, sourcePath)); - - sc.textFile(sourcePath) - .filter(l -> isEntityType(l, type)) - .map(l -> StringUtils.substringAfter(l, "|")) - .saveAsTextFile(targetPath + "/" + type, GzipCodec.class); // use repartition(XXX) ??? - } - - private static boolean isEntityType(final String line, final String type) { - return StringUtils.substringBefore(line, "|").equalsIgnoreCase(type); - } - -} diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/workflow.xml deleted file mode 100644 index 42ab59822..000000000 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/workflow.xml +++ /dev/null @@ -1,197 +0,0 @@ - - - - - workingPath - /tmp/dhp_migration - the base path to store temporary intermediate data - - - graphBasePath - the target path to store raw graph - - - reuseContent - false - should import content from the aggregator or reuse a previous version - - - postgresURL - the postgres URL to access to the database - - - postgresUser - the user postgres - - - postgresPassword - the password postgres - - - mongoURL - mongoDB url, example: mongodb://[username:password@]host[:port] - - - mongoDb - mongo database - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - - - ${jobTracker} - ${nameNode} - - - mapreduce.job.queuename - ${queueName} - - - oozie.launcher.mapred.job.queue.name - ${oozieLauncherQueueName} - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - ${wf:conf('reuseContent') eq false} - ${wf:conf('reuseContent') eq true} - - - - - - - - - - - - - - - - eu.dnetlib.dhp.migration.step1.MigrateDbEntitiesApplication - -p${workingPath}/db_records - -pgurl${postgresURL} - -pguser${postgresUser} - -pgpasswd${postgresPassword} - - - - - - - - eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication - -p${workingPath}/odf_records - -mongourl${mongoURL} - -mongodb${mongoDb} - -fODF - -lstore - -icleaned - - - - - - - - eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication - -p${workingPath}/oaf_records - -mongourl${mongoURL} - -mongodb${mongoDb} - -fOAF - -lstore - -icleaned - - - - - - - - - - - - - - - - yarn - cluster - GenerateEntities - eu.dnetlib.dhp.migration.step2.GenerateEntitiesApplication - dhp-aggregation-${projectVersion}.jar - - --executor-memory ${sparkExecutorMemory} - --executor-cores ${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" - --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" - --conf spark.sql.warehouse.dir="/user/hive/warehouse" - - -mt yarn-cluster - -s${workingPath}/db_records,${workingPath}/oaf_records,${workingPath}/odf_records - -t${workingPath}/all_entities - -pgurl${postgresURL} - -pguser${postgresUser} - -pgpasswd${postgresPassword} - - - - - - - - - - - - - - - - - yarn - cluster - GenerateGraph - eu.dnetlib.dhp.migration.step3.DispatchEntitiesApplication - dhp-aggregation-${projectVersion}.jar - - --executor-memory ${sparkExecutorMemory} - --executor-cores ${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" - --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" - --conf spark.sql.warehouse.dir="/user/hive/warehouse" - - -mt yarn-cluster - -s${workingPath}/all_entities - -g${graphBasePath}/graph_raw - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/pom.xml b/dhp-workflows/dhp-graph-mapper/pom.xml index 9876edc16..d25446bbc 100644 --- a/dhp-workflows/dhp-graph-mapper/pom.xml +++ b/dhp-workflows/dhp-graph-mapper/pom.xml @@ -40,6 +40,7 @@ dhp-schemas ${project.version} + com.jayway.jsonpath json-path @@ -48,6 +49,18 @@ org.mongodb mongo-java-driver + + dom4j + dom4j + + + jaxen + jaxen + + + org.postgresql + postgresql + diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java similarity index 90% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/AbstractMdRecordToOafMapper.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index 18a62124b..7d99a4774 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -1,45 +1,15 @@ -package eu.dnetlib.dhp.migration.step2; - -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.dataInfo; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.journal; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.keyValue; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.listFields; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.oaiIProvenance; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.qualifier; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.structuredProperty; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +package eu.dnetlib.dhp.oa.graph.raw; +import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentFactory; import org.dom4j.DocumentHelper; import org.dom4j.Node; -import eu.dnetlib.dhp.schema.oaf.Author; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.Field; -import eu.dnetlib.dhp.schema.oaf.GeoLocation; -import eu.dnetlib.dhp.schema.oaf.Instance; -import eu.dnetlib.dhp.schema.oaf.Journal; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.OAIProvenance; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.dhp.schema.oaf.Software; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import java.util.*; + +import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.*; public abstract class AbstractMdRecordToOafMapper { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java new file mode 100644 index 000000000..0b47db588 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java @@ -0,0 +1,94 @@ +package eu.dnetlib.dhp.oa.graph.raw; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +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.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +public class DispatchEntitiesApplication { + + private static final Logger log = LoggerFactory.getLogger(DispatchEntitiesApplication.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils.toString(MigrateMongoMdstoresApplication.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/graph/dispatch_entities_parameters.json"))); + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String sourcePath = parser.get("sourcePath"); + final String targetPath = parser.get("graphRawPath"); + + SparkConf conf = new SparkConf(); + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + removeOutputDir(spark, targetPath); + + processEntity(spark, Publication.class, sourcePath, targetPath); + processEntity(spark, Dataset.class, sourcePath, targetPath); + processEntity(spark, Software.class, sourcePath, targetPath); + processEntity(spark, OtherResearchProduct.class, sourcePath, targetPath); + processEntity(spark, Datasource.class, sourcePath, targetPath); + processEntity(spark, Organization.class, sourcePath, targetPath); + processEntity(spark, Project.class, sourcePath, targetPath); + processEntity(spark, Relation.class, sourcePath, targetPath); + }); + } + + private static void processEntity(final SparkSession spark, final Class clazz, final String sourcePath, final String targetPath) { + final String type = clazz.getSimpleName().toLowerCase(); + + log.info(String.format("Processing entities (%s) in file: %s", type, sourcePath)); + + spark.read() + .textFile(sourcePath) + .filter((FilterFunction) value -> isEntityType(value, type)) + .map((MapFunction) value -> StringUtils.substringAfter(value, "|"), Encoders.STRING()) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)) + .write() + .mode(SaveMode.Overwrite) + .parquet(targetPath + "/" + type); + + /* + JavaSparkContext.fromSparkContext(spark.sparkContext()) + .textFile(sourcePath) + .filter(l -> isEntityType(l, type)) + .map(l -> StringUtils.substringAfter(l, "|")) + .saveAsTextFile(targetPath + "/" + type, GzipCodec.class); // use repartition(XXX) ??? + + */ + } + + private static boolean isEntityType(final String line, final String type) { + return StringUtils.substringBefore(line, "|").equalsIgnoreCase(type); + } + + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/GenerateEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java similarity index 70% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/GenerateEntitiesApplication.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java index 7f907b0c8..a9f331f53 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/GenerateEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java @@ -1,13 +1,10 @@ -package eu.dnetlib.dhp.migration.step2; - -import java.io.IOException; -import java.sql.SQLException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; +package eu.dnetlib.dhp.oa.graph.raw; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; +import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; @@ -16,37 +13,38 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.GzipCodec; +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 com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication; -import eu.dnetlib.dhp.migration.utils.DbClient; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.Datasource; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Organization; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Project; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Software; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Tuple2; +import java.io.IOException; +import java.sql.SQLException; +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + public class GenerateEntitiesApplication { - private static final Log log = LogFactory.getLog(GenerateEntitiesApplication.class); + private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class); public static void main(final String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils.toString(MigrateMongoMdstoresApplication.class - .getResourceAsStream("/eu/dnetlib/dhp/migration/generate_entities_parameters.json"))); + .getResourceAsStream("/eu/dnetlib/dhp/oa/graph/generate_entities_parameters.json"))); parser.parseArgument(args); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + final String sourcePaths = parser.get("sourcePaths"); final String targetPath = parser.get("targetPath"); @@ -56,31 +54,27 @@ public class GenerateEntitiesApplication { final Map code2name = loadClassNames(dbUrl, dbUser, dbPassword); - try (final SparkSession spark = newSparkSession(parser); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext())) { - final List existingSourcePaths = Arrays.stream(sourcePaths.split(",")).filter(p -> exists(sc, p)).collect(Collectors.toList()); - generateEntities(sc, code2name, existingSourcePaths, targetPath); - } + SparkConf conf = new SparkConf(); + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + removeOutputDir(spark, targetPath); + generateEntities(spark, code2name, sourcePaths, targetPath); + }); } - private static SparkSession newSparkSession(final ArgumentApplicationParser parser) { - return SparkSession - .builder() - .appName(GenerateEntitiesApplication.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); - } - - private static void generateEntities(final JavaSparkContext sc, + private static void generateEntities(final SparkSession spark, final Map code2name, - final List sourcePaths, + final String sourcePaths, final String targetPath) { + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + final List existingSourcePaths = Arrays.stream(sourcePaths.split(",")).filter(p -> exists(sc, p)).collect(Collectors.toList()); + log.info("Generate entities from files:"); - sourcePaths.forEach(log::info); + existingSourcePaths.forEach(log::info); JavaRDD inputRdd = sc.emptyRDD(); - for (final String sp : sourcePaths) { + for (final String sp : existingSourcePaths) { inputRdd = inputRdd.union(sc.sequenceFile(sp, Text.class, Text.class) .map(k -> new Tuple2<>(k._1().toString(), k._2().toString())) .map(k -> convertToListOaf(k._1(), k._2(), code2name)) @@ -88,7 +82,8 @@ public class GenerateEntitiesApplication { .map(oaf -> oaf.getClass().getSimpleName().toLowerCase() + "|" + convertToJson(oaf))); } - inputRdd.saveAsTextFile(targetPath, GzipCodec.class); + inputRdd + .saveAsTextFile(targetPath, GzipCodec.class); } @@ -163,11 +158,15 @@ public class GenerateEntitiesApplication { private static boolean exists(final JavaSparkContext context, final String pathToFile) { try { - final FileSystem hdfs = org.apache.hadoop.fs.FileSystem.get(context.hadoopConfiguration()); + final FileSystem hdfs = FileSystem.get(context.hadoopConfiguration()); final Path path = new Path(pathToFile); return hdfs.exists(path); } catch (final IOException e) { throw new RuntimeException(e); } } + + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java new file mode 100644 index 000000000..85e4f3663 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java @@ -0,0 +1,151 @@ +package eu.dnetlib.dhp.oa.graph.raw; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +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.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; +import scala.Tuple2; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; + +import java.util.Objects; +import java.util.Optional; +import java.util.function.Function; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; + +public class MergeClaimsApplication { + + private static final Logger log = LoggerFactory.getLogger(MergeClaimsApplication.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils.toString(MigrateMongoMdstoresApplication.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json"))); + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String rawGraphPath = parser.get("rawGraphPath"); + log.info("rawGraphPath: {}", rawGraphPath); + + final String claimsGraphPath = parser.get("claimsGraphPath"); + log.info("claimsGraphPath: {}", claimsGraphPath); + + final String outputRawGaphPath = parser.get("outputRawGaphPath"); + log.info("outputRawGaphPath: {}", outputRawGaphPath); + + String graphTableClassName = parser.get("graphTableClassName"); + log.info("graphTableClassName: {}", graphTableClassName); + + Class clazz = (Class) Class.forName(graphTableClassName); + + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + String type = clazz.getSimpleName().toLowerCase(); + + String rawPath = rawGraphPath + "/" + type; + String claimPath = claimsGraphPath + "/" + type; + String outPath = outputRawGaphPath + "/" + type; + + removeOutputDir(spark, outPath); + mergeByType(spark, rawPath, claimPath, outPath, clazz); + }); + } + + private static void mergeByType(SparkSession spark, String rawPath, String claimPath, String outPath, Class clazz) { + Dataset> raw = readFromPath(spark, rawPath, clazz) + .map((MapFunction>) value -> new Tuple2<>(idFn().apply(value), value), Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + + final JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + Dataset> claim = jsc.broadcast(readFromPath(spark, claimPath, clazz)) + .getValue() + .map((MapFunction>) value -> new Tuple2<>(idFn().apply(value), value), Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + + /* + Dataset> claim = readFromPath(spark, claimPath, clazz) + .map((MapFunction>) value -> new Tuple2<>(idFn().apply(value), value), Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + */ + + raw.joinWith(claim, raw.col("_1").equalTo(claim.col("_1")), "full_outer") + .map((MapFunction, Tuple2>, T>) value -> { + + Optional> opRaw = Optional.ofNullable(value._1()); + Optional> opClaim = Optional.ofNullable(value._2()); + + return opRaw.isPresent() ? opRaw.get()._2() : opClaim.isPresent() ? opClaim.get()._2() : null; + }, Encoders.bean(clazz)) + .filter(Objects::nonNull) + .write() + .mode(SaveMode.Overwrite) + .parquet(outPath); + } + + private static Dataset readFromPath(SparkSession spark, String path, Class clazz) { + return spark.read() + .load(path) + .as(Encoders.bean(clazz)) + .filter((FilterFunction) value -> Objects.nonNull(idFn().apply(value))); + } + + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } + + private static Function idFn() { + return x -> { + if (isSubClass(x, Relation.class)) { + return idFnForRelation(x); + } + return idFnForOafEntity(x); + }; + } + + private static String idFnForRelation(T t) { + Relation r = (Relation) t; + return Optional.ofNullable(r.getSource()) + .map(source -> Optional.ofNullable(r.getTarget()) + .map(target -> Optional.ofNullable(r.getRelType()) + .map(relType -> Optional.ofNullable(r.getSubRelType()) + .map(subRelType -> Optional.ofNullable(r.getRelClass()) + .map(relClass -> String.join(source, target, relType, subRelType, relClass)) + .orElse(String.join(source, target, relType, subRelType)) + ) + .orElse(String.join(source, target, relType)) + ) + .orElse(String.join(source, target)) + ) + .orElse(source) + ) + .orElse(null); + } + + private static String idFnForOafEntity(T t) { + return ((OafEntity) t).getId(); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java similarity index 91% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplication.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index 7db2b1772..f8f6b58cc 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -1,14 +1,13 @@ -package eu.dnetlib.dhp.migration.step1; +package eu.dnetlib.dhp.oa.graph.raw; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.asString; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.dataInfo; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.journal; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.listFields; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.listKeyValues; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.qualifier; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.structuredProperty; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; +import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import java.io.Closeable; import java.io.IOException; @@ -22,31 +21,7 @@ import java.util.List; import java.util.function.Consumer; import java.util.function.Function; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.migration.utils.AbstractMigrationApplication; -import eu.dnetlib.dhp.migration.utils.DbClient; -import eu.dnetlib.dhp.schema.oaf.Context; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.Datasource; -import eu.dnetlib.dhp.schema.oaf.Field; -import eu.dnetlib.dhp.schema.oaf.Journal; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Organization; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Project; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.dhp.schema.oaf.Software; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.*; public class MigrateDbEntitiesApplication extends AbstractMigrationApplication implements Closeable { @@ -61,7 +36,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i public static void main(final String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils.toString(MigrateDbEntitiesApplication.class.getResourceAsStream("/eu/dnetlib/dhp/migration/migrate_db_entities_parameters.json"))); + IOUtils.toString(MigrateDbEntitiesApplication.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/migrate_db_entities_parameters.json"))); parser.parseArgument(args); @@ -111,7 +86,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i } public void execute(final String sqlFile, final Function> producer) throws Exception { - final String sql = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/migration/sql/" + sqlFile)); + final String sql = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/sql/" + sqlFile)); final Consumer consumer = rs -> producer.apply(rs).forEach(oaf -> emitOaf(oaf)); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateMongoMdstoresApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateMongoMdstoresApplication.java similarity index 89% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateMongoMdstoresApplication.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateMongoMdstoresApplication.java index b1de31326..585209ac9 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateMongoMdstoresApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateMongoMdstoresApplication.java @@ -1,18 +1,17 @@ -package eu.dnetlib.dhp.migration.step1; +package eu.dnetlib.dhp.oa.graph.raw; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; +import eu.dnetlib.dhp.oa.graph.raw.common.MdstoreClient; +import org.apache.commons.io.IOUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import java.io.Closeable; import java.io.IOException; import java.util.Map; import java.util.Map.Entry; -import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.migration.utils.AbstractMigrationApplication; -import eu.dnetlib.dhp.migration.utils.MdstoreClient; - public class MigrateMongoMdstoresApplication extends AbstractMigrationApplication implements Closeable { private static final Log log = LogFactory.getLog(MigrateMongoMdstoresApplication.class); @@ -21,7 +20,7 @@ public class MigrateMongoMdstoresApplication extends AbstractMigrationApplicatio public static void main(final String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils.toString(MigrateMongoMdstoresApplication.class.getResourceAsStream("/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json"))); + IOUtils.toString(MigrateMongoMdstoresApplication.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/migrate_mongo_mstores_parameters.json"))); parser.parseArgument(args); final String mongoBaseUrl = parser.get("mongoBaseUrl"); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java similarity index 91% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OafToOafMapper.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index 110abc486..aed582d8f 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -1,27 +1,17 @@ -package eu.dnetlib.dhp.migration.step2; +package eu.dnetlib.dhp.oa.graph.raw; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field; +import eu.dnetlib.dhp.oa.graph.raw.common.PacePerson; +import eu.dnetlib.dhp.schema.oaf.*; +import org.dom4j.Document; +import org.dom4j.Node; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.dom4j.Document; -import org.dom4j.Node; - -import eu.dnetlib.dhp.migration.utils.PacePerson; -import eu.dnetlib.dhp.schema.oaf.Author; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Field; -import eu.dnetlib.dhp.schema.oaf.GeoLocation; -import eu.dnetlib.dhp.schema.oaf.Instance; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.createOpenaireId; +import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.field; public class OafToOafMapper extends AbstractMdRecordToOafMapper { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java similarity index 93% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OdfToOafMapper.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index b4868b8f9..6a6def977 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -1,28 +1,16 @@ -package eu.dnetlib.dhp.migration.step2; +package eu.dnetlib.dhp.oa.graph.raw; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field; -import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.structuredProperty; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.Node; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.apache.commons.lang3.StringUtils; -import org.dom4j.Document; -import org.dom4j.Node; - -import eu.dnetlib.dhp.schema.oaf.Author; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Field; -import eu.dnetlib.dhp.schema.oaf.GeoLocation; -import eu.dnetlib.dhp.schema.oaf.Instance; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.*; public class OdfToOafMapper extends AbstractMdRecordToOafMapper { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/AbstractMigrationApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/AbstractMigrationApplication.java similarity index 98% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/AbstractMigrationApplication.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/AbstractMigrationApplication.java index e1a5e5fa7..aec1ea50d 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/AbstractMigrationApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/AbstractMigrationApplication.java @@ -1,9 +1,6 @@ -package eu.dnetlib.dhp.migration.utils; - -import java.io.Closeable; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicInteger; +package eu.dnetlib.dhp.oa.graph.raw.common; +import eu.dnetlib.dhp.schema.oaf.Oaf; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -12,7 +9,9 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.codehaus.jackson.map.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Oaf; +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; public class AbstractMigrationApplication implements Closeable { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/DbClient.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java similarity index 90% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/DbClient.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java index 8e9784346..9c0562946 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/DbClient.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java @@ -1,18 +1,14 @@ -package eu.dnetlib.dhp.migration.utils; - -import java.io.Closeable; -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.function.Consumer; +package eu.dnetlib.dhp.oa.graph.raw.common; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import java.io.Closeable; +import java.io.IOException; +import java.sql.*; +import java.util.function.Consumer; + public class DbClient implements Closeable { private static final Log log = LogFactory.getLog(DbClient.class); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/MdstoreClient.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MdstoreClient.java similarity index 98% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/MdstoreClient.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MdstoreClient.java index 612503da7..ac700ef63 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/MdstoreClient.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MdstoreClient.java @@ -1,4 +1,14 @@ -package eu.dnetlib.dhp.migration.utils; +package eu.dnetlib.dhp.oa.graph.raw.common; + +import com.google.common.collect.Iterables; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientURI; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.bson.Document; import java.io.Closeable; import java.io.IOException; @@ -7,17 +17,6 @@ import java.util.HashMap; import java.util.Map; import java.util.stream.StreamSupport; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.bson.Document; - -import com.google.common.collect.Iterables; -import com.mongodb.MongoClient; -import com.mongodb.MongoClientURI; -import com.mongodb.client.MongoCollection; -import com.mongodb.client.MongoDatabase; - public class MdstoreClient implements Closeable { private final MongoClient client; diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/OafMapperUtils.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java similarity index 92% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/OafMapperUtils.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java index 8e51c1858..d02070a8b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/OafMapperUtils.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java @@ -1,4 +1,8 @@ -package eu.dnetlib.dhp.migration.utils; +package eu.dnetlib.dhp.oa.graph.raw.common; + +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.utils.DHPUtils; +import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; import java.util.Arrays; @@ -6,19 +10,6 @@ import java.util.List; import java.util.Objects; import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; - -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.ExtraInfo; -import eu.dnetlib.dhp.schema.oaf.Field; -import eu.dnetlib.dhp.schema.oaf.Journal; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.OAIProvenance; -import eu.dnetlib.dhp.schema.oaf.OriginDescription; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import eu.dnetlib.dhp.utils.DHPUtils; - public class OafMapperUtils { public static KeyValue keyValue(final String k, final String v) { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/PacePerson.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/PacePerson.java similarity index 97% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/PacePerson.java rename to dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/PacePerson.java index 69e128e63..a72788728 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/PacePerson.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/PacePerson.java @@ -1,19 +1,18 @@ -package eu.dnetlib.dhp.migration.utils; - -import java.nio.charset.Charset; -import java.text.Normalizer; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.text.WordUtils; +package eu.dnetlib.dhp.oa.graph.raw.common; 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 org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.text.WordUtils; + +import java.nio.charset.Charset; +import java.text.Normalizer; +import java.util.HashSet; +import java.util.List; +import java.util.Set; public class PacePerson { @@ -105,7 +104,7 @@ public class PacePerson { private List splitTerms(final String s) { if (particles == null) { - particles = loadFromClasspath("/eu/dnetlib/dhp/migration/pace/name_particles.txt"); + particles = loadFromClasspath("/eu/dnetlib/dhp/oa/graph/pace/name_particles.txt"); } final List list = Lists.newArrayList(); diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/dispatch_entities_parameters.json similarity index 61% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/dispatch_entities_parameters.json index 8c81290ca..7d995f39a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/dispatch_entities_parameters.json @@ -1,16 +1,16 @@ [ + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "when true will stop SparkSession after job execution", + "paramRequired": false + }, { "paramName": "s", "paramLongName": "sourcePath", "paramDescription": "the source path", "paramRequired": true }, - { - "paramName": "mt", - "paramLongName": "master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName": "g", "paramLongName": "graphRawPath", diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/generate_entities_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/generate_entities_parameters.json similarity index 82% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/generate_entities_parameters.json rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/generate_entities_parameters.json index 53ee010c4..293bf041b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/generate_entities_parameters.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/generate_entities_parameters.json @@ -1,16 +1,16 @@ [ + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "when true will stop SparkSession after job execution", + "paramRequired": false + }, { "paramName": "s", "paramLongName": "sourcePaths", "paramDescription": "the HDFS source paths which contains the sequential file (comma separated)", "paramRequired": true }, - { - "paramName": "mt", - "paramLongName": "master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName": "t", "paramLongName": "targetPath", diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json new file mode 100644 index 000000000..686fea643 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json @@ -0,0 +1,32 @@ +[ + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "when true will stop SparkSession after job execution", + "paramRequired": false + }, + { + "paramName": "rgp", + "paramLongName": "rawGraphPath", + "paramDescription": "the raw graph path", + "paramRequired": true + }, + { + "paramName": "cgp", + "paramLongName": "claimsGraphPath", + "paramDescription": "the path of the claims graph", + "paramRequired": true + }, + { + "paramName": "ogp", + "paramLongName": "outputRawGaphPath", + "paramDescription": "the path of output graph, combining raw and claims", + "paramRequired": true + }, + { + "paramName": "clazz", + "paramLongName": "graphTableClassName", + "paramDescription": "class name associated to the input entity path", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_actionsets_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_actionsets_parameters.json new file mode 100644 index 000000000..c4910ec61 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_actionsets_parameters.json @@ -0,0 +1,10 @@ +[ + {"paramName":"is", "paramLongName":"isLookupUrl", "paramDescription": "URL of the isLookUp Service", "paramRequired": true}, + {"paramName":"sn", "paramLongName":"sourceNameNode", "paramDescription": "nameNode of the source cluster", "paramRequired": true}, + {"paramName":"tn", "paramLongName":"targetNameNode", "paramDescription": "namoNode of the target cluster", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingDirectory", "paramDescription": "working directory", "paramRequired": true}, + {"paramName":"nm", "paramLongName":"distcp_num_maps", "paramDescription": "maximum number of map tasks used in the distcp process", "paramRequired": true}, + {"paramName":"mm", "paramLongName":"distcp_memory_mb", "paramDescription": "memory for distcp action copying actionsets from remote cluster", "paramRequired": true}, + {"paramName":"tt", "paramLongName":"distcp_task_timeout", "paramDescription": "timeout for distcp copying actions from remote cluster", "paramRequired": true}, + {"paramName":"tr", "paramLongName":"transform_only", "paramDescription": "activate tranform-only mode. Only apply transformation step", "paramRequired": true} +] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_db_entities_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_db_entities_parameters.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_db_entities_parameters.json rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_db_entities_parameters.json diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_mongo_mstores_parameters.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/migrate_mongo_mstores_parameters.json diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/pace/name_particles.txt b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/pace/name_particles.txt similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/pace/name_particles.txt rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/pace/name_particles.txt diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml new file mode 100644 index 000000000..11c8f71a1 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml @@ -0,0 +1,538 @@ + + + + + graphBasePath + the target path to store raw graph + + + reuseContent + false + should import content from the aggregator or reuse a previous version + + + postgresURL + the postgres URL to access to the database + + + postgresUser + the user postgres + + + postgresPassword + the password postgres + + + mongoURL + mongoDB url, example: mongodb://[username:password@]host[:port] + + + mongoDb + mongo database + + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + ${jobTracker} + ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${wf:conf('reuseContent') eq false} + ${wf:conf('reuseContent') eq true} + + + + + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication + -p${workingDir}/db_claims + -pgurl${postgresURL} + -pguser${postgresUser} + -pgpasswd${postgresPassword} + -aclaims + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication + -p${workingDir}/odf_claims + -mongourl${mongoURL} + -mongodb${mongoDb} + -fODF + -lstore + -iclaim + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication + -p${workingDir}/oaf_claims + -mongourl${mongoURL} + -mongodb${mongoDb} + -fOAF + -lstore + -iclaim + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication + -p${workingDir}/db_records + -pgurl${postgresURL} + -pguser${postgresUser} + -pgpasswd${postgresPassword} + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication + -p${workingDir}/odf_records + -mongourl${mongoURL} + -mongodb${mongoDb} + -fODF + -lstore + -icleaned + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication + -p${workingDir}/oaf_records + -mongourl${mongoURL} + -mongodb${mongoDb} + -fOAF + -lstore + -icleaned + + + + + + + + + + + + + + + yarn + cluster + GenerateEntities_claim + eu.dnetlib.dhp.oa.graph.raw.GenerateEntitiesApplication + dhp-graph-mapper-${projectVersion}.jar + + --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} + + -s${workingDir}/db_claims,${workingDir}/oaf_claims,${workingDir}/odf_claims + -t${workingDir}/entities_claim + -pgurl${postgresURL} + -pguser${postgresUser} + -pgpasswd${postgresPassword} + + + + + + + + yarn + cluster + GenerateGraph_claims + eu.dnetlib.dhp.oa.graph.raw.DispatchEntitiesApplication + dhp-graph-mapper-${projectVersion}.jar + + --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} + + -s${workingDir}/entities_claim + -g${workingDir}/graph_claims + + + + + + + + yarn + cluster + GenerateEntities + eu.dnetlib.dhp.oa.graph.raw.GenerateEntitiesApplication + dhp-graph-mapper-${projectVersion}.jar + + --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} + + -s${workingDir}/db_records,${workingDir}/oaf_records,${workingDir}/odf_records + -t${workingDir}/entities + -pgurl${postgresURL} + -pguser${postgresUser} + -pgpasswd${postgresPassword} + + + + + + + + yarn + cluster + GenerateGraph + eu.dnetlib.dhp.oa.graph.raw.DispatchEntitiesApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=7680 + + -s${workingDir}/entities + -g${workingDir}/graph_raw + + + + + + + + + + + + + + + + + + + + + + yarn + cluster + MergeClaims_publication + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=7680 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Publication + + + + + + + + yarn + cluster + MergeClaims_dataset + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=7680 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Dataset + + + + + + + + yarn + cluster + MergeClaims_relation + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=3840 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Relation + + + + + + + + yarn + cluster + MergeClaims_software + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=1920 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Software + + + + + + + + yarn + cluster + MergeClaims_otherresearchproduct + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=1920 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + + + + + + + + yarn + cluster + MergeClaims_datasource + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=200 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Datasource + + + + + + + + yarn + cluster + MergeClaims_organization + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=200 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Organization + + + + + + + + yarn + cluster + MergeClaims_project + eu.dnetlib.dhp.oa.graph.raw.MergeClaimsApplication + dhp-graph-mapper-${projectVersion}.jar + + --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.shuffle.partitions=200 + + --rawGraphPath${workingDir}/graph_raw + --claimsGraphPath${workingDir}/graph_claims + --outputRawGaphPath${graphBasePath}/graph_raw + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Project + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/config-default.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/config-default.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/workflow.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/workflow.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1_onlydb/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1_onlydb/oozie_app/workflow.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1_onlydb/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step1/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1_onlydb/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step1/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step1/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/workflow.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step1/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step2/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step2/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step2/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/workflow.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step2/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step3/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step3/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step3/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/workflow.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_step3/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryClaims.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryClaims.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryClaims.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryClaims.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryDatasourceOrganization.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasourceOrganization.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryDatasourceOrganization.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasourceOrganization.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryDatasources.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryDatasources.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryOrganizations.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryOrganizations.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryOrganizationsFromOpenOrgsDB.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryOrganizationsFromOpenOrgsDB.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryProjectOrganization.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjectOrganization.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryProjectOrganization.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjectOrganization.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryProjects.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryProjects.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryProjects_production.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects_production.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/queryProjects_production.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects_production.sql diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/querySimilarityFromOpenOrgsDB.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql similarity index 100% rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/sql/querySimilarityFromOpenOrgsDB.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/transform_actionsets_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/transform_actionsets_parameters.json new file mode 100644 index 000000000..6fa10f739 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/transform_actionsets_parameters.json @@ -0,0 +1,20 @@ +[ + { + "paramName": "mt", + "paramLongName": "master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName": "is", + "paramLongName": "isLookupUrl", + "paramDescription": "URL of the isLookUp Service", + "paramRequired": true + }, + { + "paramName": "i", + "paramLongName": "inputPaths", + "paramDescription": "URL of the isLookUp Service", + "paramRequired": true + } +] diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java index 29ca46d1d..ecadbe981 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java @@ -1,34 +1,25 @@ package eu.dnetlib.dhp.oa.graph; -import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.Oaf; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.RandomStringUtils; 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.SparkSession; -import org.junit.jupiter.api.*; -import org.junit.jupiter.api.io.TempDir; +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.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Objects; public class GraphHiveImporterJobTest { private static final Logger log = LoggerFactory.getLogger(GraphHiveImporterJobTest.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - private static final ClassLoader cl = GraphHiveImporterJobTest.class.getClassLoader(); - public static final String JDBC_DERBY_TEMPLATE = "jdbc:derby:;databaseName=%s/junit_metastore_db;create=true"; private static SparkSession spark; diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step2/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java similarity index 91% rename from dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step2/MappersTest.java rename to dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 894355bcb..63d7d50db 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step2/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -1,27 +1,24 @@ -package eu.dnetlib.dhp.migration.step2; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.List; -import java.util.Map; +package eu.dnetlib.dhp.oa.graph.raw; +import com.google.common.collect.Maps; +import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Software; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) public class MappersTest { @@ -30,7 +27,7 @@ public class MappersTest { private Map code2name; @BeforeEach - void setUp() throws Exception { + public void setUp() throws Exception { when(code2name.get(anyString())).thenAnswer(invocation -> invocation.getArgument(0)); } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java similarity index 99% rename from dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplicationTest.java rename to dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java index d63bb3ee3..b1fc9131f 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplicationTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp.migration.step1; +package eu.dnetlib.dhp.oa.graph.raw; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/claimscontext_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimscontext_resultset_entry.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/claimscontext_resultset_entry.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimscontext_resultset_entry.json diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/claimsrel_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimsrel_resultset_entry.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/claimsrel_resultset_entry.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/claimsrel_resultset_entry.json diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/datasourceorganization_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasourceorganization_resultset_entry.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/datasourceorganization_resultset_entry.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasourceorganization_resultset_entry.json diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/datasources_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasources_resultset_entry.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/datasources_resultset_entry.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasources_resultset_entry.json diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2/oaf_record.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2/oaf_record.xml rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2/odf_dataset.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_dataset.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2/odf_dataset.xml rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_dataset.xml diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2/odf_software.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_software.xml similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step2/odf_software.xml rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_software.xml diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/organizations_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/organizations_resultset_entry.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/organizations_resultset_entry.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/organizations_resultset_entry.json diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/projectorganization_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/projectorganization_resultset_entry.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/projectorganization_resultset_entry.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/projectorganization_resultset_entry.json diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/projects_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/projects_resultset_entry.json similarity index 100% rename from dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/migration/step1/projects_resultset_entry.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/projects_resultset_entry.json diff --git a/pom.xml b/pom.xml index ae19ddbe5..52c601517 100644 --- a/pom.xml +++ b/pom.xml @@ -346,6 +346,12 @@ mongo-java-driver ${mongodb.driver.version} + + org.postgresql + postgresql + 42.2.10 + + org.antlr stringtemplate From 82e8341f50668c3623c924c9826a367808bcdc81 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 14 Apr 2020 15:54:41 +0200 Subject: [PATCH 2/9] reorganizing parameter names in the provision workflow --- dhp-workflows/dhp-actionmanager/pom.xml | 5 + .../dnetlib/dhp/actionmanager/ISClient.java | 122 ++++++++++++++++++ .../PartitionActionSetsByPayloadTypeJob.java | 17 ++- .../PromoteActionPayloadForGraphTableJob.java | 4 +- ...sets_by_payload_type_input_parameters.json | 12 +- .../wf/main/oozie_app/workflow.xml | 11 +- .../oa/graph/raw_all/oozie_app/workflow.xml | 18 +-- pom.xml | 6 + 8 files changed, 171 insertions(+), 24 deletions(-) create mode 100644 dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java diff --git a/dhp-workflows/dhp-actionmanager/pom.xml b/dhp-workflows/dhp-actionmanager/pom.xml index b034528ba..aa2078be5 100644 --- a/dhp-workflows/dhp-actionmanager/pom.xml +++ b/dhp-workflows/dhp-actionmanager/pom.xml @@ -53,5 +53,10 @@ ${project.version} + + eu.dnetlib + dnet-actionmanager-api + + diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java new file mode 100644 index 000000000..1cb1eb4bc --- /dev/null +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java @@ -0,0 +1,122 @@ +package eu.dnetlib.dhp.actionmanager; + +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import eu.dnetlib.actionmanager.rmi.ActionManagerException; +import eu.dnetlib.actionmanager.set.ActionManagerSet; +import eu.dnetlib.actionmanager.set.ActionManagerSet.ImpactTypes; +import eu.dnetlib.dhp.actionmanager.partition.PartitionActionSetsByPayloadTypeJob; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.dom4j.Document; +import org.dom4j.Element; +import org.dom4j.io.SAXReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; + +public class ISClient implements Serializable { + + private static final Logger log = LoggerFactory.getLogger(PartitionActionSetsByPayloadTypeJob.class); + + private static final String INPUT_ACTION_SET_ID_SEPARATOR = ","; + + public static List getLatestRawsetPaths(String isLookupUrl, String setIds) { + + ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl); + ISClient isClient = new ISClient(); + List ids = Lists.newArrayList(Splitter.on(INPUT_ACTION_SET_ID_SEPARATOR) + .omitEmptyStrings() + .trimResults() + .split(setIds)); + + return ids.stream() + .map(id -> isClient.getSet(isLookup, id)) + .map(as -> as.getPathToLatest()) + .collect(Collectors.toCollection(ArrayList::new)); + } + + private ActionManagerSet getSet(ISLookUpService isLookup, final String setId) { + + final String q = "for $x in collection('/db/DRIVER/ActionManagerSetDSResources/ActionManagerSetDSResourceType') " + + "where $x//SET/@id = '" + setId + "' return $x"; + + try { + final String basePath = getBasePathHDFS(isLookup); + final String setProfile = isLookup.getResourceProfileByQuery(q); + return getActionManagerSet(basePath, setProfile); + } catch (ISLookUpException | ActionManagerException e) { + throw new RuntimeException("Error accessing Sets, using query: " + q); + } + } + + private ActionManagerSet getActionManagerSet(final String basePath, final String profile) throws ActionManagerException { + final SAXReader reader = new SAXReader(); + final ActionManagerSet set = new ActionManagerSet(); + + try { + final Document doc = reader.read(new StringReader(profile)); + + set.setId(doc.valueOf("//SET/@id").trim()); + set.setName(doc.valueOf("//SET").trim()); + set.setImpact(ImpactTypes.valueOf(doc.valueOf("//IMPACT").trim())); + set.setLatest(doc.valueOf("//RAW_SETS/LATEST/@id"), doc.valueOf("//RAW_SETS/LATEST/@creationDate"), doc.valueOf("//RAW_SETS/LATEST/@lastUpdate")); + set.setDirectory(doc.valueOf("//SET/@directory")); + final List expiredNodes = doc.selectNodes("//RAW_SETS/EXPIRED"); + if (expiredNodes != null) { + for (int i = 0; i < expiredNodes.size(); i++) { + Element ex = (Element) expiredNodes.get(i); + set.addExpired(ex.attributeValue("id"), ex.attributeValue("creationDate"), ex.attributeValue("lastUpdate")); + } + } + + final StringBuilder sb = new StringBuilder(); + sb.append(basePath); + sb.append("/"); + sb.append(doc.valueOf("//SET/@directory")); + sb.append("/"); + sb.append(doc.valueOf("//RAW_SETS/LATEST/@id")); + set.setPathToLatest(sb.toString()); + + return set; + } catch (Exception e) { + throw new ActionManagerException("Error creating set from profile: " + profile, e); + } + } + + private String getBasePathHDFS(ISLookUpService isLookup) throws ActionManagerException { + return queryServiceProperty(isLookup, "basePath"); + } + + private String queryServiceProperty(ISLookUpService isLookup, final String propertyName) throws ActionManagerException { + final String q = "for $x in /RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='ActionManagerServiceResourceType'] return $x//SERVICE_PROPERTIES/PROPERTY[./@ key='" + + propertyName + "']/@value/string()"; + log.debug("quering for service property: " + q); + try { + final List value = isLookup.quickSearchProfile(q); + return Iterables.getOnlyElement(value); + } catch (ISLookUpException e) { + String msg = "Error accessing service profile, using query: " + q; + log.error(msg, e); + throw new ActionManagerException(msg, e); + } catch (NoSuchElementException e) { + String msg = "missing service property: " + propertyName; + log.error(msg, e); + throw new ActionManagerException(msg, e); + } catch (IllegalArgumentException e) { + String msg = "found more than one service property: " + propertyName; + log.error(msg, e); + throw new ActionManagerException(msg, e); + } + } + + +} diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java index 003f6dc6c..8ba331626 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java @@ -1,5 +1,6 @@ package eu.dnetlib.dhp.actionmanager.partition; +import eu.dnetlib.dhp.actionmanager.ISClient; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJob; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -40,8 +41,6 @@ public class PartitionActionSetsByPayloadTypeJob { StructField$.MODULE$.apply("payload", DataTypes.StringType, false, Metadata.empty()) )); - private static final String INPUT_ACTION_SET_PATHS_SEPARATOR = ","; - public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils.toString( PromoteActionPayloadForGraphTableJob.class @@ -55,21 +54,25 @@ public class PartitionActionSetsByPayloadTypeJob { .orElse(Boolean.TRUE); logger.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputActionSetPaths = parser.get("inputActionSetPaths"); - logger.info("inputActionSetPaths: {}", inputActionSetPaths); + String inputActionSetIds = parser.get("inputActionSetIds"); + logger.info("inputActionSetIds: {}", inputActionSetIds); String outputPath = parser.get("outputPath"); logger.info("outputPath: {}", outputPath); + String isLookupUrl = parser.get("isLookupUrl"); + logger.info("isLookupUrl: {}", isLookupUrl); + + List inputActionSetPaths = ISClient.getLatestRawsetPaths(isLookupUrl, inputActionSetIds); + logger.info("inputActionSetPaths: {}", String.join(",", inputActionSetPaths)); + SparkConf conf = new SparkConf(); conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); runWithSparkSession(conf, isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - readAndWriteActionSetsFromPaths(spark, - Arrays.asList(inputActionSetPaths.split(INPUT_ACTION_SET_PATHS_SEPARATOR)), - outputPath); + readAndWriteActionSetsFromPaths(spark, inputActionSetPaths, outputPath); }); } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java index abad4f210..68bb35c2b 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java @@ -121,8 +121,8 @@ public class PromoteActionPayloadForGraphTableJob { logger.info("Reading graph table from path: {}", path); return spark .read() - .textFile(path) - .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, rowClazz), Encoders.bean(rowClazz)); + .parquet(path) + .as(Encoders.bean(rowClazz)); } private static Dataset readActionPayload(SparkSession spark, diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/partition/partition_action_sets_by_payload_type_input_parameters.json b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/partition/partition_action_sets_by_payload_type_input_parameters.json index c2594ba49..ad58fe754 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/partition/partition_action_sets_by_payload_type_input_parameters.json +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/partition/partition_action_sets_by_payload_type_input_parameters.json @@ -6,9 +6,9 @@ "paramRequired": false }, { - "paramName": "iasp", - "paramLongName": "inputActionSetPaths", - "paramDescription": "comma separated list of action sets to partition by payload type", + "paramName": "iasi", + "paramLongName": "inputActionSetIds", + "paramDescription": "comma separated list of action set ids to partition by payload type", "paramRequired": true }, { @@ -16,5 +16,11 @@ "paramLongName": "outputPath", "paramDescription": "root output location for partitioned action sets", "paramRequired": true + }, + { + "paramName": "is", + "paramLongName": "isLookupUrl", + "paramDescription": "URL of the isLookUp Service", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/workflow.xml index 4f54c0699..25afc34c9 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/main/oozie_app/workflow.xml @@ -41,8 +41,12 @@ root location of input materialized graph - inputActionSetPaths - comma separated list of action sets to promote + isLookupUrl + URL of the ISLookupService + + + inputActionSetIds + comma separated list of action set ids to promote outputGraphRootPath @@ -121,8 +125,9 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --inputActionSetPaths${inputActionSetPaths} + --inputActionSetIds${inputActionSetIds} --outputPath${workingDir}/action_payload_by_type + --isLookupUrl${isLookupUrl} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml index 11c8f71a1..d33bb0211 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml @@ -2,7 +2,7 @@ - graphBasePath + graphOutputPath the target path to store raw graph @@ -343,7 +343,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Publication @@ -369,7 +369,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Dataset @@ -395,7 +395,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Relation @@ -421,7 +421,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Software @@ -447,7 +447,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct @@ -473,7 +473,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Datasource @@ -499,7 +499,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Organization @@ -525,7 +525,7 @@ --rawGraphPath${workingDir}/graph_raw --claimsGraphPath${workingDir}/graph_claims - --outputRawGaphPath${graphBasePath}/graph_raw + --outputRawGaphPath${graphOutputPath} --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Project diff --git a/pom.xml b/pom.xml index 52c601517..1d36d42f9 100644 --- a/pom.xml +++ b/pom.xml @@ -293,6 +293,12 @@ dnet-actionmanager-common 6.0.5 + + eu.dnetlib + dnet-actionmanager-api + [4.0.1,5.0.0) + + eu.dnetlib dnet-openaire-data-protos From c36239e6931564f79fb14f229bd83c65b24788c9 Mon Sep 17 00:00:00 2001 From: sandro Date: Tue, 14 Apr 2020 17:47:36 +0200 Subject: [PATCH 3/9] fixed incremental indexing --- .../dhp/schema/scholexplorer/DLIRelation.java | 15 +++ .../sx/graph/SparkSXGeneratePidSimlarity.java | 7 +- .../SparkScholexplorerCreateRawGraphJob.java | 7 +- .../parser/AbstractScholexplorerParser.java | 78 ++++++++++++++- .../parser/DatasetScholexplorerParser.java | 72 ++------------ .../PublicationScholexplorerParser.java | 50 +--------- .../dhp-graph-provision-scholexplorer/pom.xml | 5 + .../provision/scholix/ScholixResource.java | 10 +- .../provision/update/Datacite2Scholix.java | 2 +- .../update/SparkResolveScholixTarget.java | 57 ++++++++++- .../dhp/sx/index/oozie_app/config-default.xml | 10 ++ .../dhp/sx/index/oozie_app/workflow.xml | 68 +++++++++++++ .../dhp/sx/synch/oozie_app/config-default.xml | 10 ++ .../dhp/sx/synch/oozie_app/workflow.xml | 97 +++++++++++++++++++ .../dhp/provision/DataciteClientTest.java | 64 +----------- 15 files changed, 364 insertions(+), 188 deletions(-) create mode 100644 dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java create mode 100644 dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java new file mode 100644 index 000000000..1a45b64ff --- /dev/null +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java @@ -0,0 +1,15 @@ +package eu.dnetlib.dhp.schema.scholexplorer; + +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class DLIRelation extends Relation { + private String dateOfCollection; + + public String getDateOfCollection() { + return dateOfCollection; + } + + public void setDateOfCollection(String dateOfCollection) { + this.dateOfCollection = dateOfCollection; + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java index 806140160..aa2f2cc58 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.sx.graph; import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; import eu.dnetlib.dhp.utils.DHPUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.api.java.JavaPairRDD; @@ -49,15 +50,15 @@ public class SparkSXGeneratePidSimlarity { .equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::"))) .distinct(); - JavaRDD simRel = datasetSimRel.union(publicationSimRel).map(s -> { - final Relation r = new Relation(); + JavaRDD simRel = datasetSimRel.union(publicationSimRel).map(s -> { + final DLIRelation r = new DLIRelation(); r.setSource(s._1()); r.setTarget(s._2()); r.setRelType("similar"); return r; } ); - spark.createDataset(simRel.rdd(), Encoders.bean(Relation.class)).distinct().write() + spark.createDataset(simRel.rdd(), Encoders.bean(DLIRelation.class)).distinct().write() .mode(SaveMode.Overwrite).save(targetPath+"/pid_simRel"); } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java index 36d3cf540..36c94f595 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java @@ -7,6 +7,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset; import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication; +import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown; import eu.dnetlib.dhp.utils.DHPUtils; import net.minidev.json.JSONArray; @@ -135,19 +136,19 @@ public class SparkScholexplorerCreateRawGraphJob { SparkSXGeneratePidSimlarity.generateDataFrame(spark, sc, inputPath.replace("/relation",""),targetPath.replace("/relation","") ); - RDD rdd = union.mapToPair((PairFunction) f -> { + RDD rdd = union.mapToPair((PairFunction) f -> { final String source = getJPathString(SOURCEJSONPATH, f); final String target = getJPathString(TARGETJSONPATH, f); final String reltype = getJPathString(RELJSONPATH, f); ObjectMapper mapper = new ObjectMapper(); mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - return new Tuple2<>(DHPUtils.md5(String.format("%s::%s::%s", source.toLowerCase(), reltype.toLowerCase(), target.toLowerCase())), mapper.readValue(f, Relation.class)); + return new Tuple2<>(DHPUtils.md5(String.format("%s::%s::%s", source.toLowerCase(), reltype.toLowerCase(), target.toLowerCase())), mapper.readValue(f, DLIRelation.class)); }).reduceByKey((a, b) -> { a.mergeFrom(b); return a; }).map(Tuple2::_2).rdd(); - spark.createDataset(rdd, Encoders.bean(Relation.class)).write().mode(SaveMode.Overwrite).save(targetPath); + spark.createDataset(rdd, Encoders.bean(DLIRelation.class)).write().mode(SaveMode.Overwrite).save(targetPath); Dataset rel_ds =spark.read().load(targetPath).as(Encoders.bean(Relation.class)); System.out.println("LOADING PATH :"+targetPath.replace("/relation","")+"/pid_simRel"); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java index ca20c0aba..9eeff9613 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java @@ -2,10 +2,13 @@ package eu.dnetlib.dhp.sx.graph.parser; import eu.dnetlib.dhp.parser.utility.VtdUtilityParser; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset; +import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; +import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown; +import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo; import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.scholexplorer.relation.RelInfo; import eu.dnetlib.scholexplorer.relation.RelationMapper; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; @@ -15,6 +18,7 @@ import javax.xml.stream.XMLStreamReader; import java.util.*; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; public abstract class AbstractScholexplorerParser { @@ -104,6 +108,74 @@ public abstract class AbstractScholexplorerParser { return type+ DHPUtils.md5(String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim())); } + protected DLIUnknown createUnknownObject(final String pid, final String pidType, final KeyValue cf, final DataInfo di, final String dateOfCollection) { + final DLIUnknown uk = new DLIUnknown(); + uk.setId(generateId(pid, pidType, "unknown")); + ProvenaceInfo pi = new ProvenaceInfo(); + pi.setId(cf.getKey()); + pi.setName(cf.getValue()); + pi.setCompletionStatus("incomplete"); + uk.setDataInfo(di); + uk.setDlicollectedfrom(Collections.singletonList(pi)); + final StructuredProperty sourcePid = new StructuredProperty(); + sourcePid.setValue(pid); + final Qualifier pt = new Qualifier(); + pt.setClassname(pidType); + pt.setClassid(pidType); + pt.setSchemename("dnet:pid_types"); + pt.setSchemeid("dnet:pid_types"); + sourcePid.setQualifier(pt); + uk.setPid(Collections.singletonList(sourcePid)); + uk.setDateofcollection(dateOfCollection); + return uk; + } + + protected void generateRelations(RelationMapper relationMapper, Result parsedObject, List result, DataInfo di, String dateOfCollection, List relatedIdentifiers) { + if(relatedIdentifiers!= null) { + result.addAll(relatedIdentifiers.stream() + .flatMap(n -> { + final List rels = new ArrayList<>(); + DLIRelation r = new DLIRelation(); + r.setSource(parsedObject.getId()); + final String relatedPid = n.getTextValue(); + final String relatedPidType = n.getAttributes().get("relatedIdentifierType"); + final String relatedType = n.getAttributes().getOrDefault("entityType", "unknown"); + String relationSemantic = n.getAttributes().get("relationType"); + String inverseRelation; + final String targetId = generateId(relatedPid, relatedPidType, relatedType); + r.setDateOfCollection(dateOfCollection); + if (relationMapper.containsKey(relationSemantic.toLowerCase())) + { + RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase()); + relationSemantic = relInfo.getOriginal(); + inverseRelation = relInfo.getInverse(); + } + else { + relationSemantic = "Unknown"; + inverseRelation = "Unknown"; + } + r.setTarget(targetId); + r.setRelType(relationSemantic); + r.setRelClass("datacite"); + r.setCollectedFrom(parsedObject.getCollectedfrom()); + r.setDataInfo(di); + rels.add(r); + r = new DLIRelation(); + r.setDataInfo(di); + r.setSource(targetId); + r.setTarget(parsedObject.getId()); + r.setRelType(inverseRelation); + r.setRelClass("datacite"); + r.setCollectedFrom(parsedObject.getCollectedfrom()); + r.setDateOfCollection(dateOfCollection); + rels.add(r); + if("unknown".equalsIgnoreCase(relatedType)) + result.add(createUnknownObject(relatedPid, relatedPidType, parsedObject.getCollectedfrom().get(0), di, dateOfCollection)); + return rels.stream(); + }).collect(Collectors.toList())); + } + } + diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java index 2ba2bd519..f1915c5cf 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java @@ -42,7 +42,8 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { parsedObject.setOriginalId(Collections.singletonList(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='recordIdentifier']"))); parsedObject.setOriginalObjIdentifier(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='objIdentifier']")); - parsedObject.setDateofcollection(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']")); + String dateOfCollection = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']"); + parsedObject.setDateofcollection(dateOfCollection); final String resolvedDate = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='resolvedDate']"); @@ -123,7 +124,7 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { List descs = VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='description']"); if (descs != null && descs.size() > 0) parsedObject.setDescription(descs.stream() - .map(it -> it.length() < 512 ? it : it.substring(0, 512)) + .map(it -> it.length() < 10000 ? it : it.substring(0, 10000)) .map(it -> { final Field d = new Field<>(); d.setValue(it); @@ -137,48 +138,7 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { Arrays.asList("relatedIdentifierType", "relationType", "entityType", "inverseRelationType")); - if(relatedIdentifiers!= null) { - result.addAll(relatedIdentifiers.stream() - .flatMap(n -> { - final List rels = new ArrayList<>(); - Relation r = new Relation(); - r.setSource(parsedObject.getId()); - final String relatedPid = n.getTextValue(); - final String relatedPidType = n.getAttributes().get("relatedIdentifierType"); - final String relatedType = n.getAttributes().getOrDefault("entityType", "unknown"); - String relationSemantic = n.getAttributes().get("relationType"); - String inverseRelation = n.getAttributes().get("inverseRelationType"); - final String targetId = generateId(relatedPid, relatedPidType, relatedType); - - if (relationMapper.containsKey(relationSemantic.toLowerCase())) - { - RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase()); - relationSemantic = relInfo.getOriginal(); - inverseRelation = relInfo.getInverse(); - } - else { - relationSemantic = "Unknown"; - inverseRelation = "Unknown"; - } - r.setTarget(targetId); - r.setRelType(relationSemantic); - r.setRelClass("datacite"); - r.setCollectedFrom(parsedObject.getCollectedfrom()); - r.setDataInfo(di); - rels.add(r); - r = new Relation(); - r.setDataInfo(di); - r.setSource(targetId); - r.setTarget(parsedObject.getId()); - r.setRelType(inverseRelation); - r.setRelClass("datacite"); - r.setCollectedFrom(parsedObject.getCollectedfrom()); - rels.add(r); - if("unknown".equalsIgnoreCase(relatedType)) - result.add(createUnknownObject(relatedPid, relatedPidType, parsedObject.getCollectedfrom().get(0), di)); - return rels.stream(); - }).collect(Collectors.toList())); - } + generateRelations(relationMapper, parsedObject, result, di, dateOfCollection, relatedIdentifiers); final List hostedBy = @@ -199,7 +159,7 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { } - List subjects = extractSubject(VtdUtilityParser.getTextValuesWithAttributes(ap, vn, "//*[local-name()='resource']//*[local-name()='subject']", Arrays.asList("subjectScheme"))); + List subjects = extractSubject(VtdUtilityParser.getTextValuesWithAttributes(ap, vn, "//*[local-name()='resource']//*[local-name()='subject']", Collections.singletonList("subjectScheme"))); parsedObject.setSubject(subjects); @@ -265,24 +225,6 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { } - private DLIUnknown createUnknownObject(final String pid, final String pidType, final KeyValue cf, final DataInfo di) { - final DLIUnknown uk = new DLIUnknown(); - uk.setId(generateId(pid, pidType, "unknown")); - ProvenaceInfo pi = new ProvenaceInfo(); - pi.setId(cf.getKey()); - pi.setName(cf.getValue()); - pi.setCompletionStatus("incomplete"); - uk.setDataInfo(di); - uk.setDlicollectedfrom(Collections.singletonList(pi)); - final StructuredProperty sourcePid = new StructuredProperty(); - sourcePid.setValue(pid); - final Qualifier pt = new Qualifier(); - pt.setClassname(pidType); - pt.setClassid(pidType); - pt.setSchemename("dnet:pid_types"); - pt.setSchemeid("dnet:pid_types"); - sourcePid.setQualifier(pt); - uk.setPid(Collections.singletonList(sourcePid)); - return uk; - } + + } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java index b8b38515b..aa2f86076 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java @@ -38,7 +38,8 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser di.setDeletedbyinference(false); di.setInvisible(false); - parsedObject.setDateofcollection(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']")); + String dateOfCollection = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']"); + parsedObject.setDateofcollection(dateOfCollection); final String resolvedDate = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='resolvedDate']"); parsedObject.setOriginalId(Collections.singletonList(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='recordIdentifier']"))); @@ -118,48 +119,7 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser final List relatedIdentifiers = VtdUtilityParser.getTextValuesWithAttributes(ap, vn, "//*[local-name()='relatedIdentifier']", Arrays.asList("relatedIdentifierType", "relationType", "entityType", "inverseRelationType")); - - - if (relatedIdentifiers != null) { - result.addAll(relatedIdentifiers.stream() - .flatMap(n -> { - final List rels = new ArrayList<>(); - Relation r = new Relation(); - r.setSource(parsedObject.getId()); - final String relatedPid = n.getTextValue(); - final String relatedPidType = n.getAttributes().get("relatedIdentifierType"); - final String relatedType = n.getAttributes().getOrDefault("entityType", "unknown"); - String relationSemantic = n.getAttributes().get("relationType"); - String inverseRelation = "Unknown"; - final String targetId = generateId(relatedPid, relatedPidType, relatedType); - - if (relationMapper.containsKey(relationSemantic.toLowerCase())) - { - RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase()); - relationSemantic = relInfo.getOriginal(); - inverseRelation = relInfo.getInverse(); - } - else { - relationSemantic = "Unknown"; - } - r.setTarget(targetId); - r.setRelType(relationSemantic); - r.setCollectedFrom(parsedObject.getCollectedfrom()); - r.setRelClass("datacite"); - r.setDataInfo(di); - rels.add(r); - r = new Relation(); - r.setDataInfo(di); - r.setSource(targetId); - r.setTarget(parsedObject.getId()); - r.setRelType(inverseRelation); - r.setRelClass("datacite"); - r.setCollectedFrom(parsedObject.getCollectedfrom()); - rels.add(r); - - return rels.stream(); - }).collect(Collectors.toList())); - } + generateRelations(relationMapper, parsedObject, result, di, dateOfCollection, relatedIdentifiers); final List hostedBy = VtdUtilityParser.getTextValuesWithAttributes(ap, vn, "//*[local-name()='hostedBy']", Arrays.asList("id", "name")); @@ -206,8 +166,8 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser description.setValue(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='description']")); - if (StringUtils.isNotBlank(description.getValue()) && description.getValue().length() > 512) { - description.setValue(description.getValue().substring(0, 512)); + if (StringUtils.isNotBlank(description.getValue()) && description.getValue().length() > 10000) { + description.setValue(description.getValue().substring(0, 10000)); } parsedObject.setDescription(Collections.singletonList(description)); diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml index de38a01b3..03604f431 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml @@ -69,6 +69,11 @@ + + org.apache.httpcomponents + httpclient + + diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java index 49b891e65..f29722eb8 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java @@ -3,7 +3,6 @@ package eu.dnetlib.dhp.provision.scholix; import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; import java.io.Serializable; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -20,10 +19,6 @@ public class ScholixResource implements Serializable { private List publisher; private List collectedFrom; - - - - public static ScholixResource fromSummary(ScholixSummary summary) { final ScholixResource resource = new ScholixResource(); @@ -38,7 +33,7 @@ public class ScholixResource implements Serializable { resource.setObjectType(summary.getTypology().toString()); - if (summary.getTitle() != null && summary.getTitle().size()>0) + if (summary.getTitle() != null && summary.getTitle().size() > 0) resource.setTitle(summary.getTitle().get(0)); if (summary.getAuthor() != null) @@ -47,7 +42,7 @@ public class ScholixResource implements Serializable { .collect(Collectors.toList()) ); - if (summary.getDate() != null && summary.getDate().size()>0) + if (summary.getDate() != null && summary.getDate().size() > 0) resource.setPublicationDate(summary.getDate().get(0)); if (summary.getPublisher() != null) resource.setPublisher(summary.getPublisher().stream() @@ -65,6 +60,7 @@ public class ScholixResource implements Serializable { } + public List getIdentifier() { return identifier; } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java index c6617a823..ac05a8350 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java @@ -165,7 +165,7 @@ public class Datacite2Scholix { return res; } - protected String generateId(final String pid, final String pidType, final String entityType) { + public static String generateId(final String pid, final String pidType, final String entityType) { String type; switch (entityType){ case "publication": diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java index 4628c4684..35020ecdf 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java @@ -4,18 +4,25 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.provision.scholix.Scholix; import eu.dnetlib.dhp.provision.scholix.ScholixIdentifier; +import eu.dnetlib.dhp.provision.scholix.ScholixRelationship; import eu.dnetlib.dhp.provision.scholix.ScholixResource; import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.scholexplorer.relation.RelationMapper; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import scala.Tuple2; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; public class SparkResolveScholixTarget { @@ -29,8 +36,6 @@ public class SparkResolveScholixTarget { final String sourcePath = parser.get("sourcePath"); final String workingDirPath= parser.get("workingDirPath"); final String indexHost= parser.get("indexHost"); - - try (SparkSession spark = getSession(conf, master)){ final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); @@ -65,7 +70,55 @@ public class SparkResolveScholixTarget { }, Encoders.bean(ScholixResource.class)).write().mode(SaveMode.Overwrite).save(workingDirPath+"/stepB"); + Dataset s2 = spark.read().load(workingDirPath+"/stepB").as(Encoders.bean(ScholixResource.class)); + + s1.joinWith(s2, s1.col("target.identifier.identifier").equalTo(s2.col("identifier.identifier")), "left") + + .flatMap((FlatMapFunction, Scholix>) f -> + { + + final List res = new ArrayList<>(); + final Scholix s = f._1(); + final ScholixResource target = f._2(); + if (StringUtils.isNotBlank(s.getIdentifier())) + res.add(s); + else if (target == null) { + ScholixResource currentTarget = s.getTarget(); + currentTarget.setObjectType("unknown"); + currentTarget.setDnetIdentifier(Datacite2Scholix.generateId(currentTarget.getIdentifier().get(0).getIdentifier(),currentTarget.getIdentifier().get(0).getSchema(), currentTarget.getObjectType())); + + s.generateIdentifier(); + res.add(s); + final Scholix inverse = new Scholix(); + inverse.setTarget(s.getSource()); + inverse.setSource(s.getTarget()); + inverse.setLinkprovider(s.getLinkprovider()); + inverse.setPublicationDate(s.getPublicationDate()); + inverse.setPublisher(s.getPublisher()); + inverse.setRelationship(new ScholixRelationship(s.getRelationship().getInverse(), s.getRelationship().getSchema(), s.getRelationship().getName())); + inverse.generateIdentifier(); + res.add(inverse); + + } else + { + target.setIdentifier(target.getIdentifier().stream().map(d -> new ScholixIdentifier(d.getIdentifier().toLowerCase(), d.getSchema().toLowerCase())).collect(Collectors.toList())); + s.setTarget(target); + s.generateIdentifier(); + res.add(s); + final Scholix inverse = new Scholix(); + inverse.setTarget(s.getSource()); + inverse.setSource(s.getTarget()); + inverse.setLinkprovider(s.getLinkprovider()); + inverse.setPublicationDate(s.getPublicationDate()); + inverse.setPublisher(s.getPublisher()); + inverse.setRelationship(new ScholixRelationship(s.getRelationship().getInverse(), s.getRelationship().getSchema(), s.getRelationship().getName())); + inverse.generateIdentifier(); + res.add(inverse); + } + + return res.iterator(); + }, Encoders.bean(Scholix.class)).javaRDD().map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(workingDirPath+"/resolved_json"); } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/config-default.xml new file mode 100644 index 000000000..6fb2a1253 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/config-default.xml @@ -0,0 +1,10 @@ + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml new file mode 100644 index 000000000..9fc86e014 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml @@ -0,0 +1,68 @@ + + + + workingDirPath + the source path + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + index + index name + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + index Summary + eu.dnetlib.dhp.provision.SparkIndexCollectionOnES + dhp-graph-provision-scholexplorer-${projectVersion}.jar + --executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="32" + -mt yarn-cluster + --sourcePath${workingDirPath}/summary + --index${index}_object + --idPathid + --typesummary + + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + index scholix + eu.dnetlib.dhp.provision.SparkIndexCollectionOnES + dhp-graph-provision-scholexplorer-${projectVersion}.jar + --executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="8" + -mt yarn-cluster + --sourcePath${workingDirPath}/scholix_json + --index${index}_scholix + --idPathidentifier + --typescholix + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml new file mode 100644 index 000000000..6fb2a1253 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml @@ -0,0 +1,10 @@ + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml new file mode 100644 index 000000000..5e108f94f --- /dev/null +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml @@ -0,0 +1,97 @@ + + + + workingDirPath + the source path + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + index + index name + + + timestamp + timestamp from incremental harvesting + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.dhp.provision.update.RetrieveUpdateFromDatacite + -t${workingDirPath}/synch/input_json + -n${nameNode} + -ts${timestamp} + -ihip-90-147-167-25.ct1.garrservices.it + -indatacite + + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + Resolve and generate Scholix + eu.dnetlib.dhp.provision.update.SparkResolveScholixTarget + dhp-graph-provision-scholexplorer-${projectVersion}.jar + --executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="32" + -m yarn-cluster + -s${workingDirPath}/synch/input_json + -w${workingDirPath}/synch + -hip-90-147-167-25.ct1.garrservices.it + + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + index scholix + eu.dnetlib.dhp.provision.SparkIndexCollectionOnES + dhp-graph-provision-scholexplorer-${projectVersion}.jar + --executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="8" + -mt yarn-cluster + --sourcePath${workingDirPath}/synch/resolved_json + --index${index}_scholix + --idPathidentifier + --typescholix + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java index e008d72be..782784be4 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java @@ -3,24 +3,18 @@ package eu.dnetlib.dhp.provision; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.provision.scholix.Scholix; import eu.dnetlib.dhp.provision.scholix.ScholixResource; -import eu.dnetlib.dhp.provision.update.*; +import eu.dnetlib.dhp.provision.update.CrossrefClient; +import eu.dnetlib.dhp.provision.update.Datacite2Scholix; +import eu.dnetlib.dhp.provision.update.DataciteClient; import eu.dnetlib.scholexplorer.relation.RelationMapper; import org.apache.commons.io.IOUtils; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.entity.StringEntity; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClients; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.io.IOException; import java.util.List; public class DataciteClientTest { - - @Test public void dataciteSCholixTest() throws Exception { final String json = IOUtils.toString(getClass().getResourceAsStream("datacite.json")); @@ -32,66 +26,18 @@ public class DataciteClientTest { } - - public void testClient() throws Exception { - RetrieveUpdateFromDatacite.main(new String[]{ - "-n", "file:///data/new_s2.txt", - "-t", "/data/new_s2.txt", - "-ts", "1585760736", - "-ih", "ip-90-147-167-25.ct1.garrservices.it", - "-in", "datacite", - }); - - - SparkResolveScholixTarget.main(new String[]{ - "-s", "file:///data/new_s.txt", - "-m", "local[*]", - "-w", "/data/scholix/provision", - "-h", "ip-90-147-167-25.ct1.garrservices.it", - - }); - } - - public void testResolveDataset() throws Exception { - DataciteClient dc = new DataciteClient("ip-90-147-167-25.ct1.garrservices.it"); + DataciteClient dc = new DataciteClient("ip-90-147-167-25.ct1.garrservices.it"); ScholixResource datasetByDOI = dc.getDatasetByDOI("10.17182/hepdata.15392.v1/t5"); Assertions.assertNotNull(datasetByDOI); System.out.println(new ObjectMapper().writeValueAsString(datasetByDOI)); CrossrefClient cr = new CrossrefClient("ip-90-147-167-25.ct1.garrservices.it"); - ScholixResource crossrefByDOI = cr.getResourceByDOI("10.26850/1678-4618eqj.v35.1.2010.p41-46"); + ScholixResource crossrefByDOI = cr.getResourceByDOI("10.26850/1678-4618eqj.v35.1.2010.p41-46"); Assertions.assertNotNull(crossrefByDOI); System.out.println(new ObjectMapper().writeValueAsString(crossrefByDOI)); - - } - - private String getResponse(final String url,final String json ) { - CloseableHttpClient client = HttpClients.createDefault(); - try { - - HttpPost httpPost = new HttpPost(url); - if (json!= null) { - StringEntity entity = new StringEntity(json); - httpPost.setEntity(entity); - httpPost.setHeader("Accept", "application/json"); - httpPost.setHeader("Content-type", "application/json"); - } - CloseableHttpResponse response = client.execute(httpPost); - - return IOUtils.toString(response.getEntity().getContent()); - } catch (Throwable e) { - throw new RuntimeException("Error on executing request ",e); - } finally { - try { - client.close(); - } catch (IOException e) { - throw new RuntimeException("Unable to close client ",e); - } - } - } } From a68fae9bcbd621d70e1d0033d108dedb8f4ae699 Mon Sep 17 00:00:00 2001 From: Alessia Bardi Date: Tue, 14 Apr 2020 17:52:48 +0200 Subject: [PATCH 4/9] now supporting openaire 4.0 compliance --- .../resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql index 8c587f34e..9a0a45f78 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql @@ -8,6 +8,9 @@ SELECT WHEN (array_agg(DISTINCT COALESCE (a.compatibility_override, a.compatibility):: TEXT) @> ARRAY ['openaire-cris_1.1']) THEN 'openaire-cris_1.1@@@OpenAIRE CRIS v1.1@@@dnet:datasourceCompatibilityLevel@@@dnet:datasourceCompatibilityLevel' + WHEN (array_agg(DISTINCT COALESCE (a.compatibility_override, a.compatibility):: TEXT) @> ARRAY ['openaire4.0']) + THEN + 'openaire4.0@@@OpenAIRE 4.0@@@dnet:datasourceCompatibilityLevel@@@dnet:datasourceCompatibilityLevel' WHEN (array_agg(DISTINCT COALESCE (a.compatibility_override, a.compatibility):: TEXT) @> ARRAY ['driver', 'openaire2.0']) THEN 'driver-openaire2.0@@@OpenAIRE 2.0+ (DRIVER OA, EC funding)@@@dnet:datasourceCompatibilityLevel@@@dnet:datasourceCompatibilityLevel' From 74a7fac774fab83f52a11a01b159385c783d7274 Mon Sep 17 00:00:00 2001 From: sandro Date: Wed, 15 Apr 2020 15:23:54 +0200 Subject: [PATCH 5/9] fixed problem with timestamp --- .../dnetlib/dhp/provision/update/DataciteClientIterator.java | 4 +++- .../resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml | 4 ++-- .../java/eu/dnetlib/dhp/provision/DataciteClientTest.java | 5 ++--- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java index e823945ae..61c1aa39f 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java @@ -27,10 +27,12 @@ public class DataciteClientIterator implements Iterator { final String esIndex; final ObjectMapper mapper = new ObjectMapper(); - public DataciteClientIterator(final String esHost, final String esIndex, final long timestamp) throws IOException { + public DataciteClientIterator(final String esHost, final String esIndex, long timestamp) throws IOException { this.esHost = esHost; this.esIndex = esIndex; + // THIS FIX IS NECESSARY to avoid different timezone + timestamp -= (60 *60 *2); final String body =getResponse(String.format("http://%s:9200/%s/_search?scroll=1m", esHost, esIndex), String.format("{\"size\":1000, \"query\":{\"range\":{\"timestamp\":{\"gte\":%d}}}}", timestamp)); scrollId= getJPathString(scrollIdPath, body); buffer = getBlobs(body); diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml index 5e108f94f..c004eafe6 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/workflow.xml @@ -50,7 +50,7 @@ -ihip-90-147-167-25.ct1.garrservices.it -indatacite - + @@ -60,7 +60,7 @@ ${nameNode} yarn-cluster cluster - Resolve and generate Scholix + resolve and generate Scholix eu.dnetlib.dhp.provision.update.SparkResolveScholixTarget dhp-graph-provision-scholexplorer-${projectVersion}.jar --executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} --conf spark.dynamicAllocation.maxExecutors="32" diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java index 782784be4..f4a480163 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java @@ -6,11 +6,13 @@ import eu.dnetlib.dhp.provision.scholix.ScholixResource; import eu.dnetlib.dhp.provision.update.CrossrefClient; import eu.dnetlib.dhp.provision.update.Datacite2Scholix; import eu.dnetlib.dhp.provision.update.DataciteClient; +import eu.dnetlib.dhp.provision.update.DataciteClientIterator; import eu.dnetlib.scholexplorer.relation.RelationMapper; import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.io.IOException; import java.util.List; @@ -25,7 +27,6 @@ public class DataciteClientTest { System.out.println(new ObjectMapper().writeValueAsString(s)); } - public void testResolveDataset() throws Exception { DataciteClient dc = new DataciteClient("ip-90-147-167-25.ct1.garrservices.it"); ScholixResource datasetByDOI = dc.getDatasetByDOI("10.17182/hepdata.15392.v1/t5"); @@ -37,7 +38,5 @@ public class DataciteClientTest { ScholixResource crossrefByDOI = cr.getResourceByDOI("10.26850/1678-4618eqj.v35.1.2010.p41-46"); Assertions.assertNotNull(crossrefByDOI); System.out.println(new ObjectMapper().writeValueAsString(crossrefByDOI)); - - } } From ff30f99c65c04bad95af6dc887797781f6d322df Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 15 Apr 2020 16:16:20 +0200 Subject: [PATCH 6/9] using newline delimited json files for the raw graph materialization. Introduced contentPath parameter --- .../raw/DispatchEntitiesApplication.java | 6 ++-- .../oa/graph/raw/MergeClaimsApplication.java | 10 +++++- .../oa/graph/raw_all/oozie_app/workflow.xml | 32 +++++++++++-------- 3 files changed, 30 insertions(+), 18 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java index 0b47db588..4812f1c30 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java @@ -64,6 +64,7 @@ public class DispatchEntitiesApplication { log.info(String.format("Processing entities (%s) in file: %s", type, sourcePath)); + /* spark.read() .textFile(sourcePath) .filter((FilterFunction) value -> isEntityType(value, type)) @@ -73,14 +74,13 @@ public class DispatchEntitiesApplication { .mode(SaveMode.Overwrite) .parquet(targetPath + "/" + type); - /* + */ + JavaSparkContext.fromSparkContext(spark.sparkContext()) .textFile(sourcePath) .filter(l -> isEntityType(l, type)) .map(l -> StringUtils.substringAfter(l, "|")) .saveAsTextFile(targetPath + "/" + type, GzipCodec.class); // use repartition(XXX) ??? - - */ } private static boolean isEntityType(final String line, final String type) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java index 85e4f3663..4b209c68a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java @@ -100,16 +100,24 @@ public class MergeClaimsApplication { return opRaw.isPresent() ? opRaw.get()._2() : opClaim.isPresent() ? opClaim.get()._2() : null; }, Encoders.bean(clazz)) .filter(Objects::nonNull) + .map((MapFunction) value -> OBJECT_MAPPER.writeValueAsString(value), Encoders.STRING()) .write() .mode(SaveMode.Overwrite) - .parquet(outPath); + .option("compression", "gzip") + .text(outPath); } private static Dataset readFromPath(SparkSession spark, String path, Class clazz) { + return spark.read() + .textFile(path) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)) + .filter((FilterFunction) value -> Objects.nonNull(idFn().apply(value))); + /* return spark.read() .load(path) .as(Encoders.bean(clazz)) .filter((FilterFunction) value -> Objects.nonNull(idFn().apply(value))); + */ } private static void removeOutputDir(SparkSession spark, String path) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml index d33bb0211..9f91380ab 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml @@ -10,6 +10,10 @@ false should import content from the aggregator or reuse a previous version + + contentPath + path location to store (or reuse) content from the aggregator + postgresURL the postgres URL to access to the database @@ -108,10 +112,10 @@ - + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication - -p${workingDir}/db_claims + -p${contentPath}/db_claims -pgurl${postgresURL} -pguser${postgresUser} -pgpasswd${postgresPassword} @@ -124,10 +128,10 @@ - + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication - -p${workingDir}/odf_claims + -p${contentPath}/odf_claims -mongourl${mongoURL} -mongodb${mongoDb} -fODF @@ -141,10 +145,10 @@ - + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication - -p${workingDir}/oaf_claims + -p${contentPath}/oaf_claims -mongourl${mongoURL} -mongodb${mongoDb} -fOAF @@ -158,10 +162,10 @@ - + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication - -p${workingDir}/db_records + -p${contentPath}/db_records -pgurl${postgresURL} -pguser${postgresUser} -pgpasswd${postgresPassword} @@ -173,10 +177,10 @@ - + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication - -p${workingDir}/odf_records + -p${contentPath}/odf_records -mongourl${mongoURL} -mongodb${mongoDb} -fODF @@ -190,10 +194,10 @@ - + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication - -p${workingDir}/oaf_records + -p${contentPath}/oaf_records -mongourl${mongoURL} -mongodb${mongoDb} -fOAF @@ -227,7 +231,7 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - -s${workingDir}/db_claims,${workingDir}/oaf_claims,${workingDir}/odf_claims + -s${contentPath}/db_claims,${contentPath}/oaf_claims,${contentPath}/odf_claims -t${workingDir}/entities_claim -pgurl${postgresURL} -pguser${postgresUser} @@ -276,7 +280,7 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - -s${workingDir}/db_records,${workingDir}/oaf_records,${workingDir}/odf_records + -s${contentPath}/db_records,${contentPath}/oaf_records,${contentPath}/odf_records -t${workingDir}/entities -pgurl${postgresURL} -pguser${postgresUser} From c439d0c6bbeeb527c675d8e816b55719ee6b3d07 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 15 Apr 2020 16:18:33 +0200 Subject: [PATCH 7/9] PromoteActionPayloadForGraphTableJob reads directly the content pointed by the input path, adjusted promote action tests (ISLookup mock) --- .../dnetlib/dhp/actionmanager/ISClient.java | 12 +++-- .../PartitionActionSetsByPayloadTypeJob.java | 46 +++++++++++------ .../PromoteActionPayloadForGraphTableJob.java | 8 +++ ...rtitionActionSetsByPayloadTypeJobTest.java | 51 +++++++++++++++---- 4 files changed, 88 insertions(+), 29 deletions(-) diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java index 1cb1eb4bc..994108b1c 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java @@ -29,17 +29,21 @@ public class ISClient implements Serializable { private static final String INPUT_ACTION_SET_ID_SEPARATOR = ","; - public static List getLatestRawsetPaths(String isLookupUrl, String setIds) { + private ISLookUpService isLookup; + + public ISClient(String isLookupUrl) { + isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl); + } + + public List getLatestRawsetPaths(String setIds) { - ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl); - ISClient isClient = new ISClient(); List ids = Lists.newArrayList(Splitter.on(INPUT_ACTION_SET_ID_SEPARATOR) .omitEmptyStrings() .trimResults() .split(setIds)); return ids.stream() - .map(id -> isClient.getSet(isLookup, id)) + .map(id -> getSet(isLookup, id)) .map(as -> as.getPathToLatest()) .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java index 8ba331626..31a4da190 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java @@ -41,6 +41,15 @@ public class PartitionActionSetsByPayloadTypeJob { StructField$.MODULE$.apply("payload", DataTypes.StringType, false, Metadata.empty()) )); + private ISClient isClient; + + public PartitionActionSetsByPayloadTypeJob(String isLookupUrl) { + this.isClient = new ISClient(isLookupUrl); + } + + public PartitionActionSetsByPayloadTypeJob() { + } + public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils.toString( PromoteActionPayloadForGraphTableJob.class @@ -63,7 +72,12 @@ public class PartitionActionSetsByPayloadTypeJob { String isLookupUrl = parser.get("isLookupUrl"); logger.info("isLookupUrl: {}", isLookupUrl); - List inputActionSetPaths = ISClient.getLatestRawsetPaths(isLookupUrl, inputActionSetIds); + new PartitionActionSetsByPayloadTypeJob(isLookupUrl).run(isSparkSessionManaged, inputActionSetIds, outputPath); + } + + protected void run(Boolean isSparkSessionManaged, String inputActionSetIds, String outputPath) { + + List inputActionSetPaths = getIsClient().getLatestRawsetPaths(inputActionSetIds); logger.info("inputActionSetPaths: {}", String.join(",", inputActionSetPaths)); SparkConf conf = new SparkConf(); @@ -95,21 +109,15 @@ public class PartitionActionSetsByPayloadTypeJob { String path) { logger.info("Reading actions from path: {}", path); - List files = HdfsSupport.listFiles(path, spark.sparkContext().hadoopConfiguration()); - logger.info("Found files: {}", String.join(",", files)); - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - return files - .stream() - .map(file -> { - JavaRDD rdd = sc - .sequenceFile(file, Text.class, Text.class) - .map(x -> RowFactory.create(x._1().toString(), x._2().toString())); - return spark.createDataFrame(rdd, KV_SCHEMA) - .withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA)) - .select(expr("atomic_action.*")); - }) - .reduce(spark.createDataFrame(Collections.emptyList(), ATOMIC_ACTION_SCHEMA), Dataset::union); + + JavaRDD rdd = sc + .sequenceFile(path, Text.class, Text.class) + .map(x -> RowFactory.create(x._1().toString(), x._2().toString())); + + return spark.createDataFrame(rdd, KV_SCHEMA) + .withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA)) + .select(expr("atomic_action.*")); } private static void saveActions(Dataset actionDS, @@ -121,4 +129,12 @@ public class PartitionActionSetsByPayloadTypeJob { .mode(SaveMode.Append) .parquet(path); } + + public ISClient getIsClient() { + return isClient; + } + + public void setIsClient(ISClient isClient) { + this.isClient = isClient; + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java index 68bb35c2b..20b75842c 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java @@ -8,6 +8,7 @@ import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -119,10 +120,17 @@ public class PromoteActionPayloadForGraphTableJob { String path, Class rowClazz) { logger.info("Reading graph table from path: {}", path); + + return spark.read() + .textFile(path) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, rowClazz), Encoders.bean(rowClazz)); + + /* return spark .read() .parquet(path) .as(Encoders.bean(rowClazz)); + */ } private static Dataset readActionPayload(SparkSession spark, diff --git a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java index 2761a7c89..bd5dc9a5d 100644 --- a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java +++ b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java @@ -1,6 +1,8 @@ package eu.dnetlib.dhp.actionmanager.partition; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import eu.dnetlib.dhp.actionmanager.ISClient; import eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJobTest; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.hadoop.conf.Configuration; @@ -15,7 +17,11 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; import org.junit.jupiter.api.*; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; import scala.Tuple2; import scala.collection.mutable.Seq; @@ -31,6 +37,7 @@ import static org.apache.spark.sql.functions.*; import static org.junit.jupiter.api.Assertions.assertIterableEquals; import static scala.collection.JavaConversions.mutableSeqAsJavaList; +@ExtendWith(MockitoExtension.class) public class PartitionActionSetsByPayloadTypeJobTest { private static final ClassLoader cl = PartitionActionSetsByPayloadTypeJobTest.class.getClassLoader(); @@ -64,20 +71,29 @@ public class PartitionActionSetsByPayloadTypeJobTest { @Nested class Main { + @Mock + private ISClient isClient; + @Test public void shouldPartitionActionSetsByPayloadType(@TempDir Path workingDir) throws Exception { // given - Path inputActionSetsDir = workingDir.resolve("input").resolve("action_sets"); + Path inputActionSetsBaseDir = workingDir.resolve("input").resolve("action_sets"); Path outputDir = workingDir.resolve("output"); - Map> oafsByClassName = createActionSets(inputActionSetsDir); + Map> oafsByClassName = createActionSets(inputActionSetsBaseDir); + + List inputActionSetsPaths = resolveInputActionSetPaths(inputActionSetsBaseDir); // when - PartitionActionSetsByPayloadTypeJob.main(new String[]{ - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-inputActionSetPaths", inputActionSetsDir.toString(), - "-outputPath", outputDir.toString() - }); + Mockito.when(isClient.getLatestRawsetPaths(Mockito.anyString())).thenReturn(inputActionSetsPaths); + + PartitionActionSetsByPayloadTypeJob job = new PartitionActionSetsByPayloadTypeJob(); + job.setIsClient(isClient); + job.run( + Boolean.FALSE, + "", // it can be empty we're mocking the response from isClient to resolve the paths + outputDir.toString() + ); // then Files.exists(outputDir); @@ -94,10 +110,19 @@ public class PartitionActionSetsByPayloadTypeJobTest { } } + private List resolveInputActionSetPaths(Path inputActionSetsBaseDir) throws IOException { + Path inputActionSetJsonDumpsDir = getInputActionSetJsonDumpsDir(); + return Files + .list(inputActionSetJsonDumpsDir) + .map(path -> { + String inputActionSetId = path.getFileName().toString(); + return inputActionSetsBaseDir.resolve(inputActionSetId).toString(); + }) + .collect(Collectors.toCollection(ArrayList::new)); + } + private static Map> createActionSets(Path inputActionSetsDir) throws IOException { - Path inputActionSetJsonDumpsDir = Paths - .get(Objects.requireNonNull(cl.getResource("eu/dnetlib/dhp/actionmanager/partition/input/")) - .getFile()); + Path inputActionSetJsonDumpsDir = getInputActionSetJsonDumpsDir(); Map> oafsByType = new HashMap<>(); Files @@ -138,6 +163,12 @@ public class PartitionActionSetsByPayloadTypeJobTest { return oafsByType; } + private static Path getInputActionSetJsonDumpsDir() { + return Paths + .get(Objects.requireNonNull(cl.getResource("eu/dnetlib/dhp/actionmanager/partition/input/")) + .getFile()); + } + private static Dataset readActionsFromJsonDump(String path) { return spark .read() From da019495d76ae9221b1c0556a8260420047fa1d0 Mon Sep 17 00:00:00 2001 From: pjacewicz Date: Wed, 15 Apr 2020 17:56:57 +0200 Subject: [PATCH 8/9] [dhp-actionmanager] target dir removal added for distcp actions --- .../dhp/actionmanager/wf/dataset/oozie_app/workflow.xml | 6 ++++++ .../dhp/actionmanager/wf/datasource/oozie_app/workflow.xml | 3 +++ .../actionmanager/wf/organization/oozie_app/workflow.xml | 3 +++ .../wf/otherresearchproduct/oozie_app/workflow.xml | 6 ++++++ .../dhp/actionmanager/wf/project/oozie_app/workflow.xml | 3 +++ .../dhp/actionmanager/wf/publication/oozie_app/workflow.xml | 6 ++++++ .../dhp/actionmanager/wf/relation/oozie_app/workflow.xml | 3 +++ .../dhp/actionmanager/wf/software/oozie_app/workflow.xml | 6 ++++++ 8 files changed, 36 insertions(+) diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/dataset/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/dataset/oozie_app/workflow.xml index 27c272aea..f95349935 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/dataset/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/dataset/oozie_app/workflow.xml @@ -118,6 +118,9 @@ + + + -pb ${inputGraphRootPath}/dataset ${workingDir}/dataset @@ -166,6 +169,9 @@ + + + -pb ${workingDir}/dataset ${outputGraphRootPath}/dataset diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/datasource/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/datasource/oozie_app/workflow.xml index 800f29b17..c85ba4ac1 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/datasource/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/datasource/oozie_app/workflow.xml @@ -113,6 +113,9 @@ + + + -pb ${inputGraphRootPath}/datasource ${outputGraphRootPath}/datasource diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/organization/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/organization/oozie_app/workflow.xml index a6c6220e4..412cad70b 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/organization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/organization/oozie_app/workflow.xml @@ -113,6 +113,9 @@ + + + -pb ${inputGraphRootPath}/organization ${outputGraphRootPath}/organization diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/otherresearchproduct/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/otherresearchproduct/oozie_app/workflow.xml index 4b65de3df..0deb1b945 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/otherresearchproduct/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/otherresearchproduct/oozie_app/workflow.xml @@ -117,6 +117,9 @@ + + + -pb ${inputGraphRootPath}/otherresearchproduct ${workingDir}/otherresearchproduct @@ -165,6 +168,9 @@ + + + -pb ${workingDir}/otherresearchproduct ${outputGraphRootPath}/otherresearchproduct diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/project/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/project/oozie_app/workflow.xml index c73fea96e..daf48e9d7 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/project/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/project/oozie_app/workflow.xml @@ -113,6 +113,9 @@ + + + -pb ${inputGraphRootPath}/project ${outputGraphRootPath}/project diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/publication/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/publication/oozie_app/workflow.xml index 35be128da..70400a123 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/publication/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/publication/oozie_app/workflow.xml @@ -118,6 +118,9 @@ + + + -pb ${inputGraphRootPath}/publication ${workingDir}/publication @@ -166,6 +169,9 @@ + + + -pb ${workingDir}/publication ${outputGraphRootPath}/publication diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/relation/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/relation/oozie_app/workflow.xml index f64301c70..a7dce8f2f 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/relation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/relation/oozie_app/workflow.xml @@ -114,6 +114,9 @@ + + + -pb ${inputGraphRootPath}/relation ${outputGraphRootPath}/relation diff --git a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/software/oozie_app/workflow.xml b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/software/oozie_app/workflow.xml index 397184c91..396e27721 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/software/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-actionmanager/src/main/resources/eu/dnetlib/dhp/actionmanager/wf/software/oozie_app/workflow.xml @@ -117,6 +117,9 @@ + + + -pb ${inputGraphRootPath}/software ${workingDir}/software @@ -165,6 +168,9 @@ + + + -pb ${workingDir}/software ${outputGraphRootPath}/software From 5e2fa996aa76b13b08d08127a4abdb25edb1e25a Mon Sep 17 00:00:00 2001 From: sandro Date: Fri, 17 Apr 2020 12:11:51 +0200 Subject: [PATCH 9/9] fixed problem with conversion of long into string --- .../dhp/schema/oaf/StructuredProperty.java | 2 +- .../provision/update/Datacite2Scholix.java | 10 ++++++++-- .../dhp/sx/synch/oozie_app/config-default.xml | 4 ++++ .../dhp/provision/DataciteClientTest.java | 19 ++++++++++++++----- 4 files changed, 27 insertions(+), 8 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java index f6c6b7335..5df6b80f3 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java @@ -2,7 +2,7 @@ package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; -public class StructuredProperty implements Serializable { +public class StructuredProperty implements Serializable { private String value; diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java index ac05a8350..fd2e37837 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java @@ -34,8 +34,13 @@ public class Datacite2Scholix { ScholixResource resource = generateDataciteScholixResource(dJson); return relIds.stream().flatMap(s-> { - final List result = generateScholix(resource, s.get("relatedIdentifier"), s.get("relatedIdentifierType"), s.get("relationType"), updated); - return result.stream(); + try { + final List result = generateScholix(resource, ""+s.get("relatedIdentifier"), s.get("relatedIdentifierType"), s.get("relationType"), updated); + return result.stream(); + } catch (Throwable e) + { + return new ArrayList().stream(); + } }).collect(Collectors.toList()); } @@ -48,6 +53,7 @@ public class Datacite2Scholix { } private List generateScholix(ScholixResource source, final String pid, final String pidtype, final String relType, final String updated) { + if ("doi".equalsIgnoreCase(pidtype)) { ScholixResource target = new ScholixResource(); target.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, pidtype))); diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml index 6fb2a1253..7c1a43e51 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/synch/oozie_app/config-default.xml @@ -7,4 +7,8 @@ oozie.action.sharelib.for.spark spark2 + + oozie.launcher.mapreduce.user.classpath.first + true + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java index f4a480163..cc4b0047a 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java @@ -3,16 +3,12 @@ package eu.dnetlib.dhp.provision; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.provision.scholix.Scholix; import eu.dnetlib.dhp.provision.scholix.ScholixResource; -import eu.dnetlib.dhp.provision.update.CrossrefClient; -import eu.dnetlib.dhp.provision.update.Datacite2Scholix; -import eu.dnetlib.dhp.provision.update.DataciteClient; -import eu.dnetlib.dhp.provision.update.DataciteClientIterator; +import eu.dnetlib.dhp.provision.update.*; import eu.dnetlib.scholexplorer.relation.RelationMapper; import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.io.IOException; import java.util.List; @@ -27,6 +23,19 @@ public class DataciteClientTest { System.out.println(new ObjectMapper().writeValueAsString(s)); } + +// public void testS() throws Exception { +// RetrieveUpdateFromDatacite.main(new String[]{ +// "-n", "file:///data/new_s2.txt", +// "-t", "/data/new_s2.txt", +// "-ts", "1586974078", +// "-ih", "ip-90-147-167-25.ct1.garrservices.it", +// "-in", "datacite", +// }); +// +// } + + public void testResolveDataset() throws Exception { DataciteClient dc = new DataciteClient("ip-90-147-167-25.ct1.garrservices.it"); ScholixResource datasetByDOI = dc.getDatasetByDOI("10.17182/hepdata.15392.v1/t5");